From 4f5ca12baa743c8ec0e91c34c71aad2ac95ff70b Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sun, 8 Sep 2024 22:41:30 +0200 Subject: [PATCH 01/14] Rename errorClass to condition in checkError() --- .../spark/sql/avro/AvroCodecSuite.scala | 2 +- .../spark/sql/avro/AvroFunctionsSuite.scala | 6 +- .../spark/sql/avro/AvroLogicalTypeSuite.scala | 2 +- .../org/apache/spark/sql/avro/AvroSuite.scala | 16 +- .../sql/kafka010/KafkaOffsetReaderSuite.scala | 2 +- .../sql/protobuf/ProtobufFunctionsSuite.scala | 20 +- .../sql/protobuf/ProtobufSerdeSuite.scala | 6 +- .../apache/spark/JobCancellationSuite.scala | 6 +- .../org/apache/spark/SparkFunSuite.scala | 40 +-- .../spark/io/CompressionCodecSuite.scala | 4 +- .../metrics/sink/GraphiteSinkSuite.scala | 6 +- .../scala/org/apache/spark/rdd/RDDSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 4 +- .../sort/ShuffleExternalSorterSuite.scala | 2 +- .../spark/ml/tuning/CrossValidatorSuite.scala | 2 +- .../ExpressionImplUtilsSuite.scala | 4 +- .../org/apache/spark/sql/RowJsonSuite.scala | 2 +- .../CatalystTypeConvertersSuite.scala | 10 +- .../sql/catalyst/ScalaReflectionSuite.scala | 4 +- .../spark/sql/catalyst/ShuffleSpecSuite.scala | 2 +- .../analysis/AnalysisErrorSuite.scala | 12 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 +- .../sql/catalyst/analysis/AnalysisTest.scala | 2 +- .../ExpressionTypeCheckingSuite.scala | 58 ++-- .../analysis/LookupFunctionsSuite.scala | 2 +- .../NamedParameterFunctionSuite.scala | 12 +- .../ResolveLambdaVariablesSuite.scala | 4 +- .../analysis/UnsupportedOperationsSuite.scala | 2 +- .../catalog/SessionCatalogSuite.scala | 76 ++--- .../sql/catalyst/csv/CSVExprUtilsSuite.scala | 10 +- .../catalyst/csv/UnivocityParserSuite.scala | 6 +- .../encoders/EncoderErrorMessageSuite.scala | 10 +- .../encoders/EncoderResolutionSuite.scala | 24 +- .../encoders/ExpressionEncoderSuite.scala | 10 +- .../AttributeResolutionSuite.scala | 6 +- .../CallMethodViaReflectionSuite.scala | 4 +- .../CollationExpressionSuite.scala | 2 +- .../CollectionExpressionsSuite.scala | 2 +- .../expressions/ComplexTypeSuite.scala | 4 +- .../expressions/DateExpressionsSuite.scala | 4 +- .../expressions/ExpressionEvalHelper.scala | 2 +- .../GeneratorExpressionSuite.scala | 4 +- .../expressions/JsonExpressionsSuite.scala | 2 +- .../expressions/LiteralExpressionSuite.scala | 2 +- .../expressions/NullExpressionsSuite.scala | 2 +- .../expressions/ObjectExpressionsSuite.scala | 6 +- .../expressions/RegexpExpressionsSuite.scala | 10 +- .../expressions/StringExpressionsSuite.scala | 10 +- .../expressions/XmlExpressionsSuite.scala | 2 +- .../aggregate/PercentileSuite.scala | 2 +- .../codegen/BufferHolderSuite.scala | 4 +- .../expressions/codegen/CodeBlockSuite.scala | 2 +- .../codegen/UnsafeArrayWriterSuite.scala | 2 +- .../VariantExpressionEvalUtilsSuite.scala | 2 +- ...tractPythonUDFFromJoinConditionSuite.scala | 2 +- .../ReassignLambdaVariableIDSuite.scala | 2 +- ...ReplaceNullWithFalseInPredicateSuite.scala | 2 +- .../sql/catalyst/parser/DDLParserSuite.scala | 176 +++++------ .../catalyst/parser/DataTypeParserSuite.scala | 4 +- .../catalyst/parser/ErrorParserSuite.scala | 94 +++--- .../parser/ExpressionParserSuite.scala | 62 ++-- .../catalyst/parser/ParserUtilsSuite.scala | 6 +- .../sql/catalyst/parser/PlanParserSuite.scala | 90 +++--- .../parser/SqlScriptingParserSuite.scala | 14 +- .../parser/TableIdentifierParserSuite.scala | 6 +- .../parser/TableSchemaParserSuite.scala | 12 +- .../sql/catalyst/plans/JoinTypesTest.scala | 2 +- .../streaming/InternalOutputModesSuite.scala | 2 +- .../util/ArrayBasedMapBuilderSuite.scala | 12 +- .../util/ArrayDataIndexedSeqSuite.scala | 2 +- .../util/DateTimeFormatterHelperSuite.scala | 10 +- .../catalyst/util/DateTimeUtilsSuite.scala | 10 +- .../catalyst/util/IntervalUtilsSuite.scala | 24 +- .../util/TimestampFormatterSuite.scala | 2 +- .../sql/connector/catalog/CatalogSuite.scala | 14 +- ...pportsAtomicPartitionManagementSuite.scala | 4 +- .../SupportsPartitionManagementSuite.scala | 8 +- .../spark/sql/types/DataTypeSuite.scala | 16 +- .../DataTypeWriteCompatibilitySuite.scala | 46 +-- .../apache/spark/sql/types/DecimalSuite.scala | 20 +- .../spark/sql/types/StructTypeSuite.scala | 34 +- .../spark/sql/util/ArrowUtilsSuite.scala | 4 +- .../util/CaseInsensitiveStringMapSuite.scala | 2 +- .../spark/sql/util/SchemaUtilsSuite.scala | 8 +- .../planner/SparkConnectProtoSuite.scala | 4 +- .../SparkConnectPluginRegistrySuite.scala | 4 +- .../service/InterceptorRegistrySuite.scala | 6 +- .../sql/ApproximatePercentileQuerySuite.scala | 4 +- .../sql/BitmapExpressionsQuerySuite.scala | 4 +- .../sql/BloomFilterAggregateQuerySuite.scala | 22 +- .../apache/spark/sql/CachedTableSuite.scala | 2 +- .../spark/sql/CharVarcharTestSuite.scala | 24 +- .../sql/CollationSQLExpressionsSuite.scala | 18 +- .../spark/sql/CollationSQLRegexpSuite.scala | 30 +- .../sql/CollationStringExpressionsSuite.scala | 4 +- .../org/apache/spark/sql/CollationSuite.scala | 66 ++-- .../spark/sql/ColumnExpressionSuite.scala | 58 ++-- .../apache/spark/sql/CsvFunctionsSuite.scala | 26 +- .../spark/sql/DataFrameAggregateSuite.scala | 28 +- .../spark/sql/DataFrameAsOfJoinSuite.scala | 6 +- .../spark/sql/DataFrameFunctionsSuite.scala | 298 +++++++++--------- .../spark/sql/DataFrameNaFunctionsSuite.scala | 12 +- .../spark/sql/DataFramePivotSuite.scala | 2 +- .../spark/sql/DataFrameSelfJoinSuite.scala | 2 +- .../sql/DataFrameSetOperationsSuite.scala | 18 +- .../apache/spark/sql/DataFrameStatSuite.scala | 12 +- .../org/apache/spark/sql/DataFrameSuite.scala | 26 +- .../spark/sql/DataFrameToSchemaSuite.scala | 18 +- .../sql/DataFrameWindowFramesSuite.scala | 20 +- .../sql/DataFrameWindowFunctionsSuite.scala | 4 +- .../spark/sql/DataFrameWriterV2Suite.scala | 10 +- .../org/apache/spark/sql/DatasetSuite.scala | 52 +-- .../spark/sql/DatasetUnpivotSuite.scala | 30 +- .../apache/spark/sql/DateFunctionsSuite.scala | 2 +- .../spark/sql/FileBasedDataSourceSuite.scala | 58 ++-- .../spark/sql/GeneratorFunctionSuite.scala | 22 +- .../apache/spark/sql/JsonFunctionsSuite.scala | 42 +-- .../spark/sql/LateralColumnAliasSuite.scala | 28 +- .../apache/spark/sql/MetadataCacheSuite.scala | 6 +- .../apache/spark/sql/MiscFunctionsSuite.scala | 6 +- .../spark/sql/NestedDataSourceSuite.scala | 2 +- .../apache/spark/sql/ParametersSuite.scala | 30 +- .../sql/ResolveDefaultColumnsSuite.scala | 20 +- .../scala/org/apache/spark/sql/RowSuite.scala | 4 +- .../apache/spark/sql/SQLInsertTestSuite.scala | 22 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 34 +- .../sql/ScalaReflectionRelationSuite.scala | 4 +- .../apache/spark/sql/SetCommandSuite.scala | 2 +- .../spark/sql/StatisticsCollectionSuite.scala | 16 +- .../spark/sql/StringFunctionsSuite.scala | 30 +- .../org/apache/spark/sql/SubquerySuite.scala | 18 +- .../TableOptionsConstantFoldingSuite.scala | 12 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 10 +- .../apache/spark/sql/UrlFunctionsSuite.scala | 2 +- .../org/apache/spark/sql/VariantSuite.scala | 24 +- .../apache/spark/sql/XmlFunctionsSuite.scala | 18 +- .../ValidateExternalTypeSuite.scala | 2 +- .../spark/sql/connector/AlterTableTests.scala | 52 +-- .../DataSourceV2DataFrameSuite.scala | 6 +- .../connector/DataSourceV2FunctionSuite.scala | 26 +- .../sql/connector/DataSourceV2SQLSuite.scala | 202 ++++++------ .../sql/connector/DataSourceV2Suite.scala | 26 +- .../spark/sql/connector/DeleteFromTests.scala | 2 +- .../DeltaBasedDeleteFromTableSuite.scala | 2 +- .../DeltaBasedUpdateTableSuiteBase.scala | 4 +- .../FileDataSourceV2FallBackSuite.scala | 4 +- .../GroupBasedDeleteFromTableSuite.scala | 2 +- .../GroupBasedUpdateTableSuite.scala | 2 +- .../spark/sql/connector/InsertIntoTests.scala | 4 +- .../sql/connector/MetadataColumnSuite.scala | 4 +- .../sql/connector/UpdateTableSuiteBase.scala | 2 +- .../QueryCompilationErrorsDSv2Suite.scala | 6 +- .../errors/QueryCompilationErrorsSuite.scala | 100 +++--- .../QueryExecutionAnsiErrorsSuite.scala | 48 +-- .../errors/QueryExecutionErrorsSuite.scala | 108 +++---- .../sql/errors/QueryParsingErrorsSuite.scala | 128 ++++---- .../ExecuteImmediateEndToEndSuite.scala | 2 +- .../spark/sql/execution/SQLViewSuite.scala | 84 ++--- .../sql/execution/SQLViewTestSuite.scala | 30 +- .../spark/sql/execution/SparkPlanSuite.scala | 2 +- .../SparkScriptTransformationSuite.scala | 2 +- .../sql/execution/SparkSqlParserSuite.scala | 78 ++--- .../execution/WholeStageCodegenSuite.scala | 4 +- .../adaptive/AdaptiveQueryExecSuite.scala | 2 +- .../execution/columnar/ColumnTypeSuite.scala | 2 +- .../command/AlignMergeAssignmentsSuite.scala | 8 +- .../command/AlignUpdateAssignmentsSuite.scala | 8 +- .../AlterNamespaceSetLocationSuiteBase.scala | 6 +- ...terNamespaceSetPropertiesParserSuite.scala | 2 +- ...AlterNamespaceSetPropertiesSuiteBase.scala | 4 +- ...rNamespaceUnsetPropertiesParserSuite.scala | 4 +- ...terNamespaceUnsetPropertiesSuiteBase.scala | 4 +- .../AlterTableAddPartitionSuiteBase.scala | 2 +- .../AlterTableClusterBySuiteBase.scala | 2 +- .../AlterTableDropPartitionParserSuite.scala | 2 +- .../AlterTableDropPartitionSuiteBase.scala | 2 +- ...terTableRecoverPartitionsParserSuite.scala | 2 +- .../AlterTableRenameColumnParserSuite.scala | 6 +- .../command/AlterTableRenameParserSuite.scala | 4 +- .../AlterTableRenamePartitionSuiteBase.scala | 4 +- .../AlterTableSetSerdeParserSuite.scala | 2 +- ...lterTableSetTblPropertiesParserSuite.scala | 2 +- .../AlterTableSetTblPropertiesSuiteBase.scala | 4 +- ...erTableUnsetTblPropertiesParserSuite.scala | 2 +- ...lterTableUnsetTblPropertiesSuiteBase.scala | 4 +- .../command/CharVarcharDDLTestBase.scala | 10 +- .../command/CreateNamespaceParserSuite.scala | 12 +- .../command/CreateNamespaceSuiteBase.scala | 4 +- .../execution/command/DDLParserSuite.scala | 74 ++--- .../sql/execution/command/DDLSuite.scala | 130 ++++---- .../command/DeclareVariableParserSuite.scala | 4 +- .../command/DescribeNamespaceSuiteBase.scala | 2 +- .../command/DescribeTableParserSuite.scala | 2 +- .../command/DropNamespaceSuiteBase.scala | 4 +- .../command/DropVariableParserSuite.scala | 4 +- .../command/PlanResolutionSuite.scala | 78 ++--- .../command/ShowColumnsParserSuite.scala | 2 +- .../command/ShowColumnsSuiteBase.scala | 6 +- .../command/ShowPartitionsParserSuite.scala | 2 +- .../command/ShowTablesSuiteBase.scala | 10 +- .../command/TruncateTableParserSuite.scala | 2 +- .../command/TruncateTableSuiteBase.scala | 6 +- .../v1/AlterTableAddPartitionSuite.scala | 4 +- .../v1/AlterTableDropColumnSuite.scala | 2 +- .../v1/AlterTableDropPartitionSuite.scala | 2 +- .../v1/AlterTableRecoverPartitionsSuite.scala | 2 +- .../v1/AlterTableRenameColumnSuite.scala | 2 +- .../command/v1/AlterTableRenameSuite.scala | 4 +- .../v1/AlterTableSetLocationSuite.scala | 4 +- .../command/v1/AlterTableSetSerdeSuite.scala | 12 +- .../command/v1/DescribeTableSuite.scala | 6 +- .../command/v1/DropNamespaceSuite.scala | 2 +- .../command/v1/ShowColumnsSuite.scala | 2 +- .../command/v1/ShowCreateTableSuite.scala | 2 +- .../command/v1/ShowNamespacesSuite.scala | 2 +- .../command/v1/ShowPartitionsSuite.scala | 8 +- .../command/v1/ShowTablesSuite.scala | 4 +- .../command/v1/TruncateTableSuite.scala | 2 +- .../v2/AlterTableAddPartitionSuite.scala | 6 +- .../v2/AlterTableDropColumnSuite.scala | 6 +- .../v2/AlterTableDropPartitionSuite.scala | 4 +- .../v2/AlterTableRecoverPartitionsSuite.scala | 2 +- .../v2/AlterTableRenameColumnSuite.scala | 6 +- .../v2/AlterTableReplaceColumnsSuite.scala | 2 +- .../v2/AlterTableSetLocationSuite.scala | 2 +- .../command/v2/AlterTableSetSerdeSuite.scala | 2 +- .../command/v2/DescribeTableSuite.scala | 4 +- .../execution/command/v2/DropTableSuite.scala | 2 +- .../command/v2/MsckRepairTableSuite.scala | 2 +- .../command/v2/ShowNamespacesSuite.scala | 4 +- .../command/v2/ShowPartitionsSuite.scala | 4 +- .../command/v2/TruncateTableSuite.scala | 2 +- .../datasources/DataSourceSuite.scala | 8 +- .../datasources/FileIndexSuite.scala | 2 +- .../datasources/FileMetadataStructSuite.scala | 10 +- .../SaveIntoDataSourceCommandSuite.scala | 2 +- .../binaryfile/BinaryFileFormatSuite.scala | 2 +- .../execution/datasources/csv/CSVSuite.scala | 44 +-- .../datasources/jdbc/JdbcUtilsSuite.scala | 6 +- .../datasources/json/JsonSuite.scala | 50 +-- .../datasources/orc/OrcFilterSuite.scala | 2 +- .../datasources/orc/OrcQuerySuite.scala | 4 +- .../datasources/orc/OrcSourceSuite.scala | 4 +- ...rquetCompressionCodecPrecedenceSuite.scala | 2 +- .../parquet/ParquetFileFormatSuite.scala | 2 +- ...rquetFileMetadataStructRowIndexSuite.scala | 2 +- .../datasources/parquet/ParquetIOSuite.scala | 2 +- .../ParquetPartitionDiscoverySuite.scala | 6 +- .../parquet/ParquetSchemaSuite.scala | 12 +- .../datasources/text/TextSuite.scala | 2 +- .../v2/V2SessionCatalogSuite.scala | 6 +- .../v2/jdbc/DerbyTableCatalogSuite.scala | 2 +- .../v2/jdbc/JDBCTableCatalogSuite.scala | 38 +-- .../execution/datasources/xml/XmlSuite.scala | 28 +- .../execution/joins/HashedRelationSuite.scala | 2 +- .../python/PythonDataSourceSuite.scala | 10 +- .../PythonStreamingDataSourceSuite.scala | 10 +- .../sql/execution/python/PythonUDFSuite.scala | 10 +- .../execution/python/PythonUDTFSuite.scala | 10 +- .../CompactibleFileStreamLogSuite.scala | 2 +- .../MergingSessionsIteratorSuite.scala | 6 +- .../UpdatingSessionsIteratorSuite.scala | 12 +- .../sources/ForeachWriterSuite.scala | 2 +- .../RatePerMicroBatchProviderSuite.scala | 2 +- .../sources/RateStreamProviderSuite.scala | 6 +- .../sources/TextSocketStreamSuite.scala | 2 +- .../streaming/state/ListStateSuite.scala | 4 +- .../streaming/state/MapStateSuite.scala | 2 +- .../state/RocksDBStateStoreSuite.scala | 26 +- .../streaming/state/RocksDBSuite.scala | 12 +- .../streaming/state/StateStoreSuite.scala | 22 +- .../state/StatefulProcessorHandleSuite.scala | 6 +- .../streaming/state/ValueStateSuite.scala | 10 +- .../sql/execution/ui/UISeleniumSuite.scala | 2 +- .../sql/expressions/ExpressionInfoSuite.scala | 10 +- .../spark/sql/internal/CatalogSuite.scala | 2 +- .../spark/sql/internal/SQLConfSuite.scala | 14 +- .../spark/sql/internal/SharedStateSuite.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 +- .../apache/spark/sql/jdbc/JDBCV2Suite.scala | 12 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 12 +- .../SqlScriptingInterpreterSuite.scala | 14 +- .../sql/sources/BucketedWriteSuite.scala | 14 +- .../sources/CreateTableAsSelectSuite.scala | 4 +- .../sql/sources/DDLSourceLoadSuite.scala | 4 +- .../spark/sql/sources/InsertSuite.scala | 150 ++++----- .../sql/sources/PartitionedWriteSuite.scala | 2 +- .../sql/sources/ResolvedDataSourceSuite.scala | 6 +- .../sql/streaming/FileStreamSinkSuite.scala | 4 +- .../sql/streaming/FileStreamSourceSuite.scala | 2 +- .../spark/sql/streaming/GroupStateSuite.scala | 6 +- .../StreamingDeduplicationSuite.scala | 4 +- .../sql/streaming/StreamingJoinSuite.scala | 4 +- .../sql/streaming/StreamingQuerySuite.scala | 4 +- .../StreamingSessionWindowSuite.scala | 4 +- .../TransformWithMapStateSuite.scala | 4 +- .../TransformWithStateChainingSuite.scala | 2 +- .../TransformWithStateInitialStateSuite.scala | 2 +- .../streaming/TransformWithStateSuite.scala | 10 +- .../sources/StreamingDataSourceV2Suite.scala | 8 +- .../test/DataStreamReaderWriterSuite.scala | 2 +- .../test/DataStreamTableAPISuite.scala | 2 +- .../sql/test/DataFrameReaderWriterSuite.scala | 28 +- .../sql/hive/HiveMetadataCacheSuite.scala | 6 +- .../sql/hive/HiveMetastoreCatalogSuite.scala | 2 +- .../spark/sql/hive/HiveParquetSuite.scala | 2 +- .../sql/hive/HiveSQLInsertTestSuite.scala | 2 +- .../apache/spark/sql/hive/InsertSuite.scala | 12 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 26 +- .../spark/sql/hive/MultiDatabaseSuite.scala | 8 +- .../spark/sql/hive/StatisticsSuite.scala | 4 +- .../sql/hive/client/HiveClientSuite.scala | 6 +- .../execution/AggregationQuerySuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 118 +++---- .../sql/hive/execution/HiveQuerySuite.scala | 20 +- .../hive/execution/HiveResolutionSuite.scala | 2 +- .../sql/hive/execution/HiveSQLViewSuite.scala | 6 +- .../sql/hive/execution/HiveUDAFSuite.scala | 4 +- .../sql/hive/execution/HiveUDFSuite.scala | 4 +- .../hive/execution/Hive_2_1_DDLSuite.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 14 +- .../sql/hive/execution/UDAQuerySuite.scala | 2 +- .../AlterNamespaceSetLocationSuite.scala | 2 +- .../command/AlterTableAddColumnsSuite.scala | 2 +- .../command/CreateNamespaceSuite.scala | 2 +- .../command/ShowCreateTableSuite.scala | 4 +- .../sql/hive/orc/HiveOrcQuerySuite.scala | 2 +- .../sql/hive/orc/HiveOrcSourceSuite.scala | 10 +- .../sql/sources/HadoopFsRelationTest.scala | 4 +- 329 files changed, 2412 insertions(+), 2412 deletions(-) diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCodecSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCodecSuite.scala index 256b608feaa1f..0db9d284c4512 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCodecSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCodecSuite.scala @@ -54,7 +54,7 @@ class AvroCodecSuite extends FileSourceCodecSuite { s"""CREATE TABLE avro_t |USING $format OPTIONS('compression'='unsupported') |AS SELECT 1 as id""".stripMargin)), - errorClass = "CODEC_SHORT_NAME_NOT_FOUND", + condition = "CODEC_SHORT_NAME_NOT_FOUND", sqlState = Some("42704"), parameters = Map("codecName" -> "unsupported") ) diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala index 432c3fa9be3ac..47faaf7662a50 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala @@ -329,7 +329,7 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { s""" |select to_avro(s, 42) as result from t |""".stripMargin)), - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = Map("sqlExpr" -> "\"to_avro(s, 42)\"", "msg" -> ("The second argument of the TO_AVRO SQL function must be a constant string " + "containing the JSON representation of the schema to use for converting the value to " + @@ -344,7 +344,7 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { s""" |select from_avro(s, 42, '') as result from t |""".stripMargin)), - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = Map("sqlExpr" -> "\"from_avro(s, 42, )\"", "msg" -> ("The second argument of the FROM_AVRO SQL function must be a constant string " + "containing the JSON representation of the schema to use for converting the value " + @@ -359,7 +359,7 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { s""" |select from_avro(s, '$jsonFormatSchema', 42) as result from t |""".stripMargin)), - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = Map( "sqlExpr" -> s"\"from_avro(s, $jsonFormatSchema, 42)\"".stripMargin, diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala index 429f3c0deca6a..751ac275e048a 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala @@ -439,7 +439,7 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession { assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[SparkArithmeticException], - errorClass = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", parameters = Map( "value" -> "0", "precision" -> "4", diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index b20ee4b3cc231..14ed6c43e4c0f 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -894,7 +894,7 @@ abstract class AvroSuite assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[AnalysisException], - errorClass = "AVRO_INCOMPATIBLE_READ_TYPE", + condition = "AVRO_INCOMPATIBLE_READ_TYPE", parameters = Map("avroPath" -> "field 'a'", "sqlPath" -> "field 'a'", "avroType" -> "decimal\\(12,10\\)", @@ -972,7 +972,7 @@ abstract class AvroSuite assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[AnalysisException], - errorClass = "AVRO_INCOMPATIBLE_READ_TYPE", + condition = "AVRO_INCOMPATIBLE_READ_TYPE", parameters = Map("avroPath" -> "field 'a'", "sqlPath" -> "field 'a'", "avroType" -> "interval day to second", @@ -1009,7 +1009,7 @@ abstract class AvroSuite assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[AnalysisException], - errorClass = "AVRO_INCOMPATIBLE_READ_TYPE", + condition = "AVRO_INCOMPATIBLE_READ_TYPE", parameters = Map("avroPath" -> "field 'a'", "sqlPath" -> "field 'a'", "avroType" -> "interval year to month", @@ -1673,7 +1673,7 @@ abstract class AvroSuite exception = intercept[AnalysisException] { sql("select interval 1 days").write.format("avro").mode("overwrite").save(tempDir) }, - errorClass = "_LEGACY_ERROR_TEMP_1136", + condition = "_LEGACY_ERROR_TEMP_1136", parameters = Map.empty ) checkError( @@ -1681,7 +1681,7 @@ abstract class AvroSuite spark.udf.register("testType", () => new IntervalData()) sql("select testType()").write.format("avro").mode("overwrite").save(tempDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`testType()`", "columnType" -> "UDT(\"INTERVAL\")", @@ -2726,7 +2726,7 @@ abstract class AvroSuite |LOCATION '${dir}' |AS SELECT ID, IF(ID=1,1,0) FROM v""".stripMargin) }, - errorClass = "INVALID_COLUMN_NAME_AS_PATH", + condition = "INVALID_COLUMN_NAME_AS_PATH", parameters = Map( "datasource" -> "AvroFileFormat", "columnName" -> "`(IF((ID = 1), 1, 0))`") ) @@ -2831,7 +2831,7 @@ class AvroV1Suite extends AvroSuite { sql("SELECT ID, IF(ID=1,1,0) FROM v").write.mode(SaveMode.Overwrite) .format("avro").save(dir.getCanonicalPath) }, - errorClass = "INVALID_COLUMN_NAME_AS_PATH", + condition = "INVALID_COLUMN_NAME_AS_PATH", parameters = Map( "datasource" -> "AvroFileFormat", "columnName" -> "`(IF((ID = 1), 1, 0))`") ) @@ -2844,7 +2844,7 @@ class AvroV1Suite extends AvroSuite { .write.mode(SaveMode.Overwrite) .format("avro").save(dir.getCanonicalPath) }, - errorClass = "INVALID_COLUMN_NAME_AS_PATH", + condition = "INVALID_COLUMN_NAME_AS_PATH", parameters = Map( "datasource" -> "AvroFileFormat", "columnName" -> "`(IF((ID = 1), 1, 0))`") ) diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderSuite.scala index 320485a79e59d..6fc22e7ac5e03 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderSuite.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderSuite.scala @@ -153,7 +153,7 @@ class KafkaOffsetReaderSuite extends QueryTest with SharedSparkSession with Kafk } checkError( exception = ex, - errorClass = "KAFKA_START_OFFSET_DOES_NOT_MATCH_ASSIGNED", + condition = "KAFKA_START_OFFSET_DOES_NOT_MATCH_ASSIGNED", parameters = Map( "specifiedPartitions" -> "Set\\(.*,.*\\)", "assignedPartitions" -> "Set\\(.*,.*,.*\\)"), diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala index e47f98a1f3f55..3eaa91e472c43 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala @@ -708,7 +708,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot } checkError( exception = e, - errorClass = "PROTOBUF_DEPENDENCY_NOT_FOUND", + condition = "PROTOBUF_DEPENDENCY_NOT_FOUND", parameters = Map("dependencyName" -> "nestedenum.proto")) } @@ -1057,7 +1057,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot } checkError( ex, - errorClass = "PROTOBUF_DESCRIPTOR_FILE_NOT_FOUND", + condition = "PROTOBUF_DESCRIPTOR_FILE_NOT_FOUND", parameters = Map("filePath" -> "/non/existent/path.desc") ) assert(ex.getCause != null) @@ -1699,7 +1699,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot } checkError( exception = parseError, - errorClass = "CANNOT_CONVERT_SQL_VALUE_TO_PROTOBUF_ENUM_TYPE", + condition = "CANNOT_CONVERT_SQL_VALUE_TO_PROTOBUF_ENUM_TYPE", parameters = Map( "sqlColumn" -> "`basic_enum`", "protobufColumn" -> "field 'basic_enum'", @@ -1711,7 +1711,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot } checkError( exception = parseError, - errorClass = "CANNOT_CONVERT_SQL_VALUE_TO_PROTOBUF_ENUM_TYPE", + condition = "CANNOT_CONVERT_SQL_VALUE_TO_PROTOBUF_ENUM_TYPE", parameters = Map( "sqlColumn" -> "`basic_enum`", "protobufColumn" -> "field 'basic_enum'", @@ -2093,7 +2093,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot | to_protobuf(complex_struct, 42, '$testFileDescFile', map()) |FROM protobuf_test_table |""".stripMargin)), - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = Map( "sqlExpr" -> s"""\"to_protobuf(complex_struct, 42, $testFileDescFile, map())\"""", "msg" -> ("The second argument of the TO_PROTOBUF SQL function must be a constant " + @@ -2111,7 +2111,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot | to_protobuf(complex_struct, 'SimpleMessageJavaTypes', 42, map()) |FROM protobuf_test_table |""".stripMargin)), - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = Map( "sqlExpr" -> "\"to_protobuf(complex_struct, SimpleMessageJavaTypes, 42, map())\"", "msg" -> ("The third argument of the TO_PROTOBUF SQL function must be a constant " + @@ -2130,7 +2130,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot | to_protobuf(complex_struct, 'SimpleMessageJavaTypes', '$testFileDescFile', 42) |FROM protobuf_test_table |""".stripMargin)), - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = Map( "sqlExpr" -> s"""\"to_protobuf(complex_struct, SimpleMessageJavaTypes, $testFileDescFile, 42)\"""", @@ -2152,7 +2152,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot |SELECT from_protobuf(protobuf_data, 42, '$testFileDescFile', map()) |FROM ($toProtobufSql) |""".stripMargin)), - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = Map( "sqlExpr" -> s"""\"from_protobuf(protobuf_data, 42, $testFileDescFile, map())\"""", "msg" -> ("The second argument of the FROM_PROTOBUF SQL function must be a constant " + @@ -2169,7 +2169,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot |SELECT from_protobuf(protobuf_data, 'SimpleMessageJavaTypes', 42, map()) |FROM ($toProtobufSql) |""".stripMargin)), - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = Map( "sqlExpr" -> "\"from_protobuf(protobuf_data, SimpleMessageJavaTypes, 42, map())\"", "msg" -> ("The third argument of the FROM_PROTOBUF SQL function must be a constant " + @@ -2188,7 +2188,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot | from_protobuf(protobuf_data, 'SimpleMessageJavaTypes', '$testFileDescFile', 42) |FROM ($toProtobufSql) |""".stripMargin)), - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = Map( "sqlExpr" -> s"""\"from_protobuf(protobuf_data, SimpleMessageJavaTypes, $testFileDescFile, 42)\"""", diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala index 03285c73f1ff1..d71aa400c0ca7 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala @@ -222,7 +222,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { checkError( exception = e1, - errorClass = "CANNOT_PARSE_PROTOBUF_DESCRIPTOR") + condition = "CANNOT_PARSE_PROTOBUF_DESCRIPTOR") val basicMessageDescWithoutImports = descriptorSetWithoutImports( ProtobufUtils.readDescriptorFileContent( @@ -240,7 +240,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { checkError( exception = e2, - errorClass = "PROTOBUF_DEPENDENCY_NOT_FOUND", + condition = "PROTOBUF_DEPENDENCY_NOT_FOUND", parameters = Map("dependencyName" -> "nestedenum.proto")) } @@ -274,7 +274,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { assert(e.getMessage === expectMsg) checkError( exception = e, - errorClass = errorClass, + condition = errorClass, parameters = params) } diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index f0d7059e29be1..380231ce97c0b 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -208,7 +208,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft ThreadUtils.awaitReady(job, Duration.Inf).failed.foreach { case e: SparkException => checkError( exception = e, - errorClass = "SPARK_JOB_CANCELLED", + condition = "SPARK_JOB_CANCELLED", sqlState = "XXKDA", parameters = scala.collection.immutable.Map( "jobId" -> "0", @@ -222,7 +222,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft sc.setJobGroup(jobGroupName, "") sc.parallelize(1 to 100).count() }, - errorClass = "SPARK_JOB_CANCELLED", + condition = "SPARK_JOB_CANCELLED", sqlState = "XXKDA", parameters = scala.collection.immutable.Map( "jobId" -> "1", @@ -258,7 +258,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft ThreadUtils.awaitReady(job, Duration.Inf).failed.foreach { case e: SparkException => checkError( exception = e, - errorClass = "SPARK_JOB_CANCELLED", + condition = "SPARK_JOB_CANCELLED", sqlState = "XXKDA", parameters = scala.collection.immutable.Map( "jobId" -> "0", diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 1966a60c1665e..e1656583dce99 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -329,7 +329,7 @@ abstract class SparkFunSuite /** * Checks an exception with an error class against expected results. * @param exception The exception to check - * @param errorClass The expected error class identifying the error + * @param condition The expected error class identifying the error * @param sqlState Optional the expected SQLSTATE, not verified if not supplied * @param parameters A map of parameter names and values. The names are as defined * in the error-classes file. @@ -338,12 +338,12 @@ abstract class SparkFunSuite */ protected def checkError( exception: SparkThrowable, - errorClass: String, + condition: String, sqlState: Option[String] = None, parameters: Map[String, String] = Map.empty, matchPVals: Boolean = false, queryContext: Array[ExpectedContext] = Array.empty): Unit = { - assert(exception.getErrorClass === errorClass) + assert(exception.getErrorClass === condition) sqlState.foreach(state => assert(exception.getSqlState === state)) val expectedParameters = exception.getMessageParameters.asScala if (matchPVals) { @@ -390,55 +390,55 @@ abstract class SparkFunSuite protected def checkError( exception: SparkThrowable, - errorClass: String, + condition: String, sqlState: String, parameters: Map[String, String]): Unit = - checkError(exception, errorClass, Some(sqlState), parameters) + checkError(exception, condition, Some(sqlState), parameters) protected def checkError( exception: SparkThrowable, - errorClass: String, + condition: String, sqlState: String, parameters: Map[String, String], context: ExpectedContext): Unit = - checkError(exception, errorClass, Some(sqlState), parameters, false, Array(context)) + checkError(exception, condition, Some(sqlState), parameters, false, Array(context)) protected def checkError( exception: SparkThrowable, - errorClass: String, + condition: String, parameters: Map[String, String], context: ExpectedContext): Unit = - checkError(exception, errorClass, None, parameters, false, Array(context)) + checkError(exception, condition, None, parameters, false, Array(context)) protected def checkError( exception: SparkThrowable, - errorClass: String, + condition: String, sqlState: String, context: ExpectedContext): Unit = - checkError(exception, errorClass, None, Map.empty, false, Array(context)) + checkError(exception, condition, None, Map.empty, false, Array(context)) protected def checkError( exception: SparkThrowable, - errorClass: String, + condition: String, sqlState: Option[String], parameters: Map[String, String], context: ExpectedContext): Unit = - checkError(exception, errorClass, sqlState, parameters, + checkError(exception, condition, sqlState, parameters, false, Array(context)) protected def checkErrorMatchPVals( exception: SparkThrowable, - errorClass: String, + condition: String, parameters: Map[String, String]): Unit = - checkError(exception, errorClass, None, parameters, matchPVals = true) + checkError(exception, condition, None, parameters, matchPVals = true) protected def checkErrorMatchPVals( exception: SparkThrowable, - errorClass: String, + condition: String, sqlState: Option[String], parameters: Map[String, String], context: ExpectedContext): Unit = - checkError(exception, errorClass, sqlState, parameters, + checkError(exception, condition, sqlState, parameters, matchPVals = true, Array(context)) protected def checkErrorTableNotFound( @@ -446,7 +446,7 @@ abstract class SparkFunSuite tableName: String, queryContext: ExpectedContext): Unit = checkError(exception = exception, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> tableName), queryContext = Array(queryContext)) @@ -454,13 +454,13 @@ abstract class SparkFunSuite exception: SparkThrowable, tableName: String): Unit = checkError(exception = exception, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> tableName)) protected def checkErrorTableAlreadyExists(exception: SparkThrowable, tableName: String): Unit = checkError(exception = exception, - errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> tableName)) case class ExpectedContext( diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 5c09a1f965b9e..ff971b72d8910 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -132,7 +132,7 @@ class CompressionCodecSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { CompressionCodec.createCodec(conf, "foobar") }, - errorClass = "CODEC_NOT_AVAILABLE.WITH_CONF_SUGGESTION", + condition = "CODEC_NOT_AVAILABLE.WITH_CONF_SUGGESTION", parameters = Map( "codecName" -> "foobar", "configKey" -> "\"spark.io.compression.codec\"", @@ -171,7 +171,7 @@ class CompressionCodecSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { CompressionCodec.getShortName(codecClass.toUpperCase(Locale.ROOT)) }, - errorClass = "CODEC_SHORT_NAME_NOT_FOUND", + condition = "CODEC_SHORT_NAME_NOT_FOUND", parameters = Map("codecName" -> codecClass.toUpperCase(Locale.ROOT))) } } diff --git a/core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala b/core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala index 55d82aed5c3f2..817d660763361 100644 --- a/core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala @@ -88,7 +88,7 @@ class GraphiteSinkSuite extends SparkFunSuite { val e = intercept[SparkException] { new GraphiteSink(props, registry) } - checkError(e, errorClass = "GRAPHITE_SINK_PROPERTY_MISSING", + checkError(e, condition = "GRAPHITE_SINK_PROPERTY_MISSING", parameters = Map("property" -> "host")) } @@ -100,7 +100,7 @@ class GraphiteSinkSuite extends SparkFunSuite { val e = intercept[SparkException] { new GraphiteSink(props, registry) } - checkError(e, errorClass = "GRAPHITE_SINK_PROPERTY_MISSING", + checkError(e, condition = "GRAPHITE_SINK_PROPERTY_MISSING", parameters = Map("property" -> "port")) } @@ -115,7 +115,7 @@ class GraphiteSinkSuite extends SparkFunSuite { exception = intercept[SparkException] { new GraphiteSink(props, registry) }, - errorClass = "GRAPHITE_SINK_INVALID_PROTOCOL", + condition = "GRAPHITE_SINK_INVALID_PROTOCOL", parameters = Map("protocol" -> "http") ) } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 7c5db914cd5ba..8bb96a0f53c73 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -922,7 +922,7 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext with Eventually { exception = intercept[SparkIllegalArgumentException] { rdd1.cartesian(rdd2).partitions }, - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.INITIALIZE", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.INITIALIZE", sqlState = "54000", parameters = Map( "numberOfElements" -> (numSlices.toLong * numSlices.toLong).toString, diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 978ceb16b376c..243d33fe55a79 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -779,7 +779,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti assert(failureReason.isDefined) checkError( exception = failureReason.get.asInstanceOf[SparkException], - errorClass = "SPARK_JOB_CANCELLED", + condition = "SPARK_JOB_CANCELLED", sqlState = "XXKDA", parameters = scala.collection.immutable.Map("jobId" -> "0", "reason" -> "") ) @@ -901,7 +901,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti cancel(jobId) checkError( exception = failure.asInstanceOf[SparkException], - errorClass = "SPARK_JOB_CANCELLED", + condition = "SPARK_JOB_CANCELLED", sqlState = "XXKDA", parameters = scala.collection.immutable.Map("jobId" -> jobId.toString, "reason" -> "") ) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala index 5b6fb31d598ac..aad649b7b2612 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala @@ -111,7 +111,7 @@ class ShuffleExternalSorterSuite extends SparkFunSuite with LocalSparkContext wi exception = intercept[SparkOutOfMemoryError] { sorter.insertRecord(bytes, Platform.BYTE_ARRAY_OFFSET, 1, 0) }, - errorClass = "UNABLE_TO_ACQUIRE_MEMORY", + condition = "UNABLE_TO_ACQUIRE_MEMORY", parameters = Map("requestedBytes" -> "800", "receivedBytes" -> "400")) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index b961f97cd877f..f97fefa245145 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -128,7 +128,7 @@ class CrossValidatorSuite exception = intercept[SparkIllegalArgumentException] { cv.fit(datasetWithFold) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map( "fieldName" -> "`fold1`", "fields" -> "`label`, `features`, `fold`") diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtilsSuite.scala b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtilsSuite.scala index 40e6182e587b3..50e933ba97ae6 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtilsSuite.scala +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtilsSuite.scala @@ -349,7 +349,7 @@ class ExpressionImplUtilsSuite extends SparkFunSuite { exception = intercept[SparkRuntimeException] { f(t) }, - errorClass = t.expectedErrorClassOpt.get, + condition = t.expectedErrorClassOpt.get, parameters = t.errorParamsMap ) } @@ -361,7 +361,7 @@ class ExpressionImplUtilsSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { ExpressionImplUtils.validateUTF8String(str) }, - errorClass = "INVALID_UTF8_STRING", + condition = "INVALID_UTF8_STRING", parameters = Map( "str" -> str.getBytes.map(byte => f"\\x$byte%02X").mkString ) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala index 3e72dc7da24b7..cf50063baa13c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala @@ -136,7 +136,7 @@ class RowJsonSuite extends SparkFunSuite { new StructType().add("a", ObjectType(classOf[(Int, Int)]))) row.jsonValue }, - errorClass = "FAILED_ROW_TO_JSON", + condition = "FAILED_ROW_TO_JSON", parameters = Map( "value" -> toSQLValue("(1,2)"), "class" -> "class scala.Tuple2$mcII$sp", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala index 562aac766fc33..7572843f44a19 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala @@ -108,7 +108,7 @@ class CatalystTypeConvertersSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { CatalystTypeConverters.createToCatalystConverter(structType)("test") }, - errorClass = "_LEGACY_ERROR_TEMP_3219", + condition = "_LEGACY_ERROR_TEMP_3219", parameters = Map( "other" -> "test", "otherClass" -> "java.lang.String", @@ -121,7 +121,7 @@ class CatalystTypeConvertersSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { CatalystTypeConverters.createToCatalystConverter(mapType)("test") }, - errorClass = "_LEGACY_ERROR_TEMP_3221", + condition = "_LEGACY_ERROR_TEMP_3221", parameters = Map( "other" -> "test", "otherClass" -> "java.lang.String", @@ -135,7 +135,7 @@ class CatalystTypeConvertersSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { CatalystTypeConverters.createToCatalystConverter(arrayType)("test") }, - errorClass = "_LEGACY_ERROR_TEMP_3220", + condition = "_LEGACY_ERROR_TEMP_3220", parameters = Map( "other" -> "test", "otherClass" -> "java.lang.String", @@ -148,7 +148,7 @@ class CatalystTypeConvertersSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { CatalystTypeConverters.createToCatalystConverter(decimalType)("test") }, - errorClass = "_LEGACY_ERROR_TEMP_3219", + condition = "_LEGACY_ERROR_TEMP_3219", parameters = Map( "other" -> "test", "otherClass" -> "java.lang.String", @@ -160,7 +160,7 @@ class CatalystTypeConvertersSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { CatalystTypeConverters.createToCatalystConverter(StringType)(0.1) }, - errorClass = "_LEGACY_ERROR_TEMP_3219", + condition = "_LEGACY_ERROR_TEMP_3219", parameters = Map( "other" -> "0.1", "otherClass" -> "java.lang.Double", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index daa8d12613f2c..a09dadbcd4816 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -494,7 +494,7 @@ class ScalaReflectionSuite extends SparkFunSuite { exception = intercept[SparkUnsupportedOperationException] { schemaFor[TraitProductWithoutCompanion] }, - errorClass = "_LEGACY_ERROR_TEMP_2144", + condition = "_LEGACY_ERROR_TEMP_2144", parameters = Map("tpe" -> "org.apache.spark.sql.catalyst.TraitProductWithoutCompanion")) } @@ -503,7 +503,7 @@ class ScalaReflectionSuite extends SparkFunSuite { exception = intercept[SparkUnsupportedOperationException] { schemaFor[TraitProductWithNoConstructorCompanion] }, - errorClass = "_LEGACY_ERROR_TEMP_2144", + condition = "_LEGACY_ERROR_TEMP_2144", parameters = Map("tpe" -> "org.apache.spark.sql.catalyst.TraitProductWithNoConstructorCompanion")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala index 8a71496607466..fc5d39fd9c2bb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala @@ -474,7 +474,7 @@ class ShuffleSpecSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkUnsupportedOperationException] { RangeShuffleSpec(10, distribution).createPartitioning(distribution.clustering) }, - errorClass = "UNSUPPORTED_CALL.WITHOUT_SUGGESTION", + condition = "UNSUPPORTED_CALL.WITHOUT_SUGGESTION", parameters = Map( "methodName" -> "createPartitioning$", "className" -> "org.apache.spark.sql.catalyst.plans.physical.ShuffleSpec")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index cfe08e1895363..4c66c52258cbd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -509,7 +509,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { val analyzer = getAnalyzer analyzer.checkAnalysis(analyzer.execute(UnresolvedTestPlan())) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Found the unresolved operator: 'UnresolvedTestPlan")) errorTest( @@ -949,7 +949,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { exception = intercept[SparkException] { SimpleAnalyzer.checkAnalysis(join) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> """ |Failure when resolving conflicting references in Join: @@ -966,7 +966,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { exception = intercept[SparkException] { SimpleAnalyzer.checkAnalysis(intersect) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> """ |Failure when resolving conflicting references in Intersect All: @@ -983,7 +983,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { exception = intercept[SparkException] { SimpleAnalyzer.checkAnalysis(except) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> """ |Failure when resolving conflicting references in Except All: @@ -1003,7 +1003,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { exception = intercept[SparkException] { SimpleAnalyzer.checkAnalysis(asOfJoin) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> """ |Failure when resolving conflicting references in AsOfJoin: @@ -1272,7 +1272,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { exception = intercept[SparkException] { SimpleAnalyzer.checkAnalysis(plan) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Hint not found: `some_random_hint_that_does_not_exist`")) // UnresolvedHint be removed by batch `Remove Unresolved Hints` diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 1816c620414c9..e55a08409b642 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -74,7 +74,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { None, CaseInsensitiveStringMap.empty()).analyze }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Logical plan should not have output of char/varchar type.*\n"), matchPVals = true) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index dc95198574fb4..f299c078befc0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -191,7 +191,7 @@ trait AnalysisTest extends PlanTest { } checkError( exception = e, - errorClass = expectedErrorClass, + condition = expectedErrorClass, parameters = expectedMessageParameters, queryContext = queryContext ) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 4367cbbd24a89..95e118a30771c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -53,7 +53,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer expr: Expression, messageParameters: Map[String, String]): Unit = { checkError( exception = analysisException(expr), - errorClass = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", parameters = messageParameters) } @@ -61,7 +61,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer expr: Expression, messageParameters: Map[String, String]): Unit = { checkError( exception = analysisException(expr), - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", parameters = messageParameters) } @@ -69,7 +69,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer expr: Expression, messageParameters: Map[String, String]): Unit = { checkError( exception = analysisException(expr), - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", parameters = messageParameters) } @@ -77,14 +77,14 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer expr: Expression, messageParameters: Map[String, String]): Unit = { checkError( exception = analysisException(expr), - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = messageParameters) } private def assertForWrongType(expr: Expression, messageParameters: Map[String, String]): Unit = { checkError( exception = analysisException(expr), - errorClass = "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + condition = "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", parameters = messageParameters) } @@ -93,7 +93,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(BitwiseNot($"stringField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"~stringField\"", "paramIndex" -> ordinalNumber(0), @@ -426,7 +426,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(Sum($"booleanField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"sum(booleanField)\"", "paramIndex" -> ordinalNumber(0), @@ -437,7 +437,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(Average($"booleanField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"avg(booleanField)\"", "paramIndex" -> ordinalNumber(0), @@ -469,7 +469,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(coalesce) }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> toSQLId(coalesce.prettyName), "expectedNum" -> "> 0", @@ -481,7 +481,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(murmur3Hash) }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> toSQLId(murmur3Hash.prettyName), "expectedNum" -> "> 0", @@ -493,7 +493,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(xxHash64) }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> toSQLId(xxHash64.prettyName), "expectedNum" -> "> 0", @@ -504,7 +504,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(Explode($"intField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"explode(intField)\"", "paramIndex" -> ordinalNumber(0), @@ -516,7 +516,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(PosExplode($"intField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"posexplode(intField)\"", "paramIndex" -> ordinalNumber(0), @@ -529,7 +529,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer test("check types for CreateNamedStruct") { checkError( exception = analysisException(CreateNamedStruct(Seq("a", "b", 2.0))), - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`named_struct`", "expectedNum" -> "2n (n > 0)", @@ -538,21 +538,21 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer ) checkError( exception = analysisException(CreateNamedStruct(Seq(1, "a", "b", 2.0))), - errorClass = "DATATYPE_MISMATCH.CREATE_NAMED_STRUCT_WITHOUT_FOLDABLE_STRING", + condition = "DATATYPE_MISMATCH.CREATE_NAMED_STRUCT_WITHOUT_FOLDABLE_STRING", parameters = Map( "sqlExpr" -> "\"named_struct(1, a, b, 2.0)\"", "inputExprs" -> "[\"1\"]") ) checkError( exception = analysisException(CreateNamedStruct(Seq($"a".string.at(0), "a", "b", 2.0))), - errorClass = "DATATYPE_MISMATCH.CREATE_NAMED_STRUCT_WITHOUT_FOLDABLE_STRING", + condition = "DATATYPE_MISMATCH.CREATE_NAMED_STRUCT_WITHOUT_FOLDABLE_STRING", parameters = Map( "sqlExpr" -> "\"named_struct(boundreference(), a, b, 2.0)\"", "inputExprs" -> "[\"boundreference()\"]") ) checkError( exception = analysisException(CreateNamedStruct(Seq(Literal.create(null, StringType), "a"))), - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL", + condition = "DATATYPE_MISMATCH.UNEXPECTED_NULL", parameters = Map( "sqlExpr" -> "\"named_struct(NULL, a)\"", "exprName" -> "[\"NULL\"]") @@ -562,7 +562,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer test("check types for CreateMap") { checkError( exception = analysisException(CreateMap(Seq("a", "b", 2.0))), - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`map`", "expectedNum" -> "2n (n > 0)", @@ -572,7 +572,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer checkError( exception = analysisException(CreateMap(Seq(Literal(1), Literal("a"), Literal(true), Literal("b")))), - errorClass = "DATATYPE_MISMATCH.CREATE_MAP_KEY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.CREATE_MAP_KEY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"map(1, a, true, b)\"", "functionName" -> "`map`", @@ -582,7 +582,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer checkError( exception = analysisException(CreateMap(Seq(Literal("a"), Literal(1), Literal("b"), Literal(true)))), - errorClass = "DATATYPE_MISMATCH.CREATE_MAP_VALUE_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.CREATE_MAP_VALUE_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"map(a, 1, b, true)\"", "functionName" -> "`map`", @@ -599,7 +599,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(Round($"intField", $"intField")) }, - errorClass = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + condition = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", parameters = Map( "sqlExpr" -> "\"round(intField, intField)\"", "inputName" -> "`scale`", @@ -610,7 +610,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(Round($"intField", $"booleanField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"round(intField, booleanField)\"", "paramIndex" -> ordinalNumber(1), @@ -621,7 +621,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(Round($"intField", $"mapField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"round(intField, mapField)\"", "paramIndex" -> ordinalNumber(1), @@ -632,7 +632,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(Round($"booleanField", $"intField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"round(booleanField, intField)\"", "paramIndex" -> ordinalNumber(0), @@ -646,7 +646,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(BRound($"intField", $"intField")) }, - errorClass = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + condition = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", parameters = Map( "sqlExpr" -> "\"bround(intField, intField)\"", "inputName" -> "`scale`", @@ -656,7 +656,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(BRound($"intField", $"booleanField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"bround(intField, booleanField)\"", "paramIndex" -> ordinalNumber(1), @@ -667,7 +667,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(BRound($"intField", $"mapField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"bround(intField, mapField)\"", "paramIndex" -> ordinalNumber(1), @@ -678,7 +678,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[AnalysisException] { assertSuccess(BRound($"booleanField", $"intField")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"bround(booleanField, intField)\"", "paramIndex" -> ordinalNumber(0), @@ -806,7 +806,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer exception = intercept[SparkException] { wsd.checkInputDataTypes() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> ("Cannot use an UnspecifiedFrame. " + "This should have been converted during analysis.")) ) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/LookupFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/LookupFunctionsSuite.scala index 1fd81349ac720..1ae3e3fa68603 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/LookupFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/LookupFunctionsSuite.scala @@ -62,7 +62,7 @@ class LookupFunctionsSuite extends PlanTest { } checkError( exception = cause, - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`undefined_fn`", "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`db1`]")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedParameterFunctionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedParameterFunctionSuite.scala index 6bc0350a5785d..5f1789083fa3b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedParameterFunctionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedParameterFunctionSuite.scala @@ -103,13 +103,13 @@ class NamedParameterFunctionSuite extends AnalysisTest { checkError( exception = parseRearrangeException( signature, Seq(k1Arg, k2Arg, k3Arg, k4Arg, namedK1Arg), "foo"), - errorClass = errorClass, + condition = errorClass, parameters = Map("routineName" -> toSQLId("foo"), "parameterName" -> toSQLId("k1")) ) checkError( exception = parseRearrangeException( signature, Seq(k1Arg, k2Arg, k3Arg, k4Arg, k4Arg), "foo"), - errorClass = "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE", + condition = "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE", parameters = Map("routineName" -> toSQLId("foo"), "parameterName" -> toSQLId("k4")) ) } @@ -117,7 +117,7 @@ class NamedParameterFunctionSuite extends AnalysisTest { test("REQUIRED_PARAMETER_NOT_FOUND") { checkError( exception = parseRearrangeException(signature, Seq(k1Arg, k2Arg, k3Arg), "foo"), - errorClass = "REQUIRED_PARAMETER_NOT_FOUND", + condition = "REQUIRED_PARAMETER_NOT_FOUND", parameters = Map( "routineName" -> toSQLId("foo"), "parameterName" -> toSQLId("k4"), "index" -> "2")) } @@ -126,7 +126,7 @@ class NamedParameterFunctionSuite extends AnalysisTest { checkError( exception = parseRearrangeException(signature, Seq(k1Arg, k2Arg, k3Arg, k4Arg, NamedArgumentExpression("k5", Literal("k5"))), "foo"), - errorClass = "UNRECOGNIZED_PARAMETER_NAME", + condition = "UNRECOGNIZED_PARAMETER_NAME", parameters = Map("routineName" -> toSQLId("foo"), "argumentName" -> toSQLId("k5"), "proposal" -> (toSQLId("k1") + " " + toSQLId("k2") + " " + toSQLId("k3"))) ) @@ -136,7 +136,7 @@ class NamedParameterFunctionSuite extends AnalysisTest { checkError( exception = parseRearrangeException(signature, Seq(k2Arg, k3Arg, k1Arg, k4Arg), "foo"), - errorClass = "UNEXPECTED_POSITIONAL_ARGUMENT", + condition = "UNEXPECTED_POSITIONAL_ARGUMENT", parameters = Map("routineName" -> toSQLId("foo"), "parameterName" -> toSQLId("k3")) ) } @@ -147,7 +147,7 @@ class NamedParameterFunctionSuite extends AnalysisTest { s" All required arguments should come before optional arguments." checkError( exception = parseRearrangeException(illegalSignature, args, "foo"), - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> errorMessage) ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala index 5809d1e04b9cf..6e911324e0759 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala @@ -82,7 +82,7 @@ class ResolveLambdaVariablesSuite extends PlanTest { checkError( exception = intercept[AnalysisException](Analyzer.execute(p)), - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.DUPLICATE_ARG_NAMES", + condition = "INVALID_LAMBDA_FUNCTION_CALL.DUPLICATE_ARG_NAMES", parameters = Map( "args" -> "`x`, `x`", "caseSensitiveConfig" -> "\"spark.sql.caseSensitive\"") @@ -96,7 +96,7 @@ class ResolveLambdaVariablesSuite extends PlanTest { checkError( exception = intercept[AnalysisException](Analyzer.execute(p)), - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "3", "actualNumArgs" -> "1") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index ff9c0a1b34f7f..a8f479d0526c3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -1052,7 +1052,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { exception = intercept[AnalysisException] { UnsupportedOperationChecker.checkForStreaming(wrapInStreaming(plan), outputMode) }, - errorClass = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", + condition = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", sqlState = "42KDE", parameters = Map( "outputMode" -> outputMode.toString.toLowerCase(Locale.ROOT), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 7e2bad484b3a6..fbe63f71ae029 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -121,7 +121,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { func(name) }, - errorClass = "INVALID_SCHEMA_OR_RELATION_NAME", + condition = "INVALID_SCHEMA_OR_RELATION_NAME", parameters = Map("name" -> toSQLId(name)) ) } @@ -171,7 +171,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { ResolveDefaultColumns.analyze(columnC, statementType) }, - errorClass = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`c`", @@ -180,7 +180,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { ResolveDefaultColumns.analyze(columnD, statementType) }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`d`", @@ -189,7 +189,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { ResolveDefaultColumns.analyze(columnE, statementType) }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`e`", @@ -589,7 +589,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { sessionCatalog.alterTableDataSchema( TableIdentifier("t1", Some("default")), StructType(oldTab.dataSchema.drop(1))) }, - errorClass = "_LEGACY_ERROR_TEMP_1071", + condition = "_LEGACY_ERROR_TEMP_1071", parameters = Map("nonExistentColumnNames" -> "[col1]")) } } @@ -817,14 +817,14 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[NoSuchTableException] { catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1")) }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`default`.`view1`") ) checkError( exception = intercept[NoSuchTableException] { catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1", Some("default"))) }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`default`.`view1`") ) @@ -838,7 +838,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[NoSuchTableException] { catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1", Some("default"))) }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`default`.`view1`") ) } @@ -1000,7 +1000,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { TableIdentifier("tbl2", Some("db2")), Seq(part1, partWithLessColumns), ignoreIfExists = false) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a", "partitionColumnNames" -> "a, b", @@ -1011,7 +1011,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { TableIdentifier("tbl2", Some("db2")), Seq(part1, partWithMoreColumns), ignoreIfExists = true) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a, b, c", "partitionColumnNames" -> "a, b", @@ -1022,7 +1022,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { TableIdentifier("tbl2", Some("db2")), Seq(partWithUnknownColumns, part1), ignoreIfExists = true) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a, unknown", "partitionColumnNames" -> "a, b", @@ -1033,7 +1033,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { TableIdentifier("tbl2", Some("db2")), Seq(partWithEmptyValue, part1), ignoreIfExists = true) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> "The spec ([a=3, b=]) contains an empty partition column value")) } @@ -1126,7 +1126,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { purge = false, retainData = false) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> ("The spec (a, b, c) must be contained within the partition " + s"spec (a, b) defined in table '`$SESSION_CATALOG_NAME`.`db2`.`tbl2`'"))) @@ -1139,7 +1139,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { purge = false, retainData = false) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> ("The spec (a, unknown) must be contained within the partition " + s"spec (a, b) defined in table '`$SESSION_CATALOG_NAME`.`db2`.`tbl2`'"))) @@ -1152,7 +1152,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { purge = false, retainData = false) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> "The spec ([a=3, b=]) contains an empty partition column value")) } @@ -1192,7 +1192,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { catalog.getPartition(TableIdentifier("tbl1", Some("db2")), partWithLessColumns.spec) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a", "partitionColumnNames" -> "a, b", @@ -1201,7 +1201,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { catalog.getPartition(TableIdentifier("tbl1", Some("db2")), partWithMoreColumns.spec) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a, b, c", "partitionColumnNames" -> "a, b", @@ -1210,7 +1210,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { catalog.getPartition(TableIdentifier("tbl1", Some("db2")), partWithUnknownColumns.spec) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a, unknown", "partitionColumnNames" -> "a, b", @@ -1219,7 +1219,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { catalog.getPartition(TableIdentifier("tbl1", Some("db2")), partWithEmptyValue.spec) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> "The spec ([a=3, b=]) contains an empty partition column value")) } @@ -1277,7 +1277,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { TableIdentifier("tbl1", Some("db2")), Seq(part1.spec), Seq(partWithLessColumns.spec)) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a", "partitionColumnNames" -> "a, b", @@ -1288,7 +1288,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { TableIdentifier("tbl1", Some("db2")), Seq(part1.spec), Seq(partWithMoreColumns.spec)) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a, b, c", "partitionColumnNames" -> "a, b", @@ -1299,7 +1299,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { TableIdentifier("tbl1", Some("db2")), Seq(part1.spec), Seq(partWithUnknownColumns.spec)) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a, unknown", "partitionColumnNames" -> "a, b", @@ -1310,7 +1310,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { TableIdentifier("tbl1", Some("db2")), Seq(part1.spec), Seq(partWithEmptyValue.spec)) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> "The spec ([a=3, b=]) contains an empty partition column value")) } @@ -1364,7 +1364,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { catalog.alterPartitions(TableIdentifier("tbl1", Some("db2")), Seq(partWithLessColumns)) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a", "partitionColumnNames" -> "a, b", @@ -1373,7 +1373,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { catalog.alterPartitions(TableIdentifier("tbl1", Some("db2")), Seq(partWithMoreColumns)) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a, b, c", "partitionColumnNames" -> "a, b", @@ -1382,7 +1382,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { catalog.alterPartitions(TableIdentifier("tbl1", Some("db2")), Seq(partWithUnknownColumns)) }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "a, unknown", "partitionColumnNames" -> "a, b", @@ -1391,7 +1391,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[AnalysisException] { catalog.alterPartitions(TableIdentifier("tbl1", Some("db2")), Seq(partWithEmptyValue)) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> "The spec ([a=3, b=]) contains an empty partition column value")) } @@ -1423,7 +1423,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")), Some(partWithMoreColumns.spec)) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> ("The spec (a, b, c) must be contained within the partition spec (a, b) " + s"defined in table '`$SESSION_CATALOG_NAME`.`db2`.`tbl2`'"))) @@ -1432,7 +1432,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")), Some(partWithUnknownColumns.spec)) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> ("The spec (a, unknown) must be contained within the partition " + s"spec (a, b) defined in table '`$SESSION_CATALOG_NAME`.`db2`.`tbl2`'"))) @@ -1441,7 +1441,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")), Some(partWithEmptyValue.spec)) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> "The spec ([a=3, b=]) contains an empty partition column value")) } @@ -1471,7 +1471,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.listPartitions(TableIdentifier("tbl2", Some("db2")), Some(partWithMoreColumns.spec)) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> ("The spec (a, b, c) must be contained within the partition spec (a, b) " + s"defined in table '`$SESSION_CATALOG_NAME`.`db2`.`tbl2`'"))) @@ -1480,7 +1480,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.listPartitions(TableIdentifier("tbl2", Some("db2")), Some(partWithUnknownColumns.spec)) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> ("The spec (a, unknown) must be contained within the partition " + s"spec (a, b) defined in table '`$SESSION_CATALOG_NAME`.`db2`.`tbl2`'"))) @@ -1489,7 +1489,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.listPartitions(TableIdentifier("tbl2", Some("db2")), Some(partWithEmptyValue.spec)) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> "The spec ([a=3, b=]) contains an empty partition column value")) } @@ -1582,7 +1582,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { newFunc("temp1", None), overrideIfExists = false, functionBuilder = Some(tempFunc3)) } checkError(e, - errorClass = "ROUTINE_ALREADY_EXISTS", + condition = "ROUTINE_ALREADY_EXISTS", parameters = Map("routineName" -> "`temp1`", "newRoutineType" -> "routine", "existingRoutineType" -> "routine")) @@ -1601,7 +1601,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { overrideIfExists = false, None) }, - errorClass = "CANNOT_LOAD_FUNCTION_CLASS", + condition = "CANNOT_LOAD_FUNCTION_CLASS", parameters = Map( "className" -> "function_class_cannot_load", "functionName" -> "`temp2`" @@ -1712,14 +1712,14 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[NoSuchFunctionException] { catalog.lookupFunction(FunctionIdentifier("func1"), arguments) }, - errorClass = "ROUTINE_NOT_FOUND", + condition = "ROUTINE_NOT_FOUND", parameters = Map("routineName" -> "`default`.`func1`") ) checkError( exception = intercept[NoSuchTempFunctionException] { catalog.dropTempFunction("func1", ignoreIfNotExists = false) }, - errorClass = "ROUTINE_NOT_FOUND", + condition = "ROUTINE_NOT_FOUND", parameters = Map("routineName" -> "`func1`") ) catalog.dropTempFunction("func1", ignoreIfNotExists = true) @@ -1728,7 +1728,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { exception = intercept[NoSuchTempFunctionException] { catalog.dropTempFunction("func2", ignoreIfNotExists = false) }, - errorClass = "ROUTINE_NOT_FOUND", + condition = "ROUTINE_NOT_FOUND", parameters = Map("routineName" -> "`func2`") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala index 7d9015e566a8c..e8239c7523948 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala @@ -38,7 +38,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException]{ CSVExprUtils.toDelimiterStr(null) }, - errorClass = "INVALID_DELIMITER_VALUE.NULL_VALUE", + condition = "INVALID_DELIMITER_VALUE.NULL_VALUE", parameters = Map.empty) } @@ -47,7 +47,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException]{ CSVExprUtils.toChar("ab") }, - errorClass = "INVALID_DELIMITER_VALUE.DELIMITER_LONGER_THAN_EXPECTED", + condition = "INVALID_DELIMITER_VALUE.DELIMITER_LONGER_THAN_EXPECTED", parameters = Map("str" -> "ab")) } @@ -56,7 +56,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException]{ CSVExprUtils.toChar("""\1""") }, - errorClass = "INVALID_DELIMITER_VALUE.UNSUPPORTED_SPECIAL_CHARACTER", + condition = "INVALID_DELIMITER_VALUE.UNSUPPORTED_SPECIAL_CHARACTER", parameters = Map("str" -> """\1""")) } @@ -65,7 +65,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException]{ CSVExprUtils.toChar("""\""") }, - errorClass = "INVALID_DELIMITER_VALUE.SINGLE_BACKSLASH", + condition = "INVALID_DELIMITER_VALUE.SINGLE_BACKSLASH", parameters = Map.empty) } @@ -74,7 +74,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException]{ CSVExprUtils.toChar("") }, - errorClass = "INVALID_DELIMITER_VALUE.EMPTY_STRING", + condition = "INVALID_DELIMITER_VALUE.EMPTY_STRING", parameters = Map.empty) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index cbc98d2f23020..514b529ea8cc0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -308,7 +308,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { check(filters = Seq(EqualTo("invalid attr", 1)), expected = None) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`invalid attr`", "fields" -> "`i`")) checkError( @@ -319,7 +319,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { filters = Seq(EqualTo("i", 1)), expected = Some(InternalRow.empty)) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`i`", "fields" -> "")) } @@ -374,7 +374,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { check(new UnivocityParser(StructType(Seq.empty), optionsWithPattern(false))) }, - errorClass = "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", + condition = "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", parameters = Map( "c" -> "n", "pattern" -> "invalid")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala index e852b474aa18c..b7309923ac206 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderErrorMessageSuite.scala @@ -55,7 +55,7 @@ class EncoderErrorMessageSuite extends SparkFunSuite { checkError( exception = intercept[ SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable1]()), - errorClass = "ENCODER_NOT_FOUND", + condition = "ENCODER_NOT_FOUND", parameters = Map( "typeName" -> "org.apache.spark.sql.catalyst.encoders.NonEncodable", "docroot" -> SPARK_DOC_ROOT) @@ -64,7 +64,7 @@ class EncoderErrorMessageSuite extends SparkFunSuite { checkError( exception = intercept[ SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable2]()), - errorClass = "ENCODER_NOT_FOUND", + condition = "ENCODER_NOT_FOUND", parameters = Map( "typeName" -> "org.apache.spark.sql.catalyst.encoders.NonEncodable", "docroot" -> SPARK_DOC_ROOT) @@ -73,7 +73,7 @@ class EncoderErrorMessageSuite extends SparkFunSuite { checkError( exception = intercept[ SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable3]()), - errorClass = "ENCODER_NOT_FOUND", + condition = "ENCODER_NOT_FOUND", parameters = Map( "typeName" -> "org.apache.spark.sql.catalyst.encoders.NonEncodable", "docroot" -> SPARK_DOC_ROOT) @@ -82,7 +82,7 @@ class EncoderErrorMessageSuite extends SparkFunSuite { checkError( exception = intercept[ SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable4]()), - errorClass = "ENCODER_NOT_FOUND", + condition = "ENCODER_NOT_FOUND", parameters = Map( "typeName" -> "org.apache.spark.sql.catalyst.encoders.NonEncodable", "docroot" -> SPARK_DOC_ROOT) @@ -91,7 +91,7 @@ class EncoderErrorMessageSuite extends SparkFunSuite { checkError( exception = intercept[ SparkUnsupportedOperationException](ExpressionEncoder[ComplexNonEncodable5]()), - errorClass = "ENCODER_NOT_FOUND", + condition = "ENCODER_NOT_FOUND", parameters = Map( "typeName" -> "org.apache.spark.sql.catalyst.encoders.NonEncodable", "docroot" -> SPARK_DOC_ROOT) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala index e29609c741633..28796db7c02e0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala @@ -90,7 +90,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq($"arr".array(StringType)) checkError( exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), - errorClass = "CANNOT_UP_CAST_DATATYPE", + condition = "CANNOT_UP_CAST_DATATYPE", parameters = Map("expression" -> "array element", "sourceType" -> "\"STRING\"", "targetType" -> "\"BIGINT\"", "details" -> ( @@ -125,7 +125,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq($"arr".int) checkError( exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), - errorClass = "UNSUPPORTED_DESERIALIZER.DATA_TYPE_MISMATCH", + condition = "UNSUPPORTED_DESERIALIZER.DATA_TYPE_MISMATCH", parameters = Map("desiredType" -> "\"ARRAY\"", "dataType" -> "\"INT\"")) } @@ -134,7 +134,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq($"arr".array(new StructType().add("c", "int"))) checkError( exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`a`", "fields" -> "`c`")) } @@ -145,7 +145,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq($"nestedArr".array(new StructType().add("arr", "int"))) checkError( exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), - errorClass = "UNSUPPORTED_DESERIALIZER.DATA_TYPE_MISMATCH", + condition = "UNSUPPORTED_DESERIALIZER.DATA_TYPE_MISMATCH", parameters = Map("desiredType" -> "\"ARRAY\"", "dataType" -> "\"INT\"")) } @@ -154,7 +154,7 @@ class EncoderResolutionSuite extends PlanTest { .add("arr", ArrayType(new StructType().add("c", "int"))))) checkError( exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`a`", "fields" -> "`c`")) } } @@ -183,7 +183,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq($"a".string, $"b".long, $"c".int) checkError( exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), - errorClass = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", + condition = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", parameters = Map("schema" -> "\"STRUCT\"", "ordinal" -> "2")) } @@ -192,7 +192,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq($"a".string) checkError( exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), - errorClass = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", + condition = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", parameters = Map("schema" -> "\"STRUCT\"", "ordinal" -> "2")) } @@ -205,7 +205,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq($"a".string, $"b".struct($"x".long, $"y".string, $"z".int)) checkError( exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), - errorClass = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", + condition = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", parameters = Map("schema" -> "\"STRUCT\"", "ordinal" -> "2")) } @@ -214,7 +214,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq($"a".string, $"b".struct($"x".long)) checkError( exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), - errorClass = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", + condition = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", parameters = Map("schema" -> "\"STRUCT\"", "ordinal" -> "2")) } @@ -233,7 +233,7 @@ class EncoderResolutionSuite extends PlanTest { .foreach { attr => val attrs = Seq(attr) checkError(exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), - errorClass = "CANNOT_UP_CAST_DATATYPE", + condition = "CANNOT_UP_CAST_DATATYPE", parameters = Map("expression" -> "a", "sourceType" -> ("\"" + attr.dataType.sql + "\""), "targetType" -> "\"STRING\"", "details" -> ( @@ -250,7 +250,7 @@ class EncoderResolutionSuite extends PlanTest { ExpressionEncoder[StringIntClass]().resolveAndBind(Seq($"a".string, $"b".long)) } checkError(exception = e1, - errorClass = "CANNOT_UP_CAST_DATATYPE", + condition = "CANNOT_UP_CAST_DATATYPE", parameters = Map("expression" -> "b", "sourceType" -> ("\"BIGINT\""), "targetType" -> "\"INT\"", "details" -> ( @@ -267,7 +267,7 @@ class EncoderResolutionSuite extends PlanTest { } checkError(exception = e2, - errorClass = "CANNOT_UP_CAST_DATATYPE", + condition = "CANNOT_UP_CAST_DATATYPE", parameters = Map("expression" -> "b.`b`", "sourceType" -> ("\"DECIMAL(38,18)\""), "targetType" -> "\"BIGINT\"", "details" -> ( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index c67c0912980f5..879b4ef6d3745 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -435,7 +435,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes implicitly[ExpressionEncoder[Foo]]) checkError( exception = exception, - errorClass = "ENCODER_NOT_FOUND", + condition = "ENCODER_NOT_FOUND", parameters = Map( "typeName" -> "Any", "docroot" -> SPARK_DOC_ROOT) @@ -496,7 +496,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes assert(e.getCause.isInstanceOf[SparkRuntimeException]) checkError( exception = e.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "NULL_MAP_KEY", + condition = "NULL_MAP_KEY", parameters = Map.empty ) } @@ -507,7 +507,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes assert(e.getCause.isInstanceOf[SparkRuntimeException]) checkError( exception = e.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "NULL_MAP_KEY", + condition = "NULL_MAP_KEY", parameters = Map.empty ) } @@ -519,7 +519,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes exception = intercept[SparkUnsupportedOperationException] { ExpressionEncoder.tuple(encoders) }, - errorClass = "_LEGACY_ERROR_TEMP_2150", + condition = "_LEGACY_ERROR_TEMP_2150", parameters = Map.empty) } @@ -535,7 +535,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes } checkError( exception = exception, - errorClass = "UNEXPECTED_SERIALIZER_FOR_CLASS", + condition = "UNEXPECTED_SERIALIZER_FOR_CLASS", parameters = Map( "className" -> Utils.getSimpleName(encoder.clsTag.runtimeClass), "expr" -> toSQLExpr(unexpectedSerializer)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeResolutionSuite.scala index 71fa60b0c0345..29c5bf3b8d2db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeResolutionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeResolutionSuite.scala @@ -69,7 +69,7 @@ class AttributeResolutionSuite extends SparkFunSuite { exception = intercept[AnalysisException] { attrs.resolve(Seq("a"), resolver) }, - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map( "name" -> "`a`", "referenceNames" -> "[`ns1`.`ns2`.`t2`.`a`, `ns1`.`t1`.`a`]" @@ -86,7 +86,7 @@ class AttributeResolutionSuite extends SparkFunSuite { exception = intercept[AnalysisException] { attrs.resolve(Seq("ns1", "t", "a"), resolver) }, - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map( "name" -> "`ns1`.`t`.`a`", "referenceNames" -> "[`ns1`.`t`.`a`, `ns2`.`ns1`.`t`.`a`]" @@ -108,7 +108,7 @@ class AttributeResolutionSuite extends SparkFunSuite { exception = intercept[AnalysisException] { attrs.resolve(Seq("ns1", "t", "a", "cc"), resolver) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`cc`", "fields" -> "`aa`, `bb`")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala index 77fdb33e515fc..995b519bd05d7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala @@ -103,7 +103,7 @@ class CallMethodViaReflectionSuite extends SparkFunSuite with ExpressionEvalHelp exception = intercept[AnalysisException] { CallMethodViaReflection(Seq.empty).checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`reflect`", "expectedNum" -> "> 1", @@ -114,7 +114,7 @@ class CallMethodViaReflectionSuite extends SparkFunSuite with ExpressionEvalHelp exception = intercept[AnalysisException] { CallMethodViaReflection(Seq(Literal(staticClassName))).checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`reflect`", "expectedNum" -> "> 1", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala index 175dd05d5911e..e34b54c7086cd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala @@ -61,7 +61,7 @@ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { test("collate on non existing collation") { checkError( exception = intercept[SparkException] { Collate(Literal("abc"), "UTF8_BS") }, - errorClass = "COLLATION_INVALID_NAME", + condition = "COLLATION_INVALID_NAME", sqlState = "42704", parameters = Map("collationName" -> "UTF8_BS", "proposals" -> "UTF8_LCASE")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index c7e995feb5ed8..40490a423b781 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -95,7 +95,7 @@ class CollectionExpressionsSuite } checkError( exception = exception, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map( "message" -> ("The size function doesn't support the operand type " + toSQLType(StringType)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index 497b335289b11..b92b7138c153f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -328,7 +328,7 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[AnalysisException] { map3.checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`map`", "expectedNum" -> "2n (n > 0)", @@ -483,7 +483,7 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[AnalysisException] { namedStruct1.checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`named_struct`", "expectedNum" -> "2n (n > 0)", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index f1c04c7e33821..1e10da6f93469 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -2103,11 +2103,11 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("datetime function CurrentDate and localtimestamp are Unevaluable") { checkError(exception = intercept[SparkException] { CurrentDate(UTC_OPT).eval(EmptyRow) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Cannot evaluate expression: current_date(Some(UTC))")) checkError(exception = intercept[SparkException] { LocalTimestamp(UTC_OPT).eval(EmptyRow) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Cannot evaluate expression: localtimestamp(Some(UTC))")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index a063e53486ad8..87d1208257070 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -179,7 +179,7 @@ trait ExpressionEvalHelper extends ScalaCheckDrivenPropertyChecks with PlanTestB withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> fallbackMode.toString) { checkError( exception = intercept[T](eval), - errorClass = errorClass, + condition = errorClass, parameters = parameters ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala index dd512b0d83e5c..b6a3d61cb13a6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala @@ -82,7 +82,7 @@ class GeneratorExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[AnalysisException] { Stack(Seq(Literal(1))).checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`stack`", "expectedNum" -> "> 1", @@ -93,7 +93,7 @@ class GeneratorExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[AnalysisException] { Stack(Seq(Literal(1.0))).checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`stack`", "expectedNum" -> "> 1", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index a23e7f44a48d1..adb39fcd568c9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -448,7 +448,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with }.getCause checkError( exception = exception.asInstanceOf[SparkException], - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map("badRecord" -> "[null]", "failFastMode" -> "FAILFAST") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index d42e0b7d681db..b351d69d3a0bb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -97,7 +97,7 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[SparkException] { Literal.default(errType) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> s"No default value for type: ${toSQLType(errType)}.") ) }) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala index ace017b1cddc3..c74a9e35833d1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala @@ -57,7 +57,7 @@ class NullExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[SparkRuntimeException] { evaluateWithoutCodegen(AssertNotNull(Literal(null))) }, - errorClass = "NOT_NULL_ASSERT_VIOLATION", + condition = "NOT_NULL_ASSERT_VIOLATION", sqlState = "42000", parameters = Map("walkedTypePath" -> "\n\n")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index af38fc1f12f7f..d37a6928189ff 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -82,7 +82,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[SparkException] { Invoke(inputObject, "zeroArgNotExistMethod", IntegerType).eval(inputRow) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> ("Couldn't find method zeroArgNotExistMethod with arguments " + "() on class java.lang.Object.") @@ -98,7 +98,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { Seq(Literal.fromObject(UTF8String.fromString("dummyInputString"))), Seq(StringType)).eval(inputRow) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> ("Couldn't find method oneArgNotExistMethod with arguments " + "(class org.apache.spark.unsafe.types.UTF8String) on class java.lang.Object.") @@ -417,7 +417,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[SparkRuntimeException] { testMapObjects(collection, classOf[scala.collection.Map[Int, Int]], inputType) }, - errorClass = "CLASS_UNSUPPORTED_BY_MAP_OBJECTS", + condition = "CLASS_UNSUPPORTED_BY_MAP_OBJECTS", parameters = Map("cls" -> "scala.collection.Map")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index 690db18bbfa69..12aeb7d6685bf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -158,13 +158,13 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[AnalysisException] { evaluateWithoutCodegen("""a""" like """\a""") }, - errorClass = "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + condition = "INVALID_FORMAT.ESC_IN_THE_MIDDLE", parameters = Map("format" -> """'\\a'""", "char" -> "'a'")) checkError( exception = intercept[AnalysisException] { evaluateWithoutCodegen("""a""" like """a\""") }, - errorClass = "INVALID_FORMAT.ESC_AT_THE_END", + condition = "INVALID_FORMAT.ESC_AT_THE_END", parameters = Map("format" -> """'a\\'""")) // case @@ -238,7 +238,7 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[AnalysisException] { evaluateWithoutCodegen("""a""" like(s"""${escapeChar}a""", escapeChar)) }, - errorClass = "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + condition = "INVALID_FORMAT.ESC_IN_THE_MIDDLE", parameters = Map("format" -> s"'${escapeChar}a'", "char" -> "'a'")) // case @@ -283,7 +283,7 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[SparkRuntimeException] { evaluateWithoutCodegen("abbbbc" rlike "**") }, - errorClass = "INVALID_PARAMETER_VALUE.PATTERN", + condition = "INVALID_PARAMETER_VALUE.PATTERN", parameters = Map( "parameter" -> toSQLId("regexp"), "functionName" -> toSQLId("rlike"), @@ -294,7 +294,7 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val regex = $"a".string.at(0) evaluateWithoutCodegen("abbbbc" rlike regex, create_row("**")) }, - errorClass = "INVALID_PARAMETER_VALUE.PATTERN", + condition = "INVALID_PARAMETER_VALUE.PATTERN", parameters = Map( "parameter" -> toSQLId("regexp"), "functionName" -> toSQLId("rlike"), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index b39820f0d317d..69988ca86f17d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -155,7 +155,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[AnalysisException] { Elt(Seq.empty).checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`elt`", "expectedNum" -> "> 1", @@ -166,7 +166,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[AnalysisException] { Elt(Seq(Literal(1))).checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`elt`", "expectedNum" -> "> 1", @@ -1910,7 +1910,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[AnalysisException] { ParseUrl(Seq(Literal("1"))).checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`parse_url`", "expectedNum" -> "[2, 3]", @@ -1922,7 +1922,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { ParseUrl(Seq(Literal("1"), Literal("2"), Literal("3"), Literal("4"))).checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`parse_url`", "expectedNum" -> "[2, 3]", @@ -2037,7 +2037,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { exception = intercept[AnalysisException] { expr1.checkInputDataTypes() }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`elt`", "expectedNum" -> "> 1", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/XmlExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/XmlExpressionsSuite.scala index 9a10985153044..66baf6b1430fa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/XmlExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/XmlExpressionsSuite.scala @@ -64,7 +64,7 @@ class XmlExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P }.getCause checkError( exception = exception.asInstanceOf[SparkException], - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map("badRecord" -> "[null]", "failFastMode" -> "FAILFAST") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala index 389c757eefb63..f1d9fd6a36584 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala @@ -410,7 +410,7 @@ class PercentileSuite extends SparkFunSuite { agg.update(buffer, InternalRow(1, -5)) agg.eval(buffer) }, - errorClass = "_LEGACY_ERROR_TEMP_2013", + condition = "_LEGACY_ERROR_TEMP_2013", parameters = Map("frequencyExpression" -> "CAST(boundreference() AS INT)")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSuite.scala index 79f03f23eb245..ca2eaf7be0c21 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSuite.scala @@ -27,7 +27,7 @@ class BufferHolderSuite extends SparkFunSuite { exception = intercept[SparkUnsupportedOperationException] { new BufferHolder(new UnsafeRow(Int.MaxValue / 8)) }, - errorClass = "_LEGACY_ERROR_TEMP_3130", + condition = "_LEGACY_ERROR_TEMP_3130", parameters = Map("numFields" -> "268435455")) val holder = new BufferHolder(new UnsafeRow(1000)) @@ -38,7 +38,7 @@ class BufferHolderSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { holder.grow(Integer.MAX_VALUE) }, - errorClass = "_LEGACY_ERROR_TEMP_3199", + condition = "_LEGACY_ERROR_TEMP_3199", parameters = Map("neededSize" -> "2147483647", "arrayMax" -> "2147483632") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeBlockSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeBlockSuite.scala index d51647ee96df9..4f81ef49e5736 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeBlockSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeBlockSuite.scala @@ -121,7 +121,7 @@ class CodeBlockSuite extends SparkFunSuite { exception = intercept[SparkException] { code"$obj" }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> s"Can not interpolate ${obj.getClass.getName} into code block.") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala index a968b1fe53506..8a8f0afeb1224 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala @@ -30,7 +30,7 @@ class UnsafeArrayWriterSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { arrayWriter.initialize(numElements) }, - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.INITIALIZE", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.INITIALIZE", parameters = Map( "numberOfElements" -> (numElements * elementSize).toString, "maxRoundedArrayLength" -> Int.MaxValue.toString diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionEvalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionEvalUtilsSuite.scala index 2b8c64a1af679..515dc98061a13 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionEvalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionEvalUtilsSuite.scala @@ -124,7 +124,7 @@ class VariantExpressionEvalUtilsSuite extends SparkFunSuite { VariantExpressionEvalUtils.parseJson(UTF8String.fromString(json), allowDuplicateKeys = false) }, - errorClass = errorClass, + condition = errorClass, parameters = parameters ) assert(try_parse_json_output === null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala index 854a3e8f7a74d..776600bbdcf5d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala @@ -189,7 +189,7 @@ class ExtractPythonUDFFromJoinConditionSuite extends PlanTest { } checkError( exception = e, - errorClass = "UNSUPPORTED_FEATURE.PYTHON_UDF_IN_ON_CLAUSE", + condition = "UNSUPPORTED_FEATURE.PYTHON_UDF_IN_ON_CLAUSE", parameters = Map("joinType" -> joinType.sql) ) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala index bf9f922978f6d..677a5d7928fc1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala @@ -58,7 +58,7 @@ class ReassignLambdaVariableIDSuite extends PlanTest { val query = testRelation.where(var1 && var2) checkError( exception = intercept[SparkException](Optimize.execute(query)), - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map( "message" -> "LambdaVariable IDs in a query should be all positive or negative.")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala index a50842a26b2ce..eaa651e62455e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala @@ -59,7 +59,7 @@ class ReplaceNullWithFalseInPredicateSuite extends PlanTest { exception = intercept[AnalysisException] { testFilter(originalCond = Literal(null, IntegerType), expectedCond = FalseLiteral) }, - errorClass = "DATATYPE_MISMATCH.FILTER_NOT_BOOLEAN", + condition = "DATATYPE_MISMATCH.FILTER_NOT_BOOLEAN", parameters = Map("sqlExpr" -> "\"NULL\"", "filter" -> "\"NULL\"", "type" -> "\"INT\"") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index d514f777e5544..0f2bb791f3465 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -79,7 +79,7 @@ class DDLParserSuite extends AnalysisTest { val sql = "CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet" checkError( exception = parseException(sql), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "':'", "hint" -> "")) } @@ -380,7 +380,7 @@ class DDLParserSuite extends AnalysisTest { |Columns: p2 string""".stripMargin checkError( exception = parseException(createSql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> value1), context = ExpectedContext( fragment = createSql, @@ -390,7 +390,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") checkError( exception = parseException(replaceSql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> value1), context = ExpectedContext( fragment = replaceSql, @@ -405,7 +405,7 @@ class DDLParserSuite extends AnalysisTest { |Columns: p2 string""".stripMargin checkError( exception = parseException(createSqlWithExpr), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> value2), context = ExpectedContext( fragment = createSqlWithExpr, @@ -415,7 +415,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSqlWithExpr = createSqlWithExpr.replaceFirst("CREATE", "REPLACE") checkError( exception = parseException(replaceSqlWithExpr), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> value2), context = ExpectedContext( fragment = replaceSqlWithExpr, @@ -482,7 +482,7 @@ class DDLParserSuite extends AnalysisTest { "which also specifies a serde" checkError( exception = parseException(createSql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> value), context = ExpectedContext( fragment = createSql, @@ -492,7 +492,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") checkError( exception = parseException(replaceSql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> value), context = ExpectedContext( fragment = replaceSql, @@ -538,7 +538,7 @@ class DDLParserSuite extends AnalysisTest { val value = "ROW FORMAT DELIMITED is only compatible with 'textfile', not 'otherformat'" checkError( exception = parseException(createFailSql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> value), context = ExpectedContext( fragment = createFailSql, @@ -548,7 +548,7 @@ class DDLParserSuite extends AnalysisTest { val replaceFailSql = createFailSql.replaceFirst("CREATE", "REPLACE") checkError( exception = parseException(replaceFailSql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> value), context = ExpectedContext( fragment = replaceFailSql, @@ -610,7 +610,7 @@ class DDLParserSuite extends AnalysisTest { |STORED AS parquet""".stripMargin checkError( exception = parseException(createSql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "CREATE TABLE ... USING ... STORED AS PARQUET "), context = ExpectedContext( fragment = createSql, @@ -620,7 +620,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") checkError( exception = parseException(replaceSql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "REPLACE TABLE ... USING ... STORED AS PARQUET "), context = ExpectedContext( fragment = replaceSql, @@ -635,7 +635,7 @@ class DDLParserSuite extends AnalysisTest { |ROW FORMAT SERDE 'customSerde'""".stripMargin checkError( exception = parseException(createSql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "CREATE TABLE ... USING ... ROW FORMAT SERDE CUSTOMSERDE"), context = ExpectedContext( fragment = createSql, @@ -645,7 +645,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") checkError( exception = parseException(replaceSql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "REPLACE TABLE ... USING ... ROW FORMAT SERDE CUSTOMSERDE"), context = ExpectedContext( fragment = replaceSql, @@ -660,7 +660,7 @@ class DDLParserSuite extends AnalysisTest { |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""".stripMargin checkError( exception = parseException(createSql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "CREATE TABLE ... USING ... ROW FORMAT DELIMITED"), context = ExpectedContext( fragment = createSql, @@ -670,7 +670,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") checkError( exception = parseException(replaceSql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "REPLACE TABLE ... USING ... ROW FORMAT DELIMITED"), context = ExpectedContext( fragment = replaceSql, @@ -685,7 +685,7 @@ class DDLParserSuite extends AnalysisTest { val fragment = "STORED BY 'handler'" checkError( exception = parseException(createSql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "STORED BY"), context = ExpectedContext( fragment = fragment, @@ -695,7 +695,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") checkError( exception = parseException(replaceSql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "STORED BY"), context = ExpectedContext( fragment = fragment, @@ -707,7 +707,7 @@ class DDLParserSuite extends AnalysisTest { val sql1 = "CREATE TABLE my_tab (id bigint) SKEWED BY (id) ON (1,2,3)" checkError( exception = parseException(sql1), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "CREATE TABLE ... SKEWED BY"), context = ExpectedContext( fragment = sql1, @@ -717,7 +717,7 @@ class DDLParserSuite extends AnalysisTest { val sql2 = "REPLACE TABLE my_tab (id bigint) SKEWED BY (id) ON (1,2,3)" checkError( exception = parseException(sql2), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "CREATE TABLE ... SKEWED BY"), context = ExpectedContext( fragment = sql2, @@ -737,7 +737,7 @@ class DDLParserSuite extends AnalysisTest { val sql1 = createTableHeader("TBLPROPERTIES('test' = 'test2')") checkError( exception = parseException(sql1), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "TBLPROPERTIES"), context = ExpectedContext( fragment = sql1, @@ -747,7 +747,7 @@ class DDLParserSuite extends AnalysisTest { val sql2 = createTableHeader("LOCATION '/tmp/file'") checkError( exception = parseException(sql2), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "LOCATION"), context = ExpectedContext( fragment = sql2, @@ -757,7 +757,7 @@ class DDLParserSuite extends AnalysisTest { val sql3 = createTableHeader("COMMENT 'a table'") checkError( exception = parseException(sql3), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "COMMENT"), context = ExpectedContext( fragment = sql3, @@ -767,7 +767,7 @@ class DDLParserSuite extends AnalysisTest { val sql4 = createTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS") checkError( exception = parseException(sql4), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "CLUSTERED BY"), context = ExpectedContext( fragment = sql4, @@ -777,7 +777,7 @@ class DDLParserSuite extends AnalysisTest { val sql5 = createTableHeader("PARTITIONED BY (b)") checkError( exception = parseException(sql5), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "PARTITIONED BY"), context = ExpectedContext( fragment = sql5, @@ -787,7 +787,7 @@ class DDLParserSuite extends AnalysisTest { val sql6 = createTableHeader("PARTITIONED BY (c int)") checkError( exception = parseException(sql6), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "PARTITIONED BY"), context = ExpectedContext( fragment = sql6, @@ -797,7 +797,7 @@ class DDLParserSuite extends AnalysisTest { val sql7 = createTableHeader("STORED AS parquet") checkError( exception = parseException(sql7), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "STORED AS/BY"), context = ExpectedContext( fragment = sql7, @@ -807,7 +807,7 @@ class DDLParserSuite extends AnalysisTest { val sql8 = createTableHeader("STORED AS INPUTFORMAT 'in' OUTPUTFORMAT 'out'") checkError( exception = parseException(sql8), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "STORED AS/BY"), context = ExpectedContext( fragment = sql8, @@ -817,7 +817,7 @@ class DDLParserSuite extends AnalysisTest { val sql9 = createTableHeader("ROW FORMAT SERDE 'serde'") checkError( exception = parseException(sql9), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "ROW FORMAT"), context = ExpectedContext( fragment = sql9, @@ -827,7 +827,7 @@ class DDLParserSuite extends AnalysisTest { val sql10 = replaceTableHeader("TBLPROPERTIES('test' = 'test2')") checkError( exception = parseException(sql10), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "TBLPROPERTIES"), context = ExpectedContext( fragment = sql10, @@ -837,7 +837,7 @@ class DDLParserSuite extends AnalysisTest { val sql11 = replaceTableHeader("LOCATION '/tmp/file'") checkError( exception = parseException(sql11), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "LOCATION"), context = ExpectedContext( fragment = sql11, @@ -847,7 +847,7 @@ class DDLParserSuite extends AnalysisTest { val sql12 = replaceTableHeader("COMMENT 'a table'") checkError( exception = parseException(sql12), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "COMMENT"), context = ExpectedContext( fragment = sql12, @@ -857,7 +857,7 @@ class DDLParserSuite extends AnalysisTest { val sql13 = replaceTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS") checkError( exception = parseException(sql13), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "CLUSTERED BY"), context = ExpectedContext( fragment = sql13, @@ -867,7 +867,7 @@ class DDLParserSuite extends AnalysisTest { val sql14 = replaceTableHeader("PARTITIONED BY (b)") checkError( exception = parseException(sql14), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "PARTITIONED BY"), context = ExpectedContext( fragment = sql14, @@ -877,7 +877,7 @@ class DDLParserSuite extends AnalysisTest { val sql15 = replaceTableHeader("PARTITIONED BY (c int)") checkError( exception = parseException(sql15), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "PARTITIONED BY"), context = ExpectedContext( fragment = sql15, @@ -887,7 +887,7 @@ class DDLParserSuite extends AnalysisTest { val sql16 = replaceTableHeader("STORED AS parquet") checkError( exception = parseException(sql16), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "STORED AS/BY"), context = ExpectedContext( fragment = sql16, @@ -897,7 +897,7 @@ class DDLParserSuite extends AnalysisTest { val sql17 = replaceTableHeader("STORED AS INPUTFORMAT 'in' OUTPUTFORMAT 'out'") checkError( exception = parseException(sql17), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "STORED AS/BY"), context = ExpectedContext( fragment = sql17, @@ -907,7 +907,7 @@ class DDLParserSuite extends AnalysisTest { val sql18 = replaceTableHeader("ROW FORMAT SERDE 'serde'") checkError( exception = parseException(sql18), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "ROW FORMAT"), context = ExpectedContext( fragment = sql18, @@ -917,7 +917,7 @@ class DDLParserSuite extends AnalysisTest { val sql19 = createTableHeader("CLUSTER BY (a)") checkError( exception = parseException(sql19), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "CLUSTER BY"), context = ExpectedContext( fragment = sql19, @@ -927,7 +927,7 @@ class DDLParserSuite extends AnalysisTest { val sql20 = replaceTableHeader("CLUSTER BY (a)") checkError( exception = parseException(sql20), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "CLUSTER BY"), context = ExpectedContext( fragment = sql20, @@ -1231,7 +1231,7 @@ class DDLParserSuite extends AnalysisTest { val fragment = "bad_type" checkError( exception = parseException(sql), - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> "\"BAD_TYPE\""), context = ExpectedContext( fragment = fragment, @@ -1282,19 +1282,19 @@ class DDLParserSuite extends AnalysisTest { val sql1 = "ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint COMMENT 'new comment'" checkError( exception = parseException(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'COMMENT'", "hint" -> "")) val sql2 = "ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint COMMENT AFTER d" checkError( exception = parseException(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'COMMENT'", "hint" -> "")) val sql3 = "ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint COMMENT 'new comment' AFTER d" checkError( exception = parseException(sql3), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'COMMENT'", "hint" -> "")) } @@ -1364,7 +1364,7 @@ class DDLParserSuite extends AnalysisTest { val sql4 = "ALTER TABLE table_name CHANGE COLUMN a.b.c new_name INT" checkError( exception = parseException(sql4), - errorClass = "_LEGACY_ERROR_TEMP_0034", + condition = "_LEGACY_ERROR_TEMP_0034", parameters = Map( "operation" -> "Renaming column", "command" -> "ALTER COLUMN", @@ -1378,7 +1378,7 @@ class DDLParserSuite extends AnalysisTest { val sql5 = "ALTER TABLE table_name PARTITION (a='1') CHANGE COLUMN a.b.c c INT" checkError( exception = parseException(sql5), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE ... PARTITION ... CHANGE COLUMN"), context = ExpectedContext( fragment = sql5, @@ -1425,7 +1425,7 @@ class DDLParserSuite extends AnalysisTest { val sql5 = "ALTER TABLE table_name PARTITION (a='1') REPLACE COLUMNS (x string)" checkError( exception = parseException(sql5), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE ... PARTITION ... REPLACE COLUMNS"), context = ExpectedContext( fragment = sql5, @@ -1435,7 +1435,7 @@ class DDLParserSuite extends AnalysisTest { val sql6 = "ALTER TABLE table_name REPLACE COLUMNS (x string NOT NULL)" checkError( exception = parseException(sql6), - errorClass = "_LEGACY_ERROR_TEMP_0034", + condition = "_LEGACY_ERROR_TEMP_0034", parameters = Map("operation" -> "NOT NULL", "command" -> "REPLACE COLUMNS", "msg" -> ""), context = ExpectedContext( fragment = sql6, @@ -1445,7 +1445,7 @@ class DDLParserSuite extends AnalysisTest { val sql7 = "ALTER TABLE table_name REPLACE COLUMNS (x string FIRST)" checkError( exception = parseException(sql7), - errorClass = "_LEGACY_ERROR_TEMP_0034", + condition = "_LEGACY_ERROR_TEMP_0034", parameters = Map( "operation" -> "Column position", "command" -> "REPLACE COLUMNS", @@ -1458,7 +1458,7 @@ class DDLParserSuite extends AnalysisTest { val sql8 = "ALTER TABLE table_name REPLACE COLUMNS (a.b.c string)" checkError( exception = parseException(sql8), - errorClass = "_LEGACY_ERROR_TEMP_0034", + condition = "_LEGACY_ERROR_TEMP_0034", parameters = Map( "operation" -> "Replacing with a nested column", "command" -> "REPLACE COLUMNS", @@ -1471,7 +1471,7 @@ class DDLParserSuite extends AnalysisTest { val sql9 = "ALTER TABLE table_name REPLACE COLUMNS (a STRING COMMENT 'x' COMMENT 'y')" checkError( exception = parseException(sql9), - errorClass = "ALTER_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", + condition = "ALTER_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", parameters = Map( "type" -> "REPLACE", "columnName" -> "a", @@ -1646,7 +1646,7 @@ class DDLParserSuite extends AnalysisTest { |PARTITION (p1 = 3, p2) IF NOT EXISTS""".stripMargin checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "IF NOT EXISTS with dynamic partitions: p2"), context = ExpectedContext( fragment = fragment, @@ -1664,7 +1664,7 @@ class DDLParserSuite extends AnalysisTest { |PARTITION (p1 = 3) IF NOT EXISTS""".stripMargin checkError( exception = parseException(sql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "INSERT INTO ... IF NOT EXISTS"), context = ExpectedContext( fragment = fragment, @@ -1704,7 +1704,7 @@ class DDLParserSuite extends AnalysisTest { checkError( exception = parseException( "INSERT INTO TABLE t1 BY NAME (c1,c2) SELECT * FROM tmp_view"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map( "error" -> "'c1'", "hint" -> "") @@ -1729,7 +1729,7 @@ class DDLParserSuite extends AnalysisTest { val sql = "DELETE FROM testcat.ns1.ns2.tbl AS t(a,b,c,d) WHERE d = 2" checkError( exception = parseException(sql), - errorClass = "COLUMN_ALIASES_NOT_ALLOWED", + condition = "COLUMN_ALIASES_NOT_ALLOWED", parameters = Map("op" -> "DELETE"), context = ExpectedContext( fragment = sql, @@ -1771,7 +1771,7 @@ class DDLParserSuite extends AnalysisTest { |WHERE d=2""".stripMargin checkError( exception = parseException(sql), - errorClass = "COLUMN_ALIASES_NOT_ALLOWED", + condition = "COLUMN_ALIASES_NOT_ALLOWED", parameters = Map("op" -> "UPDATE"), context = ExpectedContext( fragment = sql, @@ -1931,7 +1931,7 @@ class DDLParserSuite extends AnalysisTest { """.stripMargin checkError( exception = parseException(sql), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'*'", "hint" -> "")) } @@ -1977,7 +1977,7 @@ class DDLParserSuite extends AnalysisTest { .stripMargin checkError( exception = parseException(sql), - errorClass = "COLUMN_ALIASES_NOT_ALLOWED", + condition = "COLUMN_ALIASES_NOT_ALLOWED", parameters = Map("op" -> "MERGE"), context = ExpectedContext( fragment = sql, @@ -2056,7 +2056,7 @@ class DDLParserSuite extends AnalysisTest { |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2)""".stripMargin checkError( exception = parseException(sql), - errorClass = "NON_LAST_MATCHED_CLAUSE_OMIT_CONDITION", + condition = "NON_LAST_MATCHED_CLAUSE_OMIT_CONDITION", parameters = Map.empty, context = ExpectedContext( fragment = sql, @@ -2079,7 +2079,7 @@ class DDLParserSuite extends AnalysisTest { |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2)""".stripMargin checkError( exception = parseException(sql), - errorClass = "NON_LAST_NOT_MATCHED_BY_TARGET_CLAUSE_OMIT_CONDITION", + condition = "NON_LAST_NOT_MATCHED_BY_TARGET_CLAUSE_OMIT_CONDITION", parameters = Map.empty, context = ExpectedContext( fragment = sql, @@ -2103,7 +2103,7 @@ class DDLParserSuite extends AnalysisTest { |WHEN NOT MATCHED BY SOURCE THEN DELETE""".stripMargin checkError( exception = parseException(sql), - errorClass = "NON_LAST_NOT_MATCHED_BY_SOURCE_CLAUSE_OMIT_CONDITION", + condition = "NON_LAST_NOT_MATCHED_BY_SOURCE_CLAUSE_OMIT_CONDITION", parameters = Map.empty, context = ExpectedContext( fragment = sql, @@ -2118,7 +2118,7 @@ class DDLParserSuite extends AnalysisTest { |ON target.col1 = source.col1""".stripMargin checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0008", + condition = "_LEGACY_ERROR_TEMP_0008", parameters = Map.empty, context = ExpectedContext( fragment = sql, @@ -2209,7 +2209,7 @@ class DDLParserSuite extends AnalysisTest { val sql1 = "analyze table a.b.c compute statistics xxxx" checkError( exception = parseException(sql1), - errorClass = "INVALID_SQL_SYNTAX.ANALYZE_TABLE_UNEXPECTED_NOSCAN", + condition = "INVALID_SQL_SYNTAX.ANALYZE_TABLE_UNEXPECTED_NOSCAN", parameters = Map("ctx" -> "XXXX"), context = ExpectedContext( fragment = sql1, @@ -2219,7 +2219,7 @@ class DDLParserSuite extends AnalysisTest { val sql2 = "analyze table a.b.c partition (a) compute statistics xxxx" checkError( exception = parseException(sql2), - errorClass = "INVALID_SQL_SYNTAX.ANALYZE_TABLE_UNEXPECTED_NOSCAN", + condition = "INVALID_SQL_SYNTAX.ANALYZE_TABLE_UNEXPECTED_NOSCAN", parameters = Map("ctx" -> "XXXX"), context = ExpectedContext( fragment = sql2, @@ -2238,7 +2238,7 @@ class DDLParserSuite extends AnalysisTest { val sql = "ANALYZE TABLES IN a.b.c COMPUTE STATISTICS xxxx" checkError( exception = parseException(sql), - errorClass = "INVALID_SQL_SYNTAX.ANALYZE_TABLE_UNEXPECTED_NOSCAN", + condition = "INVALID_SQL_SYNTAX.ANALYZE_TABLE_UNEXPECTED_NOSCAN", parameters = Map("ctx" -> "XXXX"), context = ExpectedContext( fragment = sql, @@ -2250,7 +2250,7 @@ class DDLParserSuite extends AnalysisTest { val sql1 = "ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS" checkError( exception = parseException(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "end of input", "hint" -> "")) comparePlans( @@ -2287,13 +2287,13 @@ class DDLParserSuite extends AnalysisTest { val sql2 = "ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value" checkError( exception = parseException(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'key'", "hint" -> "")) // expecting {, ';'} val sql3 = "ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL" checkError( exception = parseException(sql3), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "end of input", "hint" -> ": missing 'COLUMNS'")) } @@ -2370,7 +2370,7 @@ class DDLParserSuite extends AnalysisTest { val sql = "CACHE TABLE a.b.c AS SELECT * FROM testData" checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0037", + condition = "_LEGACY_ERROR_TEMP_0037", parameters = Map("quoted" -> "a.b"), context = ExpectedContext( fragment = sql, @@ -2382,7 +2382,7 @@ class DDLParserSuite extends AnalysisTest { val createTableSql = "create table test_table using my_data_source options (password)" checkError( exception = parseException(createTableSql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "A value must be specified for the key: password."), context = ExpectedContext( fragment = createTableSql, @@ -2413,7 +2413,7 @@ class DDLParserSuite extends AnalysisTest { |(dt='2009-09-09', country='uk')""".stripMargin checkError( exception = parseException(sql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER VIEW ... ADD PARTITION"), context = ExpectedContext( fragment = sql, @@ -2698,14 +2698,14 @@ class DDLParserSuite extends AnalysisTest { val sql1 = "ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT " checkError( exception = parseException(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "end of input", "hint" -> "")) // It is not possible to both SET DEFAULT and DROP DEFAULT at the same time. // This results in a parsing error. val sql2 = "ALTER TABLE t1 ALTER COLUMN a.b.c DROP DEFAULT SET DEFAULT 42" checkError( exception = parseException(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'SET'", "hint" -> "")) comparePlans( @@ -2724,7 +2724,7 @@ class DDLParserSuite extends AnalysisTest { val fragment = "b STRING NOT NULL DEFAULT \"abc\"" checkError( exception = parseException(sql), - errorClass = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION", + condition = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION", parameters = Map.empty, context = ExpectedContext( fragment = fragment, @@ -2784,7 +2784,7 @@ class DDLParserSuite extends AnalysisTest { exception = intercept[ParseException]( parsePlan( "CREATE TABLE my_tab(a INT, b STRING NOT NULL DEFAULT \"abc\" NOT NULL)")), - errorClass = "CREATE_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", + condition = "CREATE_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", parameters = Map( "columnName" -> "b", "optionName" -> "NOT NULL"), @@ -2794,7 +2794,7 @@ class DDLParserSuite extends AnalysisTest { exception = intercept[ParseException]( parsePlan( "CREATE TABLE my_tab(a INT, b STRING DEFAULT \"123\" NOT NULL DEFAULT \"abc\")")), - errorClass = "CREATE_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", + condition = "CREATE_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", parameters = Map( "columnName" -> "b", "optionName" -> "DEFAULT"), @@ -2804,7 +2804,7 @@ class DDLParserSuite extends AnalysisTest { exception = intercept[ParseException]( parsePlan( "CREATE TABLE my_tab(a INT, b STRING COMMENT \"abc\" NOT NULL COMMENT \"abc\")")), - errorClass = "CREATE_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", + condition = "CREATE_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", parameters = Map( "columnName" -> "b", "optionName" -> "COMMENT"), @@ -2836,7 +2836,7 @@ class DDLParserSuite extends AnalysisTest { checkError( exception = parseException("CREATE TABLE my_tab(a INT, " + "b INT GENERATED ALWAYS AS (a + 1) GENERATED ALWAYS AS (a + 2)) USING PARQUET"), - errorClass = "CREATE_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", + condition = "CREATE_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", parameters = Map("columnName" -> "b", "optionName" -> "GENERATED ALWAYS AS"), context = ExpectedContext( fragment = "b INT GENERATED ALWAYS AS (a + 1) GENERATED ALWAYS AS (a + 2)", @@ -2848,14 +2848,14 @@ class DDLParserSuite extends AnalysisTest { checkError( exception = parseException( "CREATE TABLE my_tab(a INT, b INT GENERATED ALWAYS AS ()) USING PARQUET"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "')'", "hint" -> "") ) // No parenthesis checkError( exception = parseException( "CREATE TABLE my_tab(a INT, b INT GENERATED ALWAYS AS a + 1) USING PARQUET"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'a'", "hint" -> ": missing '('") ) } @@ -2887,7 +2887,7 @@ class DDLParserSuite extends AnalysisTest { exception = intercept[ParseException]( parsePlan("ALTER TABLE my_tab ADD COLUMN b STRING NOT NULL DEFAULT \"abc\" NOT NULL") ), - errorClass = "ALTER_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", + condition = "ALTER_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", parameters = Map("type" -> "ADD", "columnName" -> "b", "optionName" -> "NOT NULL"), context = ExpectedContext( fragment = "b STRING NOT NULL DEFAULT \"abc\" NOT NULL", @@ -2899,7 +2899,7 @@ class DDLParserSuite extends AnalysisTest { exception = intercept[ParseException]( parsePlan("ALTER TABLE my_tab ADD COLUMN b STRING DEFAULT \"123\" NOT NULL DEFAULT \"abc\"") ), - errorClass = "ALTER_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", + condition = "ALTER_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", parameters = Map("type" -> "ADD", "columnName" -> "b", "optionName" -> "DEFAULT"), context = ExpectedContext( fragment = "b STRING DEFAULT \"123\" NOT NULL DEFAULT \"abc\"", @@ -2911,7 +2911,7 @@ class DDLParserSuite extends AnalysisTest { exception = intercept[ParseException]( parsePlan("ALTER TABLE my_tab ADD COLUMN b STRING COMMENT \"abc\" NOT NULL COMMENT \"abc\"") ), - errorClass = "ALTER_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", + condition = "ALTER_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", parameters = Map("type" -> "ADD", "columnName" -> "b", "optionName" -> "COMMENT"), context = ExpectedContext( fragment = "b STRING COMMENT \"abc\" NOT NULL COMMENT \"abc\"", @@ -2923,7 +2923,7 @@ class DDLParserSuite extends AnalysisTest { exception = intercept[ParseException]( parsePlan("ALTER TABLE my_tab ADD COLUMN b STRING FIRST COMMENT \"abc\" AFTER y") ), - errorClass = "ALTER_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", + condition = "ALTER_TABLE_COLUMN_DESCRIPTOR_DUPLICATE", parameters = Map("type" -> "ADD", "columnName" -> "b", "optionName" -> "FIRST|AFTER"), context = ExpectedContext(fragment = "b STRING FIRST COMMENT \"abc\" AFTER y", start = 30, stop = 65) @@ -2935,7 +2935,7 @@ class DDLParserSuite extends AnalysisTest { "USING parquet CLUSTERED BY (a) INTO 2 BUCKETS CLUSTER BY (a)" checkError( exception = parseException(sql1), - errorClass = "SPECIFY_CLUSTER_BY_WITH_BUCKETING_IS_NOT_ALLOWED", + condition = "SPECIFY_CLUSTER_BY_WITH_BUCKETING_IS_NOT_ALLOWED", parameters = Map.empty, context = ExpectedContext(fragment = sql1, start = 0, stop = 96) ) @@ -2946,7 +2946,7 @@ class DDLParserSuite extends AnalysisTest { "USING parquet CLUSTERED BY (a) INTO 2 BUCKETS CLUSTER BY (a)" checkError( exception = parseException(sql1), - errorClass = "SPECIFY_CLUSTER_BY_WITH_BUCKETING_IS_NOT_ALLOWED", + condition = "SPECIFY_CLUSTER_BY_WITH_BUCKETING_IS_NOT_ALLOWED", parameters = Map.empty, context = ExpectedContext(fragment = sql1, start = 0, stop = 97) ) @@ -2957,7 +2957,7 @@ class DDLParserSuite extends AnalysisTest { "USING parquet CLUSTER BY (a) PARTITIONED BY (a)" checkError( exception = parseException(sql1), - errorClass = "SPECIFY_CLUSTER_BY_WITH_PARTITIONED_BY_IS_NOT_ALLOWED", + condition = "SPECIFY_CLUSTER_BY_WITH_PARTITIONED_BY_IS_NOT_ALLOWED", parameters = Map.empty, context = ExpectedContext(fragment = sql1, start = 0, stop = 83) ) @@ -2968,7 +2968,7 @@ class DDLParserSuite extends AnalysisTest { "USING parquet CLUSTER BY (a) PARTITIONED BY (a)" checkError( exception = parseException(sql1), - errorClass = "SPECIFY_CLUSTER_BY_WITH_PARTITIONED_BY_IS_NOT_ALLOWED", + condition = "SPECIFY_CLUSTER_BY_WITH_PARTITIONED_BY_IS_NOT_ALLOWED", parameters = Map.empty, context = ExpectedContext(fragment = sql1, start = 0, stop = 84) ) @@ -2986,7 +2986,7 @@ class DDLParserSuite extends AnalysisTest { checkError( exception = internalException(insertDirSql), - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "INSERT OVERWRITE DIRECTORY is not supported.")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala index f11e920e4c07d..c416d21cfd4b0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala @@ -138,12 +138,12 @@ class DataTypeParserSuite extends SparkFunSuite with SQLHelper { test("Do not print empty parentheses for no params") { checkError( exception = intercept("unknown"), - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> "\"UNKNOWN\"") ) checkError( exception = intercept("unknown(1,2,3)"), - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> "\"UNKNOWN(1,2,3)\"") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala index e4f9b54680dc7..603d5d779769d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala @@ -34,7 +34,7 @@ class ErrorParserSuite extends AnalysisTest { test("semantic errors") { checkError( exception = parseException("select *\nfrom r\norder by q\ncluster by q"), - errorClass = "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", + condition = "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", parameters = Map.empty, context = ExpectedContext(fragment = "order by q\ncluster by q", start = 16, stop = 38)) } @@ -43,42 +43,42 @@ class ErrorParserSuite extends AnalysisTest { // scalastyle:off checkError( exception = parseException("USE \u0196pfel"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "\u0196pfel")) checkError( exception = parseException("USE \u88681"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "\u88681")) // scalastyle:on checkError( exception = parseException("USE https://www.spa.rk/bucket/pa-th.json?=&#%"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "https://www.spa.rk/bucket/pa-th.json?=&#%")) } test("hyphen in identifier - DDL tests") { checkError( exception = parseException("USE test-test"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-test")) checkError( exception = parseException("SET CATALOG test-test"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-test")) checkError( exception = parseException("CREATE DATABASE IF NOT EXISTS my-database"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "my-database")) checkError( exception = parseException( """ |ALTER DATABASE my-database |SET DBPROPERTIES ('p1'='v1')""".stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "my-database")) checkError( exception = parseException("DROP DATABASE my-database"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "my-database")) checkError( exception = parseException( @@ -87,7 +87,7 @@ class ErrorParserSuite extends AnalysisTest { |CHANGE COLUMN |test-col TYPE BIGINT """.stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-col")) checkError( exception = parseException( @@ -96,23 +96,23 @@ class ErrorParserSuite extends AnalysisTest { |DROP COLUMN |test-col, test """.stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-col")) checkError( exception = parseException("CREATE TABLE test (attri-bute INT)"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "attri-bute")) checkError( exception = parseException("CREATE FUNCTION test-func as org.test.func"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-func")) checkError( exception = parseException("DROP FUNCTION test-func as org.test.func"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-func")) checkError( exception = parseException("SHOW FUNCTIONS LIKE test-func"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-func")) checkError( exception = parseException( @@ -123,7 +123,7 @@ class ErrorParserSuite extends AnalysisTest { |LOCATION '/user/external/page_view' |TBLPROPERTIES ('p1'='v1', 'p2'='v2') |AS SELECT * FROM src""".stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "page-view")) checkError( exception = parseException( @@ -131,31 +131,31 @@ class ErrorParserSuite extends AnalysisTest { |CREATE TABLE IF NOT EXISTS tab |USING test-provider |AS SELECT * FROM src""".stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-provider")) checkError( exception = parseException("SHOW TABLES IN hyphen-database"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "hyphen-database")) checkError( exception = parseException("SHOW TABLE EXTENDED IN hyphen-db LIKE \"str\""), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "hyphen-db")) checkError( exception = parseException("DESC SCHEMA EXTENDED test-db"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-db")) checkError( exception = parseException("ANALYZE TABLE test-table PARTITION (part1)"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-table")) checkError( exception = parseException("CREATE TABLE t(c1 struct)"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-test")) checkError( exception = parseException("LOAD DATA INPATH \"path\" INTO TABLE my-tab"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "my-tab")) } @@ -163,28 +163,28 @@ class ErrorParserSuite extends AnalysisTest { // dml tests checkError( exception = parseException("SELECT * FROM table-with-hyphen"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "table-with-hyphen")) // special test case: minus in expression shouldn't be treated as hyphen in identifiers checkError( exception = parseException("SELECT a-b FROM table-with-hyphen"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "table-with-hyphen")) checkError( exception = parseException("SELECT a-b AS a-b FROM t"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "a-b")) checkError( exception = parseException("SELECT a-b FROM table-hyphen WHERE a-b = 0"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "table-hyphen")) checkError( exception = parseException("SELECT (a - test_func(b-c)) FROM test-table"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-table")) checkError( exception = parseException("WITH a-b AS (SELECT 1 FROM s) SELECT * FROM s;"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "a-b")) checkError( exception = parseException( @@ -193,7 +193,7 @@ class ErrorParserSuite extends AnalysisTest { |FROM t1 JOIN t2 |USING (a, b, at-tr) """.stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "at-tr")) checkError( exception = parseException( @@ -202,7 +202,7 @@ class ErrorParserSuite extends AnalysisTest { |OVER (PARTITION BY category ORDER BY revenue DESC) as hyphen-rank |FROM productRevenue """.stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "hyphen-rank")) checkError( exception = parseException( @@ -213,7 +213,7 @@ class ErrorParserSuite extends AnalysisTest { |GROUP BY fake-breaker |ORDER BY c """.stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "grammar-breaker")) assert(parsePlan( """ @@ -234,7 +234,7 @@ class ErrorParserSuite extends AnalysisTest { |WINDOW hyphen-window AS | (PARTITION BY a, b ORDER BY c rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) """.stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "hyphen-window")) checkError( exception = parseException( @@ -242,7 +242,7 @@ class ErrorParserSuite extends AnalysisTest { |SELECT * FROM tab |WINDOW window_ref AS window-ref """.stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "window-ref")) checkError( exception = parseException( @@ -251,7 +251,7 @@ class ErrorParserSuite extends AnalysisTest { |FROM t-a INNER JOIN tb |ON ta.a = tb.a AND ta.tag = tb.tag """.stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "t-a")) checkError( exception = parseException( @@ -260,7 +260,7 @@ class ErrorParserSuite extends AnalysisTest { |SELECT a |SELECT b """.stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-table")) checkError( exception = parseException( @@ -273,7 +273,7 @@ class ErrorParserSuite extends AnalysisTest { | FOR test-test IN ('dotNET', 'Java') |); """.stripMargin), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-test")) } @@ -281,23 +281,23 @@ class ErrorParserSuite extends AnalysisTest { // general bad types checkError( exception = parseException("SELECT cast(1 as badtype)"), - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> "\"BADTYPE\""), context = ExpectedContext(fragment = "badtype", start = 17, stop = 23)) // special handling on char and varchar checkError( exception = parseException("SELECT cast('a' as CHAR)"), - errorClass = "DATATYPE_MISSING_SIZE", + condition = "DATATYPE_MISSING_SIZE", parameters = Map("type" -> "\"CHAR\""), context = ExpectedContext(fragment = "CHAR", start = 19, stop = 22)) checkError( exception = parseException("SELECT cast('a' as Varchar)"), - errorClass = "DATATYPE_MISSING_SIZE", + condition = "DATATYPE_MISSING_SIZE", parameters = Map("type" -> "\"VARCHAR\""), context = ExpectedContext(fragment = "Varchar", start = 19, stop = 25)) checkError( exception = parseException("SELECT cast('a' as Character)"), - errorClass = "DATATYPE_MISSING_SIZE", + condition = "DATATYPE_MISSING_SIZE", parameters = Map("type" -> "\"CHARACTER\""), context = ExpectedContext(fragment = "Character", start = 19, stop = 27)) } @@ -305,32 +305,32 @@ class ErrorParserSuite extends AnalysisTest { test("'!' where only NOT should be allowed") { checkError( exception = parseException("SELECT 1 ! IN (2)"), - errorClass = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", + condition = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", parameters = Map("clause" -> "!"), context = ExpectedContext(fragment = "!", start = 9, stop = 9)) checkError( exception = parseException("SELECT 'a' ! LIKE 'b'"), - errorClass = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", + condition = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", parameters = Map("clause" -> "!"), context = ExpectedContext(fragment = "!", start = 11, stop = 11)) checkError( exception = parseException("SELECT 1 ! BETWEEN 1 AND 2"), - errorClass = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", + condition = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", parameters = Map("clause" -> "!"), context = ExpectedContext(fragment = "!", start = 9, stop = 9)) checkError( exception = parseException("SELECT 1 IS ! NULL"), - errorClass = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", + condition = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", parameters = Map("clause" -> "!"), context = ExpectedContext(fragment = "!", start = 12, stop = 12)) checkError( exception = parseException("CREATE TABLE IF ! EXISTS t(c1 INT)"), - errorClass = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", + condition = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", parameters = Map("clause" -> "!"), context = ExpectedContext(fragment = "!", start = 16, stop = 16)) checkError( exception = parseException("CREATE TABLE t(c1 INT ! NULL)"), - errorClass = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", + condition = "SYNTAX_DISCONTINUED.BANG_EQUALS_NOT", parameters = Map("clause" -> "!"), context = ExpectedContext(fragment = "!", start = 22, stop = 22)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 2654757177ee7..6d307d1cd9a87 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -201,7 +201,7 @@ class ExpressionParserSuite extends AnalysisTest { checkError( exception = parseException("a like 'pattern%' escape '##'"), - errorClass = "INVALID_ESC", + condition = "INVALID_ESC", parameters = Map("invalidEscape" -> "'##'"), context = ExpectedContext( fragment = "like 'pattern%' escape '##'", @@ -210,7 +210,7 @@ class ExpressionParserSuite extends AnalysisTest { checkError( exception = parseException("a like 'pattern%' escape ''"), - errorClass = "INVALID_ESC", + condition = "INVALID_ESC", parameters = Map("invalidEscape" -> "''"), context = ExpectedContext( fragment = "like 'pattern%' escape ''", @@ -222,7 +222,7 @@ class ExpressionParserSuite extends AnalysisTest { checkError( exception = parseException("a not like 'pattern%' escape '\"/'"), - errorClass = "INVALID_ESC", + condition = "INVALID_ESC", parameters = Map("invalidEscape" -> "'\"/'"), context = ExpectedContext( fragment = "not like 'pattern%' escape '\"/'", @@ -231,7 +231,7 @@ class ExpressionParserSuite extends AnalysisTest { checkError( exception = parseException("a not like 'pattern%' escape ''"), - errorClass = "INVALID_ESC", + condition = "INVALID_ESC", parameters = Map("invalidEscape" -> "''"), context = ExpectedContext( fragment = "not like 'pattern%' escape ''", @@ -261,7 +261,7 @@ class ExpressionParserSuite extends AnalysisTest { Seq("any", "some", "all").foreach { quantifier => checkError( exception = parseException(s"a like $quantifier()"), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> "Expected something between '(' and ')'."), context = ExpectedContext( fragment = s"like $quantifier()", @@ -328,7 +328,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("`select`(all a, b)", $"select".function($"a", $"b")) checkError( exception = parseException("foo(a x)"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'x'", "hint" -> ": extra input 'x'")) } @@ -461,7 +461,7 @@ class ExpressionParserSuite extends AnalysisTest { // We cannot use an arbitrary expression. checkError( exception = parseException("foo(*) over (partition by a order by b rows exp(b) preceding)"), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> "Frame bound value must be a literal."), context = ExpectedContext( fragment = "exp(b) preceding", @@ -540,7 +540,7 @@ class ExpressionParserSuite extends AnalysisTest { Literal(Timestamp.valueOf("2016-03-11 20:54:00.000"))) checkError( exception = parseException("timestamP_LTZ '2016-33-11 20:54:00.000'"), - errorClass = "INVALID_TYPED_LITERAL", + condition = "INVALID_TYPED_LITERAL", sqlState = "42604", parameters = Map( "valueType" -> "\"TIMESTAMP_LTZ\"", @@ -556,7 +556,7 @@ class ExpressionParserSuite extends AnalysisTest { Literal(LocalDateTime.parse("2016-03-11T20:54:00.000"))) checkError( exception = parseException("tImEstAmp_Ntz '2016-33-11 20:54:00.000'"), - errorClass = "INVALID_TYPED_LITERAL", + condition = "INVALID_TYPED_LITERAL", sqlState = "42604", parameters = Map( "valueType" -> "\"TIMESTAMP_NTZ\"", @@ -572,7 +572,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("dAte '2016-03-11'", Literal(Date.valueOf("2016-03-11"))) checkError( exception = parseException("DAtE 'mar 11 2016'"), - errorClass = "INVALID_TYPED_LITERAL", + condition = "INVALID_TYPED_LITERAL", sqlState = "42604", parameters = Map("valueType" -> "\"DATE\"", "value" -> "'mar 11 2016'"), context = ExpectedContext( @@ -585,7 +585,7 @@ class ExpressionParserSuite extends AnalysisTest { Literal(Timestamp.valueOf("2016-03-11 20:54:00.000"))) checkError( exception = parseException("timestamP '2016-33-11 20:54:00.000'"), - errorClass = "INVALID_TYPED_LITERAL", + condition = "INVALID_TYPED_LITERAL", sqlState = "42604", parameters = Map("valueType" -> "\"TIMESTAMP\"", "value" -> "'2016-33-11 20:54:00.000'"), context = ExpectedContext( @@ -600,7 +600,7 @@ class ExpressionParserSuite extends AnalysisTest { checkError( exception = parseException("timestamP '2016-33-11 20:54:00.000'"), - errorClass = "INVALID_TYPED_LITERAL", + condition = "INVALID_TYPED_LITERAL", sqlState = "42604", parameters = Map("valueType" -> "\"TIMESTAMP\"", "value" -> "'2016-33-11 20:54:00.000'"), context = ExpectedContext( @@ -621,7 +621,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("INTERVAL '1 year 2 month'", ymIntervalLiteral) checkError( exception = parseException("Interval 'interval 1 yearsss 2 monthsss'"), - errorClass = "INVALID_TYPED_LITERAL", + condition = "INVALID_TYPED_LITERAL", parameters = Map( "valueType" -> "\"INTERVAL\"", "value" -> "'interval 1 yearsss 2 monthsss'" @@ -638,7 +638,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("INTERVAL '1 day 2 hour 3 minute 4.005006 second'", dtIntervalLiteral) checkError( exception = parseException("Interval 'interval 1 daysss 2 hoursss'"), - errorClass = "INVALID_TYPED_LITERAL", + condition = "INVALID_TYPED_LITERAL", parameters = Map( "valueType" -> "\"INTERVAL\"", "value" -> "'interval 1 daysss 2 hoursss'" @@ -651,7 +651,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("-interval '1 day 2 hour 3 minute 4.005006 second'", UnaryMinus(dtIntervalLiteral)) checkError( exception = parseException("INTERVAL '1 year 2 second'"), - errorClass = "_LEGACY_ERROR_TEMP_0029", + condition = "_LEGACY_ERROR_TEMP_0029", parameters = Map("literal" -> "INTERVAL '1 year 2 second'"), context = ExpectedContext( fragment = "INTERVAL '1 year 2 second'", @@ -664,7 +664,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("INTERVAL '3 month 1 hour'", intervalLiteral) checkError( exception = parseException("Interval 'interval 3 monthsss 1 hoursss'"), - errorClass = "INVALID_TYPED_LITERAL", + condition = "INVALID_TYPED_LITERAL", parameters = Map( "valueType" -> "\"INTERVAL\"", "value" -> "'interval 3 monthsss 1 hoursss'" @@ -688,7 +688,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("x'A10C'", Literal(Array(0xa1, 0x0c).map(_.toByte))) checkError( exception = parseException("x'A1OC'"), - errorClass = "INVALID_TYPED_LITERAL", + condition = "INVALID_TYPED_LITERAL", sqlState = "42604", parameters = Map( "valueType" -> "\"X\"", @@ -701,7 +701,7 @@ class ExpressionParserSuite extends AnalysisTest { checkError( exception = parseException("GEO '(10,-6)'"), - errorClass = "UNSUPPORTED_TYPED_LITERAL", + condition = "UNSUPPORTED_TYPED_LITERAL", parameters = Map( "unsupportedType" -> "\"GEO\"", "supportedTypes" -> @@ -743,14 +743,14 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("9.e+1BD", Literal(BigDecimal("9.e+1").underlying())) checkError( exception = parseException(".e3"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'.'", "hint" -> "")) // Tiny Int Literal assertEqual("10Y", Literal(10.toByte)) checkError( exception = parseException("1000Y"), - errorClass = "INVALID_NUMERIC_LITERAL_RANGE", + condition = "INVALID_NUMERIC_LITERAL_RANGE", parameters = Map( "rawStrippedQualifier" -> "1000", "minValue" -> Byte.MinValue.toString, @@ -765,7 +765,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("10S", Literal(10.toShort)) checkError( exception = parseException("40000S"), - errorClass = "INVALID_NUMERIC_LITERAL_RANGE", + condition = "INVALID_NUMERIC_LITERAL_RANGE", parameters = Map( "rawStrippedQualifier" -> "40000", "minValue" -> Short.MinValue.toString, @@ -780,7 +780,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("10L", Literal(10L)) checkError( exception = parseException("78732472347982492793712334L"), - errorClass = "INVALID_NUMERIC_LITERAL_RANGE", + condition = "INVALID_NUMERIC_LITERAL_RANGE", parameters = Map( "rawStrippedQualifier" -> "78732472347982492793712334", "minValue" -> Long.MinValue.toString, @@ -795,7 +795,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("10.0D", Literal(10.0D)) checkError( exception = parseException("-1.8E308D"), - errorClass = "INVALID_NUMERIC_LITERAL_RANGE", + condition = "INVALID_NUMERIC_LITERAL_RANGE", parameters = Map( "rawStrippedQualifier" -> "-1.8E308", "minValue" -> BigDecimal(Double.MinValue).toString, @@ -807,7 +807,7 @@ class ExpressionParserSuite extends AnalysisTest { stop = 8)) checkError( exception = parseException("1.8E308D"), - errorClass = "INVALID_NUMERIC_LITERAL_RANGE", + condition = "INVALID_NUMERIC_LITERAL_RANGE", parameters = Map( "rawStrippedQualifier" -> "1.8E308", "minValue" -> BigDecimal(Double.MinValue).toString, @@ -825,7 +825,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("123.08BD", Literal(BigDecimal("123.08").underlying())) checkError( exception = parseException("1.20E-38BD"), - errorClass = "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + condition = "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", parameters = Map( "precision" -> "40", "maxPrecision" -> "38"), @@ -899,7 +899,7 @@ class ExpressionParserSuite extends AnalysisTest { // when ESCAPED_STRING_LITERALS is enabled. checkError( exception = parseException("'\''"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'''", "hint" -> ": extra input '''")) // The unescape special characters (e.g., "\\t") for 2.0+ don't work @@ -1082,7 +1082,7 @@ class ExpressionParserSuite extends AnalysisTest { // Unknown FROM TO intervals checkError( exception = parseException("interval '10' month to second"), - errorClass = "_LEGACY_ERROR_TEMP_0028", + condition = "_LEGACY_ERROR_TEMP_0028", parameters = Map("from" -> "month", "to" -> "second"), context = ExpectedContext( fragment = "'10' month to second", @@ -1104,7 +1104,7 @@ class ExpressionParserSuite extends AnalysisTest { } else { checkError( exception = parseException(s"interval $intervalStr"), - errorClass = "_LEGACY_ERROR_TEMP_0029", + condition = "_LEGACY_ERROR_TEMP_0029", parameters = Map("literal" -> "interval 3 monThs 4 dayS 22 sEcond 1 millisecond"), context = ExpectedContext( fragment = s"interval $intervalStr", @@ -1120,7 +1120,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("1 - f('o', o(bar))", Literal(1) - $"f".function("o", $"o".function($"bar"))) checkError( exception = parseException("1 - f('o', o(bar)) hello * world"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'*'", "hint" -> "")) } @@ -1142,7 +1142,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual(complexName.quotedString, UnresolvedAttribute(Seq("`fo`o", "`ba`r"))) checkError( exception = parseException(complexName.unquotedString), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'.'", "hint" -> "")) // Function identifier contains continuous backticks should be treated correctly. @@ -1225,7 +1225,7 @@ class ExpressionParserSuite extends AnalysisTest { Seq("any", "some", "all").foreach { quantifier => checkError( exception = parseException(s"a ilike $quantifier()"), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> "Expected something between '(' and ')'."), context = ExpectedContext( fragment = s"ilike $quantifier()", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala index 83d2557108c57..93afef60a9ddf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala @@ -159,7 +159,7 @@ class ParserUtilsSuite extends SparkFunSuite { exception = intercept[ParseException] { operationNotAllowed(errorMessage, showFuncContext) }, - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> errorMessage)) } @@ -172,7 +172,7 @@ class ParserUtilsSuite extends SparkFunSuite { exception = intercept[ParseException] { checkDuplicateKeys(properties2, createDbContext) }, - errorClass = "DUPLICATE_KEY", + condition = "DUPLICATE_KEY", parameters = Map("keyColumn" -> "`a`")) } @@ -223,7 +223,7 @@ class ParserUtilsSuite extends SparkFunSuite { exception = intercept[ParseException] { validate(f1(emptyContext), message, emptyContext) }, - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> message)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index a6a32e87b7421..6901f6e928c8a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -204,7 +204,7 @@ class PlanParserSuite extends AnalysisTest { |""".stripMargin checkError( exception = parseException(query), - errorClass = "UNCLOSED_BRACKETED_COMMENT", + condition = "UNCLOSED_BRACKETED_COMMENT", parameters = Map.empty) } @@ -222,7 +222,7 @@ class PlanParserSuite extends AnalysisTest { |""".stripMargin checkError( exception = parseException(query), - errorClass = "UNCLOSED_BRACKETED_COMMENT", + condition = "UNCLOSED_BRACKETED_COMMENT", parameters = Map.empty) } @@ -237,7 +237,7 @@ class PlanParserSuite extends AnalysisTest { val sql1 = "EXPLAIN logical SELECT 1" checkError( exception = parseException(sql1), - errorClass = "_LEGACY_ERROR_TEMP_0039", + condition = "_LEGACY_ERROR_TEMP_0039", parameters = Map.empty, context = ExpectedContext( fragment = sql1, @@ -247,7 +247,7 @@ class PlanParserSuite extends AnalysisTest { val sql2 = "EXPLAIN formatted SELECT 1" checkError( exception = parseException(sql2), - errorClass = "_LEGACY_ERROR_TEMP_0039", + condition = "_LEGACY_ERROR_TEMP_0039", parameters = Map.empty, context = ExpectedContext( fragment = sql2, @@ -295,7 +295,7 @@ class PlanParserSuite extends AnalysisTest { val sql = "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1" checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0038", + condition = "_LEGACY_ERROR_TEMP_0038", parameters = Map("duplicateNames" -> "'cte1'"), context = ExpectedContext( fragment = sql, @@ -328,13 +328,13 @@ class PlanParserSuite extends AnalysisTest { val sql1 = "from a" checkError( exception = parseException(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "end of input", "hint" -> "")) val sql2 = "from (from a union all from b) c select *" checkError( exception = parseException(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'union'", "hint" -> "")) } @@ -345,12 +345,12 @@ class PlanParserSuite extends AnalysisTest { val sql1 = "from a select * select * from x where a.s < 10" checkError( exception = parseException(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'from'", "hint" -> "")) val sql2 = "from a select * from b" checkError( exception = parseException(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'from'", "hint" -> "")) assertEqual( "from a insert into tbl1 select * insert into tbl2 select * where s < 10", @@ -393,7 +393,7 @@ class PlanParserSuite extends AnalysisTest { val sql1 = s"$baseSql order by a sort by a" checkError( exception = parseException(sql1), - errorClass = "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", + condition = "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", parameters = Map.empty, context = ExpectedContext( fragment = "order by a sort by a", @@ -403,7 +403,7 @@ class PlanParserSuite extends AnalysisTest { val sql2 = s"$baseSql cluster by a distribute by a" checkError( exception = parseException(sql2), - errorClass = "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", + condition = "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", parameters = Map.empty, context = ExpectedContext( fragment = "cluster by a distribute by a", @@ -413,7 +413,7 @@ class PlanParserSuite extends AnalysisTest { val sql3 = s"$baseSql order by a cluster by a" checkError( exception = parseException(sql3), - errorClass = "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", + condition = "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", parameters = Map.empty, context = ExpectedContext( fragment = "order by a cluster by a", @@ -423,7 +423,7 @@ class PlanParserSuite extends AnalysisTest { val sql4 = s"$baseSql order by a distribute by a" checkError( exception = parseException(sql4), - errorClass = "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", + condition = "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", parameters = Map.empty, context = ExpectedContext( fragment = "order by a distribute by a", @@ -499,7 +499,7 @@ class PlanParserSuite extends AnalysisTest { val sql1 = "SELECT a, b, count(distinct a, distinct b) as c FROM d GROUP BY a, b" checkError( exception = parseException(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'b'", "hint" -> ": extra input 'b'")) } @@ -595,7 +595,7 @@ class PlanParserSuite extends AnalysisTest { |)""".stripMargin checkError( exception = parseException(sql1), - errorClass = "NOT_ALLOWED_IN_FROM.LATERAL_WITH_PIVOT", + condition = "NOT_ALLOWED_IN_FROM.LATERAL_WITH_PIVOT", parameters = Map.empty, context = ExpectedContext( fragment = fragment1, @@ -617,7 +617,7 @@ class PlanParserSuite extends AnalysisTest { |)""".stripMargin checkError( exception = parseException(sql2), - errorClass = "NOT_ALLOWED_IN_FROM.LATERAL_WITH_UNPIVOT", + condition = "NOT_ALLOWED_IN_FROM.LATERAL_WITH_UNPIVOT", parameters = Map.empty, context = ExpectedContext( fragment = fragment2, @@ -647,7 +647,7 @@ class PlanParserSuite extends AnalysisTest { |)""".stripMargin checkError( exception = parseException(sql3), - errorClass = "NOT_ALLOWED_IN_FROM.UNPIVOT_WITH_PIVOT", + condition = "NOT_ALLOWED_IN_FROM.UNPIVOT_WITH_PIVOT", parameters = Map.empty, context = ExpectedContext( fragment = fragment3, @@ -711,7 +711,7 @@ class PlanParserSuite extends AnalysisTest { val sql1 = "select * from a natural cross join b" checkError( exception = parseException(sql1), - errorClass = "INCOMPATIBLE_JOIN_TYPES", + condition = "INCOMPATIBLE_JOIN_TYPES", parameters = Map("joinType1" -> "NATURAL", "joinType2" -> "CROSS"), sqlState = "42613", context = ExpectedContext( @@ -723,7 +723,7 @@ class PlanParserSuite extends AnalysisTest { val sql2 = "select * from a natural join b on a.id = b.id" checkError( exception = parseException(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'on'", "hint" -> "")) // Test multiple consecutive joins @@ -744,7 +744,7 @@ class PlanParserSuite extends AnalysisTest { val sql3 = "select * from t1 inner join t2 inner join t3 on col3 = col2 on col3 = col1" checkError( exception = parseException(sql3), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'on'", "hint" -> "")) // Parenthesis @@ -834,7 +834,7 @@ class PlanParserSuite extends AnalysisTest { val fragment1 = "tablesample(bucket 4 out of 10 on x)" checkError( exception = parseException(sql1), - errorClass = "_LEGACY_ERROR_TEMP_0015", + condition = "_LEGACY_ERROR_TEMP_0015", parameters = Map("msg" -> "BUCKET x OUT OF y ON colname"), context = ExpectedContext( fragment = fragment1, @@ -845,7 +845,7 @@ class PlanParserSuite extends AnalysisTest { val fragment2 = "tablesample(bucket 11 out of 10)" checkError( exception = parseException(sql2), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> "Sampling fraction (1.1) must be on interval [0, 1]"), context = ExpectedContext( fragment = fragment2, @@ -856,7 +856,7 @@ class PlanParserSuite extends AnalysisTest { val fragment3 = "TABLESAMPLE(300M)" checkError( exception = parseException(sql3), - errorClass = "_LEGACY_ERROR_TEMP_0015", + condition = "_LEGACY_ERROR_TEMP_0015", parameters = Map("msg" -> "byteLengthLiteral"), context = ExpectedContext( fragment = fragment3, @@ -867,7 +867,7 @@ class PlanParserSuite extends AnalysisTest { val fragment4 = "TABLESAMPLE(BUCKET 3 OUT OF 32 ON rand())" checkError( exception = parseException(sql4), - errorClass = "_LEGACY_ERROR_TEMP_0015", + condition = "_LEGACY_ERROR_TEMP_0015", parameters = Map("msg" -> "BUCKET x OUT OF y ON function"), context = ExpectedContext( fragment = fragment4, @@ -925,7 +925,7 @@ class PlanParserSuite extends AnalysisTest { val fragment1 = "default.range(2)" checkError( exception = parseException(sql1), - errorClass = "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", + condition = "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", parameters = Map("funcName" -> "`default`.`range`"), context = ExpectedContext( fragment = fragment1, @@ -937,7 +937,7 @@ class PlanParserSuite extends AnalysisTest { val fragment2 = "spark_catalog.default.range(2)" checkError( exception = parseException(sql2), - errorClass = "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", + condition = "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", parameters = Map("funcName" -> "`spark_catalog`.`default`.`range`"), context = ExpectedContext( fragment = fragment2, @@ -1047,14 +1047,14 @@ class PlanParserSuite extends AnalysisTest { val sql1 = "SELECT /*+ HINT() */ * FROM t" checkError( exception = parseException(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "')'", "hint" -> "")) // Disallow space as the delimiter. val sql2 = "SELECT /*+ INDEX(a b c) */ * from default.t" checkError( exception = parseException(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'b'", "hint" -> "")) assertEqual( @@ -1114,7 +1114,7 @@ class PlanParserSuite extends AnalysisTest { val sql3 = "SELECT /*+ COALESCE(30 + 50) */ * FROM t" checkError( exception = parseException(sql3), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'+'", "hint" -> "")) assertEqual( @@ -1241,13 +1241,13 @@ class PlanParserSuite extends AnalysisTest { val sql1 = "select ltrim(both 'S' from 'SS abc S'" checkError( exception = parseException(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'from'", "hint" -> "")) // expecting {')' val sql2 = "select rtrim(trailing 'S' from 'SS abc S'" checkError( exception = parseException(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'from'", "hint" -> "")) // expecting {')' assertTrimPlans( @@ -1361,7 +1361,7 @@ class PlanParserSuite extends AnalysisTest { val sql1 = "CREATE VIEW testView AS INSERT INTO jt VALUES(1, 1)" checkError( exception = parseException(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'INSERT'", "hint" -> "")) // Multi insert query @@ -1371,13 +1371,13 @@ class PlanParserSuite extends AnalysisTest { |INSERT INTO tbl2 SELECT * WHERE jt.id > 4""".stripMargin checkError( exception = parseException(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'INSERT'", "hint" -> "")) val sql3 = "ALTER VIEW testView AS INSERT INTO jt VALUES(1, 1)" checkError( exception = parseException(sql3), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'INSERT'", "hint" -> "")) // Multi insert query @@ -1387,7 +1387,7 @@ class PlanParserSuite extends AnalysisTest { |INSERT INTO tbl2 SELECT * WHERE jt.id > 4""".stripMargin checkError( exception = parseException(sql4), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'INSERT'", "hint" -> "")) } @@ -1395,13 +1395,13 @@ class PlanParserSuite extends AnalysisTest { val sql1 = "SELECT * FROM (INSERT INTO BAR VALUES (2))" checkError( exception = parseException(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'BAR'", "hint" -> ": missing ')'")) val sql2 = "SELECT * FROM S WHERE C1 IN (INSERT INTO T VALUES (2))" checkError( exception = parseException(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'IN'", "hint" -> "")) } @@ -1506,7 +1506,7 @@ class PlanParserSuite extends AnalysisTest { val sql1 = "select * from my_tvf(arg1 => table v1)" checkError( exception = parseException(sql1), - errorClass = + condition = "INVALID_SQL_SYNTAX.INVALID_TABLE_FUNCTION_IDENTIFIER_ARGUMENT_MISSING_PARENTHESES", parameters = Map("argumentName" -> "`v1`"), context = ExpectedContext( @@ -1627,14 +1627,14 @@ class PlanParserSuite extends AnalysisTest { val sql6 = "select * from my_tvf(arg1 => table(1) partition by col1 with single partition)" checkError( exception = parseException(sql6), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map( "error" -> "'partition'", "hint" -> "")) val sql7 = "select * from my_tvf(arg1 => table(1) order by col1)" checkError( exception = parseException(sql7), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map( "error" -> "'order'", "hint" -> "")) @@ -1643,7 +1643,7 @@ class PlanParserSuite extends AnalysisTest { val sql8 = s"select * from my_tvf(arg1 => $sql8tableArg $sql8partition)" checkError( exception = parseException(sql8), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map( "msg" -> ("The table function call includes a table argument with an invalid " + @@ -1766,7 +1766,7 @@ class PlanParserSuite extends AnalysisTest { |FROM testData""".stripMargin checkError( exception = parseException(sql), - errorClass = "UNSUPPORTED_FEATURE.TRANSFORM_NON_HIVE", + condition = "UNSUPPORTED_FEATURE.TRANSFORM_NON_HIVE", parameters = Map.empty, context = ExpectedContext( fragment = sql, @@ -1824,7 +1824,7 @@ class PlanParserSuite extends AnalysisTest { val fragment = "TIMESTAMP AS OF col" checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0056", + condition = "_LEGACY_ERROR_TEMP_0056", parameters = Map("reason" -> "timestamp expression cannot refer to any columns"), context = ExpectedContext( fragment = fragment, @@ -1919,11 +1919,11 @@ class PlanParserSuite extends AnalysisTest { // Invalid empty name and invalid symbol in a name checkError( exception = parseException(s"SELECT :-"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'-'", "hint" -> "")) checkError( exception = parseException(s"SELECT :"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "end of input", "hint" -> "")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index 465c2d408f265..9ae516eb77e60 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -210,7 +210,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { exception = intercept[SqlScriptingException] { parseScript(sqlScriptText) }, - errorClass = "LABELS_MISMATCH", + condition = "LABELS_MISMATCH", parameters = Map("beginLabel" -> "lbl_begin", "endLabel" -> "lbl_end")) } @@ -229,7 +229,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { exception = intercept[SqlScriptingException] { parseScript(sqlScriptText) }, - errorClass = "END_LABEL_WITHOUT_BEGIN_LABEL", + condition = "END_LABEL_WITHOUT_BEGIN_LABEL", parameters = Map("endLabel" -> "lbl")) } @@ -290,7 +290,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { exception = intercept[SqlScriptingException] { parseScript(sqlScriptText) }, - errorClass = "INVALID_VARIABLE_DECLARATION.ONLY_AT_BEGINNING", + condition = "INVALID_VARIABLE_DECLARATION.ONLY_AT_BEGINNING", parameters = Map("varName" -> "`testVariable`", "lineNumber" -> "4")) } @@ -306,7 +306,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { exception = intercept[SqlScriptingException] { parseScript(sqlScriptText) }, - errorClass = "INVALID_VARIABLE_DECLARATION.NOT_ALLOWED_IN_SCOPE", + condition = "INVALID_VARIABLE_DECLARATION.NOT_ALLOWED_IN_SCOPE", parameters = Map("varName" -> "`testVariable`", "lineNumber" -> "4")) } @@ -719,7 +719,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { exception = intercept[SqlScriptingException] { parseScript(sqlScriptText) }, - errorClass = "INVALID_LABEL_USAGE.ITERATE_IN_COMPOUND", + condition = "INVALID_LABEL_USAGE.ITERATE_IN_COMPOUND", parameters = Map("labelName" -> "LBL")) } @@ -761,7 +761,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { exception = intercept[SqlScriptingException] { parseScript(sqlScriptText) }, - errorClass = "INVALID_LABEL_USAGE.DOES_NOT_EXIST", + condition = "INVALID_LABEL_USAGE.DOES_NOT_EXIST", parameters = Map("labelName" -> "RANDOMLBL", "statementType" -> "LEAVE")) } @@ -776,7 +776,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { exception = intercept[SqlScriptingException] { parseScript(sqlScriptText) }, - errorClass = "INVALID_LABEL_USAGE.DOES_NOT_EXIST", + condition = "INVALID_LABEL_USAGE.DOES_NOT_EXIST", parameters = Map("labelName" -> "RANDOMLBL", "statementType" -> "ITERATE")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 62557ead1d2ee..6e1b5b78d9741 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -299,7 +299,7 @@ class TableIdentifierParserSuite extends SQLKeywordUtils { ).foreach { case (identifier, (errorClass, parameters)) => checkError( exception = intercept[ParseException](parseTableIdentifier(identifier)), - errorClass = errorClass, + condition = errorClass, parameters = parameters) } } @@ -318,7 +318,7 @@ class TableIdentifierParserSuite extends SQLKeywordUtils { reservedKeywordsInAnsiMode.foreach { keyword => checkError( exception = intercept[ParseException](parseTableIdentifier(keyword)), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> s"'$keyword'", "hint" -> "")) assert(TableIdentifier(keyword) === parseTableIdentifier(s"`$keyword`")) assert(TableIdentifier(keyword, Option("db")) === parseTableIdentifier(s"db.`$keyword`")) @@ -374,7 +374,7 @@ class TableIdentifierParserSuite extends SQLKeywordUtils { assert(complexName === parseTableIdentifier(complexName.quotedString)) checkError( exception = intercept[ParseException](parseTableIdentifier(complexName.unquotedString)), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'b'", "hint" -> "")) // Table identifier contains continuous backticks should be treated correctly. val complexName2 = TableIdentifier("x``y", Some("d``b")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableSchemaParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableSchemaParserSuite.scala index a56ab8616df0f..74fb5a44ab0bc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableSchemaParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableSchemaParserSuite.scala @@ -75,26 +75,26 @@ class TableSchemaParserSuite extends SparkFunSuite { checkError( exception = parseException(""), - errorClass = "PARSE_EMPTY_STATEMENT") + condition = "PARSE_EMPTY_STATEMENT") checkError( exception = parseException("a"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "end of input", "hint" -> "")) checkError( exception = parseException("a INT b long"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'b'", "hint" -> "")) checkError( exception = parseException("a INT,, b long"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "','", "hint" -> "")) checkError( exception = parseException("a INT, b long,,"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "','", "hint" -> "")) checkError( exception = parseException("a INT, b long, c int,"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "end of input", "hint" -> "")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/JoinTypesTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/JoinTypesTest.scala index 886b043ad79e6..7fa1935ccb058 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/JoinTypesTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/JoinTypesTest.scala @@ -68,7 +68,7 @@ class JoinTypesTest extends SparkFunSuite { exception = intercept[AnalysisException]( JoinType(joinType) ), - errorClass = "UNSUPPORTED_JOIN_TYPE", + condition = "UNSUPPORTED_JOIN_TYPE", sqlState = "0A000", parameters = Map( "typ" -> joinType, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala index 55afbc3acb096..3a739ccbecb64 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala @@ -41,7 +41,7 @@ class InternalOutputModesSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { InternalOutputModes(outputMode) }, - errorClass = "STREAMING_OUTPUT_MODE.INVALID", + condition = "STREAMING_OUTPUT_MODE.INVALID", parameters = Map("outputMode" -> outputMode)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala index 1d3fb835f5a77..0e872dcdb6262 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala @@ -43,7 +43,7 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite with SQLHelper { builder.put(1, null) // null value is OK checkError( exception = intercept[SparkRuntimeException](builder.put(null, 1)), - errorClass = "NULL_MAP_KEY", + condition = "NULL_MAP_KEY", parameters = Map.empty ) } @@ -53,7 +53,7 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite with SQLHelper { builder.put(1, 1) checkError( exception = intercept[SparkRuntimeException](builder.put(1, 2)), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> "1", "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") @@ -65,7 +65,7 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite with SQLHelper { builderDouble.put(-0.0, 1) checkError( exception = intercept[SparkRuntimeException](builderDouble.put(0.0, 2)), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> "0.0", "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") @@ -110,7 +110,7 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite with SQLHelper { val arr = Array(1.toByte) checkError( exception = intercept[SparkRuntimeException](builder.put(arr, 3)), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> arr.toString, "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") @@ -146,7 +146,7 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite with SQLHelper { // By default duplicated map key fails the query. checkError( exception = intercept[SparkRuntimeException](builder.put(unsafeRow, 3)), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> "[0,1]", "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") @@ -180,7 +180,7 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite with SQLHelper { // By default duplicated map key fails the query. checkError( exception = intercept[SparkRuntimeException](builder.put(unsafeArray, 3)), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> unsafeArray.toString, "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala index d55e672079484..632109a0cc8d0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala @@ -58,7 +58,7 @@ class ArrayDataIndexedSeqSuite extends SparkFunSuite { exception = intercept[SparkException] { seq(index) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map( "message" -> s"Index $index must be between 0 and the length of the ArrayData.")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala index 034010f5825b8..79b26b099565b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala @@ -42,7 +42,7 @@ class DateTimeFormatterHelperSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { convertIncompatiblePattern(s"yyyy-MM-dd $l G") }, - errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_WEEK_BASED_PATTERN", + condition = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_WEEK_BASED_PATTERN", parameters = Map("c" -> l.toString)) } unsupportedLetters.foreach { l => @@ -50,7 +50,7 @@ class DateTimeFormatterHelperSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { convertIncompatiblePattern(s"yyyy-MM-dd $l G") }, - errorClass = "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", + condition = "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", parameters = Map( "c" -> l.toString, "pattern" -> s"yyyy-MM-dd $l G")) @@ -60,7 +60,7 @@ class DateTimeFormatterHelperSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { DateTimeFormatterHelper.convertIncompatiblePattern(s"$l", isParsing = true) }, - errorClass = "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", + condition = "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", parameters = Map( "c" -> l.toString, "pattern" -> s"$l")) @@ -70,13 +70,13 @@ class DateTimeFormatterHelperSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { convertIncompatiblePattern(s"yyyy-MM-dd $style") }, - errorClass = "INVALID_DATETIME_PATTERN.LENGTH", + condition = "INVALID_DATETIME_PATTERN.LENGTH", parameters = Map("pattern" -> style)) checkError( exception = intercept[SparkIllegalArgumentException] { convertIncompatiblePattern(s"yyyy-MM-dd $style${style.head}") }, - errorClass = "INVALID_DATETIME_PATTERN.LENGTH", + condition = "INVALID_DATETIME_PATTERN.LENGTH", parameters = Map("pattern" -> style)) } assert(convertIncompatiblePattern("yyyy-MM-dd EEEE") === "uuuu-MM-dd EEEE") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 8d8669aece894..8f15e70fa5a48 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -542,7 +542,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { checkError( exception = intercept[SparkIllegalArgumentException]( dateAddInterval(input, new CalendarInterval(36, 47, 1))), - errorClass = "_LEGACY_ERROR_TEMP_2000", + condition = "_LEGACY_ERROR_TEMP_2000", parameters = Map( "message" -> "Cannot add hours, minutes or seconds, milliseconds, microseconds to a date", "ansiConfig" -> "\"spark.sql.ansi.enabled\"")) @@ -896,13 +896,13 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { exception = intercept[SparkIllegalArgumentException] { getDayOfWeekFromString(UTF8String.fromString("xx")) }, - errorClass = "_LEGACY_ERROR_TEMP_3209", + condition = "_LEGACY_ERROR_TEMP_3209", parameters = Map("string" -> "xx")) checkError( exception = intercept[SparkIllegalArgumentException] { getDayOfWeekFromString(UTF8String.fromString("\"quote")) }, - errorClass = "_LEGACY_ERROR_TEMP_3209", + condition = "_LEGACY_ERROR_TEMP_3209", parameters = Map("string" -> "\"quote")) } @@ -1043,7 +1043,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { exception = intercept[SparkIllegalArgumentException] { timestampAdd("SECS", 1, date(1969, 1, 1, 0, 0, 0, 1, getZoneId("UTC")), getZoneId("UTC")) }, - errorClass = "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + condition = "INVALID_PARAMETER_VALUE.DATETIME_UNIT", parameters = Map( "functionName" -> "`TIMESTAMPADD`", "parameter" -> "`unit`", @@ -1102,7 +1102,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { date(2022, 1, 1, 0, 0, 0, 1, getZoneId("UTC")), getZoneId("UTC")) }, - errorClass = "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + condition = "INVALID_PARAMETER_VALUE.DATETIME_UNIT", parameters = Map("functionName" -> "`TIMESTAMPDIFF`", "parameter" -> "`unit`", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 001ae2728d10f..3b89ae9be2d83 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -44,7 +44,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.INPUT_IS_NULL", + condition = "INVALID_INTERVAL_FORMAT.INPUT_IS_NULL", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"))) assert(safeStringToInterval(UTF8String.fromString(input)) === null) @@ -55,7 +55,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.INPUT_IS_EMPTY", + condition = "INVALID_INTERVAL_FORMAT.INPUT_IS_EMPTY", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"))) assert(safeStringToInterval(UTF8String.fromString(input)) === null) @@ -66,7 +66,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.INVALID_PREFIX", + condition = "INVALID_INTERVAL_FORMAT.INVALID_PREFIX", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"), "prefix" -> prefix)) @@ -78,7 +78,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.UNRECOGNIZED_NUMBER", + condition = "INVALID_INTERVAL_FORMAT.UNRECOGNIZED_NUMBER", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"), "number" -> number)) @@ -90,7 +90,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.ARITHMETIC_EXCEPTION", + condition = "INVALID_INTERVAL_FORMAT.ARITHMETIC_EXCEPTION", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"))) assert(safeStringToInterval(UTF8String.fromString(input)) === null) @@ -101,7 +101,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.INVALID_VALUE", + condition = "INVALID_INTERVAL_FORMAT.INVALID_VALUE", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"), "value" -> value)) @@ -113,7 +113,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.INVALID_PRECISION", + condition = "INVALID_INTERVAL_FORMAT.INVALID_PRECISION", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"), "value" -> value)) @@ -125,7 +125,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.INVALID_FRACTION", + condition = "INVALID_INTERVAL_FORMAT.INVALID_FRACTION", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"), "unit" -> unit)) @@ -137,7 +137,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.INVALID_UNIT", + condition = "INVALID_INTERVAL_FORMAT.INVALID_UNIT", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"), "unit" -> unit)) @@ -149,7 +149,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.MISSING_NUMBER", + condition = "INVALID_INTERVAL_FORMAT.MISSING_NUMBER", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"), "word" -> word)) @@ -161,7 +161,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.MISSING_UNIT", + condition = "INVALID_INTERVAL_FORMAT.MISSING_UNIT", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"), "word" -> word)) @@ -173,7 +173,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkIllegalArgumentException] { stringToInterval(UTF8String.fromString(input)) }, - errorClass = "INVALID_INTERVAL_FORMAT.UNKNOWN_PARSING_ERROR", + condition = "INVALID_INTERVAL_FORMAT.UNKNOWN_PARSING_ERROR", parameters = Map( "input" -> Option(input).map(_.toString).getOrElse("null"), "word" -> word)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala index 6223f9aadb593..558d7eda78b4a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala @@ -553,7 +553,7 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite { exception = intercept[SparkException] { formatter.parseWithoutTimeZone(invalidTimestampStr, allowTimeZone = false) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map( "message" -> ("Cannot parse field value '2021-13-01T25:61:61' for pattern " + "'yyyy-MM-dd HH:mm:ss' as the target spark data type \"TIMESTAMP_NTZ\".")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala index e20dfd4f60512..aca6931a0688d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala @@ -362,7 +362,7 @@ class CatalogSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { catalog.alterTable(testIdent, TableChange.addColumn(Array("data", "ts"), TimestampType)) }, - errorClass = "_LEGACY_ERROR_TEMP_3229", + condition = "_LEGACY_ERROR_TEMP_3229", parameters = Map("name" -> "data")) // the table has not changed @@ -381,7 +381,7 @@ class CatalogSuite extends SparkFunSuite { catalog.alterTable(testIdent, TableChange.addColumn(Array("missing_col", "new_field"), StringType)) }, - errorClass = "_LEGACY_ERROR_TEMP_3227", + condition = "_LEGACY_ERROR_TEMP_3227", parameters = Map("fieldName" -> "missing_col")) } @@ -427,7 +427,7 @@ class CatalogSuite extends SparkFunSuite { catalog.alterTable(testIdent, TableChange.updateColumnType(Array("missing_col"), LongType)) }, - errorClass = "_LEGACY_ERROR_TEMP_3227", + condition = "_LEGACY_ERROR_TEMP_3227", parameters = Map("fieldName" -> "missing_col")) } @@ -478,7 +478,7 @@ class CatalogSuite extends SparkFunSuite { catalog.alterTable(testIdent, TableChange.updateColumnComment(Array("missing_col"), "comment")) }, - errorClass = "_LEGACY_ERROR_TEMP_3227", + condition = "_LEGACY_ERROR_TEMP_3227", parameters = Map("fieldName" -> "missing_col")) } @@ -546,7 +546,7 @@ class CatalogSuite extends SparkFunSuite { catalog.alterTable(testIdent, TableChange.renameColumn(Array("missing_col"), "new_name")) }, - errorClass = "_LEGACY_ERROR_TEMP_3227", + condition = "_LEGACY_ERROR_TEMP_3227", parameters = Map("fieldName" -> "missing_col")) } @@ -614,7 +614,7 @@ class CatalogSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { catalog.alterTable(testIdent, TableChange.deleteColumn(Array("missing_col"), false)) }, - errorClass = "_LEGACY_ERROR_TEMP_3227", + condition = "_LEGACY_ERROR_TEMP_3227", parameters = Map("fieldName" -> "missing_col")) // with if exists it should pass @@ -636,7 +636,7 @@ class CatalogSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "z"), false)) }, - errorClass = "_LEGACY_ERROR_TEMP_3227", + condition = "_LEGACY_ERROR_TEMP_3227", parameters = Map("fieldName" -> "z")) // with if exists it should pass diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala index 1aa0b408366bf..a9d8a69128ae2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala @@ -121,7 +121,7 @@ class SupportsAtomicPartitionManagementSuite extends SparkFunSuite { exception = intercept[SparkUnsupportedOperationException] { partTable.purgePartitions(partIdents) }, - errorClass = "UNSUPPORTED_FEATURE.PURGE_PARTITION", + condition = "UNSUPPORTED_FEATURE.PURGE_PARTITION", parameters = Map.empty ) } @@ -170,7 +170,7 @@ class SupportsAtomicPartitionManagementSuite extends SparkFunSuite { partTable.truncatePartitions(Array(InternalRow("5"), InternalRow("6"))) } checkError(e, - errorClass = "PARTITIONS_NOT_FOUND", + condition = "PARTITIONS_NOT_FOUND", parameters = Map("partitionList" -> "PARTITION (`dt` = 6)", "tableName" -> "`test`.`ns`.`test_table`")) assert(partTable.rows === InternalRow(2, "zyx", "5") :: Nil) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala index 06a23e7fda207..8581d4dec1fb8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala @@ -93,7 +93,7 @@ class SupportsPartitionManagementSuite extends SparkFunSuite { exception = intercept[SparkUnsupportedOperationException] { partTable.purgePartition(InternalRow.apply("3")) }, - errorClass = "UNSUPPORTED_FEATURE.PURGE_PARTITION", + condition = "UNSUPPORTED_FEATURE.PURGE_PARTITION", parameters = Map.empty ) } @@ -217,7 +217,7 @@ class SupportsPartitionManagementSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { partTable.partitionExists(InternalRow(0)) }, - errorClass = "_LEGACY_ERROR_TEMP_3208", + condition = "_LEGACY_ERROR_TEMP_3208", parameters = Map("numFields" -> "1", "schemaLen" -> "2")) } @@ -228,7 +228,7 @@ class SupportsPartitionManagementSuite extends SparkFunSuite { partTable.renamePartition(InternalRow(0, "abc"), InternalRow(1, "abc")) } checkError(e, - errorClass = "PARTITIONS_ALREADY_EXIST", + condition = "PARTITIONS_ALREADY_EXIST", parameters = Map("partitionList" -> "PARTITION (`part0` = 1, `part1` = abc)", "tableName" -> "`test`.`ns`.`test_table`")) @@ -237,7 +237,7 @@ class SupportsPartitionManagementSuite extends SparkFunSuite { partTable.renamePartition(newPart, InternalRow(3, "abc")) } checkError(e2, - errorClass = "PARTITIONS_NOT_FOUND", + condition = "PARTITIONS_NOT_FOUND", parameters = Map("partitionList" -> "PARTITION (`part0` = 2, `part1` = xyz)", "tableName" -> "`test`.`ns`.`test_table`")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index 4343e464b2c80..3241f031a706b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -161,7 +161,7 @@ class DataTypeSuite extends SparkFunSuite { exception = intercept[SparkException] { left.merge(right) }, - errorClass = "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", + condition = "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", parameters = Map("left" -> "\"FLOAT\"", "right" -> "\"BIGINT\"" ) ) @@ -299,21 +299,21 @@ class DataTypeSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { DataType.fromJson(""""abcd"""") }, - errorClass = "INVALID_JSON_DATA_TYPE", + condition = "INVALID_JSON_DATA_TYPE", parameters = Map("invalidType" -> "abcd")) checkError( exception = intercept[SparkIllegalArgumentException] { DataType.fromJson("""{"abcd":"a"}""") }, - errorClass = "INVALID_JSON_DATA_TYPE", + condition = "INVALID_JSON_DATA_TYPE", parameters = Map("invalidType" -> """{"abcd":"a"}""")) checkError( exception = intercept[SparkIllegalArgumentException] { DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""") }, - errorClass = "INVALID_JSON_DATA_TYPE", + condition = "INVALID_JSON_DATA_TYPE", parameters = Map("invalidType" -> """{"a":123}""")) // Malformed JSON string @@ -900,7 +900,7 @@ class DataTypeSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { DataType.fromJson(json) }, - errorClass = "INVALID_JSON_DATA_TYPE_FOR_COLLATIONS", + condition = "INVALID_JSON_DATA_TYPE_FOR_COLLATIONS", parameters = Map("jsonType" -> "integer") ) } @@ -934,7 +934,7 @@ class DataTypeSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { DataType.fromJson(json) }, - errorClass = "INVALID_JSON_DATA_TYPE_FOR_COLLATIONS", + condition = "INVALID_JSON_DATA_TYPE_FOR_COLLATIONS", parameters = Map("jsonType" -> "integer") ) } @@ -968,7 +968,7 @@ class DataTypeSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { DataType.fromJson(json) }, - errorClass = "INVALID_JSON_DATA_TYPE_FOR_COLLATIONS", + condition = "INVALID_JSON_DATA_TYPE_FOR_COLLATIONS", parameters = Map("jsonType" -> "map") ) } @@ -997,7 +997,7 @@ class DataTypeSuite extends SparkFunSuite { exception = intercept[SparkException] { DataType.fromJson(json) }, - errorClass = "COLLATION_INVALID_PROVIDER", + condition = "COLLATION_INVALID_PROVIDER", parameters = Map("provider" -> "badProvider", "supportedProviders" -> "spark, icu") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala index 8c9196cc33ca5..f07ee8b35bbb2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala @@ -41,7 +41,7 @@ class StrictDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBa DataTypeUtils.canWrite("", widerPoint2, point2, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`t`.`x`", @@ -60,7 +60,7 @@ class StrictDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBa DataTypeUtils.canWrite("", arrayOfLong, arrayOfInt, true, analysis.caseSensitiveResolution, "arr", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`arr`.`element`", @@ -79,7 +79,7 @@ class StrictDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBa DataTypeUtils.canWrite("", mapOfLong, mapOfInt, true, analysis.caseSensitiveResolution, "m", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`m`.`value`", @@ -98,7 +98,7 @@ class StrictDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBa DataTypeUtils.canWrite("", mapKeyLong, mapKeyInt, true, analysis.caseSensitiveResolution, "m", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`m`.`key`", @@ -116,7 +116,7 @@ class StrictDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBa analysis.caseSensitiveResolution, "nulls", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`nulls`", @@ -143,7 +143,7 @@ class ANSIDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBase DataTypeUtils.canWrite("", mapOfString, mapOfInt, true, analysis.caseSensitiveResolution, "m", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`m`.`value`", @@ -163,7 +163,7 @@ class ANSIDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBase DataTypeUtils.canWrite("", stringPoint2, point2, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`t`.`x`", @@ -182,7 +182,7 @@ class ANSIDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBase DataTypeUtils.canWrite("", arrayOfString, arrayOfInt, true, analysis.caseSensitiveResolution, "arr", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`arr`.`element`", @@ -201,7 +201,7 @@ class ANSIDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBase DataTypeUtils.canWrite("", mapKeyString, mapKeyInt, true, analysis.caseSensitiveResolution, "arr", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`arr`.`key`", @@ -218,7 +218,7 @@ class ANSIDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBase analysis.caseSensitiveResolution, "longToTimestamp", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`longToTimestamp`", @@ -231,7 +231,7 @@ class ANSIDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBase analysis.caseSensitiveResolution, "timestampToLong", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`timestampToLong`", @@ -306,7 +306,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", w, r, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`t`", @@ -328,7 +328,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", missingRequiredField, point2, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.STRUCT_MISSING_FIELDS", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.STRUCT_MISSING_FIELDS", parameters = Map("tableName" -> "``", "colName" -> "`t`", "missingFields" -> "`y`") ) } @@ -341,7 +341,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", missingRequiredField, point2, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.UNEXPECTED_COLUMN_NAME", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.UNEXPECTED_COLUMN_NAME", parameters = Map( "expected" -> "`x`", "found" -> "`y`", @@ -369,7 +369,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", missingMiddleField, expectedStruct, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.UNEXPECTED_COLUMN_NAME", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.UNEXPECTED_COLUMN_NAME", parameters = Map( "expected" -> "`y`", "found" -> "`z`", @@ -406,7 +406,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", requiredFieldIsOptional, point2, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.NULLABLE_COLUMN", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.NULLABLE_COLUMN", parameters = Map("tableName" -> "``", "colName" -> "`t`.`x`") ) } @@ -418,7 +418,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", point3, point2, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", parameters = Map("tableName" -> "``", "colName" -> "`t`", "extraFields" -> "`z`") ) } @@ -459,7 +459,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", arrayOfOptional, arrayOfRequired, true, analysis.caseSensitiveResolution, "arr", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.NULLABLE_ARRAY_ELEMENTS", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.NULLABLE_ARRAY_ELEMENTS", parameters = Map("tableName" -> "``", "colName" -> "`arr`") ) } @@ -489,7 +489,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", mapOfOptional, mapOfRequired, true, analysis.caseSensitiveResolution, "m", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.NULLABLE_MAP_VALUES", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.NULLABLE_MAP_VALUES", parameters = Map("tableName" -> "``", "colName" -> "`m`") ) } @@ -560,7 +560,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", sqlType, udtType, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.NULLABLE_COLUMN", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.NULLABLE_COLUMN", parameters = Map( "tableName" -> "``", "colName" -> "`t`.`col2`" @@ -595,7 +595,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", udtType, sqlType, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`t`.`col2`", @@ -633,7 +633,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", sqlType, udtType, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`t`.`col2`", @@ -675,7 +675,7 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataTypeUtils.canWrite("", writeType, readType, true, analysis.caseSensitiveResolution, "t", storeAssignmentPolicy, errMsg => errs += errMsg) ), - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`t`.`a`.`element`", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala index e6d915903f9bc..8d3af48ba689f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala @@ -65,7 +65,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper checkError( exception = intercept[SparkArithmeticException](Decimal(170L, 2, 1)), - errorClass = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", parameters = Map( "value" -> "0", "precision" -> "2", @@ -73,7 +73,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper "config" -> "\"spark.sql.ansi.enabled\"")) checkError( exception = intercept[SparkArithmeticException](Decimal(170L, 2, 0)), - errorClass = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", parameters = Map( "value" -> "0", "precision" -> "2", @@ -81,7 +81,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper "config" -> "\"spark.sql.ansi.enabled\"")) checkError( exception = intercept[SparkArithmeticException](Decimal(BigDecimal("10.030"), 2, 1)), - errorClass = "NUMERIC_VALUE_OUT_OF_RANGE.WITHOUT_SUGGESTION", + condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITHOUT_SUGGESTION", parameters = Map( "roundedValue" -> "10.0", "originalValue" -> "10.030", @@ -89,7 +89,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper "scale" -> "1")) checkError( exception = intercept[SparkArithmeticException](Decimal(BigDecimal("-9.95"), 2, 1)), - errorClass = "NUMERIC_VALUE_OUT_OF_RANGE.WITHOUT_SUGGESTION", + condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITHOUT_SUGGESTION", parameters = Map( "roundedValue" -> "-10.0", "originalValue" -> "-9.95", @@ -97,7 +97,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper "scale" -> "1")) checkError( exception = intercept[SparkArithmeticException](Decimal(1e17.toLong, 17, 0)), - errorClass = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", parameters = Map( "value" -> "0", "precision" -> "17", @@ -120,7 +120,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper def checkNegativeScaleDecimal(d: => Decimal): Unit = { checkError( exception = intercept[SparkException] (d), - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> ("Negative scale is not allowed: -3. " + "Set the config \"spark.sql.legacy.allowNegativeScaleOfDecimal\" " + "to \"true\" to allow it.")) @@ -317,7 +317,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper exception = intercept[SparkException] { d.toPrecision(5, 50, BigDecimal.RoundingMode.HALF_DOWN) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Not supported rounding mode: HALF_DOWN.") ) } @@ -350,7 +350,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper checkError( exception = intercept[SparkArithmeticException]( Decimal.fromStringANSI(UTF8String.fromString(string))), - errorClass = "NUMERIC_OUT_OF_SUPPORTED_RANGE", + condition = "NUMERIC_OUT_OF_SUPPORTED_RANGE", parameters = Map("value" -> string)) } @@ -370,7 +370,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper checkError( exception = intercept[SparkNumberFormatException]( Decimal.fromStringANSI(UTF8String.fromString("str"))), - errorClass = "CAST_INVALID_INPUT", + condition = "CAST_INVALID_INPUT", parameters = Map( "expression" -> "'str'", "sourceType" -> "\"STRING\"", @@ -398,7 +398,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper checkError( exception = intercept[SparkArithmeticException]( Decimal.fromStringANSI(UTF8String.fromString(string))), - errorClass = "NUMERIC_OUT_OF_SUPPORTED_RANGE", + condition = "NUMERIC_OUT_OF_SUPPORTED_RANGE", parameters = Map("value" -> string)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala index 562febe381130..5ec1525bf9b61 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala @@ -45,21 +45,21 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { test("lookup a single missing field should output existing fields") { checkError( exception = intercept[SparkIllegalArgumentException](s("c")), - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`c`", "fields" -> "`a`, `b`")) } test("lookup a set of missing fields should output existing fields") { checkError( exception = intercept[SparkIllegalArgumentException](s(Set("a", "c"))), - errorClass = "NONEXISTENT_FIELD_NAME_IN_LIST", + condition = "NONEXISTENT_FIELD_NAME_IN_LIST", parameters = Map("nonExistFields" -> "`c`", "fieldNames" -> "`a`, `b`")) } test("lookup fieldIndex for missing field should output existing fields") { checkError( exception = intercept[SparkIllegalArgumentException](s.fieldIndex("c")), - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`c`", "fields" -> "`a`, `b`")) } @@ -341,7 +341,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`S1`.`S12`.`S123`", "path" -> "`s1`.`s12`")) @@ -352,7 +352,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "AMBIGUOUS_COLUMN_OR_FIELD", + condition = "AMBIGUOUS_COLUMN_OR_FIELD", parameters = Map("name" -> "`S2`.`x`", "n" -> "2")) caseSensitiveCheck(Seq("s2", "x"), Some(Seq("s2") -> StructField("x", IntegerType))) @@ -362,7 +362,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`m1`.`key`", "path" -> "`m1`")) @@ -373,7 +373,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`M1`.`key`.`name`", "path" -> "`m1`.`key`")) @@ -382,7 +382,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`M1`.`value`.`name`", "path" -> "`m1`.`value`")) @@ -399,7 +399,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`m2`.`key`.`A`.`name`", "path" -> "`m2`.`key`.`a`")) @@ -408,7 +408,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`M2`.`value`.`b`.`name`", "path" -> "`m2`.`value`.`b`")) @@ -418,7 +418,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`A1`.`element`", "path" -> "`a1`")) @@ -428,7 +428,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`A1`.`element`.`name`", "path" -> "`a1`.`element`")) @@ -442,7 +442,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`a2`.`element`.`C`.`name`", "path" -> "`a2`.`element`.`c`")) @@ -456,7 +456,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`M3`.`value`.`value`.`MA`.`name`", "path" -> "`m3`.`value`.`value`.`ma`")) @@ -470,7 +470,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`A3`.`element`.`element`.`D`.`name`", "path" -> "`a3`.`element`.`element`.`d`") @@ -522,7 +522,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkException] { StructType.fromDDL("c1 DECIMAL(10, 5)").merge(StructType.fromDDL("c1 DECIMAL(12, 2)")) }, - errorClass = "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", + condition = "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", parameters = Map("left" -> "\"DECIMAL(10,5)\"", "right" -> "\"DECIMAL(12,2)\"") ) @@ -530,7 +530,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { exception = intercept[SparkException] { StructType.fromDDL("c1 DECIMAL(12, 5)").merge(StructType.fromDDL("c1 DECIMAL(12, 2)")) }, - errorClass = "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", + condition = "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", parameters = Map("left" -> "\"DECIMAL(12,5)\"", "right" -> "\"DECIMAL(12,2)\"") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala index c0fa43ff9bde0..c705a6b791bd1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala @@ -54,13 +54,13 @@ class ArrowUtilsSuite extends SparkFunSuite { exception = intercept[SparkException] { roundtrip(TimestampType) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Missing timezoneId where it is mandatory.")) checkError( exception = intercept[SparkUnsupportedOperationException] { ArrowUtils.fromArrowType(new ArrowType.Int(8, false)) }, - errorClass = "UNSUPPORTED_ARROWTYPE", + condition = "UNSUPPORTED_ARROWTYPE", parameters = Map("typeName" -> "Int(8, false)") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/CaseInsensitiveStringMapSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/CaseInsensitiveStringMapSuite.scala index 98c2a3d1e2726..932fb0a733371 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/CaseInsensitiveStringMapSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/CaseInsensitiveStringMapSuite.scala @@ -67,7 +67,7 @@ class CaseInsensitiveStringMapSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException] { options.getBoolean("FOO", true) }, - errorClass = "_LEGACY_ERROR_TEMP_3206", + condition = "_LEGACY_ERROR_TEMP_3206", parameters = Map("value" -> "bar")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala index c5f19b438f27f..a277bb021c3f6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala @@ -46,20 +46,20 @@ class SchemaUtilsSuite extends SparkFunSuite { exception = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication(schema, caseSensitive) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`a`")) checkError( exception = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication(schema.map(_.name), resolver(caseSensitive)) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`a`")) checkError( exception = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( schema.map(_.name), caseSensitiveAnalysis = caseSensitive) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`a`")) } @@ -106,7 +106,7 @@ class SchemaUtilsSuite extends SparkFunSuite { exception = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication(schema) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`camelcase`")) } } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala index 03bf5a4c10dbc..cad7fe6370827 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala @@ -1042,7 +1042,7 @@ class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest { analyzePlan( transform(connectTestRelation.observe("my_metric", "id".protoAttr.cast("string")))) }, - errorClass = "INVALID_OBSERVED_METRICS.NON_AGGREGATE_FUNC_ARG_IS_ATTRIBUTE", + condition = "INVALID_OBSERVED_METRICS.NON_AGGREGATE_FUNC_ARG_IS_ATTRIBUTE", parameters = Map("expr" -> "\"CAST(id AS STRING) AS id\"")) val connectPlan2 = @@ -1073,7 +1073,7 @@ class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest { transform( connectTestRelation.observe(Observation("my_metric"), "id".protoAttr.cast("string")))) }, - errorClass = "INVALID_OBSERVED_METRICS.NON_AGGREGATE_FUNC_ARG_IS_ATTRIBUTE", + condition = "INVALID_OBSERVED_METRICS.NON_AGGREGATE_FUNC_ARG_IS_ATTRIBUTE", parameters = Map("expr" -> "\"CAST(id AS STRING) AS id\"")) } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/plugin/SparkConnectPluginRegistrySuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/plugin/SparkConnectPluginRegistrySuite.scala index 512cdad62b921..2b768875c6e20 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/plugin/SparkConnectPluginRegistrySuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/plugin/SparkConnectPluginRegistrySuite.scala @@ -217,7 +217,7 @@ class SparkConnectPluginRegistrySuite extends SharedSparkSession with SparkConne exception = intercept[SparkException] { SparkConnectPluginRegistry.loadRelationPlugins() }, - errorClass = "CONNECT.PLUGIN_CTOR_MISSING", + condition = "CONNECT.PLUGIN_CTOR_MISSING", parameters = Map("cls" -> "org.apache.spark.sql.connect.plugin.DummyPluginNoTrivialCtor")) } @@ -228,7 +228,7 @@ class SparkConnectPluginRegistrySuite extends SharedSparkSession with SparkConne exception = intercept[SparkException] { SparkConnectPluginRegistry.loadRelationPlugins() }, - errorClass = "CONNECT.PLUGIN_RUNTIME_ERROR", + condition = "CONNECT.PLUGIN_RUNTIME_ERROR", parameters = Map("msg" -> "Bad Plugin Error")) } } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala index 8f76d58a31476..f125cb2d5c6c0 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala @@ -118,7 +118,7 @@ class InterceptorRegistrySuite extends SharedSparkSession { exception = intercept[SparkException] { SparkConnectInterceptorRegistry.chainInterceptors(sb) }, - errorClass = "CONNECT.INTERCEPTOR_CTOR_MISSING", + condition = "CONNECT.INTERCEPTOR_CTOR_MISSING", parameters = Map("cls" -> "org.apache.spark.sql.connect.service.TestingInterceptorNoTrivialCtor")) } @@ -132,7 +132,7 @@ class InterceptorRegistrySuite extends SharedSparkSession { exception = intercept[SparkException] { SparkConnectInterceptorRegistry.createConfiguredInterceptors() }, - errorClass = "CONNECT.INTERCEPTOR_CTOR_MISSING", + condition = "CONNECT.INTERCEPTOR_CTOR_MISSING", parameters = Map("cls" -> "org.apache.spark.sql.connect.service.TestingInterceptorNoTrivialCtor")) } @@ -144,7 +144,7 @@ class InterceptorRegistrySuite extends SharedSparkSession { exception = intercept[SparkException] { SparkConnectInterceptorRegistry.createConfiguredInterceptors() }, - errorClass = "CONNECT.INTERCEPTOR_RUNTIME_ERROR", + condition = "CONNECT.INTERCEPTOR_RUNTIME_ERROR", parameters = Map("msg" -> "Bad Error")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala index 273e8e08fd7a5..3b987529afcb5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala @@ -349,7 +349,7 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSparkSession |FROM VALUES (0), (1), (2), (10) AS tab(col); |""".stripMargin).collect() }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL", + condition = "DATATYPE_MISMATCH.UNEXPECTED_NULL", parameters = Map( "exprName" -> "accuracy", "sqlExpr" -> "\"percentile_approx(col, array(0.5, 0.4, 0.1), NULL)\""), @@ -363,7 +363,7 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSparkSession |FROM VALUES (0), (1), (2), (10) AS tab(col); |""".stripMargin).collect() }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL", + condition = "DATATYPE_MISMATCH.UNEXPECTED_NULL", parameters = Map( "exprName" -> "percentage", "sqlExpr" -> "\"percentile_approx(col, NULL, 100)\""), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BitmapExpressionsQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/BitmapExpressionsQuerySuite.scala index 0778599d54f49..97814e3bac44b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/BitmapExpressionsQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/BitmapExpressionsQuerySuite.scala @@ -214,7 +214,7 @@ class BitmapExpressionsQuerySuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("bitmap_count(a)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"bitmap_count(a)\"", "paramIndex" -> "first", @@ -236,7 +236,7 @@ class BitmapExpressionsQuerySuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("bitmap_or_agg(a)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"bitmap_or_agg(a)\"", "paramIndex" -> "first", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala index 9b39a2295e7d6..af97856fd222e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala @@ -98,7 +98,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception, - errorClass = "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + condition = "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", parameters = Map( "exprName" -> "estimatedNumItems", "valueRange" -> "[0, positive]", @@ -126,7 +126,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception, - errorClass = "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + condition = "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", parameters = Map( "exprName" -> "numBits", "valueRange" -> "[0, positive]", @@ -159,7 +159,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception1, - errorClass = "DATATYPE_MISMATCH.BLOOM_FILTER_WRONG_TYPE", + condition = "DATATYPE_MISMATCH.BLOOM_FILTER_WRONG_TYPE", parameters = Map( "functionName" -> "`bloom_filter_agg`", "sqlExpr" -> "\"bloom_filter_agg(a, 1000000, 8388608)\"", @@ -182,7 +182,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception2, - errorClass = "DATATYPE_MISMATCH.BLOOM_FILTER_WRONG_TYPE", + condition = "DATATYPE_MISMATCH.BLOOM_FILTER_WRONG_TYPE", parameters = Map( "functionName" -> "`bloom_filter_agg`", "sqlExpr" -> "\"bloom_filter_agg(a, 2, (2 * 8))\"", @@ -205,7 +205,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception3, - errorClass = "DATATYPE_MISMATCH.BLOOM_FILTER_WRONG_TYPE", + condition = "DATATYPE_MISMATCH.BLOOM_FILTER_WRONG_TYPE", parameters = Map( "functionName" -> "`bloom_filter_agg`", "sqlExpr" -> "\"bloom_filter_agg(a, CAST(2 AS BIGINT), 5)\"", @@ -228,7 +228,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception4, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL", + condition = "DATATYPE_MISMATCH.UNEXPECTED_NULL", parameters = Map( "exprName" -> "estimatedNumItems or numBits", "sqlExpr" -> "\"bloom_filter_agg(a, NULL, 5)\"" @@ -248,7 +248,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception5, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL", + condition = "DATATYPE_MISMATCH.UNEXPECTED_NULL", parameters = Map( "exprName" -> "estimatedNumItems or numBits", "sqlExpr" -> "\"bloom_filter_agg(a, 5, NULL)\"" @@ -268,7 +268,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception1, - errorClass = "DATATYPE_MISMATCH.BLOOM_FILTER_WRONG_TYPE", + condition = "DATATYPE_MISMATCH.BLOOM_FILTER_WRONG_TYPE", parameters = Map( "sqlExpr" -> "\"might_contain(1.0, 1)\"", "functionName" -> "`might_contain`", @@ -289,7 +289,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception2, - errorClass = "DATATYPE_MISMATCH.BLOOM_FILTER_WRONG_TYPE", + condition = "DATATYPE_MISMATCH.BLOOM_FILTER_WRONG_TYPE", parameters = Map( "sqlExpr" -> "\"might_contain(NULL, 0.1)\"", "functionName" -> "`might_contain`", @@ -314,7 +314,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception1, - errorClass = "DATATYPE_MISMATCH.BLOOM_FILTER_BINARY_OP_WRONG_TYPE", + condition = "DATATYPE_MISMATCH.BLOOM_FILTER_BINARY_OP_WRONG_TYPE", parameters = Map( "sqlExpr" -> "\"might_contain(CAST(a AS BINARY), CAST(5 AS BIGINT))\"", "functionName" -> "`might_contain`", @@ -335,7 +335,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } checkError( exception = exception2, - errorClass = "DATATYPE_MISMATCH.BLOOM_FILTER_BINARY_OP_WRONG_TYPE", + condition = "DATATYPE_MISMATCH.BLOOM_FILTER_BINARY_OP_WRONG_TYPE", parameters = Map( "sqlExpr" -> "\"might_contain(scalarsubquery(a), CAST(5 AS BIGINT))\"", "functionName" -> "`might_contain`", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 3ac433f31288c..b1e53aec81637 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -157,7 +157,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils sql("CACHE TABLE tempView AS SELECT 1") } checkError(e, - errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> "`tempView`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala index 59a566a3f2967..d3b11274fe1c8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala @@ -67,7 +67,7 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { def assertLengthCheckFailure(func: () => Unit): Unit = { checkError( exception = intercept[SparkRuntimeException](func()), - errorClass = "EXCEED_LIMIT_LENGTH", + condition = "EXCEED_LIMIT_LENGTH", parameters = Map("limit" -> "5") ) } @@ -702,7 +702,7 @@ class BasicCharVarcharTestSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("""SELECT from_json('{"a": "str"}', 'a CHAR(5)')""") }, - errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING", + condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING", parameters = Map.empty, context = ExpectedContext( fragment = "from_json('{\"a\": \"str\"}', 'a CHAR(5)')", @@ -724,19 +724,19 @@ class BasicCharVarcharTestSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.createDataFrame(df.collectAsList(), schema) }, - errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" + condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" ) checkError( exception = intercept[AnalysisException] { spark.createDataFrame(df.rdd, schema) }, - errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" + condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" ) checkError( exception = intercept[AnalysisException] { spark.createDataFrame(df.toJavaRDD, schema) }, - errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" + condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" ) withSQLConf((SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key, "true")) { val df1 = spark.createDataFrame(df.collectAsList(), schema) @@ -750,12 +750,12 @@ class BasicCharVarcharTestSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.read.schema(new StructType().add("id", CharType(5))) }, - errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING") + condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING") checkError( exception = intercept[AnalysisException] { spark.read.schema("id char(5)") }, - errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" + condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" ) withSQLConf((SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key, "true")) { val ds = spark.range(10).map(_.toString) @@ -792,13 +792,13 @@ class BasicCharVarcharTestSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.udf.register("testchar", () => "B", VarcharType(1)) }, - errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" + condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" ) checkError( exception = intercept[AnalysisException] { spark.udf.register("testchar2", (x: String) => x, VarcharType(1)) }, - errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" + condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" ) withSQLConf((SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key, "true")) { spark.udf.register("testchar", () => "B", VarcharType(1)) @@ -817,13 +817,13 @@ class BasicCharVarcharTestSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.readStream.schema(new StructType().add("id", CharType(5))) }, - errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" + condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" ) checkError( exception = intercept[AnalysisException] { spark.readStream.schema("id char(5)") }, - errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" + condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" ) withSQLConf((SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key, "true")) { withTempPath { dir => @@ -845,7 +845,7 @@ class BasicCharVarcharTestSuite extends QueryTest with SharedSparkSession { val df = sql("SELECT * FROM t") checkError(exception = intercept[AnalysisException] { df.to(newSchema) - }, errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING", parameters = Map.empty) + }, condition = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING", parameters = Map.empty) withSQLConf((SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key, "true")) { val df1 = df.to(newSchema) checkAnswer(df1, df.select("v", "c")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala index 7d0f6c401c0d6..d2c0b4de2afb9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala @@ -686,7 +686,7 @@ class CollationSQLExpressionsSuite val testQuery = sql(query) testQuery.collect() }, - errorClass = "INVALID_FORMAT.MISMATCH_INPUT", + condition = "INVALID_FORMAT.MISMATCH_INPUT", parameters = Map("inputType" -> "\"STRING\"", "input" -> "xx", "format" -> "999") ) } @@ -1015,7 +1015,7 @@ class CollationSQLExpressionsSuite exception = intercept[SparkRuntimeException] { sql(query).collect() }, - errorClass = "USER_RAISED_EXCEPTION", + condition = "USER_RAISED_EXCEPTION", parameters = Map("errorMessage" -> t.errorMessage) ) } @@ -1193,7 +1193,7 @@ class CollationSQLExpressionsSuite exception = intercept[AnalysisException] { sql("SELECT mask(collate('ab-CD-12-@$','UNICODE'),collate('X','UNICODE_CI'),'x','0','#')") }, - errorClass = "COLLATION_MISMATCH.EXPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", parameters = Map("explicitTypes" -> "`string collate UNICODE`, `string collate UNICODE_CI`") ) } @@ -1385,7 +1385,7 @@ class CollationSQLExpressionsSuite val testQuery = sql(query) testQuery.collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map("badRecord" -> "{\"a\":1,", "failFastMode" -> "FAILFAST") ) } @@ -1489,7 +1489,7 @@ class CollationSQLExpressionsSuite val testQuery = sql(query) testQuery.collect() }, - errorClass = "INVALID_VARIANT_CAST", + condition = "INVALID_VARIANT_CAST", parameters = Map("value" -> "\"Spark\"", "dataType" -> "\"INT\"") ) } @@ -1770,7 +1770,7 @@ class CollationSQLExpressionsSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = params, queryContext = Array( ExpectedContext(objectType = "", @@ -1821,7 +1821,7 @@ class CollationSQLExpressionsSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = params, queryContext = Array( ExpectedContext(objectType = "", @@ -1869,7 +1869,7 @@ class CollationSQLExpressionsSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", parameters = params, queryContext = Array( ExpectedContext(objectType = "", @@ -2319,7 +2319,7 @@ class CollationSQLExpressionsSuite exception = intercept[ExtendedAnalysisException] { sql(queryFail).collect() }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_STATIC_METHOD", + condition = "DATATYPE_MISMATCH.UNEXPECTED_STATIC_METHOD", parameters = Map( "methodName" -> "toHexString", "className" -> "java.lang.Integer", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala index 40cc6f19550d8..87dbbc65a3936 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala @@ -54,7 +54,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"collate(ABC, UNICODE_CI) LIKE %b%\"", "paramIndex" -> "first", @@ -148,7 +148,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"ilike(collate(ABC, UNICODE_CI), %b%)\"", "paramIndex" -> "first", @@ -188,7 +188,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"likeall(collate(Foo, UNICODE_CI))\"", "paramIndex" -> "first", @@ -228,7 +228,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"notlikeall(collate(Foo, UNICODE_CI))\"", "paramIndex" -> "first", @@ -268,7 +268,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"likeany(collate(Foo, UNICODE_CI))\"", "paramIndex" -> "first", @@ -308,7 +308,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"notlikeany(collate(Foo, UNICODE_CI))\"", "paramIndex" -> "first", @@ -348,7 +348,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"RLIKE(collate(ABC, UNICODE_CI), .b.)\"", "paramIndex" -> "first", @@ -388,7 +388,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"split(collate(ABC, UNICODE_CI), [b], -1)\"", "paramIndex" -> "first", @@ -429,7 +429,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(s"SELECT regexp_replace(collate('ABCDE','$c1'), '.c.', collate('FFF','$c2'))") }, - errorClass = "COLLATION_MISMATCH.EXPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", parameters = Map("explicitTypes" -> "`string`, `string collate UTF8_LCASE`") ) // Unsupported collations @@ -444,7 +444,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"regexp_replace(collate(ABCDE, UNICODE_CI), .c., FFF, 1)\"", "paramIndex" -> "first", @@ -486,7 +486,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"regexp_extract(collate(ABCDE, UNICODE_CI), .c., 0)\"", "paramIndex" -> "first", @@ -528,7 +528,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"regexp_extract_all(collate(ABCDE, UNICODE_CI), .c., 0)\"", "paramIndex" -> "first", @@ -568,7 +568,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"regexp_count(collate(ABCDE, UNICODE_CI), .c.)\"", "paramIndex" -> "first", @@ -608,7 +608,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"regexp_substr(collate(ABCDE, UNICODE_CI), .c.)\"", "paramIndex" -> "first", @@ -648,7 +648,7 @@ class CollationSQLRegexpSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"regexp_instr(collate(ABCDE, UNICODE_CI), .c., 0)\"", "paramIndex" -> "first", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala index 412c003a0dbaa..6804411d470b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala @@ -141,7 +141,7 @@ class CollationStringExpressionsSuite Cast(Literal.create("a"), StringType("UTF8_LCASE"))) CollationTypeCasts.transform(expr) }, - errorClass = "COLLATION_MISMATCH.IMPLICIT", + condition = "COLLATION_MISMATCH.IMPLICIT", sqlState = "42P21", parameters = Map.empty ) @@ -152,7 +152,7 @@ class CollationStringExpressionsSuite Collate(Literal.create("a"), "UTF8_LCASE")) CollationTypeCasts.transform(expr) }, - errorClass = "COLLATION_MISMATCH.EXPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", sqlState = "42P21", parameters = Map("explicitTypes" -> "`string`, `string collate UTF8_LCASE`") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index 5e7feec149c97..7782e9c382b45 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -91,7 +91,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { sql(s"select collate($args)") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = "42605", parameters = Map( "functionName" -> "`collate`", @@ -106,7 +106,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("collate function invalid collation data type") { checkError( exception = intercept[AnalysisException](sql("select collate('abc', 123)")), - errorClass = "UNEXPECTED_INPUT_TYPE", + condition = "UNEXPECTED_INPUT_TYPE", sqlState = "42K09", Map( "functionName" -> "`collate`", @@ -122,7 +122,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { checkError( exception = intercept[AnalysisException] { sql("select collate('abc', cast(null as string))") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL", + condition = "DATATYPE_MISMATCH.UNEXPECTED_NULL", sqlState = "42K09", Map("exprName" -> "`collation`", "sqlExpr" -> "\"CAST(NULL AS STRING)\""), context = ExpectedContext( @@ -133,7 +133,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("collate function invalid input data type") { checkError( exception = intercept[ExtendedAnalysisException] { sql(s"select collate(1, 'UTF8_BINARY')") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = "42K09", parameters = Map( "sqlExpr" -> "\"collate(1, UTF8_BINARY)\"", @@ -152,7 +152,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("invalid collation name throws exception") { checkError( exception = intercept[SparkException] { sql("select 'aaa' collate UTF8_BS") }, - errorClass = "COLLATION_INVALID_NAME", + condition = "COLLATION_INVALID_NAME", sqlState = "42704", parameters = Map("collationName" -> "UTF8_BS", "proposals" -> "UTF8_LCASE")) } @@ -182,7 +182,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { createTable(bucketColumns: _*) }, - errorClass = "INVALID_BUCKET_COLUMN_DATA_TYPE", + condition = "INVALID_BUCKET_COLUMN_DATA_TYPE", parameters = Map("type" -> "\"STRING COLLATE UNICODE\"") ); } @@ -248,7 +248,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { spark.sql(s"SELECT contains(collate('$left', '$leftCollationName')," + s"collate('$right', '$rightCollationName'))") }, - errorClass = "COLLATION_MISMATCH.EXPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", sqlState = "42P21", parameters = Map( "explicitTypes" -> @@ -262,7 +262,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { spark.sql(s"SELECT startsWith(collate('$left', '$leftCollationName')," + s"collate('$right', '$rightCollationName'))") }, - errorClass = "COLLATION_MISMATCH.EXPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", sqlState = "42P21", parameters = Map( "explicitTypes" -> @@ -276,7 +276,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { spark.sql(s"SELECT endsWith(collate('$left', '$leftCollationName')," + s"collate('$right', '$rightCollationName'))") }, - errorClass = "COLLATION_MISMATCH.EXPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", sqlState = "42P21", parameters = Map( "explicitTypes" -> @@ -455,7 +455,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"SELECT c1 FROM $tableName " + s"WHERE c1 = SUBSTR(COLLATE('a', 'UNICODE'), 0)") }, - errorClass = "COLLATION_MISMATCH.IMPLICIT", + condition = "COLLATION_MISMATCH.IMPLICIT", parameters = Map.empty ) @@ -479,7 +479,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { sql(s"SELECT c1 || c2 FROM $tableName") }, - errorClass = "COLLATION_MISMATCH.IMPLICIT" + condition = "COLLATION_MISMATCH.IMPLICIT" ) @@ -494,7 +494,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { sql(s"SELECT c1 FROM $tableName WHERE c1 = c3") }, - errorClass = "COLLATION_MISMATCH.IMPLICIT" + condition = "COLLATION_MISMATCH.IMPLICIT" ) // different explicit collations are set @@ -506,7 +506,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { |WHERE COLLATE('a', 'UTF8_BINARY') = COLLATE('a', 'UNICODE')""" .stripMargin) }, - errorClass = "COLLATION_MISMATCH.EXPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", parameters = Map( "explicitTypes" -> "`string`, `string collate UNICODE`" ) @@ -518,7 +518,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"SELECT c1 FROM $tableName WHERE c1 IN " + "(COLLATE('a', 'UTF8_BINARY'), COLLATE('b', 'UNICODE'))") }, - errorClass = "COLLATION_MISMATCH.EXPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", parameters = Map( "explicitTypes" -> "`string`, `string collate UNICODE`" ) @@ -528,7 +528,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"SELECT c1 FROM $tableName WHERE COLLATE(c1, 'UNICODE') IN " + "(COLLATE('a', 'UTF8_BINARY'))") }, - errorClass = "COLLATION_MISMATCH.EXPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", parameters = Map( "explicitTypes" -> "`string collate UNICODE`, `string`" ) @@ -540,7 +540,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { sql(s"SELECT c1 FROM $tableName WHERE c1 || c3 = 'aa'") }, - errorClass = "COLLATION_MISMATCH.IMPLICIT" + condition = "COLLATION_MISMATCH.IMPLICIT" ) // concat on different implicit collations should succeed, @@ -549,7 +549,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { sql(s"SELECT * FROM $tableName ORDER BY c1 || c3") }, - errorClass = "COLLATION_MISMATCH.IMPLICIT" + condition = "COLLATION_MISMATCH.IMPLICIT" ) // concat + in @@ -566,14 +566,14 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { sql(s"SELECT * FROM $tableName WHERE contains(c1||c3, 'a')") }, - errorClass = "COLLATION_MISMATCH.IMPLICIT" + condition = "COLLATION_MISMATCH.IMPLICIT" ) checkError( exception = intercept[AnalysisException] { sql(s"SELECT array('A', 'a' COLLATE UNICODE) == array('b' COLLATE UNICODE_CI)") }, - errorClass = "COLLATION_MISMATCH.IMPLICIT" + condition = "COLLATION_MISMATCH.IMPLICIT" ) checkAnswer(sql("SELECT array_join(array('a', 'b' collate UNICODE), 'c' collate UNICODE_CI)"), @@ -592,7 +592,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { sql("select map('a' COLLATE UTF8_LCASE, 'b', 'c')") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map("functionName" -> "`map`", "expectedNum" -> "2n (n > 0)", "actualNum" -> "3", "docroot" -> "https://spark.apache.org/docs/latest") ) @@ -602,7 +602,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { sql("select map('a' COLLATE UTF8_LCASE, 'b', 'c' COLLATE UNICODE, 'c')") }, - errorClass = "COLLATION_MISMATCH.EXPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", sqlState = "42P21", parameters = Map( "explicitTypes" -> @@ -722,7 +722,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { sql(s"CREATE TABLE $newTableName AS SELECT c1 || c2 FROM $tableName") }, - errorClass = "COLLATION_MISMATCH.IMPLICIT") + condition = "COLLATION_MISMATCH.IMPLICIT") } } } @@ -784,7 +784,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { createTable(partitionColumns: _*) }, - errorClass = "INVALID_PARTITION_COLUMN_DATA_TYPE", + condition = "INVALID_PARTITION_COLUMN_DATA_TYPE", parameters = Map("type" -> "\"STRING COLLATE UNICODE\"") ); } @@ -821,7 +821,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { |USING $v2Source |""".stripMargin) }, - errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + condition = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", parameters = Map( "fieldName" -> "c2", "expressionStr" -> "SUBSTRING(c1, 0, 1)", @@ -839,7 +839,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { |USING $v2Source |""".stripMargin) }, - errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + condition = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", parameters = Map( "fieldName" -> "c2", "expressionStr" -> "LOWER(c1)", @@ -857,7 +857,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { |USING $v2Source |""".stripMargin) }, - errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + condition = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", parameters = Map( "fieldName" -> "c2", "expressionStr" -> "UCASE(struct1.a)", @@ -875,7 +875,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { checkError( exception = intercept[ParseException] (sql("SELECT cast(1 as string collate unicode)")), - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map( "typeName" -> toSQLType(StringType("UNICODE"))), context = @@ -885,7 +885,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { checkError( exception = intercept[ParseException] (sql("SELECT 'A' :: string collate unicode")), - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map( "typeName" -> toSQLType(StringType("UNICODE"))), context = ExpectedContext(fragment = s"'A' :: string collate unicode", start = 7, stop = 35) @@ -898,7 +898,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { checkError( exception = intercept[ParseException] (sql("SELECT cast(1 as string collate unicode)")), - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map( "typeName" -> toSQLType(StringType("UNICODE"))), context = @@ -958,7 +958,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val query = s"select distinct m from $table" checkError( exception = intercept[ExtendedAnalysisException](sql(query)), - errorClass = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", parameters = Map( "colName" -> "`m`", "dataType" -> toSQLType(MapType( @@ -1000,7 +1000,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val ctx = s"$tableLeft.m = $tableRight.m" checkError( exception = intercept[AnalysisException](sql(query)), - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", parameters = Map( "functionName" -> "`=`", "dataType" -> toSQLType(MapType( @@ -1127,7 +1127,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val ctx = "m" checkError( exception = intercept[AnalysisException](sql(query)), - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", parameters = Map( "functionName" -> "`sortorder`", "dataType" -> s"\"MAP\"", @@ -1180,7 +1180,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val query = s"select $ctx" checkError( exception = intercept[AnalysisException](sql(query)), - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"map(collate(aaa, utf8_lcase), 1, collate(AAA, utf8_lcase), 2)[AaA]\"", "paramIndex" -> "second", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 68a7a4b8b2412..9cd35e527df57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -455,7 +455,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.filter($"a".isin($"b")) }, - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", parameters = Map( "functionName" -> "`in`", "dataType" -> "[\"INT\", \"ARRAY\"]", @@ -523,7 +523,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.filter($"a".isInCollection(Seq($"b"))) }, - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", parameters = Map( "functionName" -> "`in`", "dataType" -> "[\"INT\", \"ARRAY\"]", @@ -734,7 +734,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"SELECT *, $f() FROM tab1 JOIN tab2 ON tab1.id = tab2.id") }, - errorClass = "MULTI_SOURCES_UNSUPPORTED_FOR_EXPRESSION", + condition = "MULTI_SOURCES_UNSUPPORTED_FOR_EXPRESSION", parameters = Map("expr" -> s""""$f()""""), context = ExpectedContext( fragment = s"$f()", @@ -753,7 +753,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(stmt) }, - errorClass = "MULTI_SOURCES_UNSUPPORTED_FOR_EXPRESSION", + condition = "MULTI_SOURCES_UNSUPPORTED_FOR_EXPRESSION", parameters = Map("expr" -> """"input_file_name()""""), context = ExpectedContext( fragment = s"input_file_name()", @@ -1055,7 +1055,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { testData.withColumn("key", $"key".withField("a", lit(2))) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"update_fields(key, WithField(2))\"", "paramIndex" -> "first", @@ -1087,14 +1087,14 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { structLevel2.withColumn("a", $"a".withField("x.b", lit(2))) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`x`", "fields" -> "`a`")) checkError( exception = intercept[AnalysisException] { structLevel3.withColumn("a", $"a".withField("a.x.b", lit(2))) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`x`", "fields" -> "`a`")) } @@ -1103,7 +1103,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { structLevel1.withColumn("a", $"a".withField("b.a", lit(2))) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"update_fields(a.b, WithField(2))\"", "paramIndex" -> "first", @@ -1129,7 +1129,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { structLevel2.withColumn("a", $"a".withField("a.b", lit(2))) }, - errorClass = "AMBIGUOUS_REFERENCE_TO_FIELDS", + condition = "AMBIGUOUS_REFERENCE_TO_FIELDS", sqlState = "42000", parameters = Map("field" -> "`a`", "count" -> "2") ) @@ -1532,7 +1532,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.withColumn("a", $"a".withField("a.b.e.f", lit(2))) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`a`", "fields" -> "`a`.`b`")) } @@ -1644,14 +1644,14 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { mixedCaseStructLevel2.withColumn("a", $"a".withField("A.a", lit(2))) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`A`", "fields" -> "`a`, `B`")) checkError( exception = intercept[AnalysisException] { mixedCaseStructLevel2.withColumn("a", $"a".withField("b.a", lit(2))) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`b`", "fields" -> "`a`, `B`")) } } @@ -1687,7 +1687,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") .select($"struct_col".withField("a.c", lit(3))) }, - errorClass = "AMBIGUOUS_REFERENCE_TO_FIELDS", + condition = "AMBIGUOUS_REFERENCE_TO_FIELDS", sqlState = "42000", parameters = Map("field" -> "`a`", "count" -> "2") ) @@ -1854,7 +1854,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { testData.withColumn("key", $"key".dropFields("a")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"update_fields(key, dropfield())\"", "paramIndex" -> "first", @@ -1878,14 +1878,14 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { structLevel2.withColumn("a", $"a".dropFields("x.b")) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`x`", "fields" -> "`a`")) checkError( exception = intercept[AnalysisException] { structLevel3.withColumn("a", $"a".dropFields("a.x.b")) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`x`", "fields" -> "`a`")) } @@ -1894,7 +1894,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { structLevel1.withColumn("a", $"a".dropFields("b.a")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"update_fields(a.b, dropfield())\"", "paramIndex" -> "first", @@ -1920,7 +1920,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { structLevel2.withColumn("a", $"a".dropFields("a.b")) }, - errorClass = "AMBIGUOUS_REFERENCE_TO_FIELDS", + condition = "AMBIGUOUS_REFERENCE_TO_FIELDS", sqlState = "42000", parameters = Map("field" -> "`a`", "count" -> "2") ) @@ -1968,7 +1968,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { structLevel1.withColumn("a", $"a".dropFields("a", "b", "c")) }, - errorClass = "DATATYPE_MISMATCH.CANNOT_DROP_ALL_FIELDS", + condition = "DATATYPE_MISMATCH.CANNOT_DROP_ALL_FIELDS", parameters = Map("sqlExpr" -> "\"update_fields(a, dropfield(), dropfield(), dropfield())\""), context = ExpectedContext( fragment = "dropFields", @@ -2158,14 +2158,14 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { mixedCaseStructLevel2.withColumn("a", $"a".dropFields("A.a")) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`A`", "fields" -> "`a`, `B`")) checkError( exception = intercept[AnalysisException] { mixedCaseStructLevel2.withColumn("a", $"a".dropFields("b.a")) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`b`", "fields" -> "`a`, `B`")) } } @@ -2243,7 +2243,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { sql("SELECT named_struct('a', 1, 'b', 2) struct_col") .select($"struct_col".dropFields("a", "b")) }, - errorClass = "DATATYPE_MISMATCH.CANNOT_DROP_ALL_FIELDS", + condition = "DATATYPE_MISMATCH.CANNOT_DROP_ALL_FIELDS", parameters = Map("sqlExpr" -> "\"update_fields(struct_col, dropfield(), dropfield())\""), context = ExpectedContext( fragment = "dropFields", @@ -2270,7 +2270,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") .select($"struct_col".dropFields("a.c")) }, - errorClass = "AMBIGUOUS_REFERENCE_TO_FIELDS", + condition = "AMBIGUOUS_REFERENCE_TO_FIELDS", sqlState = "42000", parameters = Map("field" -> "`a`", "count" -> "2") ) @@ -2420,7 +2420,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { structLevel1.select($"a".withField("d", lit(4)).withField("e", $"a.d" + 1).as("a")) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`d`", "fields" -> "`a`, `b`, `c`"), context = ExpectedContext( fragment = "$", @@ -2476,7 +2476,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { .select($"a".dropFields("c").as("a")) .select($"a".withField("z", $"a.c")).as("a") }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`c`", "fields" -> "`a`, `b`"), context = ExpectedContext( fragment = "$", @@ -2575,7 +2575,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { booleanDf.select(assert_true($"cond", lit(null.asInstanceOf[String]))).collect() }, - errorClass = "USER_RAISED_EXCEPTION", + condition = "USER_RAISED_EXCEPTION", parameters = Map("errorMessage" -> "null")) val nullDf = Seq(("first row", None), ("second row", Some(true))).toDF("n", "cond") @@ -2587,7 +2587,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { nullDf.select(assert_true($"cond", $"n")).collect() }, - errorClass = "USER_RAISED_EXCEPTION", + condition = "USER_RAISED_EXCEPTION", parameters = Map("errorMessage" -> "first row")) // assert_true(condition) @@ -2607,14 +2607,14 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { strDf.select(raise_error(lit(null.asInstanceOf[String]))).collect() }, - errorClass = "USER_RAISED_EXCEPTION", + condition = "USER_RAISED_EXCEPTION", parameters = Map("errorMessage" -> "null")) checkError( exception = intercept[SparkRuntimeException] { strDf.select(raise_error($"a")).collect() }, - errorClass = "USER_RAISED_EXCEPTION", + condition = "USER_RAISED_EXCEPTION", parameters = Map("errorMessage" -> "hello")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala index bb3c00d238ca6..6589282fd3a51 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -51,7 +51,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq("1").toDS().select(from_csv($"value", lit("ARRAY"), Map[String, String]().asJava)) }, - errorClass = "INVALID_SCHEMA.NON_STRUCT_TYPE", + condition = "INVALID_SCHEMA.NON_STRUCT_TYPE", parameters = Map( "inputSchema" -> "\"ARRAY\"", "dataType" -> "\"ARRAY\"" @@ -63,7 +63,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq("1").toDF("csv").selectExpr(s"from_csv(csv, 'ARRAY')") }, - errorClass = "INVALID_SCHEMA.NON_STRUCT_TYPE", + condition = "INVALID_SCHEMA.NON_STRUCT_TYPE", parameters = Map( "inputSchema" -> "\"ARRAY\"", "dataType" -> "\"ARRAY\"" @@ -109,7 +109,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkUpgradeException] { df2.collect() }, - errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER", + condition = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER", parameters = Map( "datetime" -> "'2013-111-11 12:13:14'", "config" -> "\"spark.sql.legacy.timeParserPolicy\"")) @@ -184,7 +184,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkUnsupportedOperationException] { df.select(from_csv(to_csv($"value"), schema, options)).collect() }, - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> toSQLType(valueType)) ) } @@ -343,7 +343,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { df.select(from_csv($"value", schema, Map("mode" -> "FAILFAST"))).collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map("badRecord" -> "[null,null,\"]", "failFastMode" -> "FAILFAST") ) @@ -351,7 +351,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(from_csv($"value", schema, Map("mode" -> "DROPMALFORMED"))).collect() }, - errorClass = "_LEGACY_ERROR_TEMP_1099", + condition = "_LEGACY_ERROR_TEMP_1099", parameters = Map( "funcName" -> "from_csv", "mode" -> "DROPMALFORMED", @@ -433,7 +433,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { Seq(("1", "i int")).toDF("csv", "schema") .select(from_csv($"csv", $"schema", options)).collect() }, - errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL", + condition = "INVALID_SCHEMA.NON_STRING_LITERAL", parameters = Map("inputSchema" -> "\"schema\""), context = ExpectedContext(fragment = "from_csv", getCurrentClassCallSitePattern) ) @@ -442,7 +442,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq("1").toDF("csv").select(from_csv($"csv", lit(1), options)).collect() }, - errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL", + condition = "INVALID_SCHEMA.NON_STRING_LITERAL", parameters = Map("inputSchema" -> "\"1\""), context = ExpectedContext(fragment = "from_csv", getCurrentClassCallSitePattern) ) @@ -493,14 +493,14 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { df.selectExpr("parsed.a").collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map("badRecord" -> "[1,null]", "failFastMode" -> "FAILFAST")) checkError( exception = intercept[SparkException] { df.selectExpr("parsed.b").collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map("badRecord" -> "[1,null]", "failFastMode" -> "FAILFAST")) } } @@ -753,7 +753,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(to_csv($"value")).collect() }, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE", parameters = Map( "functionName" -> "`to_csv`", "dataType" -> "\"STRUCT\"", @@ -765,7 +765,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkUnsupportedOperationException] { df.select(from_csv(lit("data"), valueSchema, Map.empty[String, String])).collect() }, - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> "\"VARIANT\"") ) } @@ -776,7 +776,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(to_csv($"value")).collect() }, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE", parameters = Map( "functionName" -> "`to_csv`", "dataType" -> "\"INT\"", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 66b1883b91d5f..0e9d34c3bd96a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -645,7 +645,7 @@ class DataFrameAggregateSuite extends QueryTest } checkError( exception = error, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE", parameters = Map( "functionName" -> "`collect_set`", "dataType" -> "\"MAP\"", @@ -725,7 +725,7 @@ class DataFrameAggregateSuite extends QueryTest exception = intercept[AnalysisException] { testData.groupBy(sum($"key")).count() }, - errorClass = "GROUP_BY_AGGREGATE", + condition = "GROUP_BY_AGGREGATE", parameters = Map("sqlExpr" -> "sum(key)"), context = ExpectedContext(fragment = "sum", callSitePattern = getCurrentClassCallSitePattern) ) @@ -985,7 +985,7 @@ class DataFrameAggregateSuite extends QueryTest } checkError( exception = error, - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", sqlState = None, parameters = Map( "functionName" -> "`max_by`", @@ -1055,7 +1055,7 @@ class DataFrameAggregateSuite extends QueryTest } checkError( exception = error, - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", sqlState = None, parameters = Map( "functionName" -> "`min_by`", @@ -1186,7 +1186,7 @@ class DataFrameAggregateSuite extends QueryTest exception = intercept[AnalysisException] { sql("SELECT COUNT_IF(x) FROM tempView") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"count_if(x)\"", @@ -1350,7 +1350,7 @@ class DataFrameAggregateSuite extends QueryTest exception = intercept[AnalysisException] { Seq(Tuple1(Seq(1))).toDF("col").groupBy(struct($"col.a")).count() }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"col[a]\"", "paramIndex" -> "second", @@ -1924,7 +1924,7 @@ class DataFrameAggregateSuite extends QueryTest ) .collect() }, - errorClass = "HLL_INVALID_LG_K", + condition = "HLL_INVALID_LG_K", parameters = Map( "function" -> "`hll_sketch_agg`", "min" -> "4", @@ -1940,7 +1940,7 @@ class DataFrameAggregateSuite extends QueryTest ) .collect() }, - errorClass = "HLL_INVALID_LG_K", + condition = "HLL_INVALID_LG_K", parameters = Map( "function" -> "`hll_sketch_agg`", "min" -> "4", @@ -1963,7 +1963,7 @@ class DataFrameAggregateSuite extends QueryTest .withColumn("union", hll_union("hllsketch_left", "hllsketch_right")) .collect() }, - errorClass = "HLL_UNION_DIFFERENT_LG_K", + condition = "HLL_UNION_DIFFERENT_LG_K", parameters = Map( "left" -> "12", "right" -> "20", @@ -1986,7 +1986,7 @@ class DataFrameAggregateSuite extends QueryTest ) .collect() }, - errorClass = "HLL_UNION_DIFFERENT_LG_K", + condition = "HLL_UNION_DIFFERENT_LG_K", parameters = Map( "left" -> "12", "right" -> "20", @@ -2007,7 +2007,7 @@ class DataFrameAggregateSuite extends QueryTest |""".stripMargin) checkAnswer(res, Nil) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"hll_sketch_agg(value, text)\"", "paramIndex" -> "second", @@ -2036,7 +2036,7 @@ class DataFrameAggregateSuite extends QueryTest |""".stripMargin) checkAnswer(res, Nil) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"hll_union_agg(sketch, Hll_4)\"", "paramIndex" -> "second", @@ -2078,7 +2078,7 @@ class DataFrameAggregateSuite extends QueryTest | cte1 join cte2 on cte1.id = cte2.id |""".stripMargin).collect() }, - errorClass = "HLL_UNION_DIFFERENT_LG_K", + condition = "HLL_UNION_DIFFERENT_LG_K", parameters = Map( "left" -> "12", "right" -> "20", @@ -2114,7 +2114,7 @@ class DataFrameAggregateSuite extends QueryTest |group by 1 |""".stripMargin).collect() }, - errorClass = "HLL_UNION_DIFFERENT_LG_K", + condition = "HLL_UNION_DIFFERENT_LG_K", parameters = Map( "left" -> "12", "right" -> "20", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAsOfJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAsOfJoinSuite.scala index a03f083123558..5f0ae918524e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAsOfJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAsOfJoinSuite.scala @@ -108,7 +108,7 @@ class DataFrameAsOfJoinSuite extends QueryTest joinType = "inner", tolerance = df1.col("b"), allowExactMatches = true, direction = "backward") }, - errorClass = "AS_OF_JOIN.TOLERANCE_IS_UNFOLDABLE", + condition = "AS_OF_JOIN.TOLERANCE_IS_UNFOLDABLE", parameters = Map.empty) } @@ -120,7 +120,7 @@ class DataFrameAsOfJoinSuite extends QueryTest joinType = "inner", tolerance = lit(-1), allowExactMatches = true, direction = "backward") }, - errorClass = "AS_OF_JOIN.TOLERANCE_IS_NON_NEGATIVE", + condition = "AS_OF_JOIN.TOLERANCE_IS_NON_NEGATIVE", parameters = Map.empty) } @@ -133,7 +133,7 @@ class DataFrameAsOfJoinSuite extends QueryTest joinType = "inner", tolerance = lit(-1), allowExactMatches = true, direction = direction) }, - errorClass = "AS_OF_JOIN.UNSUPPORTED_DIRECTION", + condition = "AS_OF_JOIN.UNSUPPORTED_DIRECTION", sqlState = "42604", parameters = Map( "direction" -> direction, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 37461a0a05c83..d488adc5ac3d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -166,7 +166,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df4.select(map_from_arrays($"k", $"v")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"map_from_arrays(k, v)\"", "paramIndex" -> "first", @@ -185,7 +185,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { df5.select(map_from_arrays($"k", $"v")).collect() }, - errorClass = "NULL_MAP_KEY", + condition = "NULL_MAP_KEY", parameters = Map.empty ) @@ -344,7 +344,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { var expr = nullifzero(map(lit(1), lit("a"))) checkError( intercept[AnalysisException](df.select(expr)), - errorClass = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", parameters = Map( "left" -> "\"MAP\"", "right" -> "\"INT\"", @@ -360,7 +360,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { expr = nullifzero(array(lit(1), lit(2))) checkError( intercept[AnalysisException](df.select(expr)), - errorClass = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", parameters = Map( "left" -> "\"ARRAY\"", "right" -> "\"INT\"", @@ -376,7 +376,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { expr = nullifzero(Literal.create(20201231, DateType)) checkError( intercept[AnalysisException](df.select(expr)), - errorClass = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", parameters = Map( "left" -> "\"DATE\"", "right" -> "\"INT\"", @@ -422,7 +422,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { var expr = zeroifnull(map(lit(1), lit("a"))) checkError( intercept[AnalysisException](df.select(expr)), - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", parameters = Map( "functionName" -> "`coalesce`", "dataType" -> "(\"MAP\" or \"INT\")", @@ -438,7 +438,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { expr = zeroifnull(array(lit(1), lit(2))) checkError( intercept[AnalysisException](df.select(expr)), - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", parameters = Map( "functionName" -> "`coalesce`", "dataType" -> "(\"ARRAY\" or \"INT\")", @@ -454,7 +454,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { expr = zeroifnull(Literal.create(20201231, DateType)) checkError( intercept[AnalysisException](df.select(expr)), - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", parameters = Map( "functionName" -> "`coalesce`", "dataType" -> "(\"DATE\" or \"INT\")", @@ -886,7 +886,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(array_sort(col("a"), (x, y) => x - y)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> """"array_sort\(a, lambdafunction\(`-`\(x_\d+, y_\d+\), x_\d+, y_\d+\)\)"""", "paramIndex" -> "first", @@ -953,7 +953,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = error, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"sort_array(a, true)\"", "paramIndex" -> "first", @@ -989,7 +989,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("array_sort(a)").collect() }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"array_sort(a, lambdafunction((IF(((left IS NULL) AND (right IS NULL)), 0, (IF((left IS NULL), 1, (IF((right IS NULL), -1, (IF((left < right), -1, (IF((left > right), 1, 0)))))))))), left, right))\"", @@ -1302,7 +1302,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("map_contains_key(a, null)").collect() }, - errorClass = "DATATYPE_MISMATCH.NULL_TYPE", + condition = "DATATYPE_MISMATCH.NULL_TYPE", parameters = Map( "sqlExpr" -> "\"map_contains_key(a, NULL)\"", "functionName" -> "`map_contains_key`"), @@ -1379,7 +1379,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.selectExpr("map_concat(map1, map2)").collect() }, - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", sqlState = None, parameters = Map( "sqlExpr" -> "\"map_concat(map1, map2)\"", @@ -1395,7 +1395,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.select(map_concat($"map1", $"map2")).collect() }, - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", sqlState = None, parameters = Map( "sqlExpr" -> "\"map_concat(map1, map2)\"", @@ -1411,7 +1411,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.selectExpr("map_concat(map1, 12)").collect() }, - errorClass = "DATATYPE_MISMATCH.MAP_CONCAT_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.MAP_CONCAT_DIFF_TYPES", sqlState = None, parameters = Map( "sqlExpr" -> "\"map_concat(map1, 12)\"", @@ -1427,7 +1427,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.select(map_concat($"map1", lit(12))).collect() }, - errorClass = "DATATYPE_MISMATCH.MAP_CONCAT_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.MAP_CONCAT_DIFF_TYPES", sqlState = None, parameters = Map( "sqlExpr" -> "\"map_concat(map1, 12)\"", @@ -1498,7 +1498,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { wrongTypeDF.select(map_from_entries($"a")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"map_from_entries(a)\"", "paramIndex" -> "first", @@ -1542,7 +1542,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(array_contains(df("a"), null)) }, - errorClass = "DATATYPE_MISMATCH.NULL_TYPE", + condition = "DATATYPE_MISMATCH.NULL_TYPE", parameters = Map( "sqlExpr" -> "\"array_contains(a, NULL)\"", "functionName" -> "`array_contains`" @@ -1556,7 +1556,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("array_contains(a, null)") }, - errorClass = "DATATYPE_MISMATCH.NULL_TYPE", + condition = "DATATYPE_MISMATCH.NULL_TYPE", parameters = Map( "sqlExpr" -> "\"array_contains(a, NULL)\"", "functionName" -> "`array_contains`" @@ -1567,7 +1567,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("array_contains(null, 1)") }, - errorClass = "DATATYPE_MISMATCH.NULL_TYPE", + condition = "DATATYPE_MISMATCH.NULL_TYPE", parameters = Map( "sqlExpr" -> "\"array_contains(NULL, 1)\"", "functionName" -> "`array_contains`" @@ -1623,7 +1623,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { OneRowRelation().selectExpr("array_contains(array(1), 'foo')") }, - errorClass = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_contains(array(1), foo)\"", "functionName" -> "`array_contains`", @@ -1639,7 +1639,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { OneRowRelation().selectExpr("array_contains('a string', 'foo')") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"array_contains(a string, foo)\"", "paramIndex" -> "first", @@ -1688,7 +1688,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("select arrays_overlap(array(1, 2, 3), array('a', 'b', 'c'))") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"arrays_overlap(array(1, 2, 3), array(a, b, c))\"", "functionName" -> "`arrays_overlap`", @@ -1704,7 +1704,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("select arrays_overlap(null, null)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"arrays_overlap(NULL, NULL)\"", "functionName" -> "`arrays_overlap`", @@ -1719,7 +1719,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("select arrays_overlap(map(1, 2), map(3, 4))") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"arrays_overlap(map(1, 2), map(3, 4))\"", "functionName" -> "`arrays_overlap`", @@ -1794,7 +1794,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { idf.selectExpr("array_join(x, 1)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"array_join(x, 1)\"", "paramIndex" -> "second", @@ -1808,7 +1808,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { idf.selectExpr("array_join(x, ', ', 1)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"array_join(x, , , 1)\"", "paramIndex" -> "third", @@ -1924,7 +1924,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq((true, false)).toDF().selectExpr("sequence(_1, _2)") }, - errorClass = "DATATYPE_MISMATCH.SEQUENCE_WRONG_INPUT_TYPES", + condition = "DATATYPE_MISMATCH.SEQUENCE_WRONG_INPUT_TYPES", parameters = Map( "sqlExpr" -> "\"sequence(_1, _2)\"", "functionName" -> "`sequence`", @@ -1938,7 +1938,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq((true, false, 42)).toDF().selectExpr("sequence(_1, _2, _3)") }, - errorClass = "DATATYPE_MISMATCH.SEQUENCE_WRONG_INPUT_TYPES", + condition = "DATATYPE_MISMATCH.SEQUENCE_WRONG_INPUT_TYPES", parameters = Map( "sqlExpr" -> "\"sequence(_1, _2, _3)\"", "functionName" -> "`sequence`", @@ -1952,7 +1952,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq((1, 2, 0.5)).toDF().selectExpr("sequence(_1, _2, _3)") }, - errorClass = "DATATYPE_MISMATCH.SEQUENCE_WRONG_INPUT_TYPES", + condition = "DATATYPE_MISMATCH.SEQUENCE_WRONG_INPUT_TYPES", parameters = Map( "sqlExpr" -> "\"sequence(_1, _2, _3)\"", "functionName" -> "`sequence`", @@ -2068,7 +2068,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("select reverse(struct(1, 'a'))") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"reverse(struct(1, a))\"", "paramIndex" -> "first", @@ -2083,7 +2083,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("select reverse(map(1, 'a'))") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"reverse(map(1, a))\"", "paramIndex" -> "first", @@ -2169,7 +2169,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq((null, "a")).toDF().selectExpr("array_position(_1, _2)") }, - errorClass = "DATATYPE_MISMATCH.NULL_TYPE", + condition = "DATATYPE_MISMATCH.NULL_TYPE", parameters = Map( "sqlExpr" -> "\"array_position(_1, _2)\"", "functionName" -> "`array_position`" @@ -2181,7 +2181,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq(("a string element", null)).toDF().selectExpr("array_position(_1, _2)") }, - errorClass = "DATATYPE_MISMATCH.NULL_TYPE", + condition = "DATATYPE_MISMATCH.NULL_TYPE", parameters = Map( "sqlExpr" -> "\"array_position(_1, _2)\"", "functionName" -> "`array_position`" @@ -2193,7 +2193,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq(("a string element", "a")).toDF().selectExpr("array_position(_1, _2)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"array_position(_1, _2)\"", "paramIndex" -> "first", @@ -2208,7 +2208,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { OneRowRelation().selectExpr("array_position(array(1), '1')") }, - errorClass = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_position(array(1), 1)\"", "functionName" -> "`array_position`", @@ -2281,7 +2281,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq(("a string element", 1)).toDF().selectExpr("element_at(_1, _2)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"element_at(_1, _2)\"", "paramIndex" -> "first", @@ -2311,7 +2311,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { OneRowRelation().selectExpr("element_at(array('a', 'b'), 1L)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"element_at(array(a, b), 1)\"", "paramIndex" -> "second", @@ -2358,7 +2358,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { OneRowRelation().selectExpr("element_at(map(1, 'a', 2, 'b'), '1')") }, - errorClass = "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"element_at(map(1, a, 2, b), 1)\"", "functionName" -> "`element_at`", @@ -2440,7 +2440,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df6.select(array_union($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_union(a, b)\"", "functionName" -> "`array_union`", @@ -2456,7 +2456,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df6.selectExpr("array_union(a, b)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_union(a, b)\"", "functionName" -> "`array_union`", @@ -2475,7 +2475,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df7.select(array_union($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_union(a, b)\"", "functionName" -> "`array_union`", @@ -2489,7 +2489,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df7.selectExpr("array_union(a, b)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_union(a, b)\"", "functionName" -> "`array_union`", @@ -2508,7 +2508,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df8.select(array_union($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_union(a, b)\"", "functionName" -> "`array_union`", @@ -2522,7 +2522,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df8.selectExpr("array_union(a, b)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_union(a, b)\"", "functionName" -> "`array_union`", @@ -2609,7 +2609,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("concat(i1, i2, null)") }, - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"concat(i1, i2, NULL)\"", "functionName" -> "`concat`", @@ -2622,7 +2622,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("concat(i1, array(i1, i2))") }, - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"concat(i1, array(i1, i2))\"", "functionName" -> "`concat`", @@ -2635,7 +2635,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("concat(map(1, 2), map(3, 4))") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"concat(map(1, 2), map(3, 4))\"", "paramIndex" -> "first", @@ -2746,7 +2746,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { oneRowDF.select(flatten($"arr")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"flatten(arr)\"", "paramIndex" -> "first", @@ -2761,7 +2761,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { oneRowDF.select(flatten($"i")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"flatten(i)\"", "paramIndex" -> "first", @@ -2776,7 +2776,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { oneRowDF.select(flatten($"s")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"flatten(s)\"", "paramIndex" -> "first", @@ -2791,7 +2791,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { oneRowDF.selectExpr("flatten(null)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"flatten(NULL)\"", "paramIndex" -> "first", @@ -2887,7 +2887,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { invalidTypeDF.select(array_repeat($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"array_repeat(a, b)\"", "paramIndex" -> "second", @@ -2902,7 +2902,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { invalidTypeDF.select(array_repeat($"a", lit("1"))) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"array_repeat(a, 1)\"", "paramIndex" -> "second", @@ -2917,7 +2917,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { invalidTypeDF.selectExpr("array_repeat(a, 1.0)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"array_repeat(a, 1.0)\"", "paramIndex" -> "second", @@ -2968,7 +2968,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq(("a string element", "a")).toDF().selectExpr("array_prepend(_1, _2)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "paramIndex" -> "first", "sqlExpr" -> "\"array_prepend(_1, _2)\"", @@ -2980,7 +2980,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { OneRowRelation().selectExpr("array_prepend(array(1, 2), '1')") }, - errorClass = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_prepend(array(1, 2), 1)\"", "functionName" -> "`array_prepend`", @@ -3084,7 +3084,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq(("a string element", "a")).toDF().selectExpr("array_remove(_1, _2)") }, - errorClass = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_remove(_1, _2)\"", "functionName" -> "`array_remove`", @@ -3099,7 +3099,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { OneRowRelation().selectExpr("array_remove(array(1, 2), '1')") }, - errorClass = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_remove(array(1, 2), 1)\"", "functionName" -> "`array_remove`", @@ -3232,7 +3232,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df6.select(array_except($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_except(a, b)\"", "functionName" -> "`array_except`", @@ -3247,7 +3247,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df6.selectExpr("array_except(a, b)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_except(a, b)\"", "functionName" -> "`array_except`", @@ -3262,7 +3262,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df7.select(array_except($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_except(a, b)\"", "functionName" -> "`array_except`", @@ -3277,7 +3277,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df7.selectExpr("array_except(a, b)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_except(a, b)\"", "functionName" -> "`array_except`", @@ -3292,7 +3292,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df8.select(array_except($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_except(a, b)\"", "functionName" -> "`array_except`", @@ -3307,7 +3307,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df8.selectExpr("array_except(a, b)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_except(a, b)\"", "functionName" -> "`array_except`", @@ -3322,7 +3322,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df9.select(array_except($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_except(a, b)\"", "functionName" -> "`array_except`", @@ -3337,7 +3337,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df9.selectExpr("array_except(a, b)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_except(a, b)\"", "functionName" -> "`array_except`", @@ -3393,7 +3393,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df6.select(array_intersect($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_intersect(a, b)\"", "functionName" -> "`array_intersect`", @@ -3408,7 +3408,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df6.selectExpr("array_intersect(a, b)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_intersect(a, b)\"", "functionName" -> "`array_intersect`", @@ -3424,7 +3424,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df7.select(array_intersect($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_intersect(a, b)\"", "functionName" -> "`array_intersect`", @@ -3439,7 +3439,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df7.selectExpr("array_intersect(a, b)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_intersect(a, b)\"", "functionName" -> "`array_intersect`", @@ -3455,7 +3455,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df8.select(array_intersect($"a", $"b")) }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_intersect(a, b)\"", "functionName" -> "`array_intersect`", @@ -3472,7 +3472,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df8.selectExpr("array_intersect(a, b)") }, - errorClass = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_ARRAY_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array_intersect(a, b)\"", "functionName" -> "`array_intersect`", @@ -3506,7 +3506,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { df5.selectExpr("array_insert(a, b, c)").show() }, - errorClass = "INVALID_INDEX_OF_ZERO", + condition = "INVALID_INDEX_OF_ZERO", parameters = Map.empty, context = ExpectedContext( fragment = "array_insert(a, b, c)", @@ -3748,7 +3748,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("transform(s, (x, y, z) -> x + y + z)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "3", "actualNumArgs" -> "1"), context = ExpectedContext( fragment = "(x, y, z) -> x + y + z", @@ -3758,7 +3758,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](df.selectExpr("transform(i, x -> x)")), - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"transform(i, lambdafunction(x, x))\"", @@ -3774,7 +3774,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](df.selectExpr("transform(a, x -> x)")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), context = ExpectedContext( @@ -3832,7 +3832,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("map_filter(s, (x, y, z) -> x + y + z)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "3", "actualNumArgs" -> "2"), context = ExpectedContext( fragment = "(x, y, z) -> x + y + z", @@ -3844,7 +3844,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("map_filter(s, x -> x)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "1", "actualNumArgs" -> "2"), context = ExpectedContext( fragment = "x -> x", @@ -3856,7 +3856,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("map_filter(i, (k, v) -> k > v)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"map_filter(i, lambdafunction((k > v), k, v))\"", @@ -3873,7 +3873,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(map_filter(col("i"), (k, v) => k > v)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", matchPVals = true, parameters = Map( "sqlExpr" -> """"map_filter\(i, lambdafunction\(`>`\(x_\d+, y_\d+\), x_\d+, y_\d+\)\)"""", @@ -3887,7 +3887,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](df.selectExpr("map_filter(a, (k, v) -> k > v)")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), context = ExpectedContext( @@ -4029,7 +4029,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("filter(s, (x, y, z) -> x + y)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "3", "actualNumArgs" -> "1"), context = ExpectedContext( fragment = "(x, y, z) -> x + y", @@ -4041,7 +4041,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("filter(i, x -> x)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"filter(i, lambdafunction(x, x))\"", @@ -4058,7 +4058,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(filter(col("i"), x => x)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", matchPVals = true, parameters = Map( "sqlExpr" -> """"filter\(i, lambdafunction\(x_\d+, x_\d+\)\)"""", @@ -4073,7 +4073,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("filter(s, x -> x)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"filter(s, lambdafunction(namedlambdavariable(), namedlambdavariable()))\"", "paramIndex" -> "second", @@ -4089,7 +4089,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(filter(col("s"), x => x)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"filter(s, lambdafunction(namedlambdavariable(), namedlambdavariable()))\"", "paramIndex" -> "second", @@ -4103,7 +4103,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](df.selectExpr("filter(a, x -> x)")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), context = ExpectedContext( @@ -4217,7 +4217,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("exists(s, (x, y) -> x + y)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "2", "actualNumArgs" -> "1"), context = ExpectedContext( fragment = "(x, y) -> x + y", @@ -4229,7 +4229,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("exists(i, x -> x)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"exists(i, lambdafunction(x, x))\"", @@ -4246,7 +4246,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(exists(col("i"), x => x)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", matchPVals = true, parameters = Map( "sqlExpr" -> """"exists\(i, lambdafunction\(x_\d+, x_\d+\)\)"""", @@ -4261,7 +4261,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("exists(s, x -> x)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"exists(s, lambdafunction(namedlambdavariable(), namedlambdavariable()))\"", "paramIndex" -> "second", @@ -4278,7 +4278,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(exists(df("s"), x => x)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"exists(s, lambdafunction(namedlambdavariable(), namedlambdavariable()))\"", "paramIndex" -> "second", @@ -4290,7 +4290,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](df.selectExpr("exists(a, x -> x)")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), context = ExpectedContext( @@ -4418,7 +4418,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("forall(s, (x, y) -> x + y)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "2", "actualNumArgs" -> "1"), context = ExpectedContext( fragment = "(x, y) -> x + y", @@ -4430,7 +4430,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("forall(i, x -> x)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"forall(i, lambdafunction(x, x))\"", @@ -4447,7 +4447,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(forall(col("i"), x => x)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", matchPVals = true, parameters = Map( "sqlExpr" -> """"forall\(i, lambdafunction\(x_\d+, x_\d+\)\)"""", @@ -4462,7 +4462,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("forall(s, x -> x)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"forall(s, lambdafunction(namedlambdavariable(), namedlambdavariable()))\"", "paramIndex" -> "second", @@ -4478,7 +4478,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(forall(col("s"), x => x)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"forall(s, lambdafunction(namedlambdavariable(), namedlambdavariable()))\"", "paramIndex" -> "second", @@ -4490,7 +4490,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](df.selectExpr("forall(a, x -> x)")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), context = ExpectedContext( @@ -4500,7 +4500,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](df.select(forall(col("a"), x => x))), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), queryContext = Array( ExpectedContext(fragment = "col", callSitePattern = getCurrentClassCallSitePattern))) @@ -4689,7 +4689,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr(s"$agg(s, '', x -> x)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "1", "actualNumArgs" -> "2"), context = ExpectedContext( fragment = "x -> x", @@ -4701,7 +4701,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr(s"$agg(s, '', (acc, x) -> x, (acc, x) -> x)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "2", "actualNumArgs" -> "1"), context = ExpectedContext( fragment = "(acc, x) -> x", @@ -4715,7 +4715,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr(s"$agg(i, 0, (acc, x) -> x)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> s""""$agg(i, 0, lambdafunction(x, acc, x), lambdafunction(id, id))"""", @@ -4734,7 +4734,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(aggregate(col("i"), lit(0), (_, x) => x)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", matchPVals = true, parameters = Map( "sqlExpr" -> """"aggregate\(i, 0, lambdafunction\(y_\d+, x_\d+, y_\d+\), lambdafunction\(x_\d+, x_\d+\)\)"""", @@ -4752,7 +4752,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr(s"$agg(s, 0, (acc, x) -> x)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> s""""$agg(s, 0, lambdafunction(namedlambdavariable(), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))"""", "paramIndex" -> "third", @@ -4772,7 +4772,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(aggregate(col("s"), lit(0), (acc, x) => x)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> """"aggregate(s, 0, lambdafunction(namedlambdavariable(), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))"""", "paramIndex" -> "third", @@ -4788,7 +4788,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](df.selectExpr(s"$agg(a, 0, (acc, x) -> x)")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), context = ExpectedContext( @@ -4853,7 +4853,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("map_zip_with(mii, mis, (x, y) -> x + y)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "2", "actualNumArgs" -> "3"), context = ExpectedContext( fragment = "(x, y) -> x + y", @@ -4865,7 +4865,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("map_zip_with(mis, mmi, (x, y, z) -> concat(x, y, z))") }, - errorClass = "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"map_zip_with(mis, mmi, lambdafunction(concat(x, y, z), x, y, z))\"", "functionName" -> "`map_zip_with`", @@ -4881,7 +4881,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(map_zip_with(df("mis"), col("mmi"), (x, y, z) => concat(x, y, z))) }, - errorClass = "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", matchPVals = true, parameters = Map( "sqlExpr" -> """"map_zip_with\(mis, mmi, lambdafunction\(concat\(x_\d+, y_\d+, z_\d+\), x_\d+, y_\d+, z_\d+\)\)"""", @@ -4896,7 +4896,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("map_zip_with(i, mis, (x, y, z) -> concat(x, y, z))") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"map_zip_with(i, mis, lambdafunction(concat(x, y, z), x, y, z))\"", @@ -4913,7 +4913,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(map_zip_with(col("i"), col("mis"), (x, y, z) => concat(x, y, z))) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", matchPVals = true, parameters = Map( "sqlExpr" -> """"map_zip_with\(i, mis, lambdafunction\(concat\(x_\d+, y_\d+, z_\d+\), x_\d+, y_\d+, z_\d+\)\)"""", @@ -4928,7 +4928,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("map_zip_with(mis, i, (x, y, z) -> concat(x, y, z))") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"map_zip_with(mis, i, lambdafunction(concat(x, y, z), x, y, z))\"", @@ -4945,7 +4945,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(map_zip_with(col("mis"), col("i"), (x, y, z) => concat(x, y, z))) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", matchPVals = true, parameters = Map( "sqlExpr" -> """"map_zip_with\(mis, i, lambdafunction\(concat\(x_\d+, y_\d+, z_\d+\), x_\d+, y_\d+, z_\d+\)\)"""", @@ -4960,7 +4960,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("map_zip_with(mmi, mmi, (x, y, z) -> x)") }, - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"map_zip_with(mmi, mmi, lambdafunction(x, x, y, z))\"", @@ -5080,7 +5080,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfExample1.selectExpr("transform_keys(i, k -> k)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "1", "actualNumArgs" -> "2"), context = ExpectedContext( fragment = "k -> k", @@ -5092,7 +5092,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfExample1.selectExpr("transform_keys(i, (k, v, x) -> k + 1)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "3", "actualNumArgs" -> "2"), context = ExpectedContext( fragment = "(k, v, x) -> k + 1", @@ -5104,7 +5104,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { dfExample1.selectExpr("transform_keys(i, (k, v) -> v)").show() }, - errorClass = "NULL_MAP_KEY", + condition = "NULL_MAP_KEY", parameters = Map.empty ) @@ -5112,7 +5112,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { dfExample1.select(transform_keys(col("i"), (k, v) => v)).show() }, - errorClass = "NULL_MAP_KEY", + condition = "NULL_MAP_KEY", parameters = Map.empty ) @@ -5120,7 +5120,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfExample2.selectExpr("transform_keys(j, (k, v) -> k + 1)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"transform_keys(j, lambdafunction((k + 1), k, v))\"", @@ -5356,7 +5356,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfExample1.selectExpr("transform_values(i, k -> k)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "1", "actualNumArgs" -> "2"), context = ExpectedContext( fragment = "k -> k", @@ -5368,7 +5368,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfExample2.selectExpr("transform_values(j, (k, v, x) -> k + 1)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map("expectedNumArgs" -> "3", "actualNumArgs" -> "2"), context = ExpectedContext( fragment = "(k, v, x) -> k + 1", @@ -5380,7 +5380,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfExample3.selectExpr("transform_values(x, (k, v) -> k + 1)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"transform_values(x, lambdafunction((k + 1), k, v))\"", @@ -5397,7 +5397,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfExample3.select(transform_values(col("x"), (k, v) => k + 1)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", matchPVals = true, parameters = Map( "sqlExpr" -> @@ -5480,7 +5480,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("zip_with(a1, a2, x -> x)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", + condition = "INVALID_LAMBDA_FUNCTION_CALL.NUM_ARGS_MISMATCH", parameters = Map( "expectedNumArgs" -> "1", "actualNumArgs" -> "2"), @@ -5494,7 +5494,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("zip_with(a1, a2, (x, x) -> x)") }, - errorClass = "INVALID_LAMBDA_FUNCTION_CALL.DUPLICATE_ARG_NAMES", + condition = "INVALID_LAMBDA_FUNCTION_CALL.DUPLICATE_ARG_NAMES", parameters = Map( "args" -> "`x`, `x`", "caseSensitiveConfig" -> "\"spark.sql.caseSensitive\""), @@ -5508,7 +5508,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("zip_with(a1, a2, (acc, x) -> x, (acc, x) -> x)") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> toSQLId("zip_with"), "expectedNum" -> "3", @@ -5524,7 +5524,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("zip_with(i, a2, (acc, x) -> x)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"zip_with(i, a2, lambdafunction(x, acc, x))\"", @@ -5541,7 +5541,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(zip_with(df("i"), df("a2"), (_, x) => x)) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", matchPVals = true, parameters = Map( "sqlExpr" -> @@ -5556,7 +5556,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](df.selectExpr("zip_with(a1, a, (acc, x) -> x)")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`a`", "proposal" -> "`a1`, `a2`, `i`"), context = ExpectedContext( @@ -5609,7 +5609,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(coalesce()) }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = None, parameters = Map( "functionName" -> "`coalesce`", @@ -5622,7 +5622,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("coalesce()") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = None, parameters = Map( "functionName" -> "`coalesce`", @@ -5635,7 +5635,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(hash()) }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = None, parameters = Map( "functionName" -> "`hash`", @@ -5648,7 +5648,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("hash()") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = None, parameters = Map( "functionName" -> "`hash`", @@ -5661,7 +5661,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(xxhash64()) }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = None, parameters = Map( "functionName" -> "`xxhash64`", @@ -5674,7 +5674,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("xxhash64()") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = None, parameters = Map( "functionName" -> "`xxhash64`", @@ -5687,7 +5687,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(greatest()) }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = None, parameters = Map( "functionName" -> "`greatest`", @@ -5700,7 +5700,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("greatest()") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = None, parameters = Map( "functionName" -> "`greatest`", @@ -5713,7 +5713,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(least()) }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = None, parameters = Map( "functionName" -> "`least`", @@ -5726,7 +5726,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("least()") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", sqlState = None, parameters = Map( "functionName" -> "`least`", @@ -5742,7 +5742,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { df.select(map_from_arrays(concat($"k1", $"k2"), $"v")).show() }, - errorClass = "NULL_MAP_KEY", + condition = "NULL_MAP_KEY", parameters = Map.empty ) } @@ -5801,7 +5801,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(map($"m", lit(1))) }, - errorClass = "DATATYPE_MISMATCH.INVALID_MAP_KEY_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_MAP_KEY_TYPE", parameters = Map( "sqlExpr" -> "\"map(m, 1)\"", "keyType" -> "\"MAP\"" @@ -5842,7 +5842,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("select from_json('{\"a\":1}', 1)") }, - errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL", + condition = "INVALID_SCHEMA.NON_STRING_LITERAL", parameters = Map( "inputSchema" -> "\"1\"" ), @@ -5931,7 +5931,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { invalidDatatypeDF.select(array_compact($"a")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"array_compact(a)\"", "paramIndex" -> "first", @@ -5954,7 +5954,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.select(array_append(col("a"), col("b"))) }, - errorClass = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", parameters = Map( "functionName" -> "`array_append`", "dataType" -> "\"ARRAY\"", @@ -5973,7 +5973,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("array_append(a, b)") }, - errorClass = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", parameters = Map( "functionName" -> "`array_append`", "leftType" -> "\"ARRAY\"", @@ -6005,7 +6005,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df6.selectExpr("array_append(a, b)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"array_append(a, b)\"", "paramIndex" -> "first", @@ -6110,7 +6110,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { df1.map(r => df2.count() * r.getInt(0)).collect() }, - errorClass = "CANNOT_INVOKE_IN_TRANSFORMATIONS", + condition = "CANNOT_INVOKE_IN_TRANSFORMATIONS", parameters = Map.empty ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index ab900e2135576..e2bdf1c732078 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -284,7 +284,7 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { joined_df.na.fill("", cols = Seq("f2")) }, - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map( "name" -> "`f2`", "referenceNames" -> "[`f2`, `f2`]" @@ -304,7 +304,7 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.na.drop("any", Seq("*")) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`*`", "proposal" -> "`name`, `age`, `height`") ) } @@ -411,7 +411,7 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.na.fill("hello", Seq("col2")) }, - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map( "name" -> "`col2`", "referenceNames" -> "[`col2`, `col2`]" @@ -434,7 +434,7 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.na.drop("any", Seq("col2")) }, - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map( "name" -> "`col2`", "referenceNames" -> "[`col2`, `col2`]" @@ -540,7 +540,7 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { } checkError( exception = exception, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`aa`", "proposal" -> "`Col`.`1`, `Col`.`2`") ) } @@ -551,7 +551,7 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkUnsupportedOperationException] { df.na.replace("c1.c1-1", Map("b1" ->"a1")) }, - errorClass = "UNSUPPORTED_FEATURE.REPLACE_NESTED_COLUMN", + condition = "UNSUPPORTED_FEATURE.REPLACE_NESTED_COLUMN", parameters = Map("colName" -> "`c1`.`c1-1`") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala index b3bf9405a99f2..cf4fbe61101b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -309,7 +309,7 @@ class DataFramePivotSuite extends QueryTest with SharedSparkSession { .pivot(min($"training"), Seq("Experts")) .agg(sum($"sales.earnings")) }, - errorClass = "GROUP_BY_AGGREGATE", + condition = "GROUP_BY_AGGREGATE", parameters = Map("sqlExpr" -> "min(training)"), context = ExpectedContext(fragment = "min", callSitePattern = getCurrentClassCallSitePattern) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala index 310b5a62c908a..41f4f53ae1ee4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala @@ -483,7 +483,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { df3.join(df1, year($"df1.timeStr") === year($"df3.tsStr")) ) checkError(ex, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`df1`.`timeStr`", "proposal" -> "`df3`.`timeStr`, `df1`.`tsStr`"), context = ExpectedContext(fragment = "$", getCurrentClassCallSitePattern)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala index cbc39557ce4cc..5ff737d2b57cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala @@ -354,21 +354,21 @@ class DataFrameSetOperationsSuite extends QueryTest val df = spark.range(1).select(map(lit("key"), $"id").as("m")) checkError( exception = intercept[AnalysisException](df.intersect(df)), - errorClass = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", parameters = Map( "colName" -> "`m`", "dataType" -> "\"MAP\"") ) checkError( exception = intercept[AnalysisException](df.except(df)), - errorClass = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", parameters = Map( "colName" -> "`m`", "dataType" -> "\"MAP\"") ) checkError( exception = intercept[AnalysisException](df.distinct()), - errorClass = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", parameters = Map( "colName" -> "`m`", "dataType" -> "\"MAP\"")) @@ -376,7 +376,7 @@ class DataFrameSetOperationsSuite extends QueryTest df.createOrReplaceTempView("v") checkError( exception = intercept[AnalysisException](sql("SELECT DISTINCT m FROM v")), - errorClass = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", parameters = Map( "colName" -> "`m`", "dataType" -> "\"MAP\""), @@ -546,7 +546,7 @@ class DataFrameSetOperationsSuite extends QueryTest exception = intercept[AnalysisException] { df1.unionByName(df2) }, - errorClass = "NUM_COLUMNS_MISMATCH", + condition = "NUM_COLUMNS_MISMATCH", parameters = Map( "operator" -> "UNION", "firstNumColumns" -> "2", @@ -610,7 +610,7 @@ class DataFrameSetOperationsSuite extends QueryTest exception = intercept[AnalysisException] { df1.unionByName(df2) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c0.toLowerCase(Locale.ROOT)}`")) df1 = Seq((1, 1)).toDF("c0", "c1") df2 = Seq((1, 1)).toDF(c0, c1) @@ -618,7 +618,7 @@ class DataFrameSetOperationsSuite extends QueryTest exception = intercept[AnalysisException] { df1.unionByName(df2) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c0.toLowerCase(Locale.ROOT)}`")) } } @@ -1022,7 +1022,7 @@ class DataFrameSetOperationsSuite extends QueryTest exception = intercept[AnalysisException] { df1.unionByName(df2) }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`c`", "fields" -> "`a`, `b`")) // If right side of the nested struct has extra col. @@ -1032,7 +1032,7 @@ class DataFrameSetOperationsSuite extends QueryTest exception = intercept[AnalysisException] { df1.unionByName(df2) }, - errorClass = "INCOMPATIBLE_COLUMN_TYPE", + condition = "INCOMPATIBLE_COLUMN_TYPE", parameters = Map( "tableOrdinalNumber" -> "second", "columnOrdinalNumber" -> "third", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index 8eee8fc37661c..2f7b072fb7ece 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -143,7 +143,7 @@ class DataFrameStatSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfx.stat.freqItems(Array("num")) }, - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map( "name" -> "`num`", "referenceNames" -> "[`table1`.`num`, `table2`.`num`]" @@ -155,7 +155,7 @@ class DataFrameStatSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfx.stat.approxQuantile("num", Array(0.1), 0.0) }, - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map( "name" -> "`num`", "referenceNames" -> "[`table1`.`num`, `table2`.`num`]" @@ -167,7 +167,7 @@ class DataFrameStatSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfx.stat.cov("num", "num") }, - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map( "name" -> "`num`", "referenceNames" -> "[`table1`.`num`, `table2`.`num`]" @@ -177,7 +177,7 @@ class DataFrameStatSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dfx.stat.corr("num", "num") }, - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map( "name" -> "`num`", "referenceNames" -> "[`table1`.`num`, `table2`.`num`]" @@ -588,7 +588,7 @@ class DataFrameStatSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkIllegalArgumentException] { person2.summary("foo") }, - errorClass = "_LEGACY_ERROR_TEMP_2114", + condition = "_LEGACY_ERROR_TEMP_2114", parameters = Map("stats" -> "foo") ) @@ -596,7 +596,7 @@ class DataFrameStatSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkIllegalArgumentException] { person2.summary("foo%") }, - errorClass = "_LEGACY_ERROR_TEMP_2113", + condition = "_LEGACY_ERROR_TEMP_2113", parameters = Map("stats" -> "foo%") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index b1c41033fd760..1cd5fe8ec272d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -224,7 +224,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException] { df.select(explode($"*")) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"explode(csv)\"", "paramIndex" -> "first", @@ -568,7 +568,7 @@ class DataFrameSuite extends QueryTest testData.toDF().withColumns(Seq("newCol1", "newCOL1"), Seq(col("key") + 1, col("key") + 2)) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`newcol1`")) } @@ -588,7 +588,7 @@ class DataFrameSuite extends QueryTest testData.toDF().withColumns(Seq("newCol1", "newCol1"), Seq(col("key") + 1, col("key") + 2)) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`newCol1`")) } } @@ -631,7 +631,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException] { df1.withMetadata("x1", metadata) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`x1`", "proposal" -> "`x`") ) } @@ -1116,7 +1116,7 @@ class DataFrameSuite extends QueryTest exception = intercept[org.apache.spark.sql.AnalysisException] { df(name) }, - errorClass = "_LEGACY_ERROR_TEMP_1049", + condition = "_LEGACY_ERROR_TEMP_1049", parameters = Map("name" -> name)) } @@ -1202,7 +1202,7 @@ class DataFrameSuite extends QueryTest } checkError( exception = e, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`column1`")) // multiple duplicate columns present @@ -1213,7 +1213,7 @@ class DataFrameSuite extends QueryTest } checkError( exception = f, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`column1`")) } @@ -1245,7 +1245,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException] { insertion.write.insertInto("rdd_base") }, - errorClass = "UNSUPPORTED_INSERT.RDD_BASED", + condition = "UNSUPPORTED_INSERT.RDD_BASED", parameters = Map.empty ) @@ -1256,7 +1256,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException] { insertion.write.insertInto("indirect_ds") }, - errorClass = "UNSUPPORTED_INSERT.RDD_BASED", + condition = "UNSUPPORTED_INSERT.RDD_BASED", parameters = Map.empty ) @@ -1266,7 +1266,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException] { insertion.write.insertInto("one_row") }, - errorClass = "UNSUPPORTED_INSERT.RDD_BASED", + condition = "UNSUPPORTED_INSERT.RDD_BASED", parameters = Map.empty ) } @@ -2036,7 +2036,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException] { df.groupBy($"d", $"b").as[GroupByKey, Row] }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`d`", "proposal" -> "`a`, `b`, `c`"), context = ExpectedContext(fragment = "$", callSitePattern = getCurrentClassCallSitePattern)) } @@ -2548,7 +2548,7 @@ class DataFrameSuite extends QueryTest exception = intercept[ParseException] { spark.range(1).toDF("CASE").filter("CASE").collect() }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'CASE'", "hint" -> "")) } } @@ -2560,7 +2560,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException] { spark.range(1).createTempView("AUTHORIZATION") }, - errorClass = "_LEGACY_ERROR_TEMP_1321", + condition = "_LEGACY_ERROR_TEMP_1321", parameters = Map("viewName" -> "AUTHORIZATION")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameToSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameToSchemaSuite.scala index 160f583c983d8..f166043e4d554 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameToSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameToSchemaSuite.scala @@ -58,7 +58,7 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkThrowable](Seq("a" -> "b").toDF("i", "j").to(schema)) checkError( exception = e, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`non_exist`", "proposal" -> "`i`, `j`")) @@ -69,7 +69,7 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkThrowable](Seq("a" -> "b").toDF("i", "I").to(schema)) checkError( exception = e, - errorClass = "AMBIGUOUS_COLUMN_OR_FIELD", + condition = "AMBIGUOUS_COLUMN_OR_FIELD", parameters = Map( "name" -> "`i`", "n" -> "2")) @@ -92,7 +92,7 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkThrowable](data.to(schema)) checkError( exception = e, - errorClass = "NULLABLE_COLUMN_OR_FIELD", + condition = "NULLABLE_COLUMN_OR_FIELD", parameters = Map("name" -> "`i`")) } @@ -108,7 +108,7 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkThrowable](Seq("a" -> 1).toDF("i", "j").to(schema)) checkError( exception = e, - errorClass = "INVALID_COLUMN_OR_FIELD_DATA_TYPE", + condition = "INVALID_COLUMN_OR_FIELD_DATA_TYPE", parameters = Map( "name" -> "`i`", "type" -> "\"STRING\"", @@ -160,7 +160,7 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession { } checkError( exception = e, - errorClass = "UNRESOLVED_FIELD.WITH_SUGGESTION", + condition = "UNRESOLVED_FIELD.WITH_SUGGESTION", parameters = Map( "fieldName" -> "`non_exist`", "columnPath" -> "`struct`", @@ -200,7 +200,7 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkThrowable](data.to(schema)) checkError( exception = e, - errorClass = "NULLABLE_COLUMN_OR_FIELD", + condition = "NULLABLE_COLUMN_OR_FIELD", parameters = Map("name" -> "`struct`.`i`")) } @@ -220,7 +220,7 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession { } checkError( exception = e, - errorClass = "INVALID_COLUMN_OR_FIELD_DATA_TYPE", + condition = "INVALID_COLUMN_OR_FIELD_DATA_TYPE", parameters = Map( "name" -> "`struct`.`i`", "type" -> "\"STRING\"", @@ -284,7 +284,7 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkThrowable](data.to(schema)) checkError( exception = e, - errorClass = "NOT_NULL_CONSTRAINT_VIOLATION.ARRAY_ELEMENT", + condition = "NOT_NULL_CONSTRAINT_VIOLATION.ARRAY_ELEMENT", parameters = Map("columnPath" -> "`arr`")) } @@ -362,7 +362,7 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkThrowable](data.to(schema)) checkError( exception = e, - errorClass = "NOT_NULL_CONSTRAINT_VIOLATION.MAP_VALUE", + condition = "NOT_NULL_CONSTRAINT_VIOLATION.MAP_VALUE", parameters = Map("columnPath" -> "`map`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala index c03c5e878427f..d03288d7dbcdf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala @@ -186,7 +186,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { $"key", count("key").over( Window.partitionBy($"value").orderBy($"key").rowsBetween(2147483648L, 0)))), - errorClass = "INVALID_BOUNDARY.START", + condition = "INVALID_BOUNDARY.START", parameters = Map( "invalidValue" -> "2147483648L", "boundary" -> "`start`", @@ -200,7 +200,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { $"key", count("key").over( Window.partitionBy($"value").orderBy($"key").rowsBetween(0, 2147483648L)))), - errorClass = "INVALID_BOUNDARY.END", + condition = "INVALID_BOUNDARY.END", parameters = Map( "invalidValue" -> "2147483648L", "boundary" -> "`end`", @@ -226,7 +226,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { df.select( min("key").over(window.rangeBetween(Window.unboundedPreceding, 1))) ), - errorClass = "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", + condition = "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", parameters = Map( "orderSpec" -> """key#\d+ ASC NULLS FIRST,value#\d+ ASC NULLS FIRST""", "sqlExpr" -> (""""\(ORDER BY key ASC NULLS FIRST, value ASC NULLS FIRST RANGE """ + @@ -242,7 +242,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { df.select( min("key").over(window.rangeBetween(-1, Window.unboundedFollowing))) ), - errorClass = "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", + condition = "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", parameters = Map( "orderSpec" -> """key#\d+ ASC NULLS FIRST,value#\d+ ASC NULLS FIRST""", "sqlExpr" -> (""""\(ORDER BY key ASC NULLS FIRST, value ASC NULLS FIRST RANGE """ + @@ -258,7 +258,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { df.select( min("key").over(window.rangeBetween(-1, 1))) ), - errorClass = "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", + condition = "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", parameters = Map( "orderSpec" -> """key#\d+ ASC NULLS FIRST,value#\d+ ASC NULLS FIRST""", "sqlExpr" -> (""""\(ORDER BY key ASC NULLS FIRST, value ASC NULLS FIRST RANGE """ + @@ -287,7 +287,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { df.select( min("value").over(window.rangeBetween(Window.unboundedPreceding, 1))) ), - errorClass = "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + condition = "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", parameters = Map( "location" -> "upper", "exprType" -> "\"STRING\"", @@ -303,7 +303,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { df.select( min("value").over(window.rangeBetween(-1, Window.unboundedFollowing))) ), - errorClass = "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + condition = "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", parameters = Map( "location" -> "lower", "exprType" -> "\"STRING\"", @@ -319,7 +319,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { df.select( min("value").over(window.rangeBetween(-1, 1))) ), - errorClass = "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + condition = "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", parameters = Map( "location" -> "lower", "exprType" -> "\"STRING\"", @@ -512,7 +512,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select($"key", count("key").over(windowSpec)).collect() }, - errorClass = "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"RANGE BETWEEN NULL FOLLOWING AND 2 FOLLOWING\"", "lower" -> "\"NULL\"", @@ -534,7 +534,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select($"key", count("key").over(windowSpec)).collect() }, - errorClass = "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WITHOUT_FOLDABLE", + condition = "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WITHOUT_FOLDABLE", parameters = Map( "sqlExpr" -> "\"RANGE BETWEEN nonfoldableliteral() FOLLOWING AND 2 FOLLOWING\"", "location" -> "lower", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index ace4d5b294a78..8a86aa10887c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -388,7 +388,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest df.select($"key", count("invalid").over())) checkError( exception = e, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`invalid`", "proposal" -> "`value`, `key`"), @@ -870,7 +870,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest lag($"value", 3, null, true).over(window), lag(concat($"value", $"key"), 1, null, true).over(window)).orderBy($"order").collect() }, - errorClass = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + condition = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", parameters = Map( "sqlExpr" -> "\"lag(value, nonfoldableliteral(), NULL)\"", "inputName" -> "`offset`", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index 2275d8c213978..b7ac6af22a204 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -146,7 +146,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo exception = intercept[AnalysisException] { spark.table("source").withColumnRenamed("data", "d").writeTo("testcat.table_name").append() }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map("tableName" -> "`testcat`.`table_name`", "colName" -> "`data`") ) @@ -251,7 +251,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.table("source").withColumnRenamed("data", "d") .writeTo("testcat.table_name").overwrite(lit(true)) }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map("tableName" -> "`testcat`.`table_name`", "colName" -> "`data`") ) @@ -356,7 +356,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.table("source").withColumnRenamed("data", "d") .writeTo("testcat.table_name").overwritePartitions() }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map("tableName" -> "`testcat`.`table_name`", "colName" -> "`data`") ) @@ -829,14 +829,14 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo exception = intercept[AnalysisException] { ds.write }, - errorClass = "CALL_ON_STREAMING_DATASET_UNSUPPORTED", + condition = "CALL_ON_STREAMING_DATASET_UNSUPPORTED", parameters = Map("methodName" -> "`write`")) checkError( exception = intercept[AnalysisException] { ds.writeTo("testcat.table_name") }, - errorClass = "CALL_ON_STREAMING_DATASET_UNSUPPORTED", + condition = "CALL_ON_STREAMING_DATASET_UNSUPPORTED", parameters = Map("methodName" -> "`writeTo`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index fdb2ec30fdd2d..39f30b141df03 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -345,7 +345,7 @@ class DatasetSuite extends QueryTest exception = intercept[AnalysisException] { ds.select(expr("`(_1)?+.+`").as[Int]) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> "`(_1)?+.+`", @@ -359,7 +359,7 @@ class DatasetSuite extends QueryTest exception = intercept[AnalysisException] { ds.select(expr("`(_1|_2)`").as[Int]) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> "`(_1|_2)`", @@ -373,7 +373,7 @@ class DatasetSuite extends QueryTest exception = intercept[AnalysisException] { ds.select(ds("`(_1)?+.+`")) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`(_1)?+.+`", "proposal" -> "`_1`, `_2`") ) @@ -381,7 +381,7 @@ class DatasetSuite extends QueryTest exception = intercept[AnalysisException] { ds.select(ds("`(_1|_2)`")) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`(_1|_2)`", "proposal" -> "`_1`, `_2`") ) } @@ -549,7 +549,7 @@ class DatasetSuite extends QueryTest exception = intercept[AnalysisException]( ds1.joinWith(ds2, $"a.value" === $"b.value", joinType) ), - errorClass = "INVALID_JOIN_TYPE_FOR_JOINWITH", + condition = "INVALID_JOIN_TYPE_FOR_JOINWITH", sqlState = "42613", parameters = semiErrorParameters ) @@ -611,7 +611,7 @@ class DatasetSuite extends QueryTest (g, iter) => Iterator(g, iter.mkString(", ")) } }, - errorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + condition = "INVALID_USAGE_OF_STAR_OR_REGEX", parameters = Map("elem" -> "'*'", "prettyName" -> "MapGroups"), context = ExpectedContext(fragment = "$", getCurrentClassCallSitePattern)) } @@ -640,7 +640,7 @@ class DatasetSuite extends QueryTest (g, iter) => Iterator(g, iter.mkString(", ")) } }, - errorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + condition = "INVALID_USAGE_OF_STAR_OR_REGEX", parameters = Map("elem" -> "'*'", "prettyName" -> "MapGroups"), context = ExpectedContext(fragment = "$", getCurrentClassCallSitePattern)) } @@ -1187,7 +1187,7 @@ class DatasetSuite extends QueryTest exception = intercept[AnalysisException] { df.as[KryoData] }, - errorClass = "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + condition = "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", parameters = Map( "sqlExpr" -> "\"a\"", "srcType" -> "\"DOUBLE\"", @@ -1239,7 +1239,7 @@ class DatasetSuite extends QueryTest val ds = Seq(ClassData("a", 1)).toDS() checkError( exception = intercept[AnalysisException] (ds.as[ClassData2]), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`c`", "proposal" -> "`a`, `b`")) @@ -1429,7 +1429,7 @@ class DatasetSuite extends QueryTest dataset.createTempView("tempView")) intercept[AnalysisException](dataset.createTempView("tempView")) checkError(e, - errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> "`tempView`")) dataset.sparkSession.catalog.dropTempView("tempView") @@ -1440,7 +1440,7 @@ class DatasetSuite extends QueryTest val e = intercept[AnalysisException]( dataset.createTempView("test_db.tempView")) checkError(e, - errorClass = "TEMP_VIEW_NAME_TOO_MANY_NAME_PARTS", + condition = "TEMP_VIEW_NAME_TOO_MANY_NAME_PARTS", parameters = Map("actualName" -> "test_db.tempView")) } @@ -1902,19 +1902,19 @@ class DatasetSuite extends QueryTest exception = intercept[SparkUnsupportedOperationException] { Seq(CircularReferenceClassA(null)).toDS() }, - errorClass = "_LEGACY_ERROR_TEMP_2139", + condition = "_LEGACY_ERROR_TEMP_2139", parameters = Map("t" -> "org.apache.spark.sql.CircularReferenceClassA")) checkError( exception = intercept[SparkUnsupportedOperationException] { Seq(CircularReferenceClassC(null)).toDS() }, - errorClass = "_LEGACY_ERROR_TEMP_2139", + condition = "_LEGACY_ERROR_TEMP_2139", parameters = Map("t" -> "org.apache.spark.sql.CircularReferenceClassC")) checkError( exception = intercept[SparkUnsupportedOperationException] { Seq(CircularReferenceClassD(null)).toDS() }, - errorClass = "_LEGACY_ERROR_TEMP_2139", + condition = "_LEGACY_ERROR_TEMP_2139", parameters = Map("t" -> "org.apache.spark.sql.CircularReferenceClassD")) } @@ -2056,12 +2056,12 @@ class DatasetSuite extends QueryTest val parameters = Map("walkedTypePath" -> "\n- root class: \"int\"\n") checkError( exception = intercept[SparkRuntimeException](ds.collect()), - errorClass = errorClass, + condition = errorClass, sqlState = sqlState, parameters = parameters) checkError( exception = intercept[SparkRuntimeException](ds.map(_ * 2).collect()), - errorClass = errorClass, + condition = errorClass, sqlState = sqlState, parameters = parameters) @@ -2071,12 +2071,12 @@ class DatasetSuite extends QueryTest val ds = spark.read.parquet(path.getCanonicalPath).as[Int] checkError( exception = intercept[SparkRuntimeException](ds.collect()), - errorClass = errorClass, + condition = errorClass, sqlState = sqlState, parameters = parameters) checkError( exception = intercept[SparkRuntimeException](ds.map(_ * 2).collect()), - errorClass = errorClass, + condition = errorClass, sqlState = sqlState, parameters = parameters) } @@ -2317,7 +2317,7 @@ class DatasetSuite extends QueryTest exception = intercept[AnalysisException] { ds(colName) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> colName, "proposal" -> "`field`.`1`, `field 2`") ) } @@ -2334,7 +2334,7 @@ class DatasetSuite extends QueryTest // has different semantics than ds.select(colName) ds.select(colName) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> s"`${colName.replace(".", "`.`")}`", @@ -2349,7 +2349,7 @@ class DatasetSuite extends QueryTest exception = intercept[AnalysisException] { Seq(0).toDF("the.id").select("the.id") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> "`the`.`id`", @@ -2364,7 +2364,7 @@ class DatasetSuite extends QueryTest .select(map(lit("key"), lit(1)).as("map"), lit(2).as("other.column")) .select($"`map`"($"nonexisting")).show() }, - errorClass = "UNRESOLVED_MAP_KEY.WITH_SUGGESTION", + condition = "UNRESOLVED_MAP_KEY.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> "`nonexisting`", @@ -2676,7 +2676,7 @@ class DatasetSuite extends QueryTest // Expression decoding error checkError( exception = exception, - errorClass = "EXPRESSION_DECODING_FAILED", + condition = "EXPRESSION_DECODING_FAILED", parameters = Map( "expressions" -> expressions.map( _.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")) @@ -2684,7 +2684,7 @@ class DatasetSuite extends QueryTest // class unsupported by map objects checkError( exception = exception.getCause.asInstanceOf[org.apache.spark.SparkRuntimeException], - errorClass = "CLASS_UNSUPPORTED_BY_MAP_OBJECTS", + condition = "CLASS_UNSUPPORTED_BY_MAP_OBJECTS", parameters = Map("cls" -> classOf[Array[Int]].getName)) } } @@ -2697,7 +2697,7 @@ class DatasetSuite extends QueryTest } checkError( exception = exception, - errorClass = "EXPRESSION_ENCODING_FAILED", + condition = "EXPRESSION_ENCODING_FAILED", parameters = Map( "expressions" -> enc.serializer.map( _.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")) @@ -2746,7 +2746,7 @@ class DatasetSuite extends QueryTest } checkError( exception, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map("objectName" -> "`a`", "proposal" -> "`value`"), context = ExpectedContext(fragment = "col", callSitePattern = callSitePattern)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala index 5e5e4d09c5274..dad69a9aab06d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala @@ -149,7 +149,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e, - errorClass = "UNPIVOT_REQUIRES_VALUE_COLUMNS", + condition = "UNPIVOT_REQUIRES_VALUE_COLUMNS", parameters = Map()) // ids expressions are not allowed when no values are given @@ -162,7 +162,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e2, - errorClass = "UNPIVOT_REQUIRES_ATTRIBUTES", + condition = "UNPIVOT_REQUIRES_ATTRIBUTES", parameters = Map( "given" -> "id", "empty" -> "value", @@ -178,7 +178,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e3, - errorClass = "UNPIVOT_REQUIRES_ATTRIBUTES", + condition = "UNPIVOT_REQUIRES_ATTRIBUTES", parameters = Map( "given" -> "id", "empty" -> "value", @@ -207,7 +207,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e, - errorClass = "UNPIVOT_REQUIRES_VALUE_COLUMNS", + condition = "UNPIVOT_REQUIRES_VALUE_COLUMNS", parameters = Map()) } @@ -315,7 +315,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e, - errorClass = "UNPIVOT_VALUE_DATA_TYPE_MISMATCH", + condition = "UNPIVOT_VALUE_DATA_TYPE_MISMATCH", parameters = Map( "types" -> ( """"BIGINT" (`long1`, `long2`), """ + @@ -371,7 +371,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e1, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`1`", "proposal" -> "`id`, `int1`, `str1`, `long1`, `str2`"), @@ -388,7 +388,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e2, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`does`", "proposal" -> "`id`, `int1`, `long1`, `str1`, `str2`"), @@ -404,7 +404,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e3, - errorClass = "UNPIVOT_VALUE_DATA_TYPE_MISMATCH", + condition = "UNPIVOT_VALUE_DATA_TYPE_MISMATCH", parameters = Map( "types" -> """"BIGINT" (`long1`), "INT" (`id`, `int1`), "STRING" (`str1`, `str2`)""" ) @@ -420,7 +420,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e4, - errorClass = "UNPIVOT_REQUIRES_VALUE_COLUMNS", + condition = "UNPIVOT_REQUIRES_VALUE_COLUMNS", parameters = Map() ) @@ -436,7 +436,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e5, - errorClass = "UNPIVOT_VALUE_DATA_TYPE_MISMATCH", + condition = "UNPIVOT_VALUE_DATA_TYPE_MISMATCH", parameters = Map( "types" -> """"BIGINT" (`long1`), "INT" (`id`, `int1`), "STRING" (`str1`, `str2`)""" ) @@ -452,7 +452,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e6, - errorClass = "UNPIVOT_REQUIRES_VALUE_COLUMNS", + condition = "UNPIVOT_REQUIRES_VALUE_COLUMNS", parameters = Map.empty ) } @@ -507,7 +507,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`an`.`id`", "proposal" -> "`an.id`, `int1`, `long1`, `str.one`, `str.two`"), @@ -607,7 +607,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e, - errorClass = "UNPIVOT_REQUIRES_ATTRIBUTES", + condition = "UNPIVOT_REQUIRES_ATTRIBUTES", parameters = Map( "given" -> "value", "empty" -> "id", @@ -635,7 +635,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e2, - errorClass = "UNPIVOT_REQUIRES_ATTRIBUTES", + condition = "UNPIVOT_REQUIRES_ATTRIBUTES", parameters = Map( "given" -> "value", "empty" -> "id", @@ -661,7 +661,7 @@ class DatasetUnpivotSuite extends QueryTest } checkError( exception = e, - errorClass = "UNPIVOT_VALUE_SIZE_MISMATCH", + condition = "UNPIVOT_VALUE_SIZE_MISMATCH", parameters = Map("names" -> "2")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index b261ecfb0cee4..4cab05dfd2b9b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -54,7 +54,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("SELECT CURDATE(1)") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`curdate`", "expectedNum" -> "0", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 229677d208136..2fe6a83427bca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -133,7 +133,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { spark.emptyDataFrame.write.format(format).save(outputPath.toString) }, - errorClass = "_LEGACY_ERROR_TEMP_1142", + condition = "_LEGACY_ERROR_TEMP_1142", parameters = Map.empty ) } @@ -150,7 +150,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { df.write.format(format).save(outputPath.toString) }, - errorClass = "_LEGACY_ERROR_TEMP_1142", + condition = "_LEGACY_ERROR_TEMP_1142", parameters = Map.empty ) } @@ -250,7 +250,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[SparkException] { testIgnoreMissingFiles(options) }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> ".*") ) } @@ -282,7 +282,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { Seq(1).toDF().write.text(textDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`value`", "columnType" -> "\"INT\"", @@ -293,7 +293,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { Seq(1.2).toDF().write.text(textDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`value`", "columnType" -> "\"DOUBLE\"", @@ -304,7 +304,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { Seq(true).toDF().write.text(textDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`value`", "columnType" -> "\"BOOLEAN\"", @@ -315,7 +315,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { Seq(1).toDF("a").selectExpr("struct(a)").write.text(textDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`struct(a)`", "columnType" -> "\"STRUCT\"", @@ -326,7 +326,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { Seq((Map("Tesla" -> 3))).toDF("cars").write.mode("overwrite").text(textDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`cars`", "columnType" -> "\"MAP\"", @@ -338,7 +338,7 @@ class FileBasedDataSourceSuite extends QueryTest Seq((Array("Tesla", "Chevy", "Ford"))).toDF("brands") .write.mode("overwrite").text(textDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`brands`", "columnType" -> "\"ARRAY\"", @@ -352,7 +352,7 @@ class FileBasedDataSourceSuite extends QueryTest val schema = StructType(StructField("a", IntegerType, true) :: Nil) spark.read.schema(schema).text(textDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "\"INT\"", @@ -364,7 +364,7 @@ class FileBasedDataSourceSuite extends QueryTest val schema = StructType(StructField("a", DoubleType, true) :: Nil) spark.read.schema(schema).text(textDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "\"DOUBLE\"", @@ -376,7 +376,7 @@ class FileBasedDataSourceSuite extends QueryTest val schema = StructType(StructField("a", BooleanType, true) :: Nil) spark.read.schema(schema).text(textDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "\"BOOLEAN\"", @@ -397,7 +397,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { Seq((1, "Tesla")).toDF("a", "b").selectExpr("struct(a, b)").write.csv(csvDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`struct(a, b)`", "columnType" -> "\"STRUCT\"", @@ -410,7 +410,7 @@ class FileBasedDataSourceSuite extends QueryTest spark.range(1).write.mode("overwrite").csv(csvDir) spark.read.schema(schema).csv(csvDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "\"STRUCT\"", @@ -421,7 +421,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { Seq((1, Map("Tesla" -> 3))).toDF("id", "cars").write.mode("overwrite").csv(csvDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`cars`", "columnType" -> "\"MAP\"", @@ -434,7 +434,7 @@ class FileBasedDataSourceSuite extends QueryTest spark.range(1).write.mode("overwrite").csv(csvDir) spark.read.schema(schema).csv(csvDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "\"MAP\"", @@ -446,7 +446,7 @@ class FileBasedDataSourceSuite extends QueryTest Seq((1, Array("Tesla", "Chevy", "Ford"))).toDF("id", "brands") .write.mode("overwrite").csv(csvDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`brands`", "columnType" -> "\"ARRAY\"", @@ -459,7 +459,7 @@ class FileBasedDataSourceSuite extends QueryTest spark.range(1).write.mode("overwrite").csv(csvDir) spark.read.schema(schema).csv(csvDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "\"ARRAY\"", @@ -471,7 +471,7 @@ class FileBasedDataSourceSuite extends QueryTest Seq((1, new TestUDT.MyDenseVector(Array(0.25, 2.25, 4.25)))).toDF("id", "vectors") .write.mode("overwrite").csv(csvDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`vectors`", "columnType" -> "UDT(\"ARRAY\")", @@ -484,7 +484,7 @@ class FileBasedDataSourceSuite extends QueryTest spark.range(1).write.mode("overwrite").csv(csvDir) spark.read.schema(schema).csv(csvDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "UDT(\"ARRAY\")", @@ -512,7 +512,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { sql("select interval 1 days").write.format(format).mode("overwrite").save(tempDir) }, - errorClass = "_LEGACY_ERROR_TEMP_1136", + condition = "_LEGACY_ERROR_TEMP_1136", parameters = Map.empty ) } @@ -529,7 +529,7 @@ class FileBasedDataSourceSuite extends QueryTest spark.range(1).write.format(format).mode("overwrite").save(tempDir) spark.read.schema(schema).format(format).load(tempDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "\"INTERVAL\"", @@ -542,7 +542,7 @@ class FileBasedDataSourceSuite extends QueryTest spark.range(1).write.format(format).mode("overwrite").save(tempDir) spark.read.schema(schema).format(format).load(tempDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "UDT(\"INTERVAL\")", @@ -579,7 +579,7 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { sql("select null").write.format(format).mode("overwrite").save(tempDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`NULL`", "columnType" -> "\"VOID\"", @@ -592,7 +592,7 @@ class FileBasedDataSourceSuite extends QueryTest spark.udf.register("testType", () => new NullData()) sql("select testType()").write.format(format).mode("overwrite").save(tempDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`testType()`", "columnType" -> "UDT(\"VOID\")", @@ -607,7 +607,7 @@ class FileBasedDataSourceSuite extends QueryTest spark.range(1).write.format(format).mode("overwrite").save(tempDir) spark.read.schema(schema).format(format).load(tempDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "\"VOID\"", @@ -621,7 +621,7 @@ class FileBasedDataSourceSuite extends QueryTest spark.range(1).write.format(format).mode("overwrite").save(tempDir) spark.read.schema(schema).format(format).load(tempDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "UDT(\"VOID\")", @@ -657,14 +657,14 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[SparkException] { sql(s"select b from $tableName").collect() }.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "_LEGACY_ERROR_TEMP_2093", + condition = "_LEGACY_ERROR_TEMP_2093", parameters = Map("requiredFieldName" -> "b", "matchedOrcFields" -> "[b, B]") ) checkError( exception = intercept[SparkException] { sql(s"select B from $tableName").collect() }.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "_LEGACY_ERROR_TEMP_2093", + condition = "_LEGACY_ERROR_TEMP_2093", parameters = Map("requiredFieldName" -> "b", "matchedOrcFields" -> "[b, B]") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala index 97a56bdea7be7..b9491a79cc3a4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -59,7 +59,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("stack(1.1, 1, 2, 3)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"stack(1.1, 1, 2, 3)\"", "paramIndex" -> "first", @@ -77,7 +77,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("stack(-1, 1, 2, 3)") }, - errorClass = "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + condition = "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", parameters = Map( "sqlExpr" -> "\"stack(-1, 1, 2, 3)\"", "exprName" -> "`n`", @@ -95,7 +95,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("stack(2, 1, '2.2')") }, - errorClass = "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"stack(2, 1, 2.2)\"", "columnIndex" -> "0", @@ -118,7 +118,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.selectExpr("stack(n, a, b, c)") }, - errorClass = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + condition = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", parameters = Map( "sqlExpr" -> "\"stack(n, a, b, c)\"", "inputName" -> "`n`", @@ -136,7 +136,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("stack(2, a, b)") }, - errorClass = "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"stack(2, a, b)\"", "columnIndex" -> "0", @@ -287,7 +287,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.range(2).select(inline(array())) }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"inline(array())\"", "paramIndex" -> "first", @@ -330,7 +330,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(inline(array(struct(Symbol("a")), struct(Symbol("b"))))) }, - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array(struct(a), struct(b))\"", "functionName" -> "`array`", @@ -348,7 +348,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(inline(array(struct(Symbol("a")), struct(lit(2))))) }, - errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", parameters = Map( "sqlExpr" -> "\"array(struct(a), struct(2))\"", "functionName" -> "`array`", @@ -427,7 +427,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("select 1 + explode(array(min(c2), max(c2))) from t1 group by c1") }, - errorClass = "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + condition = "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", parameters = Map( "expression" -> "\"(1 + explode(array(min(c2), max(c2))))\"")) @@ -440,7 +440,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { | posexplode(array(min(c2), max(c2))) |from t1 group by c1""".stripMargin) }, - errorClass = "UNSUPPORTED_GENERATOR.MULTI_GENERATOR", + condition = "UNSUPPORTED_GENERATOR.MULTI_GENERATOR", parameters = Map( "num" -> "2", "generators" -> ("\"explode(array(min(c2), max(c2)))\", " + @@ -453,7 +453,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("SELECT array(array(1, 2), array(3)) v").select(explode(explode($"v"))).collect() }, - errorClass = "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + condition = "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", parameters = Map("expression" -> "\"explode(explode(v))\"")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 6b4be982b3ecb..7b19ad988d308 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -190,7 +190,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { nonStringDF.select(json_tuple($"a", "1")).collect() }, - errorClass = "DATATYPE_MISMATCH.NON_STRING_TYPE", + condition = "DATATYPE_MISMATCH.NON_STRING_TYPE", parameters = Map( "sqlExpr" -> "\"json_tuple(a, 1)\"", "funcName" -> "`json_tuple`" @@ -499,7 +499,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.selectExpr("to_json(a, named_struct('a', 1))") }, - errorClass = "INVALID_OPTIONS.NON_MAP_FUNCTION", + condition = "INVALID_OPTIONS.NON_MAP_FUNCTION", parameters = Map.empty, context = ExpectedContext( fragment = "to_json(a, named_struct('a', 1))", @@ -512,7 +512,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.selectExpr("to_json(a, map('a', 1))") }, - errorClass = "INVALID_OPTIONS.NON_STRING_TYPE", + condition = "INVALID_OPTIONS.NON_STRING_TYPE", parameters = Map("mapType" -> "\"MAP\""), context = ExpectedContext( fragment = "to_json(a, map('a', 1))", @@ -543,7 +543,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("from_json(value, 1)") }, - errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL", + condition = "INVALID_SCHEMA.NON_STRING_LITERAL", parameters = Map("inputSchema" -> "\"1\""), context = ExpectedContext( fragment = "from_json(value, 1)", @@ -556,7 +556,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("""from_json(value, 'time InvalidType')""") }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map( "error" -> "'InvalidType'", @@ -572,7 +572,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("from_json(value, 'time Timestamp', named_struct('a', 1))") }, - errorClass = "INVALID_OPTIONS.NON_MAP_FUNCTION", + condition = "INVALID_OPTIONS.NON_MAP_FUNCTION", parameters = Map.empty, context = ExpectedContext( fragment = "from_json(value, 'time Timestamp', named_struct('a', 1))", @@ -584,7 +584,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("from_json(value, 'time Timestamp', map('a', 1))") }, - errorClass = "INVALID_OPTIONS.NON_STRING_TYPE", + condition = "INVALID_OPTIONS.NON_STRING_TYPE", parameters = Map("mapType" -> "\"MAP\""), context = ExpectedContext( fragment = "from_json(value, 'time Timestamp', map('a', 1))", @@ -657,7 +657,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq("""{{"f": 1}: "a"}""").toDS().select(from_json($"value", schema)) }, - errorClass = "DATATYPE_MISMATCH.INVALID_JSON_MAP_KEY_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_JSON_MAP_KEY_TYPE", parameters = Map( "schema" -> "\"MAP, STRING>\"", "sqlExpr" -> "\"entries\""), @@ -851,7 +851,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { df.select(from_json($"value", schema, Map("mode" -> "FAILFAST"))).collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[null,null,{\"a\" 1, \"b\": 11}]", "failFastMode" -> "FAILFAST") @@ -861,7 +861,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(from_json($"value", schema, Map("mode" -> "DROPMALFORMED"))).collect() }, - errorClass = "_LEGACY_ERROR_TEMP_1099", + condition = "_LEGACY_ERROR_TEMP_1099", parameters = Map( "funcName" -> "from_json", "mode" -> "DROPMALFORMED", @@ -889,14 +889,14 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { checkError( exception = ex, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[null,11,{\"a\": \"1\", \"b\": 11}]", "failFastMode" -> "FAILFAST") ) checkError( exception = ex.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "CANNOT_PARSE_JSON_FIELD", + condition = "CANNOT_PARSE_JSON_FIELD", parameters = Map( "fieldName" -> toSQLValue("a", StringType), "fieldValue" -> "1", @@ -973,7 +973,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { Seq(("""{"i":1}""", "i int")).toDF("json", "schema") .select(from_json($"json", $"schema", options)).collect() }, - errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL", + condition = "INVALID_SCHEMA.NON_STRING_LITERAL", parameters = Map("inputSchema" -> "\"schema\""), context = ExpectedContext(fragment = "from_json", getCurrentClassCallSitePattern) ) @@ -1208,7 +1208,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(from_json($"json", invalidJsonSchema, Map.empty[String, String])).collect() }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'{'", "hint" -> ""), ExpectedContext("from_json", getCurrentClassCallSitePattern) ) @@ -1218,7 +1218,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(from_json($"json", invalidDataType, Map.empty[String, String])).collect() }, - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> "\"COW\""), ExpectedContext("from_json", getCurrentClassCallSitePattern) ) @@ -1228,7 +1228,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select(from_json($"json", invalidTableSchema, Map.empty[String, String])).collect() }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'INT'", "hint" -> ""), ExpectedContext("from_json", getCurrentClassCallSitePattern) ) @@ -1247,7 +1247,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { df.select(from_json($"value", schema, Map("mode" -> "FAILFAST"))("b")).collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[null,null]", "failFastMode" -> "FAILFAST") @@ -1257,7 +1257,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { df.select(from_json($"value", schema, Map("mode" -> "FAILFAST"))("a")).collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[null,null]", "failFastMode" -> "FAILFAST") @@ -1279,7 +1279,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { df.select(from_json($"value", schema, Map("mode" -> "FAILFAST"))("b")).collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[null]", "failFastMode" -> "FAILFAST") @@ -1289,7 +1289,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { df.select(from_json($"value", schema, Map("mode" -> "FAILFAST"))("a")).collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[null]", "failFastMode" -> "FAILFAST") @@ -1401,7 +1401,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select($"a").withColumn("c", to_json(structData)).collect() }, - errorClass = "DATATYPE_MISMATCH.CANNOT_CONVERT_TO_JSON", + condition = "DATATYPE_MISMATCH.CANNOT_CONVERT_TO_JSON", parameters = Map( "sqlExpr" -> "\"to_json(NAMED_STRUCT('b', 1))\"", "name" -> "`b`", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala index 336cf12ae57c5..8ea2b2a1edb7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala @@ -184,7 +184,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { query: String, parameters: Map[String, String]): Unit = { checkError( exception = intercept[AnalysisException] {sql(query)}, - errorClass = "AMBIGUOUS_LATERAL_COLUMN_ALIAS", + condition = "AMBIGUOUS_LATERAL_COLUMN_ALIAS", sqlState = "42702", parameters = parameters ) @@ -194,7 +194,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { query: String, lca: String, windowExprRegex: String): Unit = { checkErrorMatchPVals( exception = intercept[AnalysisException] {sql(query)}, - errorClass = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_WINDOW", + condition = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_WINDOW", parameters = Map("lca" -> lca, "windowExpr" -> windowExprRegex) ) } @@ -258,7 +258,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { exception = intercept[AnalysisException] { sql(s"SELECT 10000 AS lca, count(lca) FROM $testTable GROUP BY dept") }, - errorClass = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_AGGREGATE_FUNC", + condition = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_AGGREGATE_FUNC", sqlState = "0A000", parameters = Map( "lca" -> "`lca`", @@ -269,7 +269,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { exception = intercept[AnalysisException] { sql(s"SELECT dept AS lca, avg(lca) FROM $testTable GROUP BY dept") }, - errorClass = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_AGGREGATE_FUNC", + condition = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_AGGREGATE_FUNC", sqlState = "0A000", parameters = Map( "lca" -> "`lca`", @@ -281,7 +281,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { exception = intercept[AnalysisException] { sql(s"SELECT sum(salary) AS a, avg(a) FROM $testTable") }, - errorClass = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_AGGREGATE_FUNC", + condition = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_AGGREGATE_FUNC", sqlState = "0A000", parameters = Map( "lca" -> "`a`", @@ -518,7 +518,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { exception = intercept[AnalysisException] { sql(query2) }, - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", sqlState = "42703", parameters = Map("objectName" -> s"`id1`"), context = ExpectedContext( @@ -796,7 +796,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { exception = intercept[AnalysisException] { sql(s"SELECT dept AS d, d AS new_dept, new_dep + 1 AS newer_dept FROM $testTable") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map("objectName" -> s"`new_dep`", "proposal" -> "`dept`, `name`, `bonus`, `salary`, `properties`"), @@ -809,7 +809,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { exception = intercept[AnalysisException] { sql(s"SELECT count(name) AS cnt, cnt + 1, count(unresovled) FROM $testTable GROUP BY dept") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map("objectName" -> s"`unresovled`", "proposal" -> "`name`, `bonus`, `dept`, `properties`, `salary`"), @@ -823,7 +823,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { sql(s"SELECT * FROM range(1, 7) WHERE (" + s"SELECT id2 FROM (SELECT 1 AS id, other_id + 1 AS id2)) > 5") }, - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", sqlState = "42703", parameters = Map("objectName" -> s"`other_id`"), context = ExpectedContext( @@ -898,7 +898,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { exception = intercept[AnalysisException] { sql( "SELECT dept AS a, dept, " + s"(SELECT count(col) FROM VALUES (1), (2) AS data(col) WHERE col = dept) $groupBySeg") }, - errorClass = "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + condition = "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", parameters = Map("sqlExpr" -> "\"scalarsubquery(dept)\""), context = ExpectedContext( fragment = "(SELECT count(col) FROM VALUES (1), (2) AS data(col) WHERE col = dept)", @@ -910,7 +910,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { "SELECT dept AS a, a, " + s"(SELECT count(col) FROM VALUES (1), (2) AS data(col) WHERE col = dept) $groupBySeg" ) }, - errorClass = "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + condition = "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", parameters = Map("sqlExpr" -> "\"scalarsubquery(dept)\""), context = ExpectedContext( fragment = "(SELECT count(col) FROM VALUES (1), (2) AS data(col) WHERE col = dept)", @@ -924,7 +924,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { exception = intercept[AnalysisException] { sql(s"SELECT avg(salary) AS a, avg(a) $windowExpr $groupBySeg") }, - errorClass = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_AGGREGATE_FUNC", + condition = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_AGGREGATE_FUNC", sqlState = "0A000", parameters = Map("lca" -> "`a`", "aggFunc" -> "\"avg(lateralAliasReference(a))\"") ) @@ -1009,7 +1009,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { "(partition by dept order by salary rows between n preceding and current row) as rank " + s"from $testTable where dept in (1, 6)") }, - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> "Frame bound value must be a literal."), context = ExpectedContext(fragment = "n preceding", start = 87, stop = 97) ) @@ -1338,7 +1338,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { |""".stripMargin ) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`Freq`", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala index cc0cce08162ae..ebca6b26fce95 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala @@ -56,7 +56,7 @@ abstract class MetadataCacheSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { df.count() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> ".*") ) } @@ -87,7 +87,7 @@ class MetadataCacheV1Suite extends MetadataCacheSuite { exception = intercept[SparkException] { sql("select count(*) from view_refresh").first() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> ".*") ) @@ -115,7 +115,7 @@ class MetadataCacheV1Suite extends MetadataCacheSuite { exception = intercept[SparkException] { sql("select count(*) from view_refresh").first() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> ".*") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala index b9daece4913f2..b95b7b9d4c00d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala @@ -61,7 +61,7 @@ class MiscFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(sql(s"select $func"), Row(user)) checkError( exception = intercept[ParseException](sql(s"select $func()")), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> s"'$func'", "hint" -> "")) } } @@ -238,7 +238,7 @@ class MiscFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("reflect(cast(null as string), 'fromString', a)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL", + condition = "DATATYPE_MISMATCH.UNEXPECTED_NULL", parameters = Map( "exprName" -> "`class`", "sqlExpr" -> "\"reflect(CAST(NULL AS STRING), fromString, a)\""), @@ -247,7 +247,7 @@ class MiscFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("reflect('java.util.UUID', cast(null as string), a)") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL", + condition = "DATATYPE_MISMATCH.UNEXPECTED_NULL", parameters = Map( "exprName" -> "`method`", "sqlExpr" -> "\"reflect(java.util.UUID, CAST(NULL AS STRING), a)\""), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala index f83e7b6727b16..f570fc3ab25f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala @@ -65,7 +65,7 @@ trait NestedDataSourceSuiteBase extends QueryTest with SharedSparkSession { .load(path) .collect() }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`camelcase`") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index be3669cc62023..511d1e87309a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -73,7 +73,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[AnalysisException] { spark.sql("select :P", Map("p" -> 1)) }, - errorClass = "UNBOUND_SQL_PARAMETER", + condition = "UNBOUND_SQL_PARAMETER", parameters = Map("name" -> "P"), context = ExpectedContext( fragment = ":P", @@ -246,7 +246,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[ParseException] { spark.sql(sqlText, args) }, - errorClass = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", + condition = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", parameters = Map("statement" -> "CREATE VIEW body"), context = ExpectedContext( fragment = sqlText, @@ -261,7 +261,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[ParseException] { spark.sql(sqlText, args) }, - errorClass = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", + condition = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", parameters = Map("statement" -> "CREATE VIEW body"), context = ExpectedContext( fragment = sqlText, @@ -276,7 +276,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[ParseException] { spark.sql(sqlText, args) }, - errorClass = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", + condition = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", parameters = Map("statement" -> "CREATE VIEW body"), context = ExpectedContext( fragment = sqlText, @@ -291,7 +291,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[ParseException] { spark.sql(sqlText, args) }, - errorClass = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", + condition = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", parameters = Map("statement" -> "CREATE VIEW body"), context = ExpectedContext( fragment = sqlText, @@ -310,7 +310,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[ParseException] { spark.sql(sqlText, args) }, - errorClass = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", + condition = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", parameters = Map("statement" -> "CREATE VIEW body"), context = ExpectedContext( fragment = sqlText, @@ -329,7 +329,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[ParseException] { spark.sql(sqlText, args) }, - errorClass = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", + condition = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", parameters = Map("statement" -> "CREATE VIEW body"), context = ExpectedContext( fragment = sqlText, @@ -342,7 +342,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[AnalysisException] { spark.sql("select :abc, :def", Map("abc" -> 1)) }, - errorClass = "UNBOUND_SQL_PARAMETER", + condition = "UNBOUND_SQL_PARAMETER", parameters = Map("name" -> "def"), context = ExpectedContext( fragment = ":def", @@ -352,7 +352,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[AnalysisException] { sql("select :abc").collect() }, - errorClass = "UNBOUND_SQL_PARAMETER", + condition = "UNBOUND_SQL_PARAMETER", parameters = Map("name" -> "abc"), context = ExpectedContext( fragment = ":abc", @@ -365,7 +365,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[AnalysisException] { spark.sql("select ?, ?", Array(1)) }, - errorClass = "UNBOUND_SQL_PARAMETER", + condition = "UNBOUND_SQL_PARAMETER", parameters = Map("name" -> "_10"), context = ExpectedContext( fragment = "?", @@ -375,7 +375,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[AnalysisException] { sql("select ?").collect() }, - errorClass = "UNBOUND_SQL_PARAMETER", + condition = "UNBOUND_SQL_PARAMETER", parameters = Map("name" -> "_7"), context = ExpectedContext( fragment = "?", @@ -472,7 +472,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[AnalysisException] { spark.sql("select :param1, ?", Map("param1" -> 1)) }, - errorClass = "UNBOUND_SQL_PARAMETER", + condition = "UNBOUND_SQL_PARAMETER", parameters = Map("name" -> "_16"), context = ExpectedContext( fragment = "?", @@ -483,7 +483,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { exception = intercept[AnalysisException] { spark.sql("select :param1, ?", Array(1)) }, - errorClass = "UNBOUND_SQL_PARAMETER", + condition = "UNBOUND_SQL_PARAMETER", parameters = Map("name" -> "param1"), context = ExpectedContext( fragment = ":param1", @@ -498,7 +498,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { "CREATE TABLE t11(c1 int default :parm) USING parquet", args = Map("parm" -> 5)) }, - errorClass = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", + condition = "UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT", parameters = Map("statement" -> "DEFAULT"), context = ExpectedContext( fragment = "default :parm", @@ -602,7 +602,7 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { lit(Array("a")), array(str_to_map(lit("a:1,b:2,c:3")))))) }, - errorClass = "INVALID_SQL_ARG", + condition = "INVALID_SQL_ARG", parameters = Map("name" -> "m"), context = ExpectedContext( fragment = "map_from_arrays", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ResolveDefaultColumnsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ResolveDefaultColumnsSuite.scala index e3ebbadbb829a..cb9d0909554b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ResolveDefaultColumnsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ResolveDefaultColumnsSuite.scala @@ -40,7 +40,7 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t values (timestamp'2020-12-31')") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "tableColumns" -> "`c1`, `c2`", @@ -68,7 +68,7 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t values (timestamp'2020-12-31')") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "tableColumns" -> "`c1`, `c2`", @@ -85,7 +85,7 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t values (1, 2, 3)") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "tableColumns" -> "`c1`, `c2`, `c3`, `c4`", @@ -102,7 +102,7 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t partition(c3=3, c4=4) values (1)") }, - errorClass = "INSERT_PARTITION_COLUMN_ARITY_MISMATCH", + condition = "INSERT_PARTITION_COLUMN_ARITY_MISMATCH", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "tableColumns" -> "`c1`, `c2`, `c3`, `c4`", @@ -120,7 +120,7 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t partition(c3=3, c4) values (1, 2)") }, - errorClass = "INSERT_PARTITION_COLUMN_ARITY_MISMATCH", + condition = "INSERT_PARTITION_COLUMN_ARITY_MISMATCH", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "tableColumns" -> "`c1`, `c2`, `c3`, `c4`", @@ -173,7 +173,7 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("create table demos.test_ts_other (a int default 'abc') using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`a`", @@ -184,7 +184,7 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("create table demos.test_ts_other (a timestamp default 'invalid') using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`a`", @@ -195,7 +195,7 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("create table demos.test_ts_other (a boolean default 'true') using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`a`", @@ -206,7 +206,7 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("create table demos.test_ts_other (a int default true) using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`a`", @@ -237,7 +237,7 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[SparkRuntimeException]( sql(s"CREATE TABLE t(c $typeName(3) DEFAULT 'spark') USING parquet")), - errorClass = "EXCEED_LIMIT_LENGTH", + condition = "EXCEED_LIMIT_LENGTH", parameters = Map("limit" -> "3")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 4c4560e3fc48b..5de4170a1c112 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -119,7 +119,7 @@ class RowSuite extends SparkFunSuite with SharedSparkSession { exception = intercept[SparkUnsupportedOperationException] { rowWithoutSchema.fieldIndex("foo") }, - errorClass = "UNSUPPORTED_CALL.FIELD_INDEX", + condition = "UNSUPPORTED_CALL.FIELD_INDEX", parameters = Map("methodName" -> "fieldIndex", "className" -> "Row", "fieldName" -> "`foo`") ) } @@ -132,7 +132,7 @@ class RowSuite extends SparkFunSuite with SharedSparkSession { exception = intercept[SparkRuntimeException] { rowWithNullValue.getLong(position) }, - errorClass = "ROW_VALUE_IS_NULL", + condition = "ROW_VALUE_IS_NULL", parameters = Map("index" -> position.toString) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala index 63ed26bdeddf1..fc344ca0be3f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala @@ -276,7 +276,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP checkError( exception = intercept[AnalysisException]( sql(s"INSERT INTO t1 (c1, c2, c2) values(1, 2, 3)")), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`c2`")) } } @@ -288,7 +288,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP checkError( exception = intercept[AnalysisException](sql(s"INSERT INTO t1 (c1, c2, c4) values(1, 2, 3)")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`c4`", "proposal" -> "`c1`, `c2`, `c3`"), context = ExpectedContext( @@ -307,7 +307,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP sql(s"INSERT INTO t1 (c1, c2) values(1, 2, 3)") }, sqlState = None, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", parameters = Map( "tableName" -> ".*`t1`", "tableColumns" -> "`c1`, `c2`", @@ -319,7 +319,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP sql(s"INSERT INTO t1 (c1, c2, c3) values(1, 2)") }, sqlState = None, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> ".*`t1`", "tableColumns" -> "`c1`, `c2`, `c3`", @@ -399,7 +399,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP sql("INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)") }, sqlState = None, - errorClass = "DUPLICATE_KEY", + condition = "DUPLICATE_KEY", parameters = Map("keyColumn" -> "`c`"), context = ExpectedContext("PARTITION (c='2', C='3')", 19, 42) ) @@ -441,7 +441,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP exception = intercept[SparkNumberFormatException] { sql("insert into t partition(a='ansi') values('ansi')") }, - errorClass = "CAST_INVALID_INPUT", + condition = "CAST_INVALID_INPUT", parameters = Map( "expression" -> "'ansi'", "sourceType" -> "\"STRING\"", @@ -492,7 +492,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP exception = intercept[AnalysisException] { sql("alter table t drop partition(dt='8')") }, - errorClass = "PARTITIONS_NOT_FOUND", + condition = "PARTITIONS_NOT_FOUND", sqlState = None, parameters = Map( "partitionList" -> "PARTITION \\(`dt` = 8\\)", @@ -512,7 +512,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP exception = intercept[AnalysisException] { sql("alter table t drop partition(dt='08')") }, - errorClass = "PARTITIONS_NOT_FOUND", + condition = "PARTITIONS_NOT_FOUND", sqlState = None, parameters = Map( "partitionList" -> "PARTITION \\(`dt` = 08\\)", @@ -562,7 +562,7 @@ class FileSourceSQLInsertTestSuite extends SQLInsertTestSuite with SharedSparkSe v2ErrorClass: String, v1Parameters: Map[String, String], v2Parameters: Map[String, String]): Unit = { - checkError(exception = exception, sqlState = None, errorClass = v1ErrorClass, + checkError(exception = exception, sqlState = None, condition = v1ErrorClass, parameters = v1Parameters) } @@ -582,7 +582,7 @@ class DSV2SQLInsertTestSuite extends SQLInsertTestSuite with SharedSparkSession v2ErrorClass: String, v1Parameters: Map[String, String], v2Parameters: Map[String, String]): Unit = { - checkError(exception = exception, sqlState = None, errorClass = v2ErrorClass, + checkError(exception = exception, sqlState = None, condition = v2ErrorClass, parameters = v2Parameters) } protected override def sparkConf: SparkConf = { @@ -598,7 +598,7 @@ class DSV2SQLInsertTestSuite extends SQLInsertTestSuite with SharedSparkSession exception = intercept[AnalysisException] { sql("INSERT OVERWRITE t PARTITION (c='1') (c) VALUES ('2')") }, - errorClass = "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST", + condition = "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST", parameters = Map("staticName" -> "c")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 55313c8ac2f86..b7d0039446f30 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -100,7 +100,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark val sqlText = "describe functioN abcadf" checkError( exception = intercept[AnalysisException](sql(sqlText)), - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`abcadf`", "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"), @@ -1659,7 +1659,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException] { sql("select * from json.invalid_file") }, - errorClass = "PATH_NOT_FOUND", + condition = "PATH_NOT_FOUND", parameters = Map("path" -> "file:/.*invalid_file"), matchPVals = true ) @@ -1668,7 +1668,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException] { sql(s"select id from `org.apache.spark.sql.hive.orc`.`file_path`") }, - errorClass = "_LEGACY_ERROR_TEMP_1138" + condition = "_LEGACY_ERROR_TEMP_1138" ) e = intercept[AnalysisException] { @@ -1833,7 +1833,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException]{ sql("SELECT abc.* FROM nestedStructTable") }, - errorClass = "CANNOT_RESOLVE_STAR_EXPAND", + condition = "CANNOT_RESOLVE_STAR_EXPAND", parameters = Map("targetString" -> "`abc`", "columns" -> "`record`"), context = ExpectedContext(fragment = "abc.*", start = 7, stop = 11)) } @@ -1868,7 +1868,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException]{ sql("select a.* from testData2") }, - errorClass = "_LEGACY_ERROR_TEMP_1050", + condition = "_LEGACY_ERROR_TEMP_1050", sqlState = None, parameters = Map("attributes" -> "(ArrayBuffer|List)\\(a\\)"), matchPVals = true, @@ -1922,7 +1922,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException] { sql("SELECT a.* FROM temp_table_no_cols a") }, - errorClass = "CANNOT_RESOLVE_STAR_EXPAND", + condition = "CANNOT_RESOLVE_STAR_EXPAND", parameters = Map("targetString" -> "`a`", "columns" -> ""), context = ExpectedContext(fragment = "a.*", start = 7, stop = 9)) @@ -1930,7 +1930,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException] { dfNoCols.select($"b.*") }, - errorClass = "CANNOT_RESOLVE_STAR_EXPAND", + condition = "CANNOT_RESOLVE_STAR_EXPAND", parameters = Map("targetString" -> "`b`", "columns" -> ""), context = ExpectedContext( fragment = "$", @@ -2677,7 +2677,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException] { sql("SELECT nvl(1, 2, 3)") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> toSQLId("nvl"), "expectedNum" -> "2", @@ -2738,7 +2738,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException] { sql("SELECT struct(1 a) EXCEPT (SELECT struct(2 A))") }, - errorClass = "INCOMPATIBLE_COLUMN_TYPE", + condition = "INCOMPATIBLE_COLUMN_TYPE", parameters = Map( "tableOrdinalNumber" -> "second", "columnOrdinalNumber" -> "first", @@ -2761,7 +2761,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException] { sql(query) }, - errorClass = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + condition = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", sqlState = None, parameters = Map( "sqlExpr" -> "\"(c = C)\"", @@ -3073,7 +3073,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkAnswer(sql("select s.I from t group by s.i"), Nil) } }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`I`", "fields" -> "`i`"), context = ExpectedContext( fragment = "s.I", @@ -3784,7 +3784,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException] { sql("SELECT s LIKE 'm%@ca' ESCAPE '%' FROM df").collect() }, - errorClass = "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + condition = "INVALID_FORMAT.ESC_IN_THE_MIDDLE", parameters = Map( "format" -> toSQLValue("m%@ca", StringType), "char" -> toSQLValue("@", StringType))) @@ -3801,7 +3801,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException] { sql("SELECT a LIKE 'jialiuping%' ESCAPE '%' FROM df").collect() }, - errorClass = "INVALID_FORMAT.ESC_AT_THE_END", + condition = "INVALID_FORMAT.ESC_AT_THE_END", parameters = Map("format" -> toSQLValue("jialiuping%", StringType))) } } @@ -3901,7 +3901,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException] { sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$sumFuncClass'") }, - errorClass = "CANNOT_LOAD_FUNCTION_CLASS", + condition = "CANNOT_LOAD_FUNCTION_CLASS", parameters = Map( "className" -> "org.apache.spark.examples.sql.Spark33084", "functionName" -> "`test_udf`" @@ -3996,7 +3996,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } checkError( exception = e, - errorClass = "INVALID_TEMP_OBJ_REFERENCE", + condition = "INVALID_TEMP_OBJ_REFERENCE", parameters = Map( "obj" -> "VIEW", "objName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$testViewName`", @@ -4015,7 +4015,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } checkError( exception = e2, - errorClass = "INVALID_TEMP_OBJ_REFERENCE", + condition = "INVALID_TEMP_OBJ_REFERENCE", parameters = Map( "obj" -> "VIEW", "objName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$testViewName`", @@ -4901,7 +4901,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark exception = intercept[AnalysisException]( sql(sqlText) ), - errorClass = "MISSING_WINDOW_SPECIFICATION", + condition = "MISSING_WINDOW_SPECIFICATION", parameters = Map( "windowName" -> "unspecified_window", "docroot" -> SPARK_DOC_ROOT diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 01b9fdec9be3d..16118526f2fe4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -163,7 +163,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession exception = intercept[SparkUnsupportedOperationException] { Seq(InvalidInJava(1)).toDS() }, - errorClass = "_LEGACY_ERROR_TEMP_2140", + condition = "_LEGACY_ERROR_TEMP_2140", parameters = Map( "fieldName" -> "abstract", "walkedTypePath" -> "- root class: \"org.apache.spark.sql.InvalidInJava\"")) @@ -174,7 +174,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession exception = intercept[SparkUnsupportedOperationException] { Seq(InvalidInJava2(1)).toDS() }, - errorClass = "_LEGACY_ERROR_TEMP_2140", + condition = "_LEGACY_ERROR_TEMP_2140", parameters = Map( "fieldName" -> "0", "walkedTypePath" -> diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SetCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SetCommandSuite.scala index a8b359f308a2b..f4ea87b39c39b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SetCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SetCommandSuite.scala @@ -139,7 +139,7 @@ class SetCommandSuite extends QueryTest with SharedSparkSession with ResetSystem withSQLConf(key1 -> value1) { checkError( intercept[ParseException](sql("SET ${test.password}")), - errorClass = "INVALID_SET_SYNTAX" + condition = "INVALID_SET_SYNTAX" ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index 919958d304f10..948a0e3444cd1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -76,14 +76,14 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared exception = intercept[AnalysisException] { sql(s"ANALYZE TABLE $viewName COMPUTE STATISTICS") }, - errorClass = "UNSUPPORTED_FEATURE.ANALYZE_VIEW", + condition = "UNSUPPORTED_FEATURE.ANALYZE_VIEW", parameters = Map.empty ) checkError( exception = intercept[AnalysisException] { sql(s"ANALYZE TABLE $viewName COMPUTE STATISTICS FOR COLUMNS id") }, - errorClass = "UNSUPPORTED_FEATURE.ANALYZE_VIEW", + condition = "UNSUPPORTED_FEATURE.ANALYZE_VIEW", parameters = Map.empty ) } @@ -136,7 +136,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared exception = intercept[AnalysisException] { sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS data") }, - errorClass = "UNSUPPORTED_FEATURE.ANALYZE_UNSUPPORTED_COLUMN_TYPE", + condition = "UNSUPPORTED_FEATURE.ANALYZE_UNSUPPORTED_COLUMN_TYPE", parameters = Map( "columnType" -> "\"ARRAY\"", "columnName" -> "`data`", @@ -149,7 +149,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared exception = intercept[AnalysisException] { sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS some_random_column") }, - errorClass = "COLUMN_NOT_FOUND", + condition = "COLUMN_NOT_FOUND", parameters = Map( "colName" -> "`some_random_column`", "caseSensitiveConfig" -> "\"spark.sql.caseSensitive\"" @@ -630,7 +630,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared exception = intercept[AnalysisException] { sql("ANALYZE TABLE tempView COMPUTE STATISTICS FOR COLUMNS id") }, - errorClass = "UNSUPPORTED_FEATURE.ANALYZE_UNCACHED_TEMP_VIEW", + condition = "UNSUPPORTED_FEATURE.ANALYZE_UNCACHED_TEMP_VIEW", parameters = Map("viewName" -> "`tempView`") ) @@ -656,7 +656,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared exception = intercept[AnalysisException] { sql(s"ANALYZE TABLE $globalTempDB.gTempView COMPUTE STATISTICS FOR COLUMNS id") }, - errorClass = "UNSUPPORTED_FEATURE.ANALYZE_UNCACHED_TEMP_VIEW", + condition = "UNSUPPORTED_FEATURE.ANALYZE_UNCACHED_TEMP_VIEW", parameters = Map("viewName" -> "`global_temp`.`gTempView`") ) @@ -775,7 +775,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared exception = intercept[AnalysisException] { sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS value, name, $dupCol") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`value`")) } } @@ -849,7 +849,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared sql(s"ANALYZE TABLES IN db_not_exists COMPUTE STATISTICS") } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`spark_catalog`.`db_not_exists`")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index 523b3518db48c..c98dddbfe8e9f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -727,7 +727,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("sentences()") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> toSQLId("sentences"), "expectedNum" -> "[1, 2, 3]", @@ -828,7 +828,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("select regexp_replace(collect_list(1), '1', '2')") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"regexp_replace(collect_list(1), 1, 2, 1)\"", @@ -848,7 +848,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { sql("select regexp_replace('', '[a\\\\d]{0, 2}', 'x')").collect() }, - errorClass = "INVALID_PARAMETER_VALUE.PATTERN", + condition = "INVALID_PARAMETER_VALUE.PATTERN", parameters = Map( "parameter" -> toSQLId("regexp"), "functionName" -> toSQLId("regexp_replace"), @@ -859,7 +859,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { sql("select regexp_extract('', '[a\\\\d]{0, 2}', 1)").collect() }, - errorClass = "INVALID_PARAMETER_VALUE.PATTERN", + condition = "INVALID_PARAMETER_VALUE.PATTERN", parameters = Map( "parameter" -> toSQLId("regexp"), "functionName" -> toSQLId("regexp_extract"), @@ -870,7 +870,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkRuntimeException] { sql("select rlike('', '[a\\\\d]{0, 2}')").collect() }, - errorClass = "INVALID_PARAMETER_VALUE.PATTERN", + condition = "INVALID_PARAMETER_VALUE.PATTERN", parameters = Map( "parameter" -> toSQLId("regexp"), "functionName" -> toSQLId("rlike"), @@ -920,7 +920,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.select(func(col("input"), col("format"))).collect() }, - errorClass = "NON_FOLDABLE_ARGUMENT", + condition = "NON_FOLDABLE_ARGUMENT", parameters = Map( "funcName" -> s"`$funcName`", "paramName" -> "`format`", @@ -932,7 +932,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.select(func(col("input"), lit("invalid_format"))).collect() }, - errorClass = "INVALID_PARAMETER_VALUE.BINARY_FORMAT", + condition = "INVALID_PARAMETER_VALUE.BINARY_FORMAT", parameters = Map( "parameter" -> "`format`", "functionName" -> s"`$funcName`", @@ -944,7 +944,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"select $funcName('a', 'b', 'c')") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> s"`$funcName`", "expectedNum" -> "2", @@ -955,7 +955,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"select $funcName(x'537061726b2053514c', CAST(NULL AS STRING))") }, - errorClass = "INVALID_PARAMETER_VALUE.NULL", + condition = "INVALID_PARAMETER_VALUE.NULL", parameters = Map( "functionName" -> s"`$funcName`", "parameter" -> "`format`"), @@ -1058,7 +1058,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df1.select(like(col("a"), col("b"), lit(618))).collect() }, - errorClass = "INVALID_ESCAPE_CHAR", + condition = "INVALID_ESCAPE_CHAR", parameters = Map("sqlExpr" -> "\"618\""), context = ExpectedContext("like", getCurrentClassCallSitePattern) ) @@ -1067,7 +1067,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df1.select(ilike(col("a"), col("b"), lit(618))).collect() }, - errorClass = "INVALID_ESCAPE_CHAR", + condition = "INVALID_ESCAPE_CHAR", parameters = Map("sqlExpr" -> "\"618\""), context = ExpectedContext("ilike", getCurrentClassCallSitePattern) ) @@ -1078,7 +1078,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df1.select(like(col("a"), col("b"), lit("中国"))).collect() }, - errorClass = "INVALID_ESCAPE_CHAR", + condition = "INVALID_ESCAPE_CHAR", parameters = Map("sqlExpr" -> "\"中国\""), context = ExpectedContext("like", getCurrentClassCallSitePattern) ) @@ -1087,7 +1087,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df1.select(ilike(col("a"), col("b"), lit("中国"))).collect() }, - errorClass = "INVALID_ESCAPE_CHAR", + condition = "INVALID_ESCAPE_CHAR", parameters = Map("sqlExpr" -> "\"中国\""), context = ExpectedContext("ilike", getCurrentClassCallSitePattern) ) @@ -1282,7 +1282,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { intercept[SparkRuntimeException](df.queryExecution.optimizedPlan) checkError( exception = intercept[SparkRuntimeException](df.queryExecution.explainString(FormattedMode)), - errorClass = "INVALID_PARAMETER_VALUE.PATTERN", + condition = "INVALID_PARAMETER_VALUE.PATTERN", parameters = Map( "parameter" -> toSQLId("regexp"), "functionName" -> toSQLId("regexp_replace"), @@ -1310,7 +1310,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"select concat_ws(',', collect_list(dat)) FROM $testTable") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> """"concat_ws(,, collect_list(dat))"""", "paramIndex" -> "second", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 68f14f13bbd66..23c4d51983bb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -533,7 +533,7 @@ class SubquerySuite extends QueryTest } checkError( exception, - errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + condition = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "NON_CORRELATED_COLUMNS_IN_GROUP_BY", parameters = Map("value" -> "c2"), sqlState = None, @@ -548,7 +548,7 @@ class SubquerySuite extends QueryTest } checkError( exception1, - errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + condition = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", parameters = Map.empty, context = ExpectedContext( @@ -558,7 +558,7 @@ class SubquerySuite extends QueryTest } checkErrorMatchPVals( exception2, - errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + condition = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", parameters = Map.empty[String, String], sqlState = None, @@ -850,7 +850,7 @@ class SubquerySuite extends QueryTest } checkErrorMatchPVals( exception1, - errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + condition = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", parameters = Map("treeNode" -> "(?s).*"), sqlState = None, @@ -872,7 +872,7 @@ class SubquerySuite extends QueryTest } checkErrorMatchPVals( exception2, - errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + condition = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", parameters = Map("treeNode" -> "(?s).*"), sqlState = None, @@ -893,7 +893,7 @@ class SubquerySuite extends QueryTest } checkErrorMatchPVals( exception3, - errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + condition = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", parameters = Map("treeNode" -> "(?s).*"), sqlState = None, @@ -1057,7 +1057,7 @@ class SubquerySuite extends QueryTest } checkError( exception1, - errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + condition = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", parameters = Map("sqlExprs" -> "\"explode(arr_c2)\""), context = ExpectedContext( fragment = "LATERAL VIEW explode(t2.arr_c2) q AS c2", @@ -1098,7 +1098,7 @@ class SubquerySuite extends QueryTest checkError( exception = intercept[AnalysisException](sql(query)), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> "`a`", @@ -2552,7 +2552,7 @@ class SubquerySuite extends QueryTest |""".stripMargin ).collect() }, - errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + condition = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "UNSUPPORTED_CORRELATED_REFERENCE_DATA_TYPE", parameters = Map("expr" -> "v1.x", "dataType" -> "map"), context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TableOptionsConstantFoldingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TableOptionsConstantFoldingSuite.scala index 2e56327a63136..aa82ac57089f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TableOptionsConstantFoldingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TableOptionsConstantFoldingSuite.scala @@ -70,42 +70,42 @@ class TableOptionsConstantFoldingSuite extends QueryTest with SharedSparkSession checkError( exception = intercept[AnalysisException]( sql(s"$prefix ('k' = 1 + 2 + unresolvedAttribute)")), - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", parameters = Map( "objectName" -> "`unresolvedAttribute`"), queryContext = Array(ExpectedContext("", "", 60, 78, "unresolvedAttribute"))) checkError( exception = intercept[AnalysisException]( sql(s"$prefix ('k' = true or false or unresolvedAttribute)")), - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", parameters = Map( "objectName" -> "`unresolvedAttribute`"), queryContext = Array(ExpectedContext("", "", 69, 87, "unresolvedAttribute"))) checkError( exception = intercept[AnalysisException]( sql(s"$prefix ('k' = cast(array('9', '9') as array))")), - errorClass = "INVALID_SQL_SYNTAX.OPTION_IS_INVALID", + condition = "INVALID_SQL_SYNTAX.OPTION_IS_INVALID", parameters = Map( "key" -> "k", "supported" -> "constant expressions")) checkError( exception = intercept[AnalysisException]( sql(s"$prefix ('k' = cast(map('9', '9') as map))")), - errorClass = "INVALID_SQL_SYNTAX.OPTION_IS_INVALID", + condition = "INVALID_SQL_SYNTAX.OPTION_IS_INVALID", parameters = Map( "key" -> "k", "supported" -> "constant expressions")) checkError( exception = intercept[AnalysisException]( sql(s"$prefix ('k' = raise_error('failure'))")), - errorClass = "INVALID_SQL_SYNTAX.OPTION_IS_INVALID", + condition = "INVALID_SQL_SYNTAX.OPTION_IS_INVALID", parameters = Map( "key" -> "k", "supported" -> "constant expressions")) checkError( exception = intercept[AnalysisException]( sql(s"$prefix ('k' = raise_error('failure'))")), - errorClass = "INVALID_SQL_SYNTAX.OPTION_IS_INVALID", + condition = "INVALID_SQL_SYNTAX.OPTION_IS_INVALID", parameters = Map( "key" -> "k", "supported" -> "constant expressions")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 36552d5c5487c..2e072e5afc926 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -125,7 +125,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.selectExpr("substr('abcd', 2, 3, 4)") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> toSQLId("substr"), "expectedNum" -> "[2, 3]", @@ -146,7 +146,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { spark.udf.register("foo", (_: String).length) df.selectExpr("foo(2, 3, 4)") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> toSQLId("foo"), "expectedNum" -> "1", @@ -166,7 +166,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.emptyDataFrame.selectExpr(sqlText) }, - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`a_function_that_does_not_exist`", "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"), @@ -772,7 +772,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](df.select(myUdf(Column("col")))), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`b`", "proposal" -> "`a`"), @@ -1206,7 +1206,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { ) checkError( intercept[AnalysisException](spark.range(1).select(f())), - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_ENCODER", + condition = "UNSUPPORTED_DATA_TYPE_FOR_ENCODER", sqlState = "0A000", parameters = Map("dataType" -> s"\"${dt.sql}\"") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UrlFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UrlFunctionsSuite.scala index c89ddd0e6a1f1..428065fb6986f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UrlFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UrlFunctionsSuite.scala @@ -76,7 +76,7 @@ class UrlFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkIllegalArgumentException] { sql(s"SELECT parse_url('$url', 'HOST')").collect() }, - errorClass = "INVALID_URL", + condition = "INVALID_URL", parameters = Map( "url" -> url, "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala index 0c8b0b501951f..5c215a7875b59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala @@ -97,7 +97,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval exception = intercept[SparkRuntimeException] { df.select(variant_get(v, "$.a", "binary")).collect() }, - errorClass = "INVALID_VARIANT_CAST", + condition = "INVALID_VARIANT_CAST", parameters = Map("value" -> "1", "dataType" -> "\"BINARY\"") ) @@ -223,7 +223,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval exception = intercept[AnalysisException] { query.write.partitionBy("v").parquet(tempDir) }, - errorClass = "INVALID_PARTITION_COLUMN_DATA_TYPE", + condition = "INVALID_PARTITION_COLUMN_DATA_TYPE", parameters = Map("type" -> "\"VARIANT\"") ) } @@ -239,7 +239,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval exception = intercept[AnalysisException] { query.write.partitionBy("v").saveAsTable("t") }, - errorClass = "INVALID_PARTITION_COLUMN_DATA_TYPE", + condition = "INVALID_PARTITION_COLUMN_DATA_TYPE", parameters = Map("type" -> "\"VARIANT\"") ) } @@ -255,7 +255,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval exception = intercept[AnalysisException] { spark.sql(s"CREATE TABLE t USING PARQUET PARTITIONED BY (v) AS $queryString") }, - errorClass = "INVALID_PARTITION_COLUMN_DATA_TYPE", + condition = "INVALID_PARTITION_COLUMN_DATA_TYPE", parameters = Map("type" -> "\"VARIANT\"") ) } @@ -302,7 +302,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval val e = intercept[org.apache.spark.SparkException](result.collect()) checkError( exception = e.getCause.asInstanceOf[AnalysisException], - errorClass = errorClass, + condition = errorClass, parameters = parameters ) } @@ -346,7 +346,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval exception = intercept[AnalysisException] { spark.read.format("json").option("singleVariantColumn", "var").schema("var variant") }, - errorClass = "INVALID_SINGLE_VARIANT_COLUMN", + condition = "INVALID_SINGLE_VARIANT_COLUMN", parameters = Map.empty ) checkError( @@ -354,7 +354,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval spark.read.format("json").option("singleVariantColumn", "another_name") .schema("var variant").json(file.getAbsolutePath).collect() }, - errorClass = "INVALID_SINGLE_VARIANT_COLUMN", + condition = "INVALID_SINGLE_VARIANT_COLUMN", parameters = Map.empty ) } @@ -422,7 +422,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval exception = intercept[AnalysisException] { spark.sql("select parse_json('') group by 1") }, - errorClass = "GROUP_EXPRESSION_TYPE_IS_NOT_ORDERABLE", + condition = "GROUP_EXPRESSION_TYPE_IS_NOT_ORDERABLE", parameters = Map("sqlExpr" -> "\"parse_json()\"", "dataType" -> "\"VARIANT\""), context = ExpectedContext(fragment = "parse_json('')", start = 7, stop = 20) ) @@ -431,7 +431,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval exception = intercept[AnalysisException] { spark.sql("select parse_json('') order by 1") }, - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", parameters = Map( "functionName" -> "`sortorder`", "dataType" -> "\"VARIANT\"", @@ -443,7 +443,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval exception = intercept[AnalysisException] { spark.sql("select parse_json('') sort by 1") }, - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", parameters = Map( "functionName" -> "`sortorder`", "dataType" -> "\"VARIANT\"", @@ -456,7 +456,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval spark.sql("with t as (select 1 as a, parse_json('') as v) " + "select rank() over (partition by a order by v) from t") }, - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", parameters = Map( "functionName" -> "`sortorder`", "dataType" -> "\"VARIANT\"", @@ -469,7 +469,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval spark.sql("with t as (select parse_json('') as v) " + "select t1.v from t as t1 join t as t2 on t1.v = t2.v") }, - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", parameters = Map( "functionName" -> "`=`", "dataType" -> "\"VARIANT\"", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala index 4169d53e4fc8e..f9d003572a229 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala @@ -126,7 +126,7 @@ class XmlFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq("1").toDS().select(from_xml($"value", lit("ARRAY"), Map[String, String]().asJava)) }, - errorClass = "INVALID_SCHEMA.NON_STRUCT_TYPE", + condition = "INVALID_SCHEMA.NON_STRUCT_TYPE", parameters = Map( "inputSchema" -> "\"ARRAY\"", "dataType" -> "\"ARRAY\"" @@ -138,7 +138,7 @@ class XmlFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq("1").toDF("xml").selectExpr(s"from_xml(xml, 'ARRAY')") }, - errorClass = "INVALID_SCHEMA.NON_STRUCT_TYPE", + condition = "INVALID_SCHEMA.NON_STRUCT_TYPE", parameters = Map( "inputSchema" -> "\"ARRAY\"", "dataType" -> "\"ARRAY\"" @@ -285,7 +285,7 @@ class XmlFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.selectExpr("to_xml(a, named_struct('a', 1))") }, - errorClass = "INVALID_OPTIONS.NON_MAP_FUNCTION", + condition = "INVALID_OPTIONS.NON_MAP_FUNCTION", parameters = Map.empty, context = ExpectedContext( fragment = "to_xml(a, named_struct('a', 1))", @@ -298,7 +298,7 @@ class XmlFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df2.selectExpr("to_xml(a, map('a', 1))") }, - errorClass = "INVALID_OPTIONS.NON_STRING_TYPE", + condition = "INVALID_OPTIONS.NON_STRING_TYPE", parameters = Map("mapType" -> "\"MAP\""), context = ExpectedContext( fragment = "to_xml(a, map('a', 1))", @@ -350,7 +350,7 @@ class XmlFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("from_xml(value, 1)") }, - errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL", + condition = "INVALID_SCHEMA.NON_STRING_LITERAL", parameters = Map("inputSchema" -> "\"1\""), context = ExpectedContext( fragment = "from_xml(value, 1)", @@ -362,7 +362,7 @@ class XmlFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("""from_xml(value, 'time InvalidType')""") }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map( "error" -> "'InvalidType'", @@ -378,7 +378,7 @@ class XmlFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("from_xml(value, 'time Timestamp', named_struct('a', 1))") }, - errorClass = "INVALID_OPTIONS.NON_MAP_FUNCTION", + condition = "INVALID_OPTIONS.NON_MAP_FUNCTION", parameters = Map.empty, context = ExpectedContext( fragment = "from_xml(value, 'time Timestamp', named_struct('a', 1))", @@ -390,7 +390,7 @@ class XmlFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df3.selectExpr("from_xml(value, 'time Timestamp', map('a', 1))") }, - errorClass = "INVALID_OPTIONS.NON_STRING_TYPE", + condition = "INVALID_OPTIONS.NON_STRING_TYPE", parameters = Map("mapType" -> "\"MAP\""), context = ExpectedContext( fragment = "from_xml(value, 'time Timestamp', map('a', 1))", @@ -518,7 +518,7 @@ class XmlFunctionsSuite extends QueryTest with SharedSparkSession { Seq(("""1""", "i int")).toDF("xml", "schema") .select(from_xml($"xml", $"schema", options)).collect() }, - errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL", + condition = "INVALID_SCHEMA.NON_STRING_LITERAL", parameters = Map("inputSchema" -> "\"schema\""), context = ExpectedContext(fragment = "from_xml", getCurrentClassCallSitePattern) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/catalyst/expressions/ValidateExternalTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/catalyst/expressions/ValidateExternalTypeSuite.scala index 57b9e592f31b0..6e54c2a1942ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/catalyst/expressions/ValidateExternalTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/catalyst/expressions/ValidateExternalTypeSuite.scala @@ -32,7 +32,7 @@ class ValidateExternalTypeSuite extends QueryTest with SharedSparkSession { ) )), new StructType().add("f3", StringType)).show() }.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "INVALID_EXTERNAL_TYPE", + condition = "INVALID_EXTERNAL_TYPE", parameters = Map( ("externalType", "[B"), ("type", "\"STRING\""), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index 5b0d61fb6d771..21aa57cc1eace 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -166,7 +166,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { sql(s"ALTER TABLE $t ADD COLUMN c string AFTER non_exist")) checkError( exception = e1, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`c`", "fields" -> "a, point, b") ) @@ -191,7 +191,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { sql(s"ALTER TABLE $t ADD COLUMN point.x2 int AFTER non_exist")) checkError( exception = e2, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`x2`", "fields" -> "y, x, z") ) } @@ -231,7 +231,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { sql(s"ALTER TABLE $t ADD COLUMNS (yy int AFTER xx, xx int)")) checkError( exception = e, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`yy`", "fields" -> "a, x, y, z, b, point") ) } @@ -372,7 +372,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql("alter table t add column s bigint default badvalue") }, - errorClass = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", parameters = Map( "statement" -> "ALTER TABLE", "colName" -> "`s`", @@ -383,7 +383,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql("alter table t alter column s set default badvalue") }, - errorClass = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", parameters = Map( "statement" -> "ALTER TABLE ALTER COLUMN", "colName" -> "`s`", @@ -437,7 +437,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`point`", @@ -475,7 +475,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t ADD COLUMNS $field double") }, - errorClass = "FIELD_ALREADY_EXISTS", + condition = "FIELD_ALREADY_EXISTS", parameters = expectedParameters, context = ExpectedContext( fragment = s"ALTER TABLE $t ADD COLUMNS $field double", @@ -494,7 +494,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t ADD COLUMNS (data string, data1 string, data string)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`data`")) } } @@ -507,7 +507,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t ADD COLUMNS (point.z double, point.z double, point.xx double)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> toSQLId("point.z"))) } } @@ -538,7 +538,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "CANNOT_UPDATE_FIELD.INTERVAL_TYPE", + condition = "CANNOT_UPDATE_FIELD.INTERVAL_TYPE", parameters = Map( "table" -> s"${toSQLId(prependCatalogName(t))}", "fieldName" -> "`id`"), @@ -600,7 +600,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "CANNOT_UPDATE_FIELD.STRUCT_TYPE", + condition = "CANNOT_UPDATE_FIELD.STRUCT_TYPE", parameters = Map( "table" -> s"${toSQLId(prependCatalogName(t))}", "fieldName" -> "`point`"), @@ -631,7 +631,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "CANNOT_UPDATE_FIELD.ARRAY_TYPE", + condition = "CANNOT_UPDATE_FIELD.ARRAY_TYPE", parameters = Map( "table" -> s"${toSQLId(prependCatalogName(t))}", "fieldName" -> "`points`"), @@ -675,7 +675,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "CANNOT_UPDATE_FIELD.MAP_TYPE", + condition = "CANNOT_UPDATE_FIELD.MAP_TYPE", parameters = Map( "table" -> s"${toSQLId(prependCatalogName(t))}", "fieldName" -> "`m`"), @@ -772,7 +772,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`data`", @@ -791,7 +791,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`point`.`x`", @@ -809,7 +809,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", sqlState = None, parameters = Map( "originType" -> "\"INT\"", @@ -866,7 +866,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText1) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`non_exist`", @@ -896,7 +896,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText2) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`point`.`non_exist`", @@ -989,7 +989,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`data`", @@ -1008,7 +1008,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`point`.`x`", @@ -1110,7 +1110,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`data`", @@ -1129,7 +1129,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`point`.`x`", @@ -1177,7 +1177,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t RENAME COLUMN $field TO $newName") }, - errorClass = "FIELD_ALREADY_EXISTS", + condition = "FIELD_ALREADY_EXISTS", parameters = Map( "op" -> "rename", "fieldNames" -> s"${toSQLId(expectedName)}", @@ -1282,7 +1282,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`data`", @@ -1306,7 +1306,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`point`.`x`", @@ -1392,7 +1392,7 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t REPLACE COLUMNS (data string, data1 string, data string)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`data`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index 7d48459a8a517..c1e8b70ffddbe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -184,17 +184,17 @@ class DataSourceV2DataFrameSuite val v2Writer = df.writeTo("testcat.table_name") checkError( exception = intercept[AnalysisException](v2Writer.append()), - errorClass = "_LEGACY_ERROR_TEMP_1183", + condition = "_LEGACY_ERROR_TEMP_1183", parameters = Map.empty ) checkError( exception = intercept[AnalysisException](v2Writer.overwrite(df("i"))), - errorClass = "_LEGACY_ERROR_TEMP_1183", + condition = "_LEGACY_ERROR_TEMP_1183", parameters = Map.empty ) checkError( exception = intercept[AnalysisException](v2Writer.overwritePartitions()), - errorClass = "_LEGACY_ERROR_TEMP_1183", + condition = "_LEGACY_ERROR_TEMP_1183", parameters = Map.empty ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala index 95bdb2543e376..d6599debd3b11 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala @@ -145,7 +145,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { exception = intercept[AnalysisException]( sql("SELECT testcat.non_exist('abc')").collect() ), - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`testcat`.`non_exist`", "searchPath" -> "[`system`.`builtin`, `system`.`session`, `testcat`.`default`]"), @@ -161,7 +161,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { exception = intercept[AnalysisException]( sql("SELECT testcat.strlen('abc')").collect() ), - errorClass = "_LEGACY_ERROR_TEMP_1184", + condition = "_LEGACY_ERROR_TEMP_1184", parameters = Map("plugin" -> "testcat", "ability" -> "functions") ) } @@ -174,7 +174,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { exception = intercept[AnalysisException] { sql("DESCRIBE FUNCTION testcat.abc") }, - errorClass = "_LEGACY_ERROR_TEMP_1184", + condition = "_LEGACY_ERROR_TEMP_1184", parameters = Map( "plugin" -> "testcat", "ability" -> "functions" @@ -185,7 +185,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { exception = intercept[AnalysisException] { sql("DESCRIBE FUNCTION default.ns1.ns2.fun") }, - errorClass = "REQUIRES_SINGLE_PART_NAMESPACE", + condition = "REQUIRES_SINGLE_PART_NAMESPACE", parameters = Map( "sessionCatalog" -> "spark_catalog", "namespace" -> "`default`.`ns1`.`ns2`") @@ -343,7 +343,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { checkError( exception = intercept[AnalysisException](sql("SELECT testcat.ns.strlen(42)")), - errorClass = "_LEGACY_ERROR_TEMP_1198", + condition = "_LEGACY_ERROR_TEMP_1198", parameters = Map( "unbound" -> "strlen", "arguments" -> "int", @@ -358,7 +358,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { checkError( exception = intercept[AnalysisException](sql("SELECT testcat.ns.strlen('a', 'b')")), - errorClass = "_LEGACY_ERROR_TEMP_1198", + condition = "_LEGACY_ERROR_TEMP_1198", parameters = Map( "unbound" -> "strlen", "arguments" -> "string, string", @@ -414,7 +414,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { new JavaStrLen(new JavaStrLenNoImpl)) checkError( exception = intercept[AnalysisException](sql("SELECT testcat.ns.strlen('abc')").collect()), - errorClass = "_LEGACY_ERROR_TEMP_3055", + condition = "_LEGACY_ERROR_TEMP_3055", parameters = Map("scalarFunc" -> "strlen"), context = ExpectedContext( fragment = "testcat.ns.strlen('abc')", @@ -429,7 +429,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { addFunction(Identifier.of(Array("ns"), "strlen"), StrLen(StrLenBadInputTypes)) checkError( exception = intercept[AnalysisException](sql("SELECT testcat.ns.strlen('abc')").collect()), - errorClass = "_LEGACY_ERROR_TEMP_1199", + condition = "_LEGACY_ERROR_TEMP_1199", parameters = Map( "bound" -> "strlen_bad_input_types", "argsLen" -> "1", @@ -448,7 +448,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { addFunction(Identifier.of(Array("ns"), "add"), new JavaLongAdd(new JavaLongAddMismatchMagic)) checkError( exception = intercept[AnalysisException](sql("SELECT testcat.ns.add(1L, 2L)").collect()), - errorClass = "_LEGACY_ERROR_TEMP_3055", + condition = "_LEGACY_ERROR_TEMP_3055", parameters = Map("scalarFunc" -> "long_add_mismatch_magic"), context = ExpectedContext( fragment = "testcat.ns.add(1L, 2L)", @@ -481,7 +481,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { exception = intercept[AnalysisException] { sql(sqlText).collect() }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> ".*", @@ -539,7 +539,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { checkError( exception = intercept[AnalysisException]( sql("SELECT testcat.ns.strlen('abc')")), - errorClass = "INVALID_UDF_IMPLEMENTATION", + condition = "INVALID_UDF_IMPLEMENTATION", parameters = Map( "funcName" -> "`bad_bound_func`"), context = ExpectedContext( @@ -602,7 +602,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { Seq(1.toShort, 2.toShort).toDF("i").write.saveAsTable(t) checkError( exception = intercept[AnalysisException](sql(s"SELECT testcat.ns.avg(i) from $t")), - errorClass = "_LEGACY_ERROR_TEMP_1198", + condition = "_LEGACY_ERROR_TEMP_1198", parameters = Map( "unbound" -> "iavg", "arguments" -> "smallint", @@ -637,7 +637,7 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { sql("SELECT testcat.ns.avg(*) from values " + "(date '2021-06-01' - date '2011-06-01'), (date '2000-01-01' - date '1900-01-01')") }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"v2aggregator(col1)\"", "paramIndex" -> "first", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 922bf01b541a1..998d459cd436c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -132,7 +132,7 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = analysisException(s"DESCRIBE $t invalid_col"), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`invalid_col`", "proposal" -> "`id`, `data`"), @@ -165,7 +165,7 @@ class DataSourceV2SQLSuiteV1Filter sql(s"CREATE TABLE $t (d struct) USING foo") checkError( exception = analysisException(s"describe $t d.a"), - errorClass = "_LEGACY_ERROR_TEMP_1060", + condition = "_LEGACY_ERROR_TEMP_1060", parameters = Map( "command" -> "DESC TABLE COLUMN", "column" -> "d.a")) @@ -219,7 +219,7 @@ class DataSourceV2SQLSuiteV1Filter spark.sql("CREATE TABLE testcat.table_name " + "(id bigint, data string, id2 bigint) USING bar") }, - errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> "`table_name`")) // table should not have changed @@ -302,14 +302,14 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"$action TABLE table_name (id int, value interval) USING $v2Format") }, - errorClass = "_LEGACY_ERROR_TEMP_1183", + condition = "_LEGACY_ERROR_TEMP_1183", parameters = Map.empty) checkError( exception = intercept[AnalysisException] { sql(s"$action TABLE table_name (id array) USING $v2Format") }, - errorClass = "_LEGACY_ERROR_TEMP_1183", + condition = "_LEGACY_ERROR_TEMP_1183", parameters = Map.empty) } } @@ -321,14 +321,14 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"$action TABLE table_name USING $v2Format as select interval 1 day") }, - errorClass = "_LEGACY_ERROR_TEMP_1183", + condition = "_LEGACY_ERROR_TEMP_1183", parameters = Map.empty) checkError( exception = intercept[AnalysisException] { sql(s"$action TABLE table_name USING $v2Format as select array(interval 1 day)") }, - errorClass = "_LEGACY_ERROR_TEMP_1183", + condition = "_LEGACY_ERROR_TEMP_1183", parameters = Map.empty) } } @@ -662,7 +662,7 @@ class DataSourceV2SQLSuiteV1Filter spark.sql(s"REPLACE TABLE $catalog.replaced USING $v2Source " + s"AS SELECT id, data FROM source") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`replaced`")) } } @@ -677,7 +677,7 @@ class DataSourceV2SQLSuiteV1Filter s" TBLPROPERTIES (`$SIMULATE_DROP_BEFORE_REPLACE_PROPERTY`=true)" + s" AS SELECT id, data FROM source") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`replaced`")) } @@ -720,7 +720,7 @@ class DataSourceV2SQLSuiteV1Filter spark.sql("CREATE TABLE testcat.table_name USING bar AS " + "SELECT id, data, id as id2 FROM source2") }, - errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> "`table_name`")) // table should not have changed @@ -1072,7 +1072,7 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = analysisException(s"SELECT ns1.ns2.ns3.tbl.id from $t"), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`ns1`.`ns2`.`ns3`.`tbl`.`id`", "proposal" -> "`testcat`.`ns1`.`ns2`.`tbl`.`id`, `testcat`.`ns1`.`ns2`.`tbl`.`point`"), @@ -1135,7 +1135,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"INSERT INTO $t1 VALUES(4)") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`tbl`", "tableColumns" -> "`id`, `data`", @@ -1147,7 +1147,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"INSERT INTO $t1(data, data) VALUES(5)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`data`")) } } @@ -1170,7 +1170,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"INSERT OVERWRITE $t1 VALUES(4)") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`tbl`", "tableColumns" -> "`id`, `data`", @@ -1182,7 +1182,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"INSERT OVERWRITE $t1(data, data) VALUES(5)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`data`")) } } @@ -1206,7 +1206,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"INSERT OVERWRITE $t1 VALUES('a', 4)") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`tbl`", "tableColumns" -> "`id`, `data`, `data2`", @@ -1218,7 +1218,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"INSERT OVERWRITE $t1(data, data) VALUES(5)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`data`")) } } @@ -1230,7 +1230,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql("INSERT OVERWRITE t PARTITION (c='1') (c) VALUES ('2')") }, - errorClass = "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST", + condition = "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST", parameters = Map("staticName" -> "c")) } } @@ -1240,7 +1240,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql("SHOW VIEWS FROM a.b") }, - errorClass = "_LEGACY_ERROR_TEMP_1126", + condition = "_LEGACY_ERROR_TEMP_1126", parameters = Map("catalog" -> "a.b")) } @@ -1249,7 +1249,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql("SHOW VIEWS FROM testcat") }, - errorClass = "_LEGACY_ERROR_TEMP_1184", + condition = "_LEGACY_ERROR_TEMP_1184", parameters = Map("plugin" -> "testcat", "ability" -> "views")) } @@ -1271,7 +1271,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", parameters = Map( "property" -> key, "msg" -> keyParameters.getOrElse( @@ -1288,7 +1288,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[ParseException] { sql(sql1) }, - errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", parameters = Map( "property" -> key, "msg" -> keyParameters.getOrElse( @@ -1303,7 +1303,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[ParseException] { sql(sql2) }, - errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", parameters = Map( "property" -> key, "msg" -> keyParameters.getOrElse( @@ -1348,7 +1348,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[ParseException] { sql(sql1) }, - errorClass = "_LEGACY_ERROR_TEMP_0032", + condition = "_LEGACY_ERROR_TEMP_0032", parameters = Map("pathOne" -> "foo", "pathTwo" -> "bar"), context = ExpectedContext( fragment = sql1, @@ -1361,7 +1361,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[ParseException] { sql(sql2) }, - errorClass = "_LEGACY_ERROR_TEMP_0032", + condition = "_LEGACY_ERROR_TEMP_0032", parameters = Map("pathOne" -> "foo", "pathTwo" -> "bar"), context = ExpectedContext( fragment = sql2, @@ -1453,7 +1453,7 @@ class DataSourceV2SQLSuiteV1Filter sql("USE ns1") } checkError(exception, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`spark_catalog`.`ns1`")) } @@ -1464,7 +1464,7 @@ class DataSourceV2SQLSuiteV1Filter sql("USE testcat.ns1.ns2") } checkError(exception, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`testcat`.`ns1`.`ns2`")) } @@ -1503,7 +1503,7 @@ class DataSourceV2SQLSuiteV1Filter sql("USE dummy") sql(s"$statement dummy.$tableDefinition") }, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", parameters = Map( "tableName" -> "`dummy`.`my_tab`", "operation" -> "column default value" @@ -1535,7 +1535,7 @@ class DataSourceV2SQLSuiteV1Filter sql("USE dummy") sql(s"$statement dummy.$tableDefinition USING foo") }, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", parameters = Map( "tableName" -> "`dummy`.`my_tab`", "operation" -> "generated columns" @@ -1559,7 +1559,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"$statement testcat.$tableDefinition USING foo") }, - errorClass = "GENERATED_COLUMN_WITH_DEFAULT_VALUE", + condition = "GENERATED_COLUMN_WITH_DEFAULT_VALUE", parameters = Map( "colName" -> "eventYear", "defaultValue" -> "0", @@ -1584,7 +1584,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(customTableDef.getOrElse(tableDef)) }, - errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + condition = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", parameters = Map( "fieldName" -> "b", "expressionStr" -> expr, @@ -1627,7 +1627,7 @@ class DataSourceV2SQLSuiteV1Filter sql(s"CREATE TABLE testcat.$tblName(a INT, " + "b INT GENERATED ALWAYS AS (B + 1)) USING foo") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`B`", "proposal" -> "`a`"), context = ExpectedContext(fragment = "B", start = 0, stop = 0) ) @@ -1685,7 +1685,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"CREATE TABLE testcat.$tblName(a INT, b INT GENERATED ALWAYS AS (c + 1)) USING foo") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`c`", "proposal" -> "`a`"), context = ExpectedContext(fragment = "c", start = 0, stop = 0) ) @@ -1830,7 +1830,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(statement) }, - errorClass = "_LEGACY_ERROR_TEMP_3060", + condition = "_LEGACY_ERROR_TEMP_3060", parameters = Map( "i" -> i, "schema" -> @@ -1857,22 +1857,22 @@ class DataSourceV2SQLSuiteV1Filter withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { checkError( exception = analysisException(s"CREATE TABLE t ($c0 INT, $c1 INT) USING $v2Source"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c0.toLowerCase(Locale.ROOT)}`")) checkError( exception = analysisException( s"CREATE TABLE testcat.t ($c0 INT, $c1 INT) USING $v2Source"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c0.toLowerCase(Locale.ROOT)}`")) checkError( exception = analysisException( s"CREATE OR REPLACE TABLE t ($c0 INT, $c1 INT) USING $v2Source"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c0.toLowerCase(Locale.ROOT)}`")) checkError( exception = analysisException( s"CREATE OR REPLACE TABLE testcat.t ($c0 INT, $c1 INT) USING $v2Source"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c0.toLowerCase(Locale.ROOT)}`")) } } @@ -1884,23 +1884,23 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = analysisException( s"CREATE TABLE t (d struct<$c0: INT, $c1: INT>) USING $v2Source"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> toSQLId(s"d.${c0.toLowerCase(Locale.ROOT)}")) ) checkError( exception = analysisException( s"CREATE TABLE testcat.t (d struct<$c0: INT, $c1: INT>) USING $v2Source"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> toSQLId(s"d.${c0.toLowerCase(Locale.ROOT)}"))) checkError( exception = analysisException( s"CREATE OR REPLACE TABLE t (d struct<$c0: INT, $c1: INT>) USING $v2Source"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> toSQLId(s"d.${c0.toLowerCase(Locale.ROOT)}"))) checkError( exception = analysisException( s"CREATE OR REPLACE TABLE testcat.t (d struct<$c0: INT, $c1: INT>) USING $v2Source"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> toSQLId(s"d.${c0.toLowerCase(Locale.ROOT)}"))) } } @@ -1910,7 +1910,7 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = analysisException( s"CREATE TABLE tbl (a int, b string) USING $v2Source CLUSTERED BY (c) INTO 4 BUCKETS"), - errorClass = "_LEGACY_ERROR_TEMP_3060", + condition = "_LEGACY_ERROR_TEMP_3060", parameters = Map( "i" -> "c", "schema" -> @@ -1921,7 +1921,7 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = analysisException(s"CREATE TABLE testcat.tbl (a int, b string) " + s"USING $v2Source CLUSTERED BY (c) INTO 4 BUCKETS"), - errorClass = "_LEGACY_ERROR_TEMP_3060", + condition = "_LEGACY_ERROR_TEMP_3060", parameters = Map( "i" -> "c", "schema" -> @@ -1932,7 +1932,7 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = analysisException(s"CREATE OR REPLACE TABLE tbl (a int, b string) " + s"USING $v2Source CLUSTERED BY (c) INTO 4 BUCKETS"), - errorClass = "_LEGACY_ERROR_TEMP_3060", + condition = "_LEGACY_ERROR_TEMP_3060", parameters = Map( "i" -> "c", "schema" -> @@ -1943,7 +1943,7 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = analysisException(s"CREATE OR REPLACE TABLE testcat.tbl (a int, b string) " + s"USING $v2Source CLUSTERED BY (c) INTO 4 BUCKETS"), - errorClass = "_LEGACY_ERROR_TEMP_3060", + condition = "_LEGACY_ERROR_TEMP_3060", parameters = Map( "i" -> "c", "schema" -> @@ -1978,22 +1978,22 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = analysisException( s"CREATE TABLE t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)"), - errorClass = "_LEGACY_ERROR_TEMP_3058", + condition = "_LEGACY_ERROR_TEMP_3058", parameters = Map("checkType" -> "in the partitioning", "duplicateColumns" -> dupCol)) checkError( exception = analysisException( s"CREATE TABLE testcat.t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)"), - errorClass = "_LEGACY_ERROR_TEMP_3058", + condition = "_LEGACY_ERROR_TEMP_3058", parameters = Map("checkType" -> "in the partitioning", "duplicateColumns" -> dupCol)) checkError( exception = analysisException( s"CREATE OR REPLACE TABLE t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)"), - errorClass = "_LEGACY_ERROR_TEMP_3058", + condition = "_LEGACY_ERROR_TEMP_3058", parameters = Map("checkType" -> "in the partitioning", "duplicateColumns" -> dupCol)) checkError( exception = analysisException(s"CREATE OR REPLACE TABLE testcat.t ($c0 INT) " + s"USING $v2Source PARTITIONED BY ($c0, $c1)"), - errorClass = "_LEGACY_ERROR_TEMP_3058", + condition = "_LEGACY_ERROR_TEMP_3058", parameters = Map("checkType" -> "in the partitioning", "duplicateColumns" -> dupCol)) } } @@ -2007,26 +2007,26 @@ class DataSourceV2SQLSuiteV1Filter exception = analysisException( s"CREATE TABLE t ($c0 INT) USING $v2Source " + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map( "columnName" -> s"`${c0.toLowerCase(Locale.ROOT)}`")) checkError( exception = analysisException( s"CREATE TABLE testcat.t ($c0 INT) USING $v2Source " + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c0.toLowerCase(Locale.ROOT)}`")) checkError( exception = analysisException( s"CREATE OR REPLACE TABLE t ($c0 INT) USING $v2Source " + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c0.toLowerCase(Locale.ROOT)}`")) checkError( exception = analysisException( s"CREATE OR REPLACE TABLE testcat.t ($c0 INT) USING $v2Source " + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS"), - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c0.toLowerCase(Locale.ROOT)}`")) } } @@ -2130,7 +2130,7 @@ class DataSourceV2SQLSuiteV1Filter } checkError( exception = e, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", sqlState = "0A000", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`tbl`", @@ -2148,7 +2148,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"DELETE FROM $t WHERE id = 2 AND id = id") }, - errorClass = "_LEGACY_ERROR_TEMP_1110", + condition = "_LEGACY_ERROR_TEMP_1110", parameters = Map( "table" -> "testcat.ns1.ns2.tbl", "filters" -> "[id = 2, id = id]")) @@ -2169,7 +2169,7 @@ class DataSourceV2SQLSuiteV1Filter // UPDATE non-existing table checkError( exception = analysisException("UPDATE dummy SET name='abc'"), - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`dummy`"), context = ExpectedContext( fragment = "dummy", @@ -2179,7 +2179,7 @@ class DataSourceV2SQLSuiteV1Filter // UPDATE non-existing column checkError( exception = analysisException(s"UPDATE $t SET dummy='abc'"), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`dummy`", "proposal" -> "`age`, `id`, `name`, `p`" @@ -2190,7 +2190,7 @@ class DataSourceV2SQLSuiteV1Filter stop = 41)) checkError( exception = analysisException(s"UPDATE $t SET name='abc' WHERE dummy=1"), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`dummy`", "proposal" -> "`age`, `id`, `name`, `p`" @@ -2205,7 +2205,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[SparkUnsupportedOperationException] { sql(s"UPDATE $t SET name='Robert', age=32 WHERE p=1") }, - errorClass = "_LEGACY_ERROR_TEMP_2096", + condition = "_LEGACY_ERROR_TEMP_2096", parameters = Map("ddl" -> "UPDATE TABLE") ) } @@ -2240,7 +2240,7 @@ class DataSourceV2SQLSuiteV1Filter |WHEN NOT MATCHED AND (target.col2='insert') |THEN INSERT * """.stripMargin), - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`testcat`.`ns1`.`ns2`.`dummy`"), context = ExpectedContext( fragment = "testcat.ns1.ns2.dummy", @@ -2260,7 +2260,7 @@ class DataSourceV2SQLSuiteV1Filter |WHEN NOT MATCHED AND (target.col2='insert') |THEN INSERT * """.stripMargin), - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`testcat`.`ns1`.`ns2`.`dummy`"), context = ExpectedContext( fragment = "testcat.ns1.ns2.dummy", @@ -2278,7 +2278,7 @@ class DataSourceV2SQLSuiteV1Filter |THEN INSERT *""".stripMargin checkError( exception = analysisException(sql1), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`target`.`dummy`", "proposal" -> "`age`, `id`, `name`, `p`"), @@ -2294,7 +2294,7 @@ class DataSourceV2SQLSuiteV1Filter |WHEN MATCHED AND (target.age > 10) THEN UPDATE SET target.age = source.dummy |WHEN NOT MATCHED AND (target.col2='insert') |THEN INSERT *""".stripMargin), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`source`.`dummy`", "proposal" -> "`age`, `age`, `id`, `id`, `name`, `name`, `p`, `p`"), @@ -2311,7 +2311,7 @@ class DataSourceV2SQLSuiteV1Filter |WHEN MATCHED AND (target.p > 0) THEN UPDATE SET * |WHEN NOT MATCHED THEN INSERT *""".stripMargin) }, - errorClass = "_LEGACY_ERROR_TEMP_2096", + condition = "_LEGACY_ERROR_TEMP_2096", parameters = Map("ddl" -> "MERGE INTO TABLE")) } } @@ -2324,7 +2324,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql("ALTER VIEW testcat.ns1.ns2.old RENAME TO ns1.new") }, - errorClass = "_LEGACY_ERROR_TEMP_1123", + condition = "_LEGACY_ERROR_TEMP_1123", parameters = Map.empty) } } @@ -2418,7 +2418,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"CREATE VIEW $v AS SELECT 1") }, - errorClass = "_LEGACY_ERROR_TEMP_1184", + condition = "_LEGACY_ERROR_TEMP_1184", parameters = Map("plugin" -> "testcat", "ability" -> "views")) } @@ -2456,7 +2456,7 @@ class DataSourceV2SQLSuiteV1Filter // the session catalog, not the `global_temp` v2 catalog. sql(s"CREATE TABLE $globalTempDB.ns1.ns2.tbl (id bigint, data string) USING json") }, - errorClass = "REQUIRES_SINGLE_PART_NAMESPACE", + condition = "REQUIRES_SINGLE_PART_NAMESPACE", parameters = Map( "sessionCatalog" -> "spark_catalog", "namespace" -> "`global_temp`.`ns1`.`ns2`")) @@ -2492,7 +2492,7 @@ class DataSourceV2SQLSuiteV1Filter def verify(sql: String): Unit = { checkError( exception = intercept[AnalysisException](spark.sql(sql)), - errorClass = "REQUIRES_SINGLE_PART_NAMESPACE", + condition = "REQUIRES_SINGLE_PART_NAMESPACE", parameters = Map("sessionCatalog" -> "spark_catalog", "namespace" -> "")) } @@ -2568,7 +2568,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"CREATE VIEW $sessionCatalogName.default.v AS SELECT * FROM t") }, - errorClass = "INVALID_TEMP_OBJ_REFERENCE", + condition = "INVALID_TEMP_OBJ_REFERENCE", parameters = Map( "obj" -> "VIEW", "objName" -> "`spark_catalog`.`default`.`v`", @@ -2600,7 +2600,7 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = intercept[AnalysisException](sql("COMMENT ON NAMESPACE abc IS NULL")), - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`spark_catalog`.`abc`")) // V2 non-session catalog is used. @@ -2610,7 +2610,7 @@ class DataSourceV2SQLSuiteV1Filter checkNamespaceComment("testcat.ns1", "NULL") checkError( exception = intercept[AnalysisException](sql("COMMENT ON NAMESPACE testcat.abc IS NULL")), - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`testcat`.`abc`")) } @@ -2636,7 +2636,7 @@ class DataSourceV2SQLSuiteV1Filter val sql1 = "COMMENT ON TABLE abc IS NULL" checkError( exception = intercept[AnalysisException](sql(sql1)), - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`abc`"), context = ExpectedContext(fragment = "abc", start = 17, stop = 19)) @@ -2650,7 +2650,7 @@ class DataSourceV2SQLSuiteV1Filter val sql2 = "COMMENT ON TABLE testcat.abc IS NULL" checkError( exception = intercept[AnalysisException](sql(sql2)), - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`testcat`.`abc`"), context = ExpectedContext(fragment = "testcat.abc", start = 17, stop = 27)) @@ -2660,7 +2660,7 @@ class DataSourceV2SQLSuiteV1Filter sql("create global temp view v as select 1") checkError( exception = intercept[AnalysisException](sql("COMMENT ON TABLE global_temp.v IS NULL")), - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> "`global_temp`.`v`", "operation" -> "COMMENT ON TABLE"), @@ -2700,7 +2700,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"SELECT ns1.ns2.ns3.tbl.* from $t") }, - errorClass = "CANNOT_RESOLVE_STAR_EXPAND", + condition = "CANNOT_RESOLVE_STAR_EXPAND", parameters = Map( "targetString" -> "`ns1`.`ns2`.`ns3`.`tbl`", "columns" -> "`id`, `name`"), @@ -2764,7 +2764,7 @@ class DataSourceV2SQLSuiteV1Filter val e = intercept[AnalysisException](sql(sqlStatement)) checkError( e, - errorClass = "UNSUPPORTED_FEATURE.CATALOG_OPERATION", + condition = "UNSUPPORTED_FEATURE.CATALOG_OPERATION", parameters = Map("catalogName" -> "`testcat`", "operation" -> "views")) } @@ -2823,7 +2823,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[CatalogNotFoundException] { sql("SET CATALOG not_exist_catalog") }, - errorClass = "CATALOG_NOT_FOUND", + condition = "CATALOG_NOT_FOUND", parameters = Map( "catalogName" -> "`not_exist_catalog`", "config" -> "\"spark.sql.catalog.not_exist_catalog\"")) @@ -2859,7 +2859,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`non_exist`", @@ -2871,7 +2871,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(sql2) }, - errorClass = "_LEGACY_ERROR_TEMP_1332", + condition = "_LEGACY_ERROR_TEMP_1332", parameters = Map( "errorMessage" -> "CreateIndex is not supported in this table testcat.tbl.")) } @@ -3082,7 +3082,7 @@ class DataSourceV2SQLSuiteV1Filter // a fake time travel implementation that only supports two hardcoded timestamp values. sql("SELECT * FROM t TIMESTAMP AS OF current_date()") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`t`"), context = ExpectedContext( fragment = "t", @@ -3093,7 +3093,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql("SELECT * FROM t TIMESTAMP AS OF INTERVAL 1 DAY").collect() }, - errorClass = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.INPUT", + condition = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.INPUT", parameters = Map( "expr" -> "\"INTERVAL '1' DAY\"")) @@ -3101,14 +3101,14 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql("SELECT * FROM t TIMESTAMP AS OF 'abc'").collect() }, - errorClass = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.INPUT", + condition = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.INPUT", parameters = Map("expr" -> "\"abc\"")) checkError( exception = intercept[AnalysisException] { spark.read.option("timestampAsOf", "abc").table("t").collect() }, - errorClass = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.OPTION", + condition = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.OPTION", parameters = Map("expr" -> "'abc'")) checkError( @@ -3119,27 +3119,27 @@ class DataSourceV2SQLSuiteV1Filter .table("t") .collect() }, - errorClass = "INVALID_TIME_TRAVEL_SPEC") + condition = "INVALID_TIME_TRAVEL_SPEC") checkError( exception = intercept[AnalysisException] { sql("SELECT * FROM t TIMESTAMP AS OF current_user()").collect() }, - errorClass = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.UNEVALUABLE", + condition = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.UNEVALUABLE", parameters = Map("expr" -> "\"current_user()\"")) checkError( exception = intercept[AnalysisException] { sql("SELECT * FROM t TIMESTAMP AS OF CAST(rand() AS STRING)").collect() }, - errorClass = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.NON_DETERMINISTIC", + condition = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.NON_DETERMINISTIC", parameters = Map("expr" -> "\"CAST(rand() AS STRING)\"")) checkError( exception = intercept[AnalysisException] { sql("SELECT * FROM t TIMESTAMP AS OF abs(true)").collect() }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = None, parameters = Map( "sqlExpr" -> "\"abs(true)\"", @@ -3157,7 +3157,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql("SELECT * FROM parquet.`/the/path` VERSION AS OF 1") }, - errorClass = "UNSUPPORTED_FEATURE.TIME_TRAVEL", + condition = "UNSUPPORTED_FEATURE.TIME_TRAVEL", sqlState = None, parameters = Map("relationId" -> "`parquet`.`/the/path`")) @@ -3165,7 +3165,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql("WITH x AS (SELECT 1) SELECT * FROM x VERSION AS OF 1") }, - errorClass = "UNSUPPORTED_FEATURE.TIME_TRAVEL", + condition = "UNSUPPORTED_FEATURE.TIME_TRAVEL", sqlState = None, parameters = Map("relationId" -> "`x`")) @@ -3173,7 +3173,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql("SELECT * FROM non_exist VERSION AS OF 1") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`non_exist`"), context = ExpectedContext( fragment = "non_exist", @@ -3185,7 +3185,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"SELECT * FROM t TIMESTAMP AS OF ($subquery1)").collect() }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`non_exist`"), ExpectedContext( fragment = "non_exist", @@ -3196,7 +3196,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"SELECT * FROM t TIMESTAMP AS OF (SELECT ($subquery1))").collect() }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`non_exist`"), ExpectedContext( fragment = "non_exist", @@ -3208,7 +3208,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"SELECT * FROM t TIMESTAMP AS OF ($subquery2)").collect() }, - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", parameters = Map("objectName" -> "`col`"), ExpectedContext( fragment = "col", @@ -3218,7 +3218,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"SELECT * FROM t TIMESTAMP AS OF (SELECT ($subquery2))").collect() }, - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", parameters = Map("objectName" -> "`col`"), ExpectedContext( fragment = "col", @@ -3230,7 +3230,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"SELECT * FROM t TIMESTAMP AS OF ($subquery3)").collect() }, - errorClass = + condition = "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", parameters = Map("number" -> "2"), ExpectedContext( @@ -3241,7 +3241,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"SELECT * FROM t TIMESTAMP AS OF (SELECT ($subquery3))").collect() }, - errorClass = + condition = "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", parameters = Map("number" -> "2"), ExpectedContext( @@ -3254,7 +3254,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[SparkException] { sql(s"SELECT * FROM t TIMESTAMP AS OF ($subquery4)").collect() }, - errorClass = "SCALAR_SUBQUERY_TOO_MANY_ROWS", + condition = "SCALAR_SUBQUERY_TOO_MANY_ROWS", parameters = Map.empty, ExpectedContext( fragment = "(SELECT * FROM VALUES (1), (2))", @@ -3264,7 +3264,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[SparkException] { sql(s"SELECT * FROM t TIMESTAMP AS OF (SELECT ($subquery4))").collect() }, - errorClass = "SCALAR_SUBQUERY_TOO_MANY_ROWS", + condition = "SCALAR_SUBQUERY_TOO_MANY_ROWS", parameters = Map.empty, ExpectedContext( fragment = "(SELECT * FROM VALUES (1), (2))", @@ -3368,7 +3368,7 @@ class DataSourceV2SQLSuiteV1Filter } checkError( exception, - errorClass = "UNSUPPORTED_FEATURE.OVERWRITE_BY_SUBQUERY", + condition = "UNSUPPORTED_FEATURE.OVERWRITE_BY_SUBQUERY", sqlState = "0A000", parameters = Map.empty, context = ExpectedContext( @@ -3789,7 +3789,7 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") }, - errorClass = "NOT_SUPPORTED_COMMAND_FOR_V2_TABLE", + condition = "NOT_SUPPORTED_COMMAND_FOR_V2_TABLE", sqlState = "0A000", parameters = Map("cmd" -> expectedArgument.getOrElse(sqlCommand))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 1de535df246b7..d61d554025e50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -454,7 +454,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS .write.format(cls.getName) .option("path", path).mode("ignore").save() }, - errorClass = "UNSUPPORTED_DATA_SOURCE_SAVE_MODE", + condition = "UNSUPPORTED_DATA_SOURCE_SAVE_MODE", parameters = Map( "source" -> cls.getName, "createMode" -> "\"Ignore\"" @@ -467,7 +467,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS .write.format(cls.getName) .option("path", path).mode("error").save() }, - errorClass = "UNSUPPORTED_DATA_SOURCE_SAVE_MODE", + condition = "UNSUPPORTED_DATA_SOURCE_SAVE_MODE", parameters = Map( "source" -> cls.getName, "createMode" -> "\"ErrorIfExists\"" @@ -651,7 +651,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS exception = intercept[SparkUnsupportedOperationException] { sql(s"CREATE TABLE test(a INT, b INT) USING ${cls.getName}") }, - errorClass = "CANNOT_CREATE_DATA_SOURCE_TABLE.EXTERNAL_METADATA_UNSUPPORTED", + condition = "CANNOT_CREATE_DATA_SOURCE_TABLE.EXTERNAL_METADATA_UNSUPPORTED", parameters = Map("tableName" -> "`default`.`test`", "provider" -> cls.getName) ) } @@ -732,7 +732,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS exception = intercept[AnalysisException] { sql(s"CREATE TABLE test (x INT, y INT) USING ${cls.getName}") }, - errorClass = "DATA_SOURCE_TABLE_SCHEMA_MISMATCH", + condition = "DATA_SOURCE_TABLE_SCHEMA_MISMATCH", parameters = Map( "dsSchema" -> "\"STRUCT\"", "expectedSchema" -> "\"STRUCT\"")) @@ -770,7 +770,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS exception = intercept[AnalysisException] { sql(s"CREATE TABLE test USING ${cls.getName} AS VALUES (0, 1), (1, 2)") }, - errorClass = "DATA_SOURCE_TABLE_SCHEMA_MISMATCH", + condition = "DATA_SOURCE_TABLE_SCHEMA_MISMATCH", parameters = Map( "dsSchema" -> "\"STRUCT\"", "expectedSchema" -> "\"STRUCT\"")) @@ -788,7 +788,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS |AS VALUES ('a', 'b'), ('c', 'd') t(i, j) |""".stripMargin) }, - errorClass = "DATA_SOURCE_TABLE_SCHEMA_MISMATCH", + condition = "DATA_SOURCE_TABLE_SCHEMA_MISMATCH", parameters = Map( "dsSchema" -> "\"STRUCT\"", "expectedSchema" -> "\"STRUCT\"")) @@ -839,7 +839,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS exception = intercept[SparkUnsupportedOperationException] { sql(s"CREATE TABLE test USING ${cls.getName} AS VALUES (0, 1)") }, - errorClass = "CANNOT_CREATE_DATA_SOURCE_TABLE.EXTERNAL_METADATA_UNSUPPORTED", + condition = "CANNOT_CREATE_DATA_SOURCE_TABLE.EXTERNAL_METADATA_UNSUPPORTED", parameters = Map( "tableName" -> "`default`.`test`", "provider" -> "org.apache.spark.sql.connector.SimpleDataSourceV2")) @@ -851,7 +851,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS exception = intercept[AnalysisException] { sql(s"CREATE TABLE test USING ${cls.getName} AS SELECT * FROM VALUES (0, 1)") }, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`test`", "operation" -> "append in batch mode")) @@ -881,7 +881,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS exception = intercept[AnalysisException] { sql(s"INSERT INTO test VALUES (4)") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`test`", "tableColumns" -> "`x`, `y`", @@ -893,7 +893,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS exception = intercept[AnalysisException] { sql(s"INSERT INTO test(x, x) VALUES (4, 5)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`x`")) } } @@ -935,13 +935,13 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS exception = intercept[AnalysisException] { sql("INSERT INTO test PARTITION(z = 1) VALUES (2)") }, - errorClass = "NON_PARTITION_COLUMN", + condition = "NON_PARTITION_COLUMN", parameters = Map("columnName" -> "`z`")) checkError( exception = intercept[AnalysisException] { sql("INSERT INTO test PARTITION(x, y = 1) VALUES (2, 3)") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`test`", "tableColumns" -> "`x`, `y`", @@ -959,7 +959,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS exception = intercept[AnalysisException] { sql("INSERT OVERWRITE test PARTITION(x = 1) VALUES (5)") }, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`test`", "operation" -> "overwrite by filter in batch mode") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala index eeef0566b8faf..fd022580db42b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala @@ -112,7 +112,7 @@ trait DeleteFromTests extends DatasourceV2SQLBase { checkError( exception = exc, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", sqlState = "0A000", parameters = Map("tableName" -> "`spark_catalog`.`default`.`tbl`", "operation" -> "DELETE")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedDeleteFromTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedDeleteFromTableSuite.scala index 7336b3a6e9206..be180eb89ce20 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedDeleteFromTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedDeleteFromTableSuite.scala @@ -37,7 +37,7 @@ class DeltaBasedDeleteFromTableSuite extends DeleteFromTableSuiteBase { checkError( exception = intercept[AnalysisException]( sql(s"DELETE FROM $tableNameAsString WHERE id <= 1 AND rand() > 0.5")), - errorClass = "INVALID_NON_DETERMINISTIC_EXPRESSIONS", + condition = "INVALID_NON_DETERMINISTIC_EXPRESSIONS", parameters = Map( "sqlExprs" -> "\"((id <= 1) AND (rand() > 0.5))\""), context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedUpdateTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedUpdateTableSuiteBase.scala index 46942cac1c7e3..89b42b5e6db7b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedUpdateTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedUpdateTableSuiteBase.scala @@ -32,7 +32,7 @@ abstract class DeltaBasedUpdateTableSuiteBase extends UpdateTableSuiteBase { exception = intercept[AnalysisException] { sql(s"UPDATE $tableNameAsString SET salary = -1 WHERE pk = 1") }, - errorClass = "NULLABLE_ROW_ID_ATTRIBUTES", + condition = "NULLABLE_ROW_ID_ATTRIBUTES", parameters = Map("nullableRowIdAttrs" -> "pk#\\d+") ) } @@ -62,7 +62,7 @@ abstract class DeltaBasedUpdateTableSuiteBase extends UpdateTableSuiteBase { exception = intercept[AnalysisException] { sql(s"UPDATE $tableNameAsString SET dep = 'invalid' WHERE id <= 1 AND rand() > 0.5") }, - errorClass = "INVALID_NON_DETERMINISTIC_EXPRESSIONS", + condition = "INVALID_NON_DETERMINISTIC_EXPRESSIONS", parameters = Map("sqlExprs" -> "\"((id <= 1) AND (rand() > 0.5))\""), context = ExpectedContext( fragment = "UPDATE cat.ns1.test_table SET dep = 'invalid' WHERE id <= 1 AND rand() > 0.5", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala index c6060dcdd51a7..2a0ab21ddb09c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala @@ -103,7 +103,7 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { spark.read.format(dummyReadOnlyFileSourceV2).load(path).collect() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Dummy file reader")) } } @@ -131,7 +131,7 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { spark.read.format(dummyReadOnlyFileSourceV2).load(path).collect() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Dummy file reader")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/GroupBasedDeleteFromTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/GroupBasedDeleteFromTableSuite.scala index 0aeab95f58a7b..1be318f948fd9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/GroupBasedDeleteFromTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/GroupBasedDeleteFromTableSuite.scala @@ -34,7 +34,7 @@ class GroupBasedDeleteFromTableSuite extends DeleteFromTableSuiteBase { checkError( exception = intercept[AnalysisException]( sql(s"DELETE FROM $tableNameAsString WHERE id <= 1 AND rand() > 0.5")), - errorClass = "INVALID_NON_DETERMINISTIC_EXPRESSIONS", + condition = "INVALID_NON_DETERMINISTIC_EXPRESSIONS", parameters = Map( "sqlExprs" -> "\"((id <= 1) AND (rand() > 0.5))\", \"((id <= 1) AND (rand() > 0.5))\""), context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/GroupBasedUpdateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/GroupBasedUpdateTableSuite.scala index 3e736421a315c..774ae97734d25 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/GroupBasedUpdateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/GroupBasedUpdateTableSuite.scala @@ -122,7 +122,7 @@ class GroupBasedUpdateTableSuite extends UpdateTableSuiteBase { exception = intercept[AnalysisException] { sql(s"UPDATE $tableNameAsString SET dep = 'invalid' WHERE id <= 1 AND rand() > 0.5") }, - errorClass = "INVALID_NON_DETERMINISTIC_EXPRESSIONS", + condition = "INVALID_NON_DETERMINISTIC_EXPRESSIONS", parameters = Map( "sqlExprs" -> "\"((id <= 1) AND (rand() > 0.5))\", \"((id <= 1) AND (rand() > 0.5))\""), context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala index fa30969d65c52..d6e86bc93c9d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala @@ -135,7 +135,7 @@ abstract class InsertIntoTests( exception = intercept[AnalysisException] { doInsert(t1, df) }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> tableName, "tableColumns" -> "`id`, `data`, `missing`", @@ -158,7 +158,7 @@ abstract class InsertIntoTests( exception = intercept[AnalysisException] { doInsert(t1, df) }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", parameters = Map( "tableName" -> tableName, "tableColumns" -> "`id`, `data`", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala index b043bf2f5be23..741e30a739f5e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala @@ -303,7 +303,7 @@ class MetadataColumnSuite extends DatasourceV2SQLBase { exception = intercept[AnalysisException] { df.metadataColumn("foo") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`foo`", "proposal" -> "`index`, `_partition`"), queryContext = Array(ExpectedContext("select index from testcat.t", 0, 26))) @@ -312,7 +312,7 @@ class MetadataColumnSuite extends DatasourceV2SQLBase { exception = intercept[AnalysisException] { df.metadataColumn("data") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`data`", "proposal" -> "`index`, `_partition`"), queryContext = Array(ExpectedContext("select index from testcat.t", 0, 26))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/UpdateTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/UpdateTableSuiteBase.scala index c2ae5f40cfaf6..f659ca6329e2f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/UpdateTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/UpdateTableSuiteBase.scala @@ -579,7 +579,7 @@ abstract class UpdateTableSuiteBase extends RowLevelOperationSuiteBase { exception = intercept[SparkRuntimeException] { sql(s"UPDATE $tableNameAsString SET s = named_struct('n_i', null, 'n_l', -1L) WHERE pk = 1") }, - errorClass = "NOT_NULL_ASSERT_VIOLATION", + condition = "NOT_NULL_ASSERT_VIOLATION", sqlState = "42000", parameters = Map("walkedTypePath" -> "\ns\nn_i\n")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala index 39809c785af92..48d4e45ebf354 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala @@ -51,7 +51,7 @@ class QueryCompilationErrorsDSv2Suite checkAnswer(spark.table(tbl), spark.emptyDataFrame) checkError( exception = e, - errorClass = "UNSUPPORTED_FEATURE.INSERT_PARTITION_SPEC_IF_NOT_EXISTS", + condition = "UNSUPPORTED_FEATURE.INSERT_PARTITION_SPEC_IF_NOT_EXISTS", parameters = Map("tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`"), sqlState = "0A000") } @@ -70,7 +70,7 @@ class QueryCompilationErrorsDSv2Suite verifyTable(t1, spark.emptyDataFrame) checkError( exception = e, - errorClass = "NON_PARTITION_COLUMN", + condition = "NON_PARTITION_COLUMN", parameters = Map("columnName" -> "`id`")) } } @@ -87,7 +87,7 @@ class QueryCompilationErrorsDSv2Suite verifyTable(t1, spark.emptyDataFrame) checkError( exception = e, - errorClass = "NON_PARTITION_COLUMN", + condition = "NON_PARTITION_COLUMN", parameters = Map("columnName" -> "`data`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index 371a615828de3..47a6143bad1d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -51,7 +51,7 @@ class QueryCompilationErrorsSuite } checkError( exception = e1, - errorClass = "CANNOT_UP_CAST_DATATYPE", + condition = "CANNOT_UP_CAST_DATATYPE", parameters = Map("expression" -> "b", "sourceType" -> "\"BIGINT\"", "targetType" -> "\"INT\"", "details" -> ( s""" @@ -68,7 +68,7 @@ class QueryCompilationErrorsSuite } checkError( exception = e2, - errorClass = "CANNOT_UP_CAST_DATATYPE", + condition = "CANNOT_UP_CAST_DATATYPE", parameters = Map("expression" -> "b.`b`", "sourceType" -> "\"DECIMAL(38,18)\"", "targetType" -> "\"BIGINT\"", "details" -> ( @@ -94,7 +94,7 @@ class QueryCompilationErrorsSuite } checkError( exception = e, - errorClass = "UNSUPPORTED_GROUPING_EXPRESSION", + condition = "UNSUPPORTED_GROUPING_EXPRESSION", parameters = Map[String, String]()) } } @@ -112,7 +112,7 @@ class QueryCompilationErrorsSuite } checkError( exception = e, - errorClass = "UNSUPPORTED_GROUPING_EXPRESSION", + condition = "UNSUPPORTED_GROUPING_EXPRESSION", parameters = Map[String, String]()) } } @@ -123,7 +123,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { sql("select format_string('%0$s', 'Hello')") }, - errorClass = "INVALID_PARAMETER_VALUE.ZERO_INDEX", + condition = "INVALID_PARAMETER_VALUE.ZERO_INDEX", parameters = Map( "parameter" -> "`strfmt`", "functionName" -> "`format_string`"), @@ -156,7 +156,7 @@ class QueryCompilationErrorsSuite checkError( exception = e, - errorClass = "INVALID_PANDAS_UDF_PLACEMENT", + condition = "INVALID_PANDAS_UDF_PLACEMENT", parameters = Map("functionList" -> "`pandas_udf_1`, `pandas_udf_2`")) } @@ -183,7 +183,7 @@ class QueryCompilationErrorsSuite checkError( exception = e, - errorClass = "UNSUPPORTED_FEATURE.PYTHON_UDF_IN_ON_CLAUSE", + condition = "UNSUPPORTED_FEATURE.PYTHON_UDF_IN_ON_CLAUSE", parameters = Map("joinType" -> "LEFT OUTER"), sqlState = Some("0A000")) } @@ -205,7 +205,7 @@ class QueryCompilationErrorsSuite checkError( exception = e, - errorClass = "UNSUPPORTED_FEATURE.PANDAS_UDAF_IN_PIVOT", + condition = "UNSUPPORTED_FEATURE.PANDAS_UDAF_IN_PIVOT", parameters = Map[String, String](), sqlState = "0A000") } @@ -224,7 +224,7 @@ class QueryCompilationErrorsSuite ) checkError( exception = e, - errorClass = "NO_HANDLER_FOR_UDAF", + condition = "NO_HANDLER_FOR_UDAF", parameters = Map("functionName" -> "org.apache.spark.sql.errors.MyCastToString"), context = ExpectedContext( fragment = "myCast(123)", start = 7, stop = 17)) @@ -234,7 +234,7 @@ class QueryCompilationErrorsSuite test("UNTYPED_SCALA_UDF: use untyped Scala UDF should fail by default") { checkError( exception = intercept[AnalysisException](udf((x: Int) => x, IntegerType)), - errorClass = "UNTYPED_SCALA_UDF", + condition = "UNTYPED_SCALA_UDF", parameters = Map[String, String]()) } @@ -248,7 +248,7 @@ class QueryCompilationErrorsSuite ) checkError( exception = e, - errorClass = "NO_UDF_INTERFACE", + condition = "NO_UDF_INTERFACE", parameters = Map("className" -> className)) } @@ -262,7 +262,7 @@ class QueryCompilationErrorsSuite ) checkError( exception = e, - errorClass = "MULTI_UDF_INTERFACE_ERROR", + condition = "MULTI_UDF_INTERFACE_ERROR", parameters = Map("className" -> className)) } @@ -276,7 +276,7 @@ class QueryCompilationErrorsSuite ) checkError( exception = e, - errorClass = "UNSUPPORTED_FEATURE.TOO_MANY_TYPE_ARGUMENTS_FOR_UDF_CLASS", + condition = "UNSUPPORTED_FEATURE.TOO_MANY_TYPE_ARGUMENTS_FOR_UDF_CLASS", parameters = Map("num" -> "24"), sqlState = "0A000") } @@ -287,7 +287,7 @@ class QueryCompilationErrorsSuite } checkError( exception = groupingColMismatchEx, - errorClass = "GROUPING_COLUMN_MISMATCH", + condition = "GROUPING_COLUMN_MISMATCH", parameters = Map("grouping" -> "earnings.*", "groupingColumns" -> "course.*,year.*"), sqlState = Some("42803"), matchPVals = true) @@ -299,7 +299,7 @@ class QueryCompilationErrorsSuite } checkError( exception = groupingIdColMismatchEx, - errorClass = "GROUPING_ID_COLUMN_MISMATCH", + condition = "GROUPING_ID_COLUMN_MISMATCH", parameters = Map("groupingIdColumn" -> "earnings.*", "groupByColumns" -> "course.*,year.*"), sqlState = Some("42803"), @@ -322,14 +322,14 @@ class QueryCompilationErrorsSuite withSQLConf(SQLConf.LEGACY_INTEGER_GROUPING_ID.key -> "true") { checkError( exception = intercept[AnalysisException] { testGroupingIDs(33) }, - errorClass = "GROUPING_SIZE_LIMIT_EXCEEDED", + condition = "GROUPING_SIZE_LIMIT_EXCEEDED", parameters = Map("maxSize" -> "32")) } withSQLConf(SQLConf.LEGACY_INTEGER_GROUPING_ID.key -> "false") { checkError( exception = intercept[AnalysisException] { testGroupingIDs(65) }, - errorClass = "GROUPING_SIZE_LIMIT_EXCEEDED", + condition = "GROUPING_SIZE_LIMIT_EXCEEDED", parameters = Map("maxSize" -> "64")) } } @@ -354,7 +354,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { sql(s"DESC TABLE $tempViewName PARTITION (c='Us', d=1)") }, - errorClass = "FORBIDDEN_OPERATION", + condition = "FORBIDDEN_OPERATION", parameters = Map("statement" -> "DESC PARTITION", "objectType" -> "TEMPORARY VIEW", "objectName" -> s"`$tempViewName`")) } @@ -380,7 +380,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { sql(s"DESC TABLE $viewName PARTITION (c='Us', d=1)") }, - errorClass = "FORBIDDEN_OPERATION", + condition = "FORBIDDEN_OPERATION", parameters = Map("statement" -> "DESC PARTITION", "objectType" -> "VIEW", "objectName" -> s"`$viewName`")) } @@ -394,7 +394,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { sql("select date_add('1982-08-15', 'x')").collect() }, - errorClass = "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", + condition = "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", parameters = Map("functionName" -> "date_add"), sqlState = "22023") } @@ -408,7 +408,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { spark.read.schema(schema).json(spark.emptyDataset[String]) }, - errorClass = "INVALID_JSON_SCHEMA_MAP_TYPE", + condition = "INVALID_JSON_SCHEMA_MAP_TYPE", parameters = Map("jsonSchema" -> "\"STRUCT NOT NULL>\"") ) } @@ -418,7 +418,7 @@ class QueryCompilationErrorsSuite val query = "select m[a] from (select map('a', 'b') as m, 'aa' as aa)" checkError( exception = intercept[AnalysisException] {sql(query)}, - errorClass = "UNRESOLVED_MAP_KEY.WITH_SUGGESTION", + condition = "UNRESOLVED_MAP_KEY.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`a`", "proposal" -> "`aa`, `m`"), @@ -433,7 +433,7 @@ class QueryCompilationErrorsSuite val query = "select m[a] from (select map('a', 'b') as m, 'aa' as `a.a`)" checkError( exception = intercept[AnalysisException] {sql(query)}, - errorClass = "UNRESOLVED_MAP_KEY.WITH_SUGGESTION", + condition = "UNRESOLVED_MAP_KEY.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> "`a`", @@ -468,7 +468,7 @@ class QueryCompilationErrorsSuite |order by struct.a, struct.b |""".stripMargin) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> "`struct`.`a`", @@ -489,7 +489,7 @@ class QueryCompilationErrorsSuite val query = "SELECT v.i from (SELECT i FROM v)" checkError( exception = intercept[AnalysisException](sql(query)), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> "`v`.`i`", @@ -522,7 +522,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "AMBIGUOUS_ALIAS_IN_NESTED_CTE", + condition = "AMBIGUOUS_ALIAS_IN_NESTED_CTE", parameters = Map( "name" -> "`t`", "config" -> toSQLConf(SQLConf.LEGACY_CTE_PRECEDENCE_POLICY.key), @@ -542,7 +542,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "AMBIGUOUS_COLUMN_OR_FIELD", + condition = "AMBIGUOUS_COLUMN_OR_FIELD", parameters = Map("name" -> "`c`.`X`", "n" -> "2"), context = ExpectedContext( fragment = query, start = 0, stop = 52)) @@ -566,7 +566,7 @@ class QueryCompilationErrorsSuite struct(lit("java"), lit("Dummies")))). agg(sum($"earnings")).collect() }, - errorClass = "PIVOT_VALUE_DATA_TYPE_MISMATCH", + condition = "PIVOT_VALUE_DATA_TYPE_MISMATCH", parameters = Map("value" -> "struct(col1, dotnet, col2, Experts)", "valueType" -> "struct", "pivotType" -> "int")) @@ -581,7 +581,7 @@ class QueryCompilationErrorsSuite } checkError( exception = e, - errorClass = "INVALID_FIELD_NAME", + condition = "INVALID_FIELD_NAME", parameters = Map("fieldName" -> "`m`.`n`", "path" -> "`m`"), context = ExpectedContext( fragment = "m.n int", start = 27, stop = 33)) @@ -603,7 +603,7 @@ class QueryCompilationErrorsSuite pivot(df("year"), Seq($"earnings")). agg(sum($"earnings")).collect() }, - errorClass = "NON_LITERAL_PIVOT_VALUES", + condition = "NON_LITERAL_PIVOT_VALUES", parameters = Map("expression" -> "\"earnings\"")) } @@ -613,7 +613,7 @@ class QueryCompilationErrorsSuite } checkError( exception = e, - errorClass = "UNSUPPORTED_DESERIALIZER.DATA_TYPE_MISMATCH", + condition = "UNSUPPORTED_DESERIALIZER.DATA_TYPE_MISMATCH", parameters = Map("desiredType" -> "\"ARRAY\"", "dataType" -> "\"INT\"")) } @@ -626,7 +626,7 @@ class QueryCompilationErrorsSuite } checkError( exception = e1, - errorClass = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", + condition = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", parameters = Map( "schema" -> "\"STRUCT\"", "ordinal" -> "3")) @@ -636,7 +636,7 @@ class QueryCompilationErrorsSuite } checkError( exception = e2, - errorClass = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", + condition = "UNSUPPORTED_DESERIALIZER.FIELD_NUMBER_MISMATCH", parameters = Map("schema" -> "\"STRUCT\"", "ordinal" -> "1")) } @@ -649,7 +649,7 @@ class QueryCompilationErrorsSuite checkError( exception = e, - errorClass = "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + condition = "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", parameters = Map("expression" -> "\"(explode(array(1, 2, 3)) + 1)\"")) } @@ -660,7 +660,7 @@ class QueryCompilationErrorsSuite checkError( exception = e, - errorClass = "UNSUPPORTED_GENERATOR.OUTSIDE_SELECT", + condition = "UNSUPPORTED_GENERATOR.OUTSIDE_SELECT", parameters = Map("plan" -> "'Sort [explode(array(1, 2, 3)) ASC NULLS FIRST], true")) } @@ -675,7 +675,7 @@ class QueryCompilationErrorsSuite checkError( exception = e, - errorClass = "UNSUPPORTED_GENERATOR.NOT_GENERATOR", + condition = "UNSUPPORTED_GENERATOR.NOT_GENERATOR", sqlState = None, parameters = Map( "functionName" -> "`array_contains`", @@ -690,7 +690,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { Seq("""{"a":1}""").toDF("a").select(from_json($"a", IntegerType)).collect() }, - errorClass = "DATATYPE_MISMATCH.INVALID_JSON_SCHEMA", + condition = "DATATYPE_MISMATCH.INVALID_JSON_SCHEMA", parameters = Map("schema" -> "\"INT\"", "sqlExpr" -> "\"from_json(a)\""), context = ExpectedContext(fragment = "from_json", callSitePattern = getCurrentClassCallSitePattern)) @@ -701,7 +701,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { sql("SELECT CAST(1)") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> "`cast`", "expectedNum" -> "0", @@ -717,7 +717,7 @@ class QueryCompilationErrorsSuite exception = intercept[ParseException] { sql("CREATE TEMPORARY VIEW db_name.schema_name.view_name AS SELECT '1' as test_column") }, - errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", + condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", sqlState = "42601", parameters = Map("identifier" -> "`db_name`.`schema_name`.`view_name`") ) @@ -738,7 +738,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $tableName RENAME TO db_name.schema_name.new_table_name") }, - errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", + condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", sqlState = "42601", parameters = Map("identifier" -> "`db_name`.`schema_name`.`new_table_name`") ) @@ -762,7 +762,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { df.select($"name.firstname") }, - errorClass = "AMBIGUOUS_REFERENCE_TO_FIELDS", + condition = "AMBIGUOUS_REFERENCE_TO_FIELDS", sqlState = "42000", parameters = Map("field" -> "`firstname`", "count" -> "2"), context = ExpectedContext(fragment = "$", callSitePattern = getCurrentClassCallSitePattern) @@ -776,7 +776,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { df.select($"firstname.test_field") }, - errorClass = "INVALID_EXTRACT_BASE_FIELD_TYPE", + condition = "INVALID_EXTRACT_BASE_FIELD_TYPE", sqlState = "42000", parameters = Map("base" -> "\"firstname\"", "other" -> "\"STRING\""), context = ExpectedContext(fragment = "$", callSitePattern = getCurrentClassCallSitePattern) @@ -802,7 +802,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { df.select(struct($"name"(struct("test")))) }, - errorClass = "INVALID_EXTRACT_FIELD_TYPE", + condition = "INVALID_EXTRACT_FIELD_TYPE", sqlState = "42000", parameters = Map("extraction" -> "\"struct(test)\"")) @@ -810,7 +810,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { df.select($"name"(array("test"))) }, - errorClass = "INVALID_EXTRACT_FIELD_TYPE", + condition = "INVALID_EXTRACT_FIELD_TYPE", sqlState = "42000", parameters = Map("extraction" -> "\"array(test)\"")) } @@ -831,7 +831,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { sql(query) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "proposal" -> "`c1`, `v1`.`c2`, `v2`.`c2`", "objectName" -> "`b`"), @@ -849,7 +849,7 @@ class QueryCompilationErrorsSuite exception = intercept[SparkIllegalArgumentException] { coalesce.dataType }, - errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.NO_INPUTS", + condition = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.NO_INPUTS", parameters = Map("expression" -> "\"coalesce()\"")) } @@ -861,7 +861,7 @@ class QueryCompilationErrorsSuite exception = intercept[SparkIllegalArgumentException] { coalesce.dataType }, - errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.MISMATCHED_TYPES", + condition = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.MISMATCHED_TYPES", parameters = Map( "expression" -> "\"coalesce(1, a, a)\"", "inputTypes" -> "[\"INT\", \"STRING\", \"STRING\"]")) @@ -872,7 +872,7 @@ class QueryCompilationErrorsSuite exception = intercept[SparkUnsupportedOperationException] { new UnsafeRow(1).update(0, 1) }, - errorClass = "UNSUPPORTED_CALL.WITHOUT_SUGGESTION", + condition = "UNSUPPORTED_CALL.WITHOUT_SUGGESTION", parameters = Map( "methodName" -> "update", "className" -> "org.apache.spark.sql.catalyst.expressions.UnsafeRow")) @@ -891,7 +891,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { converter.convertField(StructField("test", dummyDataType)) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Cannot convert Spark data type \"DUMMY\" to any Parquet type.") ) } @@ -919,7 +919,7 @@ class QueryCompilationErrorsSuite exception = intercept[AnalysisException] { sql(test.query) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`dummy`", "proposal" -> "`a`, `b`"), context = ExpectedContext(fragment = "dummy", start = test.pos, stop = test.pos + 4) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala index 83495e0670240..c330e944cae86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala @@ -45,7 +45,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { sql("select CAST(TIMESTAMP '9999-12-31T12:13:14.56789Z' AS INT)").collect() }, - errorClass = "CAST_OVERFLOW", + condition = "CAST_OVERFLOW", parameters = Map("value" -> "TIMESTAMP '9999-12-31 04:13:14.56789'", "sourceType" -> "\"TIMESTAMP\"", "targetType" -> "\"INT\"", @@ -58,7 +58,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { sql("select 6/0").collect() }, - errorClass = "DIVIDE_BY_ZERO", + condition = "DIVIDE_BY_ZERO", sqlState = "22012", parameters = Map("config" -> ansiConf), context = ExpectedContext(fragment = "6/0", start = 7, stop = 9)) @@ -67,7 +67,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { OneRowRelation().select(lit(5) / lit(0)).collect() }, - errorClass = "DIVIDE_BY_ZERO", + condition = "DIVIDE_BY_ZERO", sqlState = "22012", parameters = Map("config" -> ansiConf), context = ExpectedContext(fragment = "div", callSitePattern = getCurrentClassCallSitePattern)) @@ -76,7 +76,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { OneRowRelation().select(lit(5).divide(lit(0))).collect() }, - errorClass = "DIVIDE_BY_ZERO", + condition = "DIVIDE_BY_ZERO", sqlState = "22012", parameters = Map("config" -> ansiConf), context = ExpectedContext( @@ -89,7 +89,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { sql("select interval 1 day / 0").collect() }, - errorClass = "INTERVAL_DIVIDED_BY_ZERO", + condition = "INTERVAL_DIVIDED_BY_ZERO", sqlState = "22012", parameters = Map.empty[String, String], context = ExpectedContext(fragment = "interval 1 day / 0", start = 7, stop = 24)) @@ -100,7 +100,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkDateTimeException] { sql("select make_timestamp(2012, 11, 30, 9, 19, 60.66666666)").collect() }, - errorClass = "INVALID_FRACTION_OF_SECOND", + condition = "INVALID_FRACTION_OF_SECOND", sqlState = "22023", parameters = Map("ansiConfig" -> ansiConf)) } @@ -110,7 +110,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { sql("select CAST('66666666666666.666' AS DECIMAL(8, 1))").collect() }, - errorClass = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", sqlState = "22003", parameters = Map( "value" -> "66666666666666.666", @@ -126,7 +126,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { OneRowRelation().select(lit("66666666666666.666").cast("DECIMAL(8, 1)")).collect() }, - errorClass = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", sqlState = "22003", parameters = Map( "value" -> "66666666666666.666", @@ -143,7 +143,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArrayIndexOutOfBoundsException] { sql("select array(1, 2, 3, 4, 5)[8]").collect() }, - errorClass = "INVALID_ARRAY_INDEX", + condition = "INVALID_ARRAY_INDEX", parameters = Map("indexValue" -> "8", "arraySize" -> "5", "ansiConfig" -> ansiConf), context = ExpectedContext(fragment = "array(1, 2, 3, 4, 5)[8]", start = 7, stop = 29)) @@ -151,7 +151,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArrayIndexOutOfBoundsException] { OneRowRelation().select(lit(Array(1, 2, 3, 4, 5))(8)).collect() }, - errorClass = "INVALID_ARRAY_INDEX", + condition = "INVALID_ARRAY_INDEX", parameters = Map("indexValue" -> "8", "arraySize" -> "5", "ansiConfig" -> ansiConf), context = ExpectedContext( fragment = "apply", @@ -163,7 +163,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArrayIndexOutOfBoundsException] { sql("select element_at(array(1, 2, 3, 4, 5), 8)").collect() }, - errorClass = "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", + condition = "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", parameters = Map("indexValue" -> "8", "arraySize" -> "5", "ansiConfig" -> ansiConf), context = ExpectedContext( fragment = "element_at(array(1, 2, 3, 4, 5), 8)", @@ -174,7 +174,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArrayIndexOutOfBoundsException] { OneRowRelation().select(element_at(lit(Array(1, 2, 3, 4, 5)), 8)).collect() }, - errorClass = "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", + condition = "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", parameters = Map("indexValue" -> "8", "arraySize" -> "5", "ansiConfig" -> ansiConf), context = ExpectedContext(fragment = "element_at", callSitePattern = getCurrentClassCallSitePattern)) @@ -185,7 +185,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkRuntimeException]( sql("select element_at(array(1, 2, 3, 4, 5), 0)").collect() ), - errorClass = "INVALID_INDEX_OF_ZERO", + condition = "INVALID_INDEX_OF_ZERO", parameters = Map.empty, context = ExpectedContext( fragment = "element_at(array(1, 2, 3, 4, 5), 0)", @@ -197,7 +197,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkRuntimeException]( OneRowRelation().select(element_at(lit(Array(1, 2, 3, 4, 5)), 0)).collect() ), - errorClass = "INVALID_INDEX_OF_ZERO", + condition = "INVALID_INDEX_OF_ZERO", parameters = Map.empty, context = ExpectedContext(fragment = "element_at", callSitePattern = getCurrentClassCallSitePattern)) @@ -208,7 +208,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkNumberFormatException] { sql("select CAST('111111111111xe23' AS DOUBLE)").collect() }, - errorClass = "CAST_INVALID_INPUT", + condition = "CAST_INVALID_INPUT", parameters = Map( "expression" -> "'111111111111xe23'", "sourceType" -> "\"STRING\"", @@ -223,7 +223,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkNumberFormatException] { OneRowRelation().select(lit("111111111111xe23").cast("DOUBLE")).collect() }, - errorClass = "CAST_INVALID_INPUT", + condition = "CAST_INVALID_INPUT", parameters = Map( "expression" -> "'111111111111xe23'", "sourceType" -> "\"STRING\"", @@ -239,7 +239,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkDateTimeException] { sql("select to_timestamp('abc', 'yyyy-MM-dd HH:mm:ss')").collect() }, - errorClass = "CANNOT_PARSE_TIMESTAMP", + condition = "CANNOT_PARSE_TIMESTAMP", parameters = Map( "message" -> "Text 'abc' could not be parsed at index 0", "ansiConfig" -> ansiConf) @@ -255,7 +255,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { sql(s"insert into $tableName values 12345678901234567890D") }, - errorClass = "CAST_OVERFLOW_IN_TABLE_INSERT", + condition = "CAST_OVERFLOW_IN_TABLE_INSERT", parameters = Map( "sourceType" -> "\"DOUBLE\"", "targetType" -> ("\"" + targetType + "\""), @@ -272,7 +272,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { CheckOverflowInTableInsert(caseWhen, "col").eval(null) }, - errorClass = "CAST_OVERFLOW", + condition = "CAST_OVERFLOW", parameters = Map("value" -> "1.2345678901234567E19D", "sourceType" -> "\"DOUBLE\"", "targetType" -> ("\"TINYINT\""), @@ -291,7 +291,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { sql(insertCmd).collect() }, - errorClass = "CAST_OVERFLOW", + condition = "CAST_OVERFLOW", parameters = Map("value" -> "-1.2345678901234567E19D", "sourceType" -> "\"DOUBLE\"", "targetType" -> "\"TINYINT\"", @@ -306,7 +306,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest CheckOverflowInTableInsert( Cast(Literal.apply(12345678901234567890D), ByteType), "col").eval(null) }.asInstanceOf[SparkThrowable], - errorClass = "CAST_OVERFLOW_IN_TABLE_INSERT", + condition = "CAST_OVERFLOW_IN_TABLE_INSERT", parameters = Map( "sourceType" -> "\"DOUBLE\"", "targetType" -> ("\"TINYINT\""), @@ -322,7 +322,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest exception = intercept[SparkArithmeticException] { CheckOverflowInTableInsert(proxy, "col").eval(null) }.asInstanceOf[SparkThrowable], - errorClass = "CAST_OVERFLOW_IN_TABLE_INSERT", + condition = "CAST_OVERFLOW_IN_TABLE_INSERT", parameters = Map( "sourceType" -> "\"DOUBLE\"", "targetType" -> ("\"TINYINT\""), @@ -366,7 +366,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest checkError( // If error is user-facing, it will be thrown directly. exception = intercept[SparkArithmeticException](df3.collect()), - errorClass = "DIVIDE_BY_ZERO", + condition = "DIVIDE_BY_ZERO", parameters = Map("config" -> ansiConf), context = ExpectedContext( fragment = "div", @@ -381,7 +381,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest val df4 = spark.range(0, 10, 1, 2).select(lit(1) / $"id") checkError( exception = intercept[SparkArithmeticException](df4.collect()), - errorClass = "DIVIDE_BY_ZERO", + condition = "DIVIDE_BY_ZERO", parameters = Map("config" -> ansiConf), context = ExpectedContext( fragment = "div", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 349b124970e32..307e70d8c6b79 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -77,7 +77,7 @@ class QueryExecutionErrorsSuite } checkError( exception, - errorClass = "CONVERSION_INVALID_INPUT", + condition = "CONVERSION_INVALID_INPUT", parameters = Map( "str" -> "'???'", "fmt" -> "'BASE64'", @@ -95,7 +95,7 @@ class QueryExecutionErrorsSuite } checkError( exception, - errorClass = "CONVERSION_INVALID_INPUT", + condition = "CONVERSION_INVALID_INPUT", parameters = Map( "str" -> "'???'", "fmt" -> "'HEX'", @@ -129,7 +129,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { df.collect() }, - errorClass = "INVALID_PARAMETER_VALUE.AES_KEY_LENGTH", + condition = "INVALID_PARAMETER_VALUE.AES_KEY_LENGTH", parameters = Map( "parameter" -> "`key`", "functionName" -> "`aes_encrypt`/`aes_decrypt`", @@ -166,7 +166,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { df2.selectExpr(s"aes_decrypt(unbase64($colName), binary('$key'), 'ECB')").collect() }, - errorClass = "INVALID_PARAMETER_VALUE.AES_CRYPTO_ERROR", + condition = "INVALID_PARAMETER_VALUE.AES_CRYPTO_ERROR", parameters = Map("parameter" -> "`expr`, `key`", "functionName" -> "`aes_encrypt`/`aes_decrypt`", "detailMessage" -> ("Given final block not properly padded. " + @@ -184,7 +184,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { df.collect() }, - errorClass = "UNSUPPORTED_FEATURE.AES_MODE", + condition = "UNSUPPORTED_FEATURE.AES_MODE", parameters = Map("mode" -> mode, "padding" -> padding, "functionName" -> "`aes_encrypt`/`aes_decrypt`"), @@ -212,7 +212,7 @@ class QueryExecutionErrorsSuite def checkUnsupportedTypeInLiteral(v: Any, literal: String, dataType: String): Unit = { checkError( exception = intercept[SparkRuntimeException] { spark.expression(lit(v)) }, - errorClass = "UNSUPPORTED_FEATURE.LITERAL_TYPE", + condition = "UNSUPPORTED_FEATURE.LITERAL_TYPE", parameters = Map("value" -> literal, "type" -> dataType), sqlState = "0A000") } @@ -230,7 +230,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e2, - errorClass = "UNSUPPORTED_FEATURE.PIVOT_TYPE", + condition = "UNSUPPORTED_FEATURE.PIVOT_TYPE", parameters = Map("value" -> "[dotnet,Dummies]", "type" -> "unknown"), sqlState = "0A000") @@ -247,7 +247,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e1, - errorClass = "REPEATED_CLAUSE", + condition = "REPEATED_CLAUSE", parameters = Map("clause" -> "PIVOT", "operation" -> "SUBQUERY"), sqlState = "42614") @@ -260,7 +260,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e2, - errorClass = "UNSUPPORTED_FEATURE.PIVOT_AFTER_GROUP_BY", + condition = "UNSUPPORTED_FEATURE.PIVOT_AFTER_GROUP_BY", parameters = Map[String, String](), sqlState = "0A000") } @@ -281,7 +281,7 @@ class QueryExecutionErrorsSuite val option = "\"datetimeRebaseMode\"" checkError( exception = e, - errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.READ_ANCIENT_DATETIME", + condition = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.READ_ANCIENT_DATETIME", parameters = Map("format" -> format, "config" -> config, "option" -> option)) } @@ -298,7 +298,7 @@ class QueryExecutionErrorsSuite val config = "\"" + SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key + "\"" checkError( exception = e.getCause.asInstanceOf[SparkUpgradeException], - errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.WRITE_ANCIENT_DATETIME", + condition = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.WRITE_ANCIENT_DATETIME", parameters = Map("format" -> format, "config" -> config)) } } @@ -314,7 +314,7 @@ class QueryExecutionErrorsSuite assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "UNSUPPORTED_FEATURE.ORC_TYPE_CAST", + condition = "UNSUPPORTED_FEATURE.ORC_TYPE_CAST", parameters = Map("orcType" -> "\"TIMESTAMP\"", "toType" -> "\"TIMESTAMP_NTZ\""), sqlState = "0A000") @@ -336,7 +336,7 @@ class QueryExecutionErrorsSuite assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "UNSUPPORTED_FEATURE.ORC_TYPE_CAST", + condition = "UNSUPPORTED_FEATURE.ORC_TYPE_CAST", parameters = Map("orcType" -> "\"TIMESTAMP_NTZ\"", "toType" -> "\"TIMESTAMP\""), sqlState = "0A000") @@ -349,7 +349,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkArithmeticException] { sql("select timestampadd(YEAR, 1000000, timestamp'2022-03-09 01:02:03')").collect() }, - errorClass = "DATETIME_OVERFLOW", + condition = "DATETIME_OVERFLOW", parameters = Map("operation" -> "add 1000000 YEAR to TIMESTAMP '2022-03-09 01:02:03'"), sqlState = "22008") } @@ -385,7 +385,7 @@ class QueryExecutionErrorsSuite checkError( exception = e2.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "CANNOT_PARSE_DECIMAL", + condition = "CANNOT_PARSE_DECIMAL", parameters = Map[String, String](), sqlState = "22018") } @@ -397,7 +397,7 @@ class QueryExecutionErrorsSuite sql(s"SELECT from_json('$jsonStr', 'a INT, b DOUBLE', map('mode','FAILFAST') )") .collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.CANNOT_PARSE_JSON_ARRAYS_AS_STRUCTS", + condition = "MALFORMED_RECORD_IN_PARSING.CANNOT_PARSE_JSON_ARRAYS_AS_STRUCTS", parameters = Map( "badRecord" -> jsonStr, "failFastMode" -> "FAILFAST" @@ -429,7 +429,7 @@ class QueryExecutionErrorsSuite .createOrReplaceTempView("words") spark.sql("select luckyCharOfWord(word, index) from words").collect() }, - errorClass = "FAILED_EXECUTE_UDF", + condition = "FAILED_EXECUTE_UDF", parameters = Map( "functionName" -> functionNameRegex, "signature" -> "string, int", @@ -458,7 +458,7 @@ class QueryExecutionErrorsSuite val words = Seq(("Jacek", 5), ("Agata", 5), ("Sweet", 6)).toDF("word", "index") words.select(luckyCharOfWord($"word", $"index")).collect() }, - errorClass = "FAILED_EXECUTE_UDF", + condition = "FAILED_EXECUTE_UDF", parameters = Map("functionName" -> functionNameRegex, "signature" -> "string, int", "result" -> "string", @@ -487,7 +487,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e, - errorClass = "INCOMPARABLE_PIVOT_COLUMN", + condition = "INCOMPARABLE_PIVOT_COLUMN", parameters = Map("columnName" -> "`map`"), sqlState = "42818") } @@ -500,7 +500,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e1, - errorClass = "UNSUPPORTED_SAVE_MODE.NON_EXISTENT_PATH", + condition = "UNSUPPORTED_SAVE_MODE.NON_EXISTENT_PATH", parameters = Map("saveMode" -> "NULL")) Utils.createDirectory(path) @@ -511,7 +511,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e2, - errorClass = "UNSUPPORTED_SAVE_MODE.EXISTENT_PATH", + condition = "UNSUPPORTED_SAVE_MODE.EXISTENT_PATH", parameters = Map("saveMode" -> "NULL")) } } @@ -521,7 +521,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkException] { RuleIdCollection.getRuleId("incorrect") }, - errorClass = "RULE_ID_NOT_FOUND", + condition = "RULE_ID_NOT_FOUND", parameters = Map("ruleName" -> "incorrect") ) } @@ -540,7 +540,7 @@ class QueryExecutionErrorsSuite checkError( exception = e.getCause.asInstanceOf[SparkSecurityException], - errorClass = "CANNOT_RESTORE_PERMISSIONS_FOR_PATH", + condition = "CANNOT_RESTORE_PERMISSIONS_FOR_PATH", parameters = Map("permission" -> ".+", "path" -> ".+"), matchPVals = true) @@ -569,7 +569,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e, - errorClass = "INCOMPATIBLE_DATASOURCE_REGISTER", + condition = "INCOMPATIBLE_DATASOURCE_REGISTER", parameters = Map("message" -> ("Illegal configuration-file syntax: " + "META-INF/services/org.apache.spark.sql.sources.DataSourceRegister"))) } @@ -650,7 +650,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkSQLException] { spark.read.jdbc(urlWithUserAndPass, tableName, new Properties()).collect() }, - errorClass = "UNRECOGNIZED_SQL_TYPE", + condition = "UNRECOGNIZED_SQL_TYPE", parameters = Map("typeName" -> unrecognizedColumnTypeName, "jdbcType" -> "DATALINK"), sqlState = "42704") @@ -675,7 +675,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkException] { aggregated.count() }, - errorClass = "INVALID_BUCKET_FILE", + condition = "INVALID_BUCKET_FILE", parameters = Map("path" -> ".+"), matchPVals = true) } @@ -688,7 +688,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkException] { sql("select (select a from (select 1 as a union all select 2 as a) t) as b").collect() }, - errorClass = "SCALAR_SUBQUERY_TOO_MANY_ROWS", + condition = "SCALAR_SUBQUERY_TOO_MANY_ROWS", queryContext = Array( ExpectedContext( fragment = "(select a from (select 1 as a union all select 2 as a) t)", @@ -704,7 +704,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkArithmeticException]( sql("select add_months('5500000-12-31', 10000000)").collect() ), - errorClass = "ARITHMETIC_OVERFLOW", + condition = "ARITHMETIC_OVERFLOW", parameters = Map( "message" -> "integer overflow", "alternative" -> "", @@ -717,7 +717,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { StructType.fromString(raw) }, - errorClass = "FAILED_PARSE_STRUCT_TYPE", + condition = "FAILED_PARSE_STRUCT_TYPE", parameters = Map("raw" -> s"'$raw'")) } @@ -730,7 +730,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkArithmeticException] { sql(s"select CAST($sourceValue AS $it)").collect() }, - errorClass = "CAST_OVERFLOW", + condition = "CAST_OVERFLOW", parameters = Map( "value" -> sourceValue, "sourceType" -> s""""${sourceType.sql}"""", @@ -747,7 +747,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { sql(s"""SELECT from_json('$jsonStr', 'a FLOAT', map('mode','FAILFAST'))""").collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.CANNOT_PARSE_STRING_AS_DATATYPE", + condition = "MALFORMED_RECORD_IN_PARSING.CANNOT_PARSE_STRING_AS_DATATYPE", parameters = Map( "badRecord" -> jsonStr, "failFastMode" -> "FAILFAST", @@ -764,7 +764,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkArithmeticException] { sql(s"select 127Y + 5Y").collect() }, - errorClass = "BINARY_ARITHMETIC_OVERFLOW", + condition = "BINARY_ARITHMETIC_OVERFLOW", parameters = Map( "value1" -> "127S", "symbol" -> "+", @@ -779,7 +779,7 @@ class QueryExecutionErrorsSuite val row = spark.sparkContext.parallelize(Seq(1, 2)).map(Row(_)) spark.createDataFrame(row, StructType.fromString("StructType()")) }, - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map( "typeName" -> "StructType()[1.1] failure: 'TimestampType' expected but 'S' found\n\nStructType()\n^" @@ -810,7 +810,7 @@ class QueryExecutionErrorsSuite val expectedPath = p.toURI checkError( exception = e, - errorClass = "RENAME_SRC_PATH_NOT_FOUND", + condition = "RENAME_SRC_PATH_NOT_FOUND", matchPVals = true, parameters = Map("sourcePath" -> s"$expectedPath.+") ) @@ -871,7 +871,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkSQLFeatureNotSupportedException] { sql("alter TABLE h2.test.people SET TBLPROPERTIES (xx='xx', yy='yy')") }, - errorClass = "UNSUPPORTED_FEATURE.MULTI_ACTION_ALTER", + condition = "UNSUPPORTED_FEATURE.MULTI_ACTION_ALTER", parameters = Map("tableName" -> "\"test\".\"people\"")) JdbcDialects.unregisterDialect(testH2DialectUnsupportedJdbcTransaction) @@ -927,7 +927,7 @@ class QueryExecutionErrorsSuite exceptions.flatten.map { e => checkError( e, - errorClass = "CONCURRENT_QUERY", + condition = "CONCURRENT_QUERY", sqlState = Some("0A000"), parameters = e.getMessageParameters.asScala.toMap ) @@ -948,7 +948,7 @@ class QueryExecutionErrorsSuite checkError( exception = e, - errorClass = "UNSUPPORTED_EXPR_FOR_WINDOW", + condition = "UNSUPPORTED_EXPR_FOR_WINDOW", parameters = Map( "sqlExpr" -> "\"to_date(2009-07-30 04:17:52)\"" ), @@ -969,7 +969,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Cannot evaluate expression: namedparameter(foo)"), sqlState = "XX000") } @@ -981,7 +981,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map( "message" -> ("Cannot generate code for expression: " + "grouping(namedparameter(foo))")), @@ -994,7 +994,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Cannot terminate expression: 'foo()"), sqlState = "XX000") } @@ -1008,7 +1008,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map( "message" -> ("""A method named "nonexistent" is not declared in """ + "any enclosing class nor any supertype")), @@ -1021,7 +1021,7 @@ class QueryExecutionErrorsSuite } checkError( exception = e, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map( "message" -> "The aggregate window function `row_number` does not support merging."), sqlState = "XX000") @@ -1032,7 +1032,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkArrayIndexOutOfBoundsException] { sql("select bitmap_construct_agg(col) from values (32768) as tab(col)").collect() }, - errorClass = "INVALID_BITMAP_POSITION", + condition = "INVALID_BITMAP_POSITION", parameters = Map( "bitPosition" -> "32768", "bitmapNumBytes" -> "4096", @@ -1045,7 +1045,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkArrayIndexOutOfBoundsException] { sql("select bitmap_construct_agg(col) from values (-1) as tab(col)").collect() }, - errorClass = "INVALID_BITMAP_POSITION", + condition = "INVALID_BITMAP_POSITION", parameters = Map( "bitPosition" -> "-1", "bitmapNumBytes" -> "4096", @@ -1060,7 +1060,7 @@ class QueryExecutionErrorsSuite maxBroadcastTableBytes = 1024 * 1024 * 1024, dataSize = 2 * 1024 * 1024 * 1024 - 1) }, - errorClass = "_LEGACY_ERROR_TEMP_2249", + condition = "_LEGACY_ERROR_TEMP_2249", parameters = Map("maxBroadcastTableBytes" -> "1024.0 MiB", "dataSize" -> "2048.0 MiB")) } @@ -1071,7 +1071,7 @@ class QueryExecutionErrorsSuite exception = intercept[AnalysisException] { sql("SELECT * FROM t TIMESTAMP AS OF '2021-01-29 00:00:00'").collect() }, - errorClass = "UNSUPPORTED_FEATURE.TIME_TRAVEL", + condition = "UNSUPPORTED_FEATURE.TIME_TRAVEL", parameters = Map("relationId" -> "`spark_catalog`.`default`.`t`") ) } @@ -1082,7 +1082,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { sql("select slice(array(1,2,3), 0, 1)").collect() }, - errorClass = "INVALID_PARAMETER_VALUE.START", + condition = "INVALID_PARAMETER_VALUE.START", parameters = Map( "parameter" -> toSQLId("start"), "functionName" -> toSQLId("slice") @@ -1095,7 +1095,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { sql("select slice(array(1,2,3), 1, -1)").collect() }, - errorClass = "INVALID_PARAMETER_VALUE.LENGTH", + condition = "INVALID_PARAMETER_VALUE.LENGTH", parameters = Map( "parameter" -> toSQLId("length"), "length" -> (-1).toString, @@ -1112,7 +1112,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { Concat(Seq(Literal.create(array, ArrayType(BooleanType)))).eval(EmptyRow) }, - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.FUNCTION", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.FUNCTION", parameters = Map( "numberOfElements" -> Int.MaxValue.toString, "maxRoundedArrayLength" -> MAX_ROUNDED_ARRAY_LENGTH.toString, @@ -1129,7 +1129,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { Flatten(CreateArray(Seq(Literal.create(array, ArrayType(BooleanType))))).eval(EmptyRow) }, - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.FUNCTION", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.FUNCTION", parameters = Map( "numberOfElements" -> Int.MaxValue.toString, "maxRoundedArrayLength" -> MAX_ROUNDED_ARRAY_LENGTH.toString, @@ -1144,7 +1144,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { sql(s"select array_repeat(1, $count)").collect() }, - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", parameters = Map( "parameter" -> toSQLId("count"), "numberOfElements" -> count.toString, @@ -1164,7 +1164,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { Seq(KryoData(1), KryoData(2)).toDS() }, - errorClass = "INVALID_EXPRESSION_ENCODER", + condition = "INVALID_EXPRESSION_ENCODER", parameters = Map( "encoderType" -> kryoEncoder.getClass.getName, "docroot" -> SPARK_DOC_ROOT @@ -1183,7 +1183,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { spark.createDataset(Seq(Row(1))).collect() }, - errorClass = "NOT_UNRESOLVED_ENCODER", + condition = "NOT_UNRESOLVED_ENCODER", parameters = Map( "attr" -> deserializer.toString ) @@ -1206,7 +1206,7 @@ class QueryExecutionErrorsSuite exception = intercept[SparkRuntimeException] { expr.eval(EmptyRow) }, - errorClass = "CLASS_NOT_OVERRIDE_EXPECTED_METHOD", + condition = "CLASS_NOT_OVERRIDE_EXPECTED_METHOD", parameters = Map( "className" -> expr.getClass.getName, "method1" -> "eval", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index b7fb65091ef73..da7b6e7f63c85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -39,7 +39,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL } checkError( exception = parseException(query), - errorClass = "FAILED_TO_PARSE_TOO_COMPLEX", + condition = "FAILED_TO_PARSE_TOO_COMPLEX", parameters = Map(), context = ExpectedContext( query, @@ -53,7 +53,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL ", 2 as first, 3 as second, 4 as second, 5 as third" checkError( exception = parseException(query), - errorClass = "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES", + condition = "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES", parameters = Map("aliases" -> "`second`, `first`"), context = ExpectedContext( "USING 1 as first, 2 as first, 3 as second, 4 as second, 5 as third", @@ -66,7 +66,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val query = "EXECUTE IMMEDIATE 'SELCT 1707 WHERE ? = 1' INTO a USING 1" checkError( exception = parseException(query), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'SELCT'", "hint" -> ""), context = ExpectedContext( start = 0, @@ -79,7 +79,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL withSQLConf("spark.sql.allowNamedFunctionArguments" -> "false") { checkError( exception = parseException("SELECT explode(arr => array(10, 20))"), - errorClass = "NAMED_PARAMETER_SUPPORT_DISABLED", + condition = "NAMED_PARAMETER_SUPPORT_DISABLED", parameters = Map("functionName"-> toSQLId("explode"), "argument" -> toSQLId("arr")) ) } @@ -88,7 +88,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("UNSUPPORTED_FEATURE: LATERAL join with NATURAL join not supported") { checkError( exception = parseException("SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2)"), - errorClass = "INCOMPATIBLE_JOIN_TYPES", + condition = "INCOMPATIBLE_JOIN_TYPES", parameters = Map("joinType1" -> "LATERAL", "joinType2" -> "NATURAL"), sqlState = "42613", context = ExpectedContext( @@ -100,7 +100,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("UNSUPPORTED_FEATURE: LATERAL join with USING join not supported") { checkError( exception = parseException("SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2)"), - errorClass = "UNSUPPORTED_FEATURE.LATERAL_JOIN_USING", + condition = "UNSUPPORTED_FEATURE.LATERAL_JOIN_USING", sqlState = "0A000", context = ExpectedContext( fragment = "JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2)", @@ -116,7 +116,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL "LEFT ANTI" -> (17, 72)).foreach { case (joinType, (start, stop)) => checkError( exception = parseException(s"SELECT * FROM t1 $joinType JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3"), - errorClass = "INVALID_LATERAL_JOIN_TYPE", + condition = "INVALID_LATERAL_JOIN_TYPE", parameters = Map("joinType" -> joinType), context = ExpectedContext( fragment = s"$joinType JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3", @@ -136,7 +136,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL ).foreach { case (sqlText, (fragment, start, stop)) => checkError( exception = parseException(s"SELECT * FROM t1$sqlText"), - errorClass = "INVALID_SQL_SYNTAX.LATERAL_WITHOUT_SUBQUERY_OR_TABLE_VALUED_FUNC", + condition = "INVALID_SQL_SYNTAX.LATERAL_WITHOUT_SUBQUERY_OR_TABLE_VALUED_FUNC", sqlState = "42000", context = ExpectedContext(fragment, start, stop)) } @@ -145,7 +145,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("UNSUPPORTED_FEATURE: NATURAL CROSS JOIN is not supported") { checkError( exception = parseException("SELECT * FROM a NATURAL CROSS JOIN b"), - errorClass = "INCOMPATIBLE_JOIN_TYPES", + condition = "INCOMPATIBLE_JOIN_TYPES", parameters = Map("joinType1" -> "NATURAL", "joinType2" -> "CROSS"), sqlState = "42613", context = ExpectedContext( @@ -157,7 +157,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("INVALID_SQL_SYNTAX.REPETITIVE_WINDOW_DEFINITION: redefine window") { checkError( exception = parseException("SELECT min(a) OVER win FROM t1 WINDOW win AS win, win AS win2"), - errorClass = "INVALID_SQL_SYNTAX.REPETITIVE_WINDOW_DEFINITION", + condition = "INVALID_SQL_SYNTAX.REPETITIVE_WINDOW_DEFINITION", sqlState = "42000", parameters = Map("windowName" -> "`win`"), context = ExpectedContext( @@ -169,7 +169,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("INVALID_SQL_SYNTAX.INVALID_WINDOW_REFERENCE: invalid window reference") { checkError( exception = parseException("SELECT min(a) OVER win FROM t1 WINDOW win AS win"), - errorClass = "INVALID_SQL_SYNTAX.INVALID_WINDOW_REFERENCE", + condition = "INVALID_SQL_SYNTAX.INVALID_WINDOW_REFERENCE", sqlState = "42000", parameters = Map("windowName" -> "`win`"), context = ExpectedContext( @@ -181,7 +181,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("INVALID_SQL_SYNTAX.UNRESOLVED_WINDOW_REFERENCE: window reference cannot be resolved") { checkError( exception = parseException("SELECT min(a) OVER win FROM t1 WINDOW win AS win2"), - errorClass = "INVALID_SQL_SYNTAX.UNRESOLVED_WINDOW_REFERENCE", + condition = "INVALID_SQL_SYNTAX.UNRESOLVED_WINDOW_REFERENCE", sqlState = "42000", parameters = Map("windowName" -> "`win2`"), context = ExpectedContext( @@ -194,7 +194,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText = "SELECT TRANSFORM(DISTINCT a) USING 'a' FROM t" checkError( exception = parseException(sqlText), - errorClass = "UNSUPPORTED_FEATURE.TRANSFORM_DISTINCT_ALL", + condition = "UNSUPPORTED_FEATURE.TRANSFORM_DISTINCT_ALL", sqlState = "0A000", context = ExpectedContext( fragment = sqlText, @@ -207,7 +207,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL "'org.apache.hadoop.hive.serde2.OpenCSVSerde' USING 'a' FROM t" checkError( exception = parseException(sqlText), - errorClass = "UNSUPPORTED_FEATURE.TRANSFORM_NON_HIVE", + condition = "UNSUPPORTED_FEATURE.TRANSFORM_NON_HIVE", sqlState = "0A000", context = ExpectedContext( fragment = sqlText, @@ -218,7 +218,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("INVALID_SQL_SYNTAX.TRANSFORM_WRONG_NUM_ARGS: Wrong number arguments for transform") { checkError( exception = parseException("CREATE TABLE table(col int) PARTITIONED BY (years(col,col))"), - errorClass = "INVALID_SQL_SYNTAX.TRANSFORM_WRONG_NUM_ARGS", + condition = "INVALID_SQL_SYNTAX.TRANSFORM_WRONG_NUM_ARGS", sqlState = "42000", parameters = Map( "transform" -> "`years`", @@ -233,7 +233,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME: Invalid table value function name") { checkError( exception = parseException("SELECT * FROM db.func()"), - errorClass = "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", + condition = "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", sqlState = "42000", parameters = Map("funcName" -> "`db`.`func`"), context = ExpectedContext( @@ -243,7 +243,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL checkError( exception = parseException("SELECT * FROM ns.db.func()"), - errorClass = "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", + condition = "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", sqlState = "42000", parameters = Map("funcName" -> "`ns`.`db`.`func`"), context = ExpectedContext( @@ -256,7 +256,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText = "SHOW sys FUNCTIONS" checkError( exception = parseException(sqlText), - errorClass = "INVALID_SQL_SYNTAX.SHOW_FUNCTIONS_INVALID_SCOPE", + condition = "INVALID_SQL_SYNTAX.SHOW_FUNCTIONS_INVALID_SCOPE", sqlState = "42000", parameters = Map("scope" -> "`sys`"), context = ExpectedContext( @@ -269,7 +269,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText1 = "SHOW FUNCTIONS IN db f1" checkError( exception = parseException(sqlText1), - errorClass = "INVALID_SQL_SYNTAX.SHOW_FUNCTIONS_INVALID_PATTERN", + condition = "INVALID_SQL_SYNTAX.SHOW_FUNCTIONS_INVALID_PATTERN", sqlState = "42000", parameters = Map("pattern" -> "`f1`"), context = ExpectedContext( @@ -279,7 +279,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText2 = "SHOW FUNCTIONS IN db LIKE f1" checkError( exception = parseException(sqlText2), - errorClass = "INVALID_SQL_SYNTAX.SHOW_FUNCTIONS_INVALID_PATTERN", + condition = "INVALID_SQL_SYNTAX.SHOW_FUNCTIONS_INVALID_PATTERN", sqlState = "42000", parameters = Map("pattern" -> "`f1`"), context = ExpectedContext( @@ -297,7 +297,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL checkError( exception = parseException(sqlText), - errorClass = "INVALID_SQL_SYNTAX.CREATE_ROUTINE_WITH_IF_NOT_EXISTS_AND_REPLACE", + condition = "INVALID_SQL_SYNTAX.CREATE_ROUTINE_WITH_IF_NOT_EXISTS_AND_REPLACE", sqlState = "42000", context = ExpectedContext( fragment = sqlText, @@ -314,7 +314,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL checkError( exception = parseException(sqlText), - errorClass = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_IF_NOT_EXISTS", + condition = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_IF_NOT_EXISTS", sqlState = "42000", context = ExpectedContext( fragment = sqlText, @@ -330,7 +330,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL checkError( exception = parseException(sqlText), - errorClass = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + condition = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", sqlState = "42000", parameters = Map( "statement" -> "CREATE TEMPORARY FUNCTION", @@ -350,7 +350,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL checkError( exception = parseException(sqlText), - errorClass = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_DATABASE", + condition = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_DATABASE", sqlState = "42000", parameters = Map("database" -> "`db`"), context = ExpectedContext( @@ -363,7 +363,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText = "DROP TEMPORARY FUNCTION db.func" checkError( exception = parseException(sqlText), - errorClass = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + condition = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", sqlState = "42000", parameters = Map( "statement" -> "DROP TEMPORARY FUNCTION", @@ -377,7 +377,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("DUPLICATE_KEY: Found duplicate partition keys") { checkError( exception = parseException("INSERT OVERWRITE TABLE table PARTITION(p1='1', p1='1') SELECT 'col1', 'col2'"), - errorClass = "DUPLICATE_KEY", + condition = "DUPLICATE_KEY", sqlState = "23505", parameters = Map("keyColumn" -> "`p1`"), context = ExpectedContext( @@ -389,7 +389,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("DUPLICATE_KEY: in table properties") { checkError( exception = parseException("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('key1' = '1', 'key1' = '2')"), - errorClass = "DUPLICATE_KEY", + condition = "DUPLICATE_KEY", sqlState = "23505", parameters = Map("keyColumn" -> "`key1`"), context = ExpectedContext( @@ -401,24 +401,24 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("PARSE_EMPTY_STATEMENT: empty input") { checkError( exception = parseException(""), - errorClass = "PARSE_EMPTY_STATEMENT", + condition = "PARSE_EMPTY_STATEMENT", sqlState = Some("42617")) checkError( exception = parseException(" "), - errorClass = "PARSE_EMPTY_STATEMENT", + condition = "PARSE_EMPTY_STATEMENT", sqlState = Some("42617")) checkError( exception = parseException(" \n"), - errorClass = "PARSE_EMPTY_STATEMENT", + condition = "PARSE_EMPTY_STATEMENT", sqlState = Some("42617")) } test("PARSE_SYNTAX_ERROR: no viable input") { checkError( exception = parseException("select ((r + 1) "), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "end of input", "hint" -> "")) } @@ -426,7 +426,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL def checkParseSyntaxError(sqlCommand: String, errorString: String, hint: String = ""): Unit = { checkError( exception = parseException(sqlCommand), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> errorString, "hint" -> hint) ) @@ -444,13 +444,13 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("PARSE_SYNTAX_ERROR: extraneous input") { checkError( exception = parseException("select 1 1"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "'1'", "hint" -> ": extra input '1'")) checkError( exception = parseException("select *\nfrom r as q t"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "'t'", "hint" -> ": extra input 't'")) } @@ -458,13 +458,13 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("PARSE_SYNTAX_ERROR: mismatched input") { checkError( exception = parseException("select * from r order by q from t"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "'from'", "hint" -> "")) checkError( exception = parseException("select *\nfrom r\norder by q\nfrom t"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "'from'", "hint" -> "")) } @@ -473,13 +473,13 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL // '' -> end of input checkError( exception = parseException("select count(*"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "end of input", "hint" -> "")) checkError( exception = parseException("select 1 as a from"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "end of input", "hint" -> "")) } @@ -488,19 +488,19 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL "misleading error message due to problematic antlr grammar") { checkError( exception = parseException("select * from a left join_ b on a.id = b.id"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "'join_'", "hint" -> ": missing 'JOIN'")) checkError( exception = parseException("select * from test where test.t is like 'test'"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "'is'", "hint" -> "")) checkError( exception = parseException("SELECT * FROM test WHERE x NOT NULL"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "'NOT'", "hint" -> "")) } @@ -508,7 +508,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("INVALID_SQL_SYNTAX.EMPTY_PARTITION_VALUE: show table partition key must set value") { checkError( exception = parseException("SHOW TABLE EXTENDED IN default LIKE 'employee' PARTITION (grade)"), - errorClass = "INVALID_SQL_SYNTAX.EMPTY_PARTITION_VALUE", + condition = "INVALID_SQL_SYNTAX.EMPTY_PARTITION_VALUE", sqlState = "42000", parameters = Map("partKey" -> "`grade`"), context = ExpectedContext( @@ -522,7 +522,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL checkError( exception = parseException("CREATE TABLE my_tab(a INT, b STRING) " + "USING parquet PARTITIONED BY (bucket(32, a, 66))"), - errorClass = "INVALID_SQL_SYNTAX.INVALID_COLUMN_REFERENCE", + condition = "INVALID_SQL_SYNTAX.INVALID_COLUMN_REFERENCE", sqlState = "42000", parameters = Map( "transform" -> "`bucket`", @@ -537,7 +537,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText = "DESCRIBE TABLE EXTENDED customer PARTITION (grade = 'A') customer.age" checkError( exception = parseException(sqlText), - errorClass = "UNSUPPORTED_FEATURE.DESC_TABLE_COLUMN_PARTITION", + condition = "UNSUPPORTED_FEATURE.DESC_TABLE_COLUMN_PARTITION", sqlState = "0A000", context = ExpectedContext( fragment = sqlText, @@ -549,7 +549,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText = "DESCRIBE TABLE EXTENDED customer PARTITION (grade)" checkError( exception = parseException(sqlText), - errorClass = "INVALID_SQL_SYNTAX.EMPTY_PARTITION_VALUE", + condition = "INVALID_SQL_SYNTAX.EMPTY_PARTITION_VALUE", sqlState = "42000", parameters = Map("partKey" -> "`grade`"), context = ExpectedContext( @@ -562,7 +562,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText = "CREATE NAMESPACE IF NOT EXISTS a.b.c WITH PROPERTIES ('location'='/home/user/db')" checkError( exception = parseException(sqlText), - errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", sqlState = "0A000", parameters = Map( "property" -> "location", @@ -578,7 +578,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL "USING PARQUET TBLPROPERTIES ('provider'='parquet')" checkError( exception = parseException(sqlText), - errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", sqlState = "0A000", parameters = Map( "property" -> "provider", @@ -593,7 +593,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText = "set =`value`" checkError( exception = parseException(sqlText), - errorClass = "INVALID_PROPERTY_KEY", + condition = "INVALID_PROPERTY_KEY", parameters = Map("key" -> "\"\"", "value" -> "\"value\""), context = ExpectedContext( fragment = sqlText, @@ -604,7 +604,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("INVALID_PROPERTY_VALUE: invalid property value for set quoted configuration") { checkError( exception = parseException("set `key`=1;2;;"), - errorClass = "INVALID_PROPERTY_VALUE", + condition = "INVALID_PROPERTY_VALUE", parameters = Map("value" -> "\"1;2;;\"", "key" -> "\"key\""), context = ExpectedContext( fragment = "set `key`=1;2", @@ -617,7 +617,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL "WITH DBPROPERTIES('a'='a', 'b'='b', 'c'='c')" checkError( exception = parseException(sqlText), - errorClass = "UNSUPPORTED_FEATURE.SET_PROPERTIES_AND_DBPROPERTIES", + condition = "UNSUPPORTED_FEATURE.SET_PROPERTIES_AND_DBPROPERTIES", sqlState = "0A000", context = ExpectedContext( fragment = sqlText, @@ -629,28 +629,28 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL // Cast simple array without specifying element type checkError( exception = parseException("SELECT CAST(array(1,2,3) AS ARRAY)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.ARRAY", + condition = "INCOMPLETE_TYPE_DEFINITION.ARRAY", sqlState = "42K01", parameters = Map("elementType" -> ""), context = ExpectedContext(fragment = "ARRAY", start = 28, stop = 32)) // Cast array of array without specifying element type for inner array checkError( exception = parseException("SELECT CAST(array(array(3)) AS ARRAY)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.ARRAY", + condition = "INCOMPLETE_TYPE_DEFINITION.ARRAY", sqlState = "42K01", parameters = Map("elementType" -> ""), context = ExpectedContext(fragment = "ARRAY", start = 37, stop = 41)) // Create column of array type without specifying element type checkError( exception = parseException("CREATE TABLE tbl_120691 (col1 ARRAY)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.ARRAY", + condition = "INCOMPLETE_TYPE_DEFINITION.ARRAY", sqlState = "42K01", parameters = Map("elementType" -> ""), context = ExpectedContext(fragment = "ARRAY", start = 30, stop = 34)) // Create column of array type without specifying element type in lowercase checkError( exception = parseException("CREATE TABLE tbl_120691 (col1 array)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.ARRAY", + condition = "INCOMPLETE_TYPE_DEFINITION.ARRAY", sqlState = "42K01", parameters = Map("elementType" -> ""), context = ExpectedContext(fragment = "array", start = 30, stop = 34)) @@ -660,31 +660,31 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL // Cast simple struct without specifying field type checkError( exception = parseException("SELECT CAST(struct(1,2,3) AS STRUCT)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.STRUCT", + condition = "INCOMPLETE_TYPE_DEFINITION.STRUCT", sqlState = "42K01", context = ExpectedContext(fragment = "STRUCT", start = 29, stop = 34)) // Cast array of struct without specifying field type in struct checkError( exception = parseException("SELECT CAST(array(struct(1,2)) AS ARRAY)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.STRUCT", + condition = "INCOMPLETE_TYPE_DEFINITION.STRUCT", sqlState = "42K01", context = ExpectedContext(fragment = "STRUCT", start = 40, stop = 45)) // Create column of struct type without specifying field type checkError( exception = parseException("CREATE TABLE tbl_120691 (col1 STRUCT)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.STRUCT", + condition = "INCOMPLETE_TYPE_DEFINITION.STRUCT", sqlState = "42K01", context = ExpectedContext(fragment = "STRUCT", start = 30, stop = 35)) // Invalid syntax `STRUCT` without field name checkError( exception = parseException("SELECT CAST(struct(1,2,3) AS STRUCT)"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "'<'", "hint" -> ": missing ')'")) // Create column of struct type without specifying field type in lowercase checkError( exception = parseException("CREATE TABLE tbl_120691 (col1 struct)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.STRUCT", + condition = "INCOMPLETE_TYPE_DEFINITION.STRUCT", sqlState = "42K01", context = ExpectedContext(fragment = "struct", start = 30, stop = 35)) } @@ -693,25 +693,25 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL // Cast simple map without specifying element type checkError( exception = parseException("SELECT CAST(map(1,'2') AS MAP)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.MAP", + condition = "INCOMPLETE_TYPE_DEFINITION.MAP", sqlState = "42K01", context = ExpectedContext(fragment = "MAP", start = 26, stop = 28)) // Create column of map type without specifying key/value types checkError( exception = parseException("CREATE TABLE tbl_120691 (col1 MAP)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.MAP", + condition = "INCOMPLETE_TYPE_DEFINITION.MAP", sqlState = "42K01", context = ExpectedContext(fragment = "MAP", start = 30, stop = 32)) // Invalid syntax `MAP` with only key type checkError( exception = parseException("SELECT CAST(map('1',2) AS MAP)"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", sqlState = "42601", parameters = Map("error" -> "'<'", "hint" -> ": missing ')'")) // Create column of map type without specifying key/value types in lowercase checkError( exception = parseException("SELECT CAST(map('1',2) AS map)"), - errorClass = "INCOMPLETE_TYPE_DEFINITION.MAP", + condition = "INCOMPLETE_TYPE_DEFINITION.MAP", sqlState = "42K01", context = ExpectedContext(fragment = "map", start = 26, stop = 28)) } @@ -719,7 +719,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL test("INVALID_ESC: Escape string must contain only one character") { checkError( exception = parseException("select * from test where test.t like 'pattern%' escape '##'"), - errorClass = "INVALID_ESC", + condition = "INVALID_ESC", parameters = Map("invalidEscape" -> "'##'"), context = ExpectedContext( fragment = "like 'pattern%' escape '##'", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala index cae22eda32f80..91b1bfd7bf213 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala @@ -51,7 +51,7 @@ class ExecuteImmediateEndToEndSuite extends QueryTest with SharedSparkSession { checkError( exception = e, - errorClass = "FAILED_TO_PARSE_TOO_COMPLEX", + condition = "FAILED_TO_PARSE_TOO_COMPLEX", parameters = Map(), context = ExpectedContext( query, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index f54a4f4606061..9a45d43656b35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -85,7 +85,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql("CREATE VIEW jtv1 AS SELECT * FROM temp_jtv1 WHERE id < 6") }, - errorClass = "INVALID_TEMP_OBJ_REFERENCE", + condition = "INVALID_TEMP_OBJ_REFERENCE", parameters = Map( "obj" -> "VIEW", "objName" -> s"`$SESSION_CATALOG_NAME`.`default`.`jtv1`", @@ -97,7 +97,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"CREATE VIEW jtv1 AS SELECT * FROM $globalTempDB.global_temp_jtv1 WHERE id < 6") }, - errorClass = "INVALID_TEMP_OBJ_REFERENCE", + condition = "INVALID_TEMP_OBJ_REFERENCE", parameters = Map( "obj" -> "VIEW", "objName" -> s"`$SESSION_CATALOG_NAME`.`default`.`jtv1`", @@ -115,7 +115,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql("CREATE OR REPLACE VIEW tab1 AS SELECT * FROM jt") }, - errorClass = "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE", + condition = "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE", parameters = Map( "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`", "operation" -> "CREATE OR REPLACE VIEW") @@ -124,7 +124,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql("CREATE VIEW tab1 AS SELECT * FROM jt") }, - errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map( "relationName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`") ) @@ -132,7 +132,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql("ALTER VIEW tab1 AS SELECT * FROM jt") }, - errorClass = "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE", + condition = "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE", parameters = Map( "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`", "operation" -> "ALTER VIEW ... AS" @@ -161,7 +161,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')") }, - errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP", + condition = "EXPECT_PERMANENT_VIEW_NOT_TEMP", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER VIEW ... SET TBLPROPERTIES" @@ -176,7 +176,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") }, - errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP", + condition = "EXPECT_PERMANENT_VIEW_NOT_TEMP", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER VIEW ... UNSET TBLPROPERTIES" @@ -198,7 +198,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName SET SERDE 'whatever'") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", + condition = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]" @@ -209,7 +209,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 'whatever'") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", + condition = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]" @@ -220,7 +220,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", + condition = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]" @@ -231,7 +231,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... RENAME TO PARTITION" @@ -242,7 +242,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName RECOVER PARTITIONS") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... RECOVER PARTITIONS" @@ -253,7 +253,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... SET LOCATION ..." @@ -264,7 +264,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... SET LOCATION ..." @@ -275,7 +275,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... ADD PARTITION ..." @@ -286,7 +286,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... DROP PARTITION ..." @@ -297,7 +297,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName SET TBLPROPERTIES ('p' = 'an')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", + condition = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... SET TBLPROPERTIES" @@ -308,7 +308,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $viewName UNSET TBLPROPERTIES ('p')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", + condition = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ALTER TABLE ... UNSET TBLPROPERTIES" @@ -327,7 +327,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"INSERT INTO TABLE $viewName SELECT 1") }, - errorClass = "UNSUPPORTED_INSERT.RDD_BASED", + condition = "UNSUPPORTED_INSERT.RDD_BASED", parameters = Map.empty ) @@ -338,7 +338,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "LOAD DATA" @@ -353,7 +353,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"SHOW CREATE TABLE $viewName") }, - errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP", + condition = "EXPECT_PERMANENT_VIEW_NOT_TEMP", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "SHOW CREATE TABLE" @@ -368,7 +368,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ANALYZE TABLE $viewName COMPUTE STATISTICS") }, - errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP", + condition = "EXPECT_PERMANENT_VIEW_NOT_TEMP", parameters = Map( "viewName" -> s"`$viewName`", "operation" -> "ANALYZE TABLE" @@ -383,7 +383,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ANALYZE TABLE $viewName COMPUTE STATISTICS FOR COLUMNS id") }, - errorClass = "UNSUPPORTED_FEATURE.ANALYZE_UNCACHED_TEMP_VIEW", + condition = "UNSUPPORTED_FEATURE.ANALYZE_UNCACHED_TEMP_VIEW", parameters = Map("viewName" -> s"`$viewName`") ) } @@ -394,7 +394,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { parameters: Map[String, String], context: ExpectedContext): Unit = { val e = intercept[AnalysisException](sql(query)) - checkError(e, errorClass = errorClass, parameters = parameters, context = context) + checkError(e, condition = errorClass, parameters = parameters, context = context) } test("error handling: insert/load table commands against a view") { @@ -405,7 +405,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"INSERT INTO TABLE $viewName SELECT 1") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`testview`", "operation" -> "INSERT" @@ -420,7 +420,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`testview`", "operation" -> "LOAD DATA"), @@ -513,7 +513,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW"), context = ExpectedContext(sqlText, 0, 77)) } @@ -867,7 +867,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql("CREATE VIEW testView2(x, y, z) AS SELECT * FROM tab1") }, - errorClass = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`testView2`", "viewColumns" -> "`x`, `y`, `z`", @@ -884,7 +884,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { .write.mode(SaveMode.Overwrite).saveAsTable("tab1") checkError( exception = intercept[AnalysisException](sql("SELECT * FROM testView")), - errorClass = "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + condition = "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`testview`", "actualCols" -> "[]", @@ -914,7 +914,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { df2.write.format("json").mode(SaveMode.Overwrite).saveAsTable("tab1") checkError( exception = intercept[AnalysisException](sql("SELECT * FROM testView")), - errorClass = "CANNOT_UP_CAST_DATATYPE", + condition = "CANNOT_UP_CAST_DATATYPE", parameters = Map( "expression" -> s"$SESSION_CATALOG_NAME.default.tab1.id", "sourceType" -> "\"DOUBLE\"", @@ -930,7 +930,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { df3.write.format("json").mode(SaveMode.Overwrite).saveAsTable("tab1") checkError( exception = intercept[AnalysisException](sql("SELECT * FROM testView")), - errorClass = "CANNOT_UP_CAST_DATATYPE", + condition = "CANNOT_UP_CAST_DATATYPE", parameters = Map( "expression" -> s"$SESSION_CATALOG_NAME.default.tab1.id1", "sourceType" -> "\"ARRAY\"", @@ -956,7 +956,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql("ALTER VIEW view1 AS SELECT * FROM view2") }, - errorClass = "RECURSIVE_VIEW", + condition = "RECURSIVE_VIEW", parameters = Map( "viewIdent" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`", "newPath" -> (s"`$SESSION_CATALOG_NAME`.`default`.`view1` -> " + @@ -970,7 +970,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql("ALTER VIEW view1 AS SELECT * FROM view3 JOIN view2") }, - errorClass = "RECURSIVE_VIEW", + condition = "RECURSIVE_VIEW", parameters = Map( "viewIdent" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`", "newPath" -> (s"`$SESSION_CATALOG_NAME`.`default`.`view1` -> " + @@ -985,7 +985,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql("CREATE OR REPLACE VIEW view1 AS SELECT * FROM view2") }, - errorClass = "RECURSIVE_VIEW", + condition = "RECURSIVE_VIEW", parameters = Map( "viewIdent" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`", "newPath" -> (s"`$SESSION_CATALOG_NAME`.`default`.`view1` -> " + @@ -999,7 +999,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql("ALTER VIEW view1 AS SELECT * FROM jt WHERE EXISTS (SELECT 1 FROM view2)") }, - errorClass = "RECURSIVE_VIEW", + condition = "RECURSIVE_VIEW", parameters = Map( "viewIdent" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`", "newPath" -> (s"`$SESSION_CATALOG_NAME`.`default`.`view1` -> " + @@ -1071,7 +1071,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[SparkException] { sql("SELECT * FROM v1").collect() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> ".*") ) } @@ -1091,7 +1091,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[SparkException] { sql("SELECT * FROM v1").collect() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> ".*") ) } @@ -1157,7 +1157,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { sql("SELECT * FROM v1") } checkError(e, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> "`C1`", @@ -1178,7 +1178,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { sql("SELECT * FROM v3") } checkError(e, - errorClass = "MISSING_AGGREGATION", + condition = "MISSING_AGGREGATION", parameters = Map( "expression" -> "\"c1\"", "expressionAnyValue" -> "\"any_value(c1)\"")) @@ -1188,7 +1188,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { sql("SELECT * FROM v4") } checkError(e, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map( "objectName" -> "`a`", @@ -1206,7 +1206,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[SparkArithmeticException] { sql("SELECT * FROM v5").collect() }, - errorClass = "DIVIDE_BY_ZERO", + condition = "DIVIDE_BY_ZERO", parameters = Map("config" -> "\"spark.sql.ansi.enabled\""), context = ExpectedContext( objectType = "VIEW", @@ -1225,7 +1225,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { exception = intercept[SparkArithmeticException] { sql("SELECT * FROM v1").collect() }, - errorClass = "DIVIDE_BY_ZERO", + condition = "DIVIDE_BY_ZERO", parameters = Map("config" -> "\"spark.sql.ansi.enabled\""), context = ExpectedContext( objectType = "VIEW", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala index aa6295fa8815f..0faace9227dd1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala @@ -182,7 +182,7 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { createView("v1", s"SELECT * FROM $viewName2", replace = true) }, - errorClass = "RECURSIVE_VIEW", + condition = "RECURSIVE_VIEW", parameters = Map( "viewIdent" -> tableIdentifier("v1").quotedString, "newPath" -> (s"${tableIdentifier("v1").quotedString} " + @@ -203,7 +203,7 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER VIEW $viewName1 AS SELECT * FROM $viewName2") }, - errorClass = "RECURSIVE_VIEW", + condition = "RECURSIVE_VIEW", parameters = Map( "viewIdent" -> tableIdentifier("v1").quotedString, "newPath" -> (s"${tableIdentifier("v1").quotedString} " + @@ -227,7 +227,7 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(s"SELECT * FROM ${viewNames.last}") }, - errorClass = "VIEW_EXCEED_MAX_NESTED_DEPTH", + condition = "VIEW_EXCEED_MAX_NESTED_DEPTH", parameters = Map( "viewName" -> tableIdentifier("view0").quotedString, "maxNestedDepth" -> "10"), @@ -363,7 +363,7 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { sql("CREATE TABLE t(s STRUCT) USING json") checkError( exception = intercept[AnalysisException](spark.table(viewName)), - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`i`", "fields" -> "`j`"), context = ExpectedContext( fragment = "s.i", @@ -399,7 +399,7 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { } else { checkErrorMatchPVals( exception = intercept[AnalysisException](spark.table(viewName).collect()), - errorClass = "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + condition = "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", parameters = Map( "viewName" -> ".*test[v|V]iew.*", "actualCols" -> "\\[COL,col,col\\]", @@ -436,7 +436,7 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException]( sql(s"SELECT * FROM $viewName VERSION AS OF 1").collect() ), - errorClass = "UNSUPPORTED_FEATURE.TIME_TRAVEL", + condition = "UNSUPPORTED_FEATURE.TIME_TRAVEL", parameters = Map("relationId" -> ".*test[v|V]iew.*") ) @@ -444,7 +444,7 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException]( sql(s"SELECT * FROM $viewName TIMESTAMP AS OF '2000-10-10'").collect() ), - errorClass = "UNSUPPORTED_FEATURE.TIME_TRAVEL", + condition = "UNSUPPORTED_FEATURE.TIME_TRAVEL", parameters = Map("relationId" -> ".*test[v|V]iew.*") ) } @@ -489,7 +489,7 @@ abstract class TempViewTestSuite extends SQLViewTestSuite { exception = intercept[AnalysisException] { sql(s"SHOW CREATE TABLE ${formattedViewName(viewName)}") }, - errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP", + condition = "EXPECT_PERMANENT_VIEW_NOT_TEMP", parameters = Map( "viewName" -> toSQLId(tableIdentifier(viewName).nameParts), "operation" -> "SHOW CREATE TABLE"), @@ -577,7 +577,7 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { exception = intercept[AnalysisException] { sql("CREATE VIEW v AS SELECT count(*) FROM VALUES (1), (2), (3) t(a)") }, - errorClass = "CREATE_PERMANENT_VIEW_WITHOUT_ALIAS", + condition = "CREATE_PERMANENT_VIEW_WITHOUT_ALIAS", parameters = Map("name" -> tableIdentifier("v").quotedString, "attr" -> "\"count(1)\"") ) sql("CREATE VIEW v AS SELECT count(*) AS cnt FROM VALUES (1), (2), (3) t(a)") @@ -591,7 +591,7 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { exception = intercept[AnalysisException] { sql("CREATE VIEW v AS SELECT * FROM (SELECT a + b FROM VALUES (1, 2) t(a, b))") }, - errorClass = "CREATE_PERMANENT_VIEW_WITHOUT_ALIAS", + condition = "CREATE_PERMANENT_VIEW_WITHOUT_ALIAS", parameters = Map("name" -> tableIdentifier("v").quotedString, "attr" -> "\"(a + b)\"") ) sql("CREATE VIEW v AS SELECT * FROM (SELECT a + b AS col FROM VALUES (1, 2) t(a, b))") @@ -606,7 +606,7 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { exception = intercept[AnalysisException] { sql("ALTER VIEW v AS SELECT count(*) FROM VALUES (1), (2), (3) t(a)") }, - errorClass = "CREATE_PERMANENT_VIEW_WITHOUT_ALIAS", + condition = "CREATE_PERMANENT_VIEW_WITHOUT_ALIAS", parameters = Map("name" -> tableIdentifier("v").quotedString, "attr" -> "\"count(1)\"") ) } @@ -639,7 +639,7 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { val unquotedViewName = tableIdentifier("test_view").unquotedString checkError( exception = e, - errorClass = "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + condition = "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", parameters = Map( "viewName" -> tableIdentifier("test_view").quotedString, "suggestion" -> s"CREATE OR REPLACE VIEW $unquotedViewName AS SELECT * FROM t", @@ -665,7 +665,7 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { exception = intercept[AnalysisException] { sql("ALTER VIEW v1 AS SELECT * FROM v2") }, - errorClass = "INVALID_TEMP_OBJ_REFERENCE", + condition = "INVALID_TEMP_OBJ_REFERENCE", parameters = Map( "obj" -> "VIEW", "objName" -> tableIdentifier("v1").quotedString, @@ -679,7 +679,7 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"ALTER VIEW v1 AS SELECT $tempFunctionName(id) from t") }, - errorClass = "INVALID_TEMP_OBJ_REFERENCE", + condition = "INVALID_TEMP_OBJ_REFERENCE", parameters = Map( "obj" -> "VIEW", "objName" -> tableIdentifier("v1").quotedString, @@ -724,7 +724,7 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { exception = intercept[AnalysisException] { sql("SELECT * FROM v") }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'DROP'", "hint" -> ""), context = ExpectedContext( objectType = "VIEW", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index 966f4e747122a..8dc07e2df99fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala @@ -112,7 +112,7 @@ class SparkPlanSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] { planner.plan(deduplicate) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map( "message" -> ("Deduplicate operator for non streaming data source should have been " + "replaced by aggregate in the optimizer"))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkScriptTransformationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkScriptTransformationSuite.scala index 52378f7370930..dbb8e9697089e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkScriptTransformationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkScriptTransformationSuite.scala @@ -54,7 +54,7 @@ class SparkScriptTransformationSuite extends BaseScriptTransformationSuite with |FROM v""".stripMargin checkError( exception = intercept[ParseException](sql(sqlText)), - errorClass = "UNSUPPORTED_FEATURE.TRANSFORM_NON_HIVE", + condition = "UNSUPPORTED_FEATURE.TRANSFORM_NON_HIVE", parameters = Map.empty, context = ExpectedContext(sqlText, 0, 185)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index f60df77b7e9bd..decfb5555dd87 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -83,12 +83,12 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { checkError( exception = parseException("SET k=`v` /*"), - errorClass = "UNCLOSED_BRACKETED_COMMENT", + condition = "UNCLOSED_BRACKETED_COMMENT", parameters = Map.empty) checkError( exception = parseException("SET `k`=`v` /*"), - errorClass = "UNCLOSED_BRACKETED_COMMENT", + condition = "UNCLOSED_BRACKETED_COMMENT", parameters = Map.empty) } @@ -120,7 +120,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "SET spark.sql.key value" checkError( exception = parseException(sql1), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql1, @@ -130,7 +130,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "SET spark.sql.key 'value'" checkError( exception = parseException(sql2), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql2, @@ -140,7 +140,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql3 = "SET spark.sql.key \"value\" " checkError( exception = parseException(sql3), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = "SET spark.sql.key \"value\"", @@ -150,7 +150,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql4 = "SET spark.sql.key value1 value2" checkError( exception = parseException(sql4), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql4, @@ -160,7 +160,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql5 = "SET spark. sql.key=value" checkError( exception = parseException(sql5), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql5, @@ -170,7 +170,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql6 = "SET spark :sql:key=value" checkError( exception = parseException(sql6), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql6, @@ -180,7 +180,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql7 = "SET spark . sql.key=value" checkError( exception = parseException(sql7), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql7, @@ -190,7 +190,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql8 = "SET spark.sql. key=value" checkError( exception = parseException(sql8), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql8, @@ -200,7 +200,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql9 = "SET spark.sql :key=value" checkError( exception = parseException(sql9), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql9, @@ -210,7 +210,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql10 = "SET spark.sql . key=value" checkError( exception = parseException(sql10), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql10, @@ -220,7 +220,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql11 = "SET =" checkError( exception = parseException(sql11), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql11, @@ -230,7 +230,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql12 = "SET =value" checkError( exception = parseException(sql12), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql12, @@ -251,7 +251,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "RESET spark.sql.key1 key2" checkError( exception = parseException(sql1), - errorClass = "_LEGACY_ERROR_TEMP_0043", + condition = "_LEGACY_ERROR_TEMP_0043", parameters = Map.empty, context = ExpectedContext( fragment = sql1, @@ -261,7 +261,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "RESET spark. sql.key1 key2" checkError( exception = parseException(sql2), - errorClass = "_LEGACY_ERROR_TEMP_0043", + condition = "_LEGACY_ERROR_TEMP_0043", parameters = Map.empty, context = ExpectedContext( fragment = sql2, @@ -271,7 +271,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql3 = "RESET spark.sql.key1 key2 key3" checkError( exception = parseException(sql3), - errorClass = "_LEGACY_ERROR_TEMP_0043", + condition = "_LEGACY_ERROR_TEMP_0043", parameters = Map.empty, context = ExpectedContext( fragment = sql3, @@ -281,7 +281,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql4 = "RESET spark: sql:key" checkError( exception = parseException(sql4), - errorClass = "_LEGACY_ERROR_TEMP_0043", + condition = "_LEGACY_ERROR_TEMP_0043", parameters = Map.empty, context = ExpectedContext( fragment = sql4, @@ -291,7 +291,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql5 = "RESET spark .sql.key" checkError( exception = parseException(sql5), - errorClass = "_LEGACY_ERROR_TEMP_0043", + condition = "_LEGACY_ERROR_TEMP_0043", parameters = Map.empty, context = ExpectedContext( fragment = sql5, @@ -301,7 +301,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql6 = "RESET spark : sql:key" checkError( exception = parseException(sql6), - errorClass = "_LEGACY_ERROR_TEMP_0043", + condition = "_LEGACY_ERROR_TEMP_0043", parameters = Map.empty, context = ExpectedContext( fragment = sql6, @@ -311,7 +311,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql7 = "RESET spark.sql: key" checkError( exception = parseException(sql7), - errorClass = "_LEGACY_ERROR_TEMP_0043", + condition = "_LEGACY_ERROR_TEMP_0043", parameters = Map.empty, context = ExpectedContext( fragment = sql7, @@ -321,7 +321,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql8 = "RESET spark.sql .key" checkError( exception = parseException(sql8), - errorClass = "_LEGACY_ERROR_TEMP_0043", + condition = "_LEGACY_ERROR_TEMP_0043", parameters = Map.empty, context = ExpectedContext( fragment = sql8, @@ -331,7 +331,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql9 = "RESET spark.sql : key" checkError( exception = parseException(sql9), - errorClass = "_LEGACY_ERROR_TEMP_0043", + condition = "_LEGACY_ERROR_TEMP_0043", parameters = Map.empty, context = ExpectedContext( fragment = sql9, @@ -354,7 +354,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "SET a=1; SELECT 1" checkError( exception = parseException(sql1), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql1, @@ -364,7 +364,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "SET a=1;2;;" checkError( exception = parseException(sql2), - errorClass = "INVALID_SET_SYNTAX", + condition = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = "SET a=1;2", @@ -374,7 +374,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql3 = "SET a b=`1;;`" checkError( exception = parseException(sql3), - errorClass = "INVALID_PROPERTY_KEY", + condition = "INVALID_PROPERTY_KEY", parameters = Map("key" -> "\"a b\"", "value" -> "\"1;;\""), context = ExpectedContext( fragment = sql3, @@ -384,7 +384,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql4 = "SET `a`=1;2;;" checkError( exception = parseException(sql4), - errorClass = "INVALID_PROPERTY_VALUE", + condition = "INVALID_PROPERTY_VALUE", parameters = Map("value" -> "\"1;2;;\"", "key" -> "\"a\""), context = ExpectedContext( fragment = "SET `a`=1;2", @@ -407,7 +407,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "REFRESH a b" checkError( exception = parseException(sql1), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> errMsg1), context = ExpectedContext( fragment = sql1, @@ -417,7 +417,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "REFRESH a\tb" checkError( exception = parseException(sql2), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> errMsg1), context = ExpectedContext( fragment = sql2, @@ -427,7 +427,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql3 = "REFRESH a\nb" checkError( exception = parseException(sql3), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> errMsg1), context = ExpectedContext( fragment = sql3, @@ -437,7 +437,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql4 = "REFRESH a\rb" checkError( exception = parseException(sql4), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> errMsg1), context = ExpectedContext( fragment = sql4, @@ -447,7 +447,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql5 = "REFRESH a\r\nb" checkError( exception = parseException(sql5), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> errMsg1), context = ExpectedContext( fragment = sql5, @@ -457,7 +457,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql6 = "REFRESH @ $a$" checkError( exception = parseException(sql6), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> errMsg1), context = ExpectedContext( fragment = sql6, @@ -468,7 +468,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql7 = "REFRESH " checkError( exception = parseException(sql7), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> errMsg2), context = ExpectedContext( fragment = "REFRESH", @@ -478,7 +478,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql8 = "REFRESH" checkError( exception = parseException(sql8), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> errMsg2), context = ExpectedContext( fragment = sql8, @@ -741,7 +741,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { |FROM v""".stripMargin checkError( exception = parseException(sql1), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> errMsg), context = ExpectedContext( fragment = sql1, @@ -763,7 +763,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { |FROM v""".stripMargin checkError( exception = parseException(sql2), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> errMsg), context = ExpectedContext( fragment = sql2, @@ -780,7 +780,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { s"CREATE TABLE target LIKE source TBLPROPERTIES (${TableCatalog.PROP_OWNER}='howdy')" checkError( exception = parseException(sql1), - errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", parameters = Map("property" -> TableCatalog.PROP_OWNER, "msg" -> "it will be set to the current user"), context = ExpectedContext( @@ -792,7 +792,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { s"CREATE TABLE target LIKE source TBLPROPERTIES (${TableCatalog.PROP_PROVIDER}='howdy')" checkError( exception = parseException(sql2), - errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", parameters = Map("property" -> TableCatalog.PROP_PROVIDER, "msg" -> "please use the USING clause to specify it"), context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 4d2d465828924..a3cfdc5a240a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -878,7 +878,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession exception = intercept[SparkException] { sql(query).collect() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> expectedErrMsg), matchPVals = true) } @@ -903,7 +903,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession exception = intercept[SparkException] { sql(query).collect() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> expectedErrMsg), matchPVals = true) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 938a96a86b015..75f016d050de9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -1082,7 +1082,7 @@ class AdaptiveQueryExecSuite val doExecute = PrivateMethod[Unit](Symbol("doExecute")) c.invokePrivate(doExecute()) }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "operating on canonicalized plan")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index a95bda9bf71df..cb97066098f20 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -186,7 +186,7 @@ class ColumnTypeSuite extends SparkFunSuite { exception = intercept[SparkUnsupportedOperationException] { ColumnType(invalidType) }, - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> "\"INVALID TYPE NAME\"") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignMergeAssignmentsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignMergeAssignmentsSuite.scala index 488b4d31bd923..cd099a2a94813 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignMergeAssignmentsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignMergeAssignmentsSuite.scala @@ -591,7 +591,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { if (policy == StoreAssignmentPolicy.ANSI) { checkError( exception = e, - errorClass = "DATATYPE_MISMATCH.INVALID_ROW_LEVEL_OPERATION_ASSIGNMENTS", + condition = "DATATYPE_MISMATCH.INVALID_ROW_LEVEL_OPERATION_ASSIGNMENTS", parameters = Map( "sqlExpr" -> "\"s.n_i = 1\", \"s.n_s = NULL\", \"s.n_i = -1\"", "errors" -> "\n- Multiple assignments for 's.n_i': 1, -1") @@ -599,7 +599,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } else { checkError( exception = e, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`s`.`n_s`", @@ -701,7 +701,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } checkError( exception = e, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map("tableName" -> "``", "colName" -> "`s`.`n_s`.`dn_l`") ) @@ -847,7 +847,7 @@ class AlignMergeAssignmentsSuite extends AlignAssignmentsSuiteBase { } checkError( exception = e, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map("tableName" -> "``", "colName" -> "`s`.`n_s`.`dn_l`") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignUpdateAssignmentsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignUpdateAssignmentsSuite.scala index 599f3e994ef8a..3c8ce44f8167b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignUpdateAssignmentsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlignUpdateAssignmentsSuite.scala @@ -478,7 +478,7 @@ class AlignUpdateAssignmentsSuite extends AlignAssignmentsSuiteBase { if (policy == StoreAssignmentPolicy.ANSI) { checkError( exception = e, - errorClass = "DATATYPE_MISMATCH.INVALID_ROW_LEVEL_OPERATION_ASSIGNMENTS", + condition = "DATATYPE_MISMATCH.INVALID_ROW_LEVEL_OPERATION_ASSIGNMENTS", parameters = Map( "sqlExpr" -> "\"s.n_i = 1\", \"s.n_s = NULL\", \"s.n_i = -1\"", "errors" -> "\n- Multiple assignments for 's.n_i': 1, -1") @@ -486,7 +486,7 @@ class AlignUpdateAssignmentsSuite extends AlignAssignmentsSuiteBase { } else { checkError( exception = e, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "``", "colName" -> "`s`.`n_s`", @@ -538,7 +538,7 @@ class AlignUpdateAssignmentsSuite extends AlignAssignmentsSuiteBase { } checkError( exception = e, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map("tableName" -> "``", "colName" -> "`s`.`n_s`.`dn_l`") ) @@ -591,7 +591,7 @@ class AlignUpdateAssignmentsSuite extends AlignAssignmentsSuiteBase { } checkError( exception = e, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map("tableName" -> "``", "colName" -> "`s`.`n_s`.`dn_l`") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala index 2d4277e5499e8..64491f9ad9741 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala @@ -51,7 +51,7 @@ trait AlterNamespaceSetLocationSuiteBase extends QueryTest with DDLCommandTestUt exception = intercept[SparkIllegalArgumentException] { sql(sqlText) }, - errorClass = "INVALID_EMPTY_LOCATION", + condition = "INVALID_EMPTY_LOCATION", parameters = Map("location" -> "")) } } @@ -66,7 +66,7 @@ trait AlterNamespaceSetLocationSuiteBase extends QueryTest with DDLCommandTestUt } checkError( exception = e, - errorClass = "INVALID_LOCATION", + condition = "INVALID_LOCATION", parameters = Map("location" -> "file:tmp")) } } @@ -77,7 +77,7 @@ trait AlterNamespaceSetLocationSuiteBase extends QueryTest with DDLCommandTestUt sql(s"ALTER DATABASE $catalog.$ns SET LOCATION 'loc'") } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> s"`$catalog`.`$ns`")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesParserSuite.scala index 9d70ceeef578e..70abfe8af5266 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesParserSuite.scala @@ -43,7 +43,7 @@ class AlterNamespaceSetPropertiesParserSuite extends AnalysisTest { val sql = "ALTER NAMESPACE my_db SET PROPERTIES('key_without_value', 'key_with_value'='x')" checkError( exception = parseException(parsePlan)(sql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "Values must be specified for key(s): [key_without_value]"), context = ExpectedContext( fragment = sql, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala index d2f2d75d86ce9..3b0ac1d408234 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala @@ -49,7 +49,7 @@ trait AlterNamespaceSetPropertiesSuiteBase extends QueryTest with DDLCommandTest sql(s"ALTER DATABASE $catalog.$ns SET PROPERTIES ('d'='d')") } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> s"`$catalog`.`$ns`")) } @@ -88,7 +88,7 @@ trait AlterNamespaceSetPropertiesSuiteBase extends QueryTest with DDLCommandTest exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", parameters = Map("property" -> key, "msg" -> ".*"), sqlState = None, context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala index 72d307c816664..11e0f6c29bef5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala @@ -54,7 +54,7 @@ class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest with SharedS val sql = "ALTER NAMESPACE my_db UNSET PROPERTIES('key_without_value', 'key_with_value'='x')" checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "Values should not be specified for key(s): [key_with_value]"), context = ExpectedContext( fragment = sql, @@ -68,7 +68,7 @@ class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest with SharedS val sql = s"ALTER $nsToken a.b.c UNSET $propToken IF EXISTS ('a', 'b', 'c')" checkError( exception = parseException(sql), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'IF'", "hint" -> ": missing '('") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala index c00f3f99f41f9..42550ef844361 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala @@ -56,7 +56,7 @@ trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTe sql(s"ALTER NAMESPACE $catalog.$ns UNSET PROPERTIES ('d')") } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> s"`$catalog`.`$ns`")) } @@ -90,7 +90,7 @@ trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTe exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", parameters = Map("property" -> key, "msg" -> ".*"), sqlState = None, context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala index 3feb4f13c73f2..8c338f71e8a30 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala @@ -234,7 +234,7 @@ trait AlterTableAddPartitionSuiteBase extends QueryTest with DDLCommandTestUtils exception = intercept[SparkNumberFormatException] { sql(s"ALTER TABLE $t ADD PARTITION (p='aaa')") }, - errorClass = "CAST_INVALID_INPUT", + condition = "CAST_INVALID_INPUT", parameters = Map( "ansiConfig" -> "\"spark.sql.ansi.enabled\"", "expression" -> "'aaa'", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableClusterBySuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableClusterBySuiteBase.scala index 73a80cd910698..c0fd0a67d06aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableClusterBySuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableClusterBySuiteBase.scala @@ -83,7 +83,7 @@ trait AlterTableClusterBySuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $tbl CLUSTER BY (unknown)") }, - errorClass = "_LEGACY_ERROR_TEMP_3060", + condition = "_LEGACY_ERROR_TEMP_3060", parameters = Map("i" -> "unknown", "schema" -> """root diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala index 9b4b026480a16..2aa77dac711d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala @@ -90,7 +90,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe val sql = "ALTER VIEW table_name DROP PARTITION (p=1)" checkError( exception = parseException(parsePlan)(sql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER VIEW ... DROP PARTITION"), context = ExpectedContext( fragment = sql, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala index ef9ae47847405..279042f675cd5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala @@ -146,7 +146,7 @@ trait AlterTableDropPartitionSuiteBase extends QueryTest with DDLCommandTestUtil "`test_catalog`.`ns`.`tbl`" } checkError(e, - errorClass = "PARTITIONS_NOT_FOUND", + condition = "PARTITIONS_NOT_FOUND", parameters = Map("partitionList" -> "PARTITION (`id` = 2)", "tableName" -> expectedTableName)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala index 936b1a3dfdb20..babf490729564 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala @@ -28,7 +28,7 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa val sql = "ALTER TABLE RECOVER PARTITIONS" checkError( exception = parseException(parsePlan)(sql), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'PARTITIONS'", "hint" -> "")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnParserSuite.scala index 62ee8aa57a760..1df4800fa7542 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnParserSuite.scala @@ -37,12 +37,12 @@ class AlterTableRenameColumnParserSuite extends AnalysisTest with SharedSparkSes checkError( exception = parseException(parsePlan)( "ALTER TABLE t RENAME COLUMN test-col TO test"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-col")) checkError( exception = parseException(parsePlan)( "ALTER TABLE t RENAME COLUMN test TO test-col"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-col")) } @@ -50,7 +50,7 @@ class AlterTableRenameColumnParserSuite extends AnalysisTest with SharedSparkSes checkError( exception = parseException(parsePlan)( "ALTER TABLE t RENAME COLUMN point.x to point.y"), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'.'", "hint" -> "")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameParserSuite.scala index 098750c929ecd..83d590e2bb35c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameParserSuite.scala @@ -44,13 +44,13 @@ class AlterTableRenameParserSuite extends AnalysisTest { val sql1 = "ALTER TABLE RENAME TO x.y.z" checkError( exception = parseException(parsePlan)(sql1), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'TO'", "hint" -> "")) val sql2 = "ALTER TABLE _ RENAME TO .z" checkError( exception = parseException(parsePlan)(sql2), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'.'", "hint" -> "")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala index d91085956e330..905e6cfb9caaa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala @@ -81,7 +81,7 @@ trait AlterTableRenamePartitionSuiteBase extends QueryTest with DDLCommandTestUt sql(s"ALTER TABLE $t PARTITION (id = 3) RENAME TO PARTITION (id = 2)") } checkError(e, - errorClass = "PARTITIONS_NOT_FOUND", + condition = "PARTITIONS_NOT_FOUND", parameters = Map("partitionList" -> "PARTITION (`id` = 3)", "tableName" -> parsed)) } @@ -103,7 +103,7 @@ trait AlterTableRenamePartitionSuiteBase extends QueryTest with DDLCommandTestUt sql(s"ALTER TABLE $t PARTITION (id = 1) RENAME TO PARTITION (id = 2)") } checkError(e, - errorClass = "PARTITIONS_ALREADY_EXIST", + condition = "PARTITIONS_ALREADY_EXIST", parameters = Map("partitionList" -> "PARTITION (`id` = 2)", "tableName" -> parsed)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetSerdeParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetSerdeParserSuite.scala index 8a3bfd47c6ea3..dcd3ad5681b06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetSerdeParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetSerdeParserSuite.scala @@ -29,7 +29,7 @@ class AlterTableSetSerdeParserSuite extends AnalysisTest with SharedSparkSession "WITH SERDEPROPERTIES('key_without_value', 'key_with_value'='x')" checkError( exception = parseException(parsePlan)(sql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "Values must be specified for key(s): [key_without_value]"), context = ExpectedContext( fragment = sql, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesParserSuite.scala index 78abd1a8b7fd3..6b2c7069c4211 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesParserSuite.scala @@ -45,7 +45,7 @@ class AlterTableSetTblPropertiesParserSuite extends AnalysisTest with SharedSpar val sql = "ALTER TABLE my_tab SET TBLPROPERTIES('key_without_value', 'key_with_value'='x')" checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "Values must be specified for key(s): [key_without_value]"), context = ExpectedContext( fragment = sql, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala index ac3c84dff718c..52a90497fdd37 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala @@ -52,7 +52,7 @@ trait AlterTableSetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestU exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> toSQLId(t)), context = ExpectedContext( fragment = t, @@ -96,7 +96,7 @@ trait AlterTableSetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestU exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", parameters = Map( "property" -> key, "msg" -> keyParameters.getOrElse( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesParserSuite.scala index 1e675a64f2235..c9582a75aa8cd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesParserSuite.scala @@ -55,7 +55,7 @@ class AlterTableUnsetTblPropertiesParserSuite extends AnalysisTest with SharedSp val sql = "ALTER TABLE my_tab UNSET TBLPROPERTIES('key_without_value', 'key_with_value'='x')" checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "Values should not be specified for key(s): [key_with_value]"), context = ExpectedContext( fragment = sql, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala index be8d85d2ef670..0013919fca08f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala @@ -52,7 +52,7 @@ trait AlterTableUnsetTblPropertiesSuiteBase extends QueryTest with DDLCommandTes exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> toSQLId(t)), context = ExpectedContext( fragment = t, @@ -116,7 +116,7 @@ trait AlterTableUnsetTblPropertiesSuiteBase extends QueryTest with DDLCommandTes exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", + condition = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", parameters = Map( "property" -> key, "msg" -> keyParameters.getOrElse( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala index 9c7f370278128..a5bb3058bedd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala @@ -53,7 +53,7 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(alterSQL) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"CHAR(4)\"", "newType" -> "\"CHAR(5)\"", @@ -74,7 +74,7 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"STRING\"", "newType" -> "\"CHAR(5)\"", @@ -95,7 +95,7 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"INT\"", "newType" -> "\"CHAR(5)\"", @@ -124,7 +124,7 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"VARCHAR(4)\"", "newType" -> "\"VARCHAR(3)\"", @@ -301,7 +301,7 @@ class DSV2CharVarcharDDLTestSuite extends CharVarcharDDLTestBase exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"CHAR(4)\"", "newType" -> "\"VARCHAR(3)\"", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceParserSuite.scala index 46ccc0b1312da..469e1a06920a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceParserSuite.scala @@ -70,7 +70,7 @@ class CreateNamespaceParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = createNamespace("COMMENT 'namespace_comment'") checkError( exception = parseException(sql1), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "COMMENT"), context = ExpectedContext( fragment = sql1, @@ -80,7 +80,7 @@ class CreateNamespaceParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = createNamespace("LOCATION '/home/user/db'") checkError( exception = parseException(sql2), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "LOCATION"), context = ExpectedContext( fragment = sql2, @@ -90,7 +90,7 @@ class CreateNamespaceParserSuite extends AnalysisTest with SharedSparkSession { val sql3 = createNamespace("WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c')") checkError( exception = parseException(sql3), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "WITH PROPERTIES"), context = ExpectedContext( fragment = sql3, @@ -100,7 +100,7 @@ class CreateNamespaceParserSuite extends AnalysisTest with SharedSparkSession { val sql4 = createNamespace("WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") checkError( exception = parseException(sql4), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "WITH DBPROPERTIES"), context = ExpectedContext( fragment = sql4, @@ -112,7 +112,7 @@ class CreateNamespaceParserSuite extends AnalysisTest with SharedSparkSession { val sql = "CREATE NAMESPACE a.b.c WITH PROPERTIES('key_without_value', 'key_with_value'='x')" checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "Values must be specified for key(s): [key_without_value]"), context = ExpectedContext( fragment = sql, @@ -127,7 +127,7 @@ class CreateNamespaceParserSuite extends AnalysisTest with SharedSparkSession { |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')""".stripMargin checkError( exception = parseException(sql), - errorClass = "UNSUPPORTED_FEATURE.SET_PROPERTIES_AND_DBPROPERTIES", + condition = "UNSUPPORTED_FEATURE.SET_PROPERTIES_AND_DBPROPERTIES", parameters = Map.empty, context = ExpectedContext( fragment = sql, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala index bfc32a761d57c..9733b104beecb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala @@ -74,7 +74,7 @@ trait CreateNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[SparkIllegalArgumentException] { sql(sqlText) }, - errorClass = "INVALID_EMPTY_LOCATION", + condition = "INVALID_EMPTY_LOCATION", parameters = Map("location" -> "")) val uri = new Path(path).toUri sql(s"CREATE NAMESPACE $ns LOCATION '$uri'") @@ -99,7 +99,7 @@ trait CreateNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"CREATE NAMESPACE $ns") } checkError(e, - errorClass = "SCHEMA_ALREADY_EXISTS", + condition = "SCHEMA_ALREADY_EXISTS", parameters = Map("schemaName" -> parsed)) // The following will be no-op since the namespace already exists. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 70276051defa9..d1a6021d829f9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -64,7 +64,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val v2 = "INSERT OVERWRITE DIRECTORY USING parquet SELECT 1 as a" checkError( exception = parseException(v2), - errorClass = "_LEGACY_ERROR_TEMP_0049", + condition = "_LEGACY_ERROR_TEMP_0049", parameters = Map.empty, context = ExpectedContext( fragment = "INSERT OVERWRITE DIRECTORY USING parquet", @@ -99,7 +99,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { | OPTIONS ('path' '/tmp/file', a 1, b 0.1, c TRUE)""".stripMargin checkError( exception = parseException(v4), - errorClass = "_LEGACY_ERROR_TEMP_0049", + condition = "_LEGACY_ERROR_TEMP_0049", parameters = Map.empty, context = ExpectedContext( fragment = fragment4, @@ -113,7 +113,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { |(dt='2008-08-08', country='us') WITH TABLE table_name_2""".stripMargin checkError( exception = parseException(sql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE EXCHANGE PARTITION"), context = ExpectedContext( fragment = sql, @@ -125,7 +125,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql = "ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')" checkError( exception = parseException(sql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE ARCHIVE PARTITION"), context = ExpectedContext( fragment = sql, @@ -137,7 +137,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql = "ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')" checkError( exception = parseException(sql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE UNARCHIVE PARTITION"), context = ExpectedContext( fragment = sql, @@ -149,7 +149,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' OUTPUTFORMAT 'test'" checkError( exception = parseException(sql1), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE SET FILEFORMAT"), context = ExpectedContext( fragment = sql1, @@ -160,7 +160,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { "SET FILEFORMAT PARQUET" checkError( exception = parseException(sql2), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE SET FILEFORMAT"), context = ExpectedContext( fragment = sql2, @@ -172,7 +172,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "ALTER TABLE table_name TOUCH" checkError( exception = parseException(sql1), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE TOUCH"), context = ExpectedContext( fragment = sql1, @@ -182,7 +182,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')" checkError( exception = parseException(sql2), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE TOUCH"), context = ExpectedContext( fragment = sql2, @@ -194,7 +194,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "ALTER TABLE table_name COMPACT 'compaction_type'" checkError( exception = parseException(sql1), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE COMPACT"), context = ExpectedContext( fragment = sql1, @@ -206,7 +206,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { |COMPACT 'MAJOR'""".stripMargin checkError( exception = parseException(sql2), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE COMPACT"), context = ExpectedContext( fragment = sql2, @@ -218,7 +218,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "ALTER TABLE table_name CONCATENATE" checkError( exception = parseException(sql1), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE CONCATENATE"), context = ExpectedContext( fragment = sql1, @@ -228,7 +228,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') CONCATENATE" checkError( exception = parseException(sql2), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE CONCATENATE"), context = ExpectedContext( fragment = sql2, @@ -240,7 +240,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "ALTER TABLE table_name CLUSTERED BY (col_name) SORTED BY (col2_name) INTO 3 BUCKETS" checkError( exception = parseException(sql1), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE CLUSTERED BY"), context = ExpectedContext( fragment = sql1, @@ -250,7 +250,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "ALTER TABLE table_name CLUSTERED BY (col_name) INTO 3 BUCKETS" checkError( exception = parseException(sql2), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE CLUSTERED BY"), context = ExpectedContext( fragment = sql2, @@ -260,7 +260,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql3 = "ALTER TABLE table_name NOT CLUSTERED" checkError( exception = parseException(sql3), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE NOT CLUSTERED"), context = ExpectedContext( fragment = sql3, @@ -270,7 +270,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql4 = "ALTER TABLE table_name NOT SORTED" checkError( exception = parseException(sql4), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE NOT SORTED"), context = ExpectedContext( fragment = sql4, @@ -282,7 +282,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "ALTER TABLE table_name NOT SKEWED" checkError( exception = parseException(sql1), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE NOT SKEWED"), context = ExpectedContext( fragment = sql1, @@ -292,7 +292,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "ALTER TABLE table_name NOT STORED AS DIRECTORIES" checkError( exception = parseException(sql2), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE NOT STORED AS DIRECTORIES"), context = ExpectedContext( fragment = sql2, @@ -302,7 +302,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql3 = "ALTER TABLE table_name SET SKEWED LOCATION (col_name1=\"location1\"" checkError( exception = parseException(sql3), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE SET SKEWED LOCATION"), context = ExpectedContext( fragment = sql3, @@ -312,7 +312,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql4 = "ALTER TABLE table_name SKEWED BY (key) ON (1,5,6) STORED AS DIRECTORIES" checkError( exception = parseException(sql4), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE SKEWED BY"), context = ExpectedContext( fragment = sql4, @@ -326,7 +326,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT""".stripMargin checkError( exception = parseException(sql), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE REPLACE COLUMNS"), context = ExpectedContext( fragment = sql, @@ -351,7 +351,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { checkError( exception = parseException(sql1), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map( "message" -> "CREATE TEMPORARY TABLE ... AS ..., use CREATE TEMPORARY VIEW instead"), context = ExpectedContext( @@ -365,7 +365,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { |AS SELECT key, value FROM src ORDER BY key, value""".stripMargin checkError( exception = parseException(sql2), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map( "message" -> "Schema may not be specified in a Create Table As Select (CTAS) statement"), context = ExpectedContext( @@ -379,7 +379,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { |AS SELECT key, value FROM src ORDER BY key, value""".stripMargin checkError( exception = parseException(sql3), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "CREATE TABLE ... SKEWED BY"), context = ExpectedContext( fragment = sql3, @@ -392,7 +392,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { |FROM testData""".stripMargin checkError( exception = parseException(sql4), - errorClass = "_LEGACY_ERROR_TEMP_0048", + condition = "_LEGACY_ERROR_TEMP_0048", parameters = Map.empty, context = ExpectedContext( fragment = sql4, @@ -407,7 +407,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val fragment1 = "'42-32' year to month" checkError( exception = parseException(sql1), - errorClass = "_LEGACY_ERROR_TEMP_0063", + condition = "_LEGACY_ERROR_TEMP_0063", parameters = Map("msg" -> value1), context = ExpectedContext( fragment = fragment1, @@ -418,7 +418,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val fragment2 = "'5 49:12:15' day to second" checkError( exception = parseException(sql2), - errorClass = "_LEGACY_ERROR_TEMP_0063", + condition = "_LEGACY_ERROR_TEMP_0063", parameters = Map("msg" -> "requirement failed: hour 49 outside range [0, 23]"), context = ExpectedContext( fragment = fragment2, @@ -429,7 +429,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val fragment3 = "'23:61:15' hour to second" checkError( exception = parseException(sql3), - errorClass = "_LEGACY_ERROR_TEMP_0063", + condition = "_LEGACY_ERROR_TEMP_0063", parameters = Map("msg" -> "requirement failed: minute 61 outside range [0, 59]"), context = ExpectedContext( fragment = fragment3, @@ -524,7 +524,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val v3 = "CREATE TEMPORARY VIEW a.b AS SELECT 1" checkError( exception = parseException(v3), - errorClass = "TEMP_VIEW_NAME_TOO_MANY_NAME_PARTS", + condition = "TEMP_VIEW_NAME_TOO_MANY_NAME_PARTS", parameters = Map("actualName" -> "`a`.`b`"), context = ExpectedContext( fragment = v3, @@ -579,7 +579,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val v1 = "CREATE VIEW view1 partitioned on (ds, hr) as select * from srcpart" checkError( exception = parseException(v1), - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "CREATE VIEW ... PARTITIONED ON"), context = ExpectedContext( fragment = v1, @@ -599,7 +599,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = createViewStatement("COMMENT 'BLABLA'") checkError( exception = parseException(sql1), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "COMMENT"), context = ExpectedContext( fragment = sql1, @@ -609,7 +609,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = createViewStatement("TBLPROPERTIES('prop1Key'=\"prop1Val\")") checkError( exception = parseException(sql2), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "TBLPROPERTIES"), context = ExpectedContext( fragment = sql2, @@ -655,7 +655,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql = "CREATE FUNCTION a as 'fun' USING OTHER 'o'" checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "CREATE FUNCTION with resource type 'other'"), context = ExpectedContext( fragment = sql, @@ -687,7 +687,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "DROP TEMPORARY FUNCTION a.b" checkError( exception = parseException(sql1), - errorClass = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + condition = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", parameters = Map("statement" -> "DROP TEMPORARY FUNCTION", "funcName" -> "`a`.`b`"), context = ExpectedContext( fragment = sql1, @@ -697,7 +697,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "DROP TEMPORARY FUNCTION IF EXISTS a.b" checkError( exception = parseException(sql2), - errorClass = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + condition = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", parameters = Map("statement" -> "DROP TEMPORARY FUNCTION", "funcName" -> "`a`.`b`"), context = ExpectedContext( fragment = sql2, @@ -713,7 +713,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { |AS SELECT * FROM tab1""".stripMargin checkError( exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW"), context = ExpectedContext( fragment = sql, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 5c1090c288ed5..6e58b0e62ed63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -88,7 +88,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"CREATE TABLE $tabName (i INT, j STRING) STORED AS parquet") }, - errorClass = "NOT_SUPPORTED_COMMAND_WITHOUT_HIVE_SUPPORT", + condition = "NOT_SUPPORTED_COMMAND_WITHOUT_HIVE_SUPPORT", parameters = Map("cmd" -> "CREATE Hive TABLE (AS SELECT)") ) } @@ -108,7 +108,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { |LOCATION '${tempDir.toURI}' """.stripMargin) }, - errorClass = "NOT_SUPPORTED_COMMAND_WITHOUT_HIVE_SUPPORT", + condition = "NOT_SUPPORTED_COMMAND_WITHOUT_HIVE_SUPPORT", parameters = Map("cmd" -> "CREATE Hive TABLE (AS SELECT)") ) } @@ -122,7 +122,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { exception = intercept[AnalysisException] { sql("CREATE TABLE t STORED AS parquet SELECT 1 as a, 1 as b") }, - errorClass = "NOT_SUPPORTED_COMMAND_WITHOUT_HIVE_SUPPORT", + condition = "NOT_SUPPORTED_COMMAND_WITHOUT_HIVE_SUPPORT", parameters = Map("cmd" -> "CREATE Hive TABLE (AS SELECT)") ) @@ -131,7 +131,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { exception = intercept[AnalysisException] { sql("CREATE TABLE t STORED AS parquet SELECT a, b from t1") }, - errorClass = "NOT_SUPPORTED_COMMAND_WITHOUT_HIVE_SUPPORT", + condition = "NOT_SUPPORTED_COMMAND_WITHOUT_HIVE_SUPPORT", parameters = Map("cmd" -> "CREATE Hive TABLE (AS SELECT)") ) } @@ -195,7 +195,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { exception = intercept[AnalysisException] { sql("CREATE TABLE t LIKE s USING org.apache.spark.sql.hive.orc") }, - errorClass = "_LEGACY_ERROR_TEMP_1138", + condition = "_LEGACY_ERROR_TEMP_1138", parameters = Map.empty ) } @@ -209,7 +209,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { } checkError( exception = e, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", sqlState = "0A000", parameters = Map("tableName" -> "`spark_catalog`.`default`.`t`", "operation" -> "ALTER COLUMN ... FIRST | AFTER")) @@ -379,7 +379,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[SparkRuntimeException] { sql(createStmt) }, - errorClass = "LOCATION_ALREADY_EXISTS", + condition = "LOCATION_ALREADY_EXISTS", parameters = Map( "location" -> expectedLoc, "identifier" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`")) @@ -392,7 +392,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[SparkRuntimeException] { sql(s"CREATE TABLE IF NOT EXISTS tab1 LIKE tab2") }, - errorClass = "LOCATION_ALREADY_EXISTS", + condition = "LOCATION_ALREADY_EXISTS", parameters = Map( "location" -> expectedLoc, "identifier" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`")) @@ -425,7 +425,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { if (userSpecifiedSchema.isEmpty && userSpecifiedPartitionCols.nonEmpty) { checkError( exception = intercept[AnalysisException](sql(sqlCreateTable)), - errorClass = "SPECIFY_PARTITION_IS_NOT_ALLOWED", + condition = "SPECIFY_PARTITION_IS_NOT_ALLOWED", parameters = Map.empty ) } else { @@ -529,7 +529,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql(s"CREATE TABLE t($c0 INT, $c1 INT) USING parquet") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c1.toLowerCase(Locale.ROOT)}`")) } } @@ -540,7 +540,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int, b string) USING json PARTITIONED BY (c)") }, - errorClass = "COLUMN_NOT_DEFINED_IN_TABLE", + condition = "COLUMN_NOT_DEFINED_IN_TABLE", parameters = Map( "colType" -> "partition", "colName" -> "`c`", @@ -553,7 +553,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int, b string) USING json CLUSTERED BY (c) INTO 4 BUCKETS") }, - errorClass = "COLUMN_NOT_DEFINED_IN_TABLE", + condition = "COLUMN_NOT_DEFINED_IN_TABLE", parameters = Map( "colType" -> "bucket", "colName" -> "`c`", @@ -568,7 +568,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql(s"CREATE TABLE t($c0 INT) USING parquet PARTITIONED BY ($c0, $c1)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c1.toLowerCase(Locale.ROOT)}`")) } } @@ -581,7 +581,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql(s"CREATE TABLE t($c0 INT) USING parquet CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c1.toLowerCase(Locale.ROOT)}`")) checkError( @@ -591,7 +591,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { | SORTED BY ($c0, $c1) INTO 2 BUCKETS """.stripMargin) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c1.toLowerCase(Locale.ROOT)}`")) } } @@ -618,7 +618,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { .option("path", dir2.getCanonicalPath) .saveAsTable("path_test") }, - errorClass = "_LEGACY_ERROR_TEMP_1160", + condition = "_LEGACY_ERROR_TEMP_1160", parameters = Map( "identifier" -> s"`$SESSION_CATALOG_NAME`.`default`.`path_test`", "existingTableLoc" -> ".*", @@ -687,7 +687,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql(s"CREATE VIEW t AS SELECT * FROM VALUES (1, 1) AS t($c0, $c1)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c1.toLowerCase(Locale.ROOT)}`")) } } @@ -798,7 +798,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { |USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat |OPTIONS (PATH '${tmpFile.toURI}') """.stripMargin)}, - errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> "`testview`")) } } @@ -821,7 +821,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("ALTER TABLE tab1 RENAME TO default.tab2") }, - errorClass = "_LEGACY_ERROR_TEMP_1074", + condition = "_LEGACY_ERROR_TEMP_1074", parameters = Map( "oldName" -> "`tab1`", "newName" -> "`default`.`tab2`", @@ -850,7 +850,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("ALTER TABLE view1 RENAME TO default.tab2") }, - errorClass = "_LEGACY_ERROR_TEMP_1074", + condition = "_LEGACY_ERROR_TEMP_1074", parameters = Map( "oldName" -> "`view1`", "newName" -> "`default`.`tab2`", @@ -872,7 +872,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { checkAnswer(spark.table("tab1"), spark.range(10).toDF()) checkError( exception = intercept[AnalysisException] { spark.table("tab2") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`tab2`") ) } @@ -959,7 +959,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[ParseException] { sql(sql1) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE CLUSTERED BY"), context = ExpectedContext(fragment = sql1, start = 0, stop = 70)) val sql2 = "ALTER TABLE dbx.tab1 CLUSTERED BY (fuji) SORTED BY (grape) INTO 5 BUCKETS" @@ -967,7 +967,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[ParseException] { sql(sql2) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE CLUSTERED BY"), context = ExpectedContext(fragment = sql2, start = 0, stop = 72)) val sql3 = "ALTER TABLE dbx.tab1 NOT CLUSTERED" @@ -975,7 +975,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[ParseException] { sql(sql3) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE NOT CLUSTERED"), context = ExpectedContext(fragment = sql3, start = 0, stop = 33)) val sql4 = "ALTER TABLE dbx.tab1 NOT SORTED" @@ -983,7 +983,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[ParseException] { sql(sql4) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE NOT SORTED"), context = ExpectedContext(fragment = sql4, start = 0, stop = 30)) } @@ -999,7 +999,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[ParseException] { sql(sql1) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE SKEWED BY"), context = ExpectedContext(fragment = sql1, start = 0, stop = 113) ) @@ -1009,7 +1009,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[ParseException] { sql(sql2) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE SKEWED BY"), context = ExpectedContext(fragment = sql2, start = 0, stop = 113) ) @@ -1018,7 +1018,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[ParseException] { sql(sql3) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE NOT SKEWED"), context = ExpectedContext(fragment = sql3, start = 0, stop = 30) ) @@ -1027,7 +1027,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[ParseException] { sql(sql4) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER TABLE NOT STORED AS DIRECTORIES"), context = ExpectedContext(fragment = sql4, start = 0, stop = 45) ) @@ -1039,7 +1039,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[ParseException] { sql(sql1) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER VIEW ... ADD PARTITION"), context = ExpectedContext(fragment = sql1, start = 0, stop = 54) ) @@ -1051,7 +1051,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[ParseException] { sql(sql1) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "ALTER VIEW ... DROP PARTITION"), context = ExpectedContext(fragment = sql1, start = 0, stop = 51) ) @@ -1085,7 +1085,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { } checkError( exception = e, - errorClass = "WRONG_COMMAND_FOR_OBJECT_TYPE", + condition = "WRONG_COMMAND_FOR_OBJECT_TYPE", parameters = Map( "alternative" -> "DROP TABLE", "operation" -> "DROP VIEW", @@ -1125,21 +1125,21 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("DROP TEMPORARY FUNCTION year") }, - errorClass = "_LEGACY_ERROR_TEMP_1255", + condition = "_LEGACY_ERROR_TEMP_1255", parameters = Map("functionName" -> "year") ) checkError( exception = intercept[AnalysisException] { sql("DROP TEMPORARY FUNCTION YeAr") }, - errorClass = "_LEGACY_ERROR_TEMP_1255", + condition = "_LEGACY_ERROR_TEMP_1255", parameters = Map("functionName" -> "YeAr") ) checkError( exception = intercept[AnalysisException] { sql("DROP TEMPORARY FUNCTION `YeAr`") }, - errorClass = "_LEGACY_ERROR_TEMP_1255", + condition = "_LEGACY_ERROR_TEMP_1255", parameters = Map("functionName" -> "YeAr") ) } @@ -1216,7 +1216,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { checkError( exception = intercept[AnalysisException] { sql("CREATE TABLE tab1 USING json") }, - errorClass = "UNABLE_TO_INFER_SCHEMA", + condition = "UNABLE_TO_INFER_SCHEMA", parameters = Map("format" -> "JSON") ) @@ -1244,7 +1244,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { |CLUSTERED BY (nonexistentColumnA) SORTED BY (nonexistentColumnB) INTO 2 BUCKETS """.stripMargin) }, - errorClass = "SPECIFY_BUCKETING_IS_NOT_ALLOWED", + condition = "SPECIFY_BUCKETING_IS_NOT_ALLOWED", parameters = Map.empty ) } @@ -1271,7 +1271,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("CREATE TEMPORARY VIEW view1 (col1, col3) AS SELECT * FROM tab1") }, - errorClass = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "viewName" -> "`view1`", "viewColumns" -> "`col1`, `col3`", @@ -1298,7 +1298,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("CREATE TEMPORARY TABLE t_temp (c3 int, c4 string) USING JSON") } checkError(e, - errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> "`t_temp`")) } } @@ -1310,7 +1310,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("CREATE TEMPORARY VIEW t_temp (c3 int, c4 string) USING JSON") } checkError(e, - errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> "`t_temp`")) } } @@ -1325,7 +1325,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { df.write.mode("append").partitionBy("a").saveAsTable("partitionedTable") }, - errorClass = "_LEGACY_ERROR_TEMP_1163", + condition = "_LEGACY_ERROR_TEMP_1163", parameters = Map( "tableName" -> "spark_catalog.default.partitionedtable", "specifiedPartCols" -> "a", @@ -1336,7 +1336,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { df.write.mode("append").partitionBy("b", "a").saveAsTable("partitionedTable") }, - errorClass = "_LEGACY_ERROR_TEMP_1163", + condition = "_LEGACY_ERROR_TEMP_1163", parameters = Map( "tableName" -> "spark_catalog.default.partitionedtable", "specifiedPartCols" -> "b, a", @@ -1347,7 +1347,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { df.write.mode("append").saveAsTable("partitionedTable") }, - errorClass = "_LEGACY_ERROR_TEMP_1163", + condition = "_LEGACY_ERROR_TEMP_1163", parameters = Map( "tableName" -> "spark_catalog.default.partitionedtable", "specifiedPartCols" -> "", "existingPartCols" -> "a, b") @@ -1934,7 +1934,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("ALTER TABLE t1 ADD COLUMNS (c2 int)") }, - errorClass = "_LEGACY_ERROR_TEMP_1260", + condition = "_LEGACY_ERROR_TEMP_1260", parameters = Map( "tableType" -> ("org\\.apache\\.spark\\.sql\\.execution\\." + "datasources\\.v2\\.text\\.TextDataSourceV2.*"), @@ -1950,7 +1950,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> "`tmp_v`", "operation" -> "ALTER TABLE ... ADD COLUMNS"), @@ -1969,7 +1969,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`v1`", "operation" -> "ALTER TABLE ... ADD COLUMNS"), @@ -1988,7 +1988,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("ALTER TABLE t1 ADD COLUMNS (c1 string)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`c1`")) } } @@ -2003,7 +2003,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`c1`")) } else { sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") @@ -2058,7 +2058,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql(s"SET ${config.CPUS_PER_TASK.key} = 4") }, - errorClass = "CANNOT_MODIFY_CONFIG", + condition = "CANNOT_MODIFY_CONFIG", parameters = Map( "key" -> "\"spark.task.cpus\"", "docroot" -> "https://spark.apache.org/docs/latest")) @@ -2120,7 +2120,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { } checkError( exception = e1, - errorClass = "DATA_SOURCE_NOT_FOUND", + condition = "DATA_SOURCE_NOT_FOUND", parameters = Map("provider" -> "unknown") ) @@ -2151,7 +2151,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[SparkException] { sql(s"ADD FILE $testDir") }, - errorClass = "UNSUPPORTED_ADD_FILE.DIRECTORY", + condition = "UNSUPPORTED_ADD_FILE.DIRECTORY", parameters = Map("path" -> s"file:${testDir.getCanonicalPath}/") ) } @@ -2163,7 +2163,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("REFRESH FUNCTION md5") }, - errorClass = "_LEGACY_ERROR_TEMP_1017", + condition = "_LEGACY_ERROR_TEMP_1017", parameters = Map( "name" -> "md5", "cmd" -> "REFRESH FUNCTION", "hintStr" -> ""), @@ -2172,7 +2172,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("REFRESH FUNCTION default.md5") }, - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`default`.`md5`", "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"), @@ -2187,7 +2187,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("REFRESH FUNCTION func1") }, - errorClass = "_LEGACY_ERROR_TEMP_1017", + condition = "_LEGACY_ERROR_TEMP_1017", parameters = Map("name" -> "func1", "cmd" -> "REFRESH FUNCTION", "hintStr" -> ""), context = ExpectedContext( fragment = "func1", @@ -2203,7 +2203,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("REFRESH FUNCTION func1") }, - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`func1`", "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"), @@ -2219,7 +2219,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("REFRESH FUNCTION func2") }, - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`func2`", "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"), @@ -2235,7 +2235,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("REFRESH FUNCTION func1") }, - errorClass = "ROUTINE_NOT_FOUND", + condition = "ROUTINE_NOT_FOUND", parameters = Map("routineName" -> "`default`.`func1`") ) @@ -2248,7 +2248,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("REFRESH FUNCTION func1") }, - errorClass = "CANNOT_LOAD_FUNCTION_CLASS", + condition = "CANNOT_LOAD_FUNCTION_CLASS", parameters = Map( "className" -> "test.non.exists.udf", "functionName" -> "`spark_catalog`.`default`.`func1`" @@ -2267,7 +2267,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("REFRESH FUNCTION rand") }, - errorClass = "_LEGACY_ERROR_TEMP_1017", + condition = "_LEGACY_ERROR_TEMP_1017", parameters = Map("name" -> "rand", "cmd" -> "REFRESH FUNCTION", "hintStr" -> ""), context = ExpectedContext(fragment = "rand", start = 17, stop = 20) ) @@ -2282,7 +2282,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql(s"create table t(a int, b int generated always as (a + 1)) using parquet") }, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", parameters = Map("tableName" -> "`spark_catalog`.`default`.`t`", "operation" -> "generated columns") ) @@ -2295,7 +2295,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("ALTER TABLE t ALTER COLUMN i COMMENT 'comment'") }, - errorClass = "CANNOT_ALTER_PARTITION_COLUMN", + condition = "CANNOT_ALTER_PARTITION_COLUMN", sqlState = "428FR", parameters = Map("tableName" -> "`spark_catalog`.`default`.`t`", "columnName" -> "`i`") @@ -2318,7 +2318,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql(alterInt) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"STRING COLLATE UTF8_LCASE\"", "originName" -> "`col`", @@ -2354,7 +2354,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql(alterMap) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"MAP\"", "originName" -> "`col`", @@ -2381,7 +2381,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("ALTER TABLE t1 ALTER COLUMN col TYPE STRING COLLATE UTF8_LCASE") }, - errorClass = "CANNOT_ALTER_PARTITION_COLUMN", + condition = "CANNOT_ALTER_PARTITION_COLUMN", sqlState = "428FR", parameters = Map("tableName" -> "`spark_catalog`.`default`.`t1`", "columnName" -> "`col`") ) @@ -2390,7 +2390,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { exception = intercept[AnalysisException] { sql("ALTER TABLE t2 ALTER COLUMN col TYPE STRING COLLATE UTF8_LCASE") }, - errorClass = "CANNOT_ALTER_COLLATION_BUCKET_COLUMN", + condition = "CANNOT_ALTER_COLLATION_BUCKET_COLUMN", sqlState = "428FR", parameters = Map("tableName" -> "`spark_catalog`.`default`.`t2`", "columnName" -> "`col`") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DeclareVariableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DeclareVariableParserSuite.scala index bc42937b93a92..02f1d012297bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DeclareVariableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DeclareVariableParserSuite.scala @@ -179,7 +179,7 @@ class DeclareVariableParserSuite extends AnalysisTest with SharedSparkSession { exception = intercept[ParseException] { parsePlan("DECLARE VARIABLE IF NOT EXISTS var1 INT") }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'EXISTS'", "hint" -> "") ) @@ -189,7 +189,7 @@ class DeclareVariableParserSuite extends AnalysisTest with SharedSparkSession { exception = intercept[ParseException] { parsePlan(sqlText) }, - errorClass = "INVALID_SQL_SYNTAX.VARIABLE_TYPE_OR_DEFAULT_REQUIRED", + condition = "INVALID_SQL_SYNTAX.VARIABLE_TYPE_OR_DEFAULT_REQUIRED", parameters = Map.empty, context = ExpectedContext(fragment = sqlText, start = 0, stop = 20) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala index 6945352564e1e..36b17568d4716 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala @@ -43,7 +43,7 @@ trait DescribeNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"DESCRIBE NAMESPACE EXTENDED $catalog.$ns") } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> s"`$catalog`.`$ns`")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala index ee1b588741cd4..944f20bf8e924 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala @@ -78,7 +78,7 @@ class DescribeTableParserSuite extends AnalysisTest { val sql = "DESCRIBE TABLE t PARTITION (ds='1970-01-01') col" checkError( exception = parseException(parsePlan)(sql), - errorClass = "UNSUPPORTED_FEATURE.DESC_TABLE_COLUMN_PARTITION", + condition = "UNSUPPORTED_FEATURE.DESC_TABLE_COLUMN_PARTITION", parameters = Map.empty, context = ExpectedContext( fragment = sql, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala index 2243517550b2c..2a7fe53a848e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala @@ -64,7 +64,7 @@ trait DropNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"DROP NAMESPACE $catalog.unknown") } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> s"`$catalog`.`unknown`")) } @@ -78,7 +78,7 @@ trait DropNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"DROP NAMESPACE $catalog.ns") } checkError(e, - errorClass = "SCHEMA_NOT_EMPTY", + condition = "SCHEMA_NOT_EMPTY", parameters = Map("schemaName" -> "`ns`")) sql(s"DROP TABLE $catalog.ns.table") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropVariableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropVariableParserSuite.scala index f2af7e5dedca0..bc5e8c60ec812 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropVariableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropVariableParserSuite.scala @@ -42,14 +42,14 @@ class DropVariableParserSuite extends AnalysisTest with SharedSparkSession { exception = intercept[ParseException] { parsePlan("DROP VARIABLE var1") }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'VARIABLE'", "hint" -> "") ) checkError( exception = intercept[ParseException] { parsePlan("DROP VAR var1") }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'VAR'", "hint" -> "") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 73bcde1e6e5be..5a4d7c86761fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -264,7 +264,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { parsePlan(sql) }, - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = parameters, context = context ) @@ -306,7 +306,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[SparkUnsupportedOperationException] { parseAndResolve(query) }, - errorClass = "_LEGACY_ERROR_TEMP_2067", + condition = "_LEGACY_ERROR_TEMP_2067", parameters = Map("transform" -> transform)) } } @@ -323,7 +323,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[SparkUnsupportedOperationException] { parseAndResolve(query) }, - errorClass = "UNSUPPORTED_FEATURE.MULTIPLE_BUCKET_TRANSFORMS", + condition = "UNSUPPORTED_FEATURE.MULTIPLE_BUCKET_TRANSFORMS", parameters = Map.empty) } } @@ -417,7 +417,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { parseAndResolve(v2) }, - errorClass = "_LEGACY_ERROR_TEMP_0032", + condition = "_LEGACY_ERROR_TEMP_0032", parameters = Map("pathOne" -> "/tmp/file", "pathTwo" -> "/tmp/file"), context = ExpectedContext( fragment = v2, @@ -763,7 +763,7 @@ class PlanResolutionSuite extends AnalysisTest { } checkError( e, - errorClass = "UNSUPPORTED_FEATURE.CATALOG_OPERATION", + condition = "UNSUPPORTED_FEATURE.CATALOG_OPERATION", parameters = Map("catalogName" -> "`testcat`", "operation" -> "views")) } @@ -1207,7 +1207,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[AnalysisException] { parseAndResolve(sql6, checkAnalysis = true) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`DEFAULT`", "proposal" -> "`i`, `s`"), context = ExpectedContext( fragment = "DEFAULT", @@ -1219,7 +1219,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[AnalysisException] { parseAndResolve(sql7, checkAnalysis = true) }, - errorClass = "NO_DEFAULT_COLUMN_VALUE_AVAILABLE", + condition = "NO_DEFAULT_COLUMN_VALUE_AVAILABLE", parameters = Map("colName" -> "`x`") ) } @@ -1267,7 +1267,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[AnalysisException] { parseAndResolve(sql2, checkAnalysis = true) }, - errorClass = "NO_DEFAULT_COLUMN_VALUE_AVAILABLE", + condition = "NO_DEFAULT_COLUMN_VALUE_AVAILABLE", parameters = Map("colName" -> "`x`") ) @@ -1276,7 +1276,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[AnalysisException] { parseAndResolve(sql3, checkAnalysis = true) }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`testcat`.`tab2`", "tableColumns" -> "`i`, `x`", @@ -1337,7 +1337,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[AnalysisException] { parseAndResolve(sql3) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`j`", @@ -1350,7 +1350,7 @@ class PlanResolutionSuite extends AnalysisTest { } checkError( exception = e2, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", sqlState = "0A000", parameters = Map("tableName" -> "`spark_catalog`.`default`.`v1Table`", "operation" -> "ALTER COLUMN with qualified column")) @@ -1359,7 +1359,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[AnalysisException] { parseAndResolve(s"ALTER TABLE $tblName ALTER COLUMN i SET NOT NULL") }, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", sqlState = "0A000", parameters = Map("tableName" -> "`spark_catalog`.`default`.`v1Table`", "operation" -> "ALTER COLUMN ... SET NOT NULL")) @@ -1407,7 +1407,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { parseAndResolve(sql) }, - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "ALTER TABLE table ALTER COLUMN requires a TYPE, a SET/DROP, a COMMENT, or a FIRST/AFTER"), context = ExpectedContext(fragment = sql, start = 0, stop = 33)) @@ -1423,7 +1423,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[AnalysisException] { parseAndResolve(sql) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`I`", @@ -1944,7 +1944,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[AnalysisException] { parseAndResolve(mergeWithDefaultReferenceInMergeCondition, checkAnalysis = true) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`DEFAULT`", "proposal" -> "`target`.`i`, `source`.`i`, `target`.`s`, `source`.`s`"), context = ExpectedContext( @@ -1973,7 +1973,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[AnalysisException] { parseAndResolve(mergeWithDefaultReferenceAsPartOfComplexExpression) }, - errorClass = "DEFAULT_PLACEMENT_INVALID", + condition = "DEFAULT_PLACEMENT_INVALID", parameters = Map.empty) val mergeWithDefaultReferenceForNonNullableCol = @@ -1988,7 +1988,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[AnalysisException] { parseAndResolve(mergeWithDefaultReferenceForNonNullableCol) }, - errorClass = "NO_DEFAULT_COLUMN_VALUE_AVAILABLE", + condition = "NO_DEFAULT_COLUMN_VALUE_AVAILABLE", parameters = Map("colName" -> "`x`") ) @@ -2093,7 +2093,7 @@ class PlanResolutionSuite extends AnalysisTest { // resolve column `i` as it's ambiguous. checkError( exception = intercept[AnalysisException](parseAndResolve(sql2)), - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map("name" -> "`i`", "referenceNames" -> referenceNames(target, "i")), context = ExpectedContext( fragment = "i", @@ -2109,7 +2109,7 @@ class PlanResolutionSuite extends AnalysisTest { // resolve column `s` as it's ambiguous. checkError( exception = intercept[AnalysisException](parseAndResolve(sql3)), - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map("name" -> "`s`", "referenceNames" -> referenceNames(target, "s")), context = ExpectedContext( fragment = "s", @@ -2125,7 +2125,7 @@ class PlanResolutionSuite extends AnalysisTest { // resolve column `s` as it's ambiguous. checkError( exception = intercept[AnalysisException](parseAndResolve(sql4)), - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map("name" -> "`s`", "referenceNames" -> referenceNames(target, "s")), context = ExpectedContext( fragment = "s", @@ -2141,7 +2141,7 @@ class PlanResolutionSuite extends AnalysisTest { // resolve column `s` as it's ambiguous. checkError( exception = intercept[AnalysisException](parseAndResolve(sql5)), - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map("name" -> "`s`", "referenceNames" -> referenceNames(target, "s")), context = ExpectedContext( fragment = "s", @@ -2201,7 +2201,7 @@ class PlanResolutionSuite extends AnalysisTest { // update value in not matched by source clause can only reference the target table. checkError( exception = intercept[AnalysisException](parseAndResolve(sql7)), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> s"${toSQLId(source)}.`s`", "proposal" -> "`i`, `s`"), context = ExpectedContext( fragment = s"$source.s", @@ -2235,7 +2235,7 @@ class PlanResolutionSuite extends AnalysisTest { |WHEN MATCHED THEN UPDATE SET *""".stripMargin checkError( exception = intercept[AnalysisException](parseAndResolve(sql2)), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`s`", "proposal" -> "`i`, `x`"), context = ExpectedContext(fragment = sql2, start = 0, stop = 80)) @@ -2247,7 +2247,7 @@ class PlanResolutionSuite extends AnalysisTest { |WHEN NOT MATCHED THEN INSERT *""".stripMargin checkError( exception = intercept[AnalysisException](parseAndResolve(sql3)), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`s`", "proposal" -> "`i`, `x`"), context = ExpectedContext(fragment = sql3, start = 0, stop = 80)) @@ -2442,7 +2442,7 @@ class PlanResolutionSuite extends AnalysisTest { val sql = "CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING)" checkError( exception = parseException(parsePlan)(sql), - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "':'", "hint" -> "")) } @@ -2591,49 +2591,49 @@ class PlanResolutionSuite extends AnalysisTest { val sql1 = createTableHeader("TBLPROPERTIES('test' = 'test2')") checkError( exception = parseException(parsePlan)(sql1), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "TBLPROPERTIES"), context = ExpectedContext(fragment = sql1, start = 0, stop = 117)) val sql2 = createTableHeader("LOCATION '/tmp/file'") checkError( exception = parseException(parsePlan)(sql2), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "LOCATION"), context = ExpectedContext(fragment = sql2, start = 0, stop = 95)) val sql3 = createTableHeader("COMMENT 'a table'") checkError( exception = parseException(parsePlan)(sql3), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "COMMENT"), context = ExpectedContext(fragment = sql3, start = 0, stop = 89)) val sql4 = createTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS") checkError( exception = parseException(parsePlan)(sql4), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "CLUSTERED BY"), context = ExpectedContext(fragment = sql4, start = 0, stop = 119)) val sql5 = createTableHeader("PARTITIONED BY (k int)") checkError( exception = parseException(parsePlan)(sql5), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "PARTITIONED BY"), context = ExpectedContext(fragment = sql5, start = 0, stop = 99)) val sql6 = createTableHeader("STORED AS parquet") checkError( exception = parseException(parsePlan)(sql6), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "STORED AS/BY"), context = ExpectedContext(fragment = sql6, start = 0, stop = 89)) val sql7 = createTableHeader("ROW FORMAT SERDE 'parquet.hive.serde.ParquetHiveSerDe'") checkError( exception = parseException(parsePlan)(sql7), - errorClass = "DUPLICATE_CLAUSES", + condition = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "ROW FORMAT"), context = ExpectedContext(fragment = sql7, start = 0, stop = 163)) } @@ -2774,7 +2774,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { extractTableDesc(s4) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "STORED BY"), context = ExpectedContext( fragment = "STORED BY 'storage.handler.class.name'", @@ -2867,7 +2867,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { parsePlan(query) }, - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map( "message" -> "CREATE TEMPORARY TABLE ..., use CREATE TEMPORARY VIEW instead"), context = ExpectedContext(fragment = query, start = 0, stop = 48)) @@ -2939,7 +2939,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { parsePlan(query1) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "CREATE TABLE ... SKEWED BY"), context = ExpectedContext(fragment = query1, start = 0, stop = 72)) @@ -2948,7 +2948,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { parsePlan(query2) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "CREATE TABLE ... SKEWED BY"), context = ExpectedContext(fragment = query2, start = 0, stop = 96)) @@ -2957,7 +2957,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { parsePlan(query3) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "CREATE TABLE ... SKEWED BY"), context = ExpectedContext(fragment = query3, start = 0, stop = 118)) } @@ -3012,7 +3012,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { parsePlan(query1) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "STORED BY"), context = ExpectedContext( fragment = "STORED BY 'org.papachi.StorageHandler'", @@ -3024,7 +3024,7 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { parsePlan(query2) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> "STORED BY"), context = ExpectedContext( fragment = "STORED BY 'org.mamachi.StorageHandler' WITH SERDEPROPERTIES ('k1'='v1')", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowColumnsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowColumnsParserSuite.scala index 17a6df87aa0e4..c93beaa10ec13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowColumnsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowColumnsParserSuite.scala @@ -47,7 +47,7 @@ class ShowColumnsParserSuite extends AnalysisTest { test("illegal characters in unquoted identifier") { checkError( exception = parseException(parsePlan)("SHOW COLUMNS IN t FROM test-db"), - errorClass = "INVALID_IDENTIFIER", + condition = "INVALID_IDENTIFIER", sqlState = "42602", parameters = Map("ident" -> "test-db") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowColumnsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowColumnsSuiteBase.scala index c6f4e0bbd01a1..54bc10d0024f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowColumnsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowColumnsSuiteBase.scala @@ -57,7 +57,7 @@ trait ShowColumnsSuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[AnalysisException] { sql(s"SHOW COLUMNS IN tbl IN ns1") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`ns1`.`tbl`"), context = ExpectedContext(fragment = "tbl", start = 16, stop = 18) ) @@ -75,7 +75,7 @@ trait ShowColumnsSuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[AnalysisException] { sql(sqlText1) }, - errorClass = "SHOW_COLUMNS_WITH_CONFLICT_NAMESPACE", + condition = "SHOW_COLUMNS_WITH_CONFLICT_NAMESPACE", parameters = Map( "namespaceA" -> s"`ns1`", "namespaceB" -> s"`ns`" @@ -88,7 +88,7 @@ trait ShowColumnsSuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[AnalysisException] { sql(sqlText2) }, - errorClass = "SHOW_COLUMNS_WITH_CONFLICT_NAMESPACE", + condition = "SHOW_COLUMNS_WITH_CONFLICT_NAMESPACE", parameters = Map( "namespaceA" -> s"`${"ns".toUpperCase(Locale.ROOT)}`", "namespaceB" -> "`ns`" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala index 3a5d57c5c7821..455689026d7d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala @@ -47,7 +47,7 @@ class ShowPartitionsParserSuite extends AnalysisTest { test("empty values in non-optional partition specs") { checkError( exception = parseException(parsePlan)("SHOW PARTITIONS dbx.tab1 PARTITION (a='1', b)"), - errorClass = "INVALID_SQL_SYNTAX.EMPTY_PARTITION_VALUE", + condition = "INVALID_SQL_SYNTAX.EMPTY_PARTITION_VALUE", sqlState = "42000", parameters = Map("partKey" -> "`b`"), context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index 1890726a376ba..f6a5f6a7da26a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -168,7 +168,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[AnalysisException] { sql(s"SHOW TABLES IN $catalog.nonexist") }, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> s"`$catalog`.`nonexist`")) } @@ -177,7 +177,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[AnalysisException] { sql(s"SHOW TABLE EXTENDED IN $catalog.nonexist LIKE '*tbl*'") }, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> s"`$catalog`.`nonexist`")) } @@ -202,7 +202,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[AnalysisException] { sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 2)") }, - errorClass = "PARTITIONS_NOT_FOUND", + condition = "PARTITIONS_NOT_FOUND", parameters = Map( "partitionList" -> "PARTITION (`id` = 2)", "tableName" -> "`ns1`.`tbl`" @@ -220,7 +220,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") } val (errorClass, parameters) = extendedPartInNonPartedTableError(catalog, namespace, table) - checkError(exception = e, errorClass = errorClass, parameters = parameters) + checkError(exception = e, condition = errorClass, parameters = parameters) } } @@ -261,7 +261,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace " + s"LIKE '$table' PARTITION(id1 = 1)") }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "id1", "partitionColumnNames" -> "id1, id2", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala index b903681e341f9..be37495acad05 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala @@ -47,7 +47,7 @@ class TruncateTableParserSuite extends AnalysisTest { test("empty values in non-optional partition specs") { checkError( exception = parseException(parsePlan)("TRUNCATE TABLE dbx.tab1 PARTITION (a='1', b)"), - errorClass = "INVALID_SQL_SYNTAX.EMPTY_PARTITION_VALUE", + condition = "INVALID_SQL_SYNTAX.EMPTY_PARTITION_VALUE", sqlState = "42000", parameters = Map("partKey" -> "`b`"), context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala index 982c568d09a79..8c985ea1f0527 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala @@ -181,7 +181,7 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[AnalysisException] { sql("TRUNCATE TABLE v0") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> "`spark_catalog`.`default`.`v0`", "operation" -> "TRUNCATE TABLE"), @@ -198,7 +198,7 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[AnalysisException] { sql("TRUNCATE TABLE v1") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> "`v1`", "operation" -> "TRUNCATE TABLE"), @@ -213,7 +213,7 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils { exception = intercept[AnalysisException] { sql(s"TRUNCATE TABLE $v2") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> "`global_temp`.`v2`", "operation" -> "TRUNCATE TABLE"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala index dac99c8ff7023..fea0d07278c1a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala @@ -43,7 +43,7 @@ trait AlterTableAddPartitionSuiteBase extends command.AlterTableAddPartitionSuit exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t ADD PARTITION (p1 = '')") }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> "The spec ([p1=]) contains an empty partition column value" ) @@ -155,7 +155,7 @@ trait AlterTableAddPartitionSuiteBase extends command.AlterTableAddPartitionSuit " PARTITION (id=2) LOCATION 'loc1'") } checkError(e, - errorClass = "PARTITIONS_ALREADY_EXIST", + condition = "PARTITIONS_ALREADY_EXIST", parameters = Map("partitionList" -> "PARTITION (`id` = 2)", "tableName" -> "`ns`.`tbl`")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropColumnSuite.scala index 6370a834746a5..85c7e66bdbe57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropColumnSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropColumnSuite.scala @@ -40,7 +40,7 @@ trait AlterTableDropColumnSuiteBase extends command.AlterTableDropColumnSuiteBas exception = intercept[AnalysisException]( sql(s"ALTER TABLE $t DROP COLUMN id") ), - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", parameters = Map( "tableName" -> toSQLId(t), "operation" -> "DROP COLUMN" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropPartitionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropPartitionSuite.scala index 8d403429ca5d2..384aadfb3a6f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropPartitionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropPartitionSuite.scala @@ -83,7 +83,7 @@ class AlterTableDropPartitionSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t DROP PARTITION (p1 = '')") }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map("details" -> "The spec ([p1=]) contains an empty partition column value") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala index b219e21a3d881..54c0e7883ccda 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala @@ -123,7 +123,7 @@ trait AlterTableRecoverPartitionsSuiteBase extends command.AlterTableRecoverPart } checkError( exception = exception, - errorClass = "NOT_A_PARTITIONED_TABLE", + condition = "NOT_A_PARTITIONED_TABLE", parameters = Map( "operation" -> "ALTER TABLE RECOVER PARTITIONS", "tableIdentWithDB" -> "`spark_catalog`.`default`.`tbl`") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameColumnSuite.scala index 86b34311bfb3d..a6b43ad4d5a78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameColumnSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameColumnSuite.scala @@ -39,7 +39,7 @@ trait AlterTableRenameColumnSuiteBase extends command.AlterTableRenameColumnSuit exception = intercept[AnalysisException]( sql(s"ALTER TABLE $t RENAME COLUMN col1 TO col3") ), - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", parameters = Map( "tableName" -> toSQLId(t), "operation" -> "RENAME COLUMN" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameSuite.scala index dfbdc6a4ca78e..f8708d5bff25a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameSuite.scala @@ -41,7 +41,7 @@ trait AlterTableRenameSuiteBase extends command.AlterTableRenameSuiteBase with Q exception = intercept[AnalysisException] { sql(s"ALTER TABLE $src RENAME TO dst_ns.dst_tbl") }, - errorClass = "_LEGACY_ERROR_TEMP_1073", + condition = "_LEGACY_ERROR_TEMP_1073", parameters = Map("db" -> "src_ns", "newDb" -> "dst_ns") ) } @@ -75,7 +75,7 @@ trait AlterTableRenameSuiteBase extends command.AlterTableRenameSuiteBase with Q exception = intercept[SparkRuntimeException] { sql(s"ALTER TABLE $src RENAME TO ns.dst_tbl") }, - errorClass = "LOCATION_ALREADY_EXISTS", + condition = "LOCATION_ALREADY_EXISTS", parameters = Map( "location" -> s"'$dst_dir'", "identifier" -> toSQLId(dst))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala index 53b9853f36c8c..8f5af2e1f2e76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala @@ -93,7 +93,7 @@ trait AlterTableSetLocationSuiteBase extends command.AlterTableSetLocationSuiteB exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t PARTITION (A='1', B='2') SET LOCATION '/path/to/part/ways3'") }, - errorClass = "_LEGACY_ERROR_TEMP_1231", + condition = "_LEGACY_ERROR_TEMP_1231", parameters = Map("key" -> "A", "tblName" -> "`spark_catalog`.`ns`.`tbl`") ) } @@ -127,7 +127,7 @@ trait AlterTableSetLocationSuiteBase extends command.AlterTableSetLocationSuiteB exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t PARTITION (b='2') SET LOCATION '/mister/spark'") }, - errorClass = "_LEGACY_ERROR_TEMP_1232", + condition = "_LEGACY_ERROR_TEMP_1232", parameters = Map( "specKeys" -> "b", "partitionColumnNames" -> "a, b", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala index 6e4d6a8a0c8f0..259c4cb52a0fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala @@ -86,14 +86,14 @@ class AlterTableSetSerdeSuite extends AlterTableSetSerdeSuiteBase with CommandSu exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t SET SERDE 'whatever'") }, - errorClass = "_LEGACY_ERROR_TEMP_1248", + condition = "_LEGACY_ERROR_TEMP_1248", parameters = Map.empty) checkError( exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t SET SERDE 'org.apache.madoop' " + "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") }, - errorClass = "_LEGACY_ERROR_TEMP_1248", + condition = "_LEGACY_ERROR_TEMP_1248", parameters = Map.empty) // set serde properties only @@ -133,14 +133,14 @@ class AlterTableSetSerdeSuite extends AlterTableSetSerdeSuiteBase with CommandSu exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t PARTITION (a=1, b=2) SET SERDE 'whatever'") }, - errorClass = "_LEGACY_ERROR_TEMP_1247", + condition = "_LEGACY_ERROR_TEMP_1247", parameters = Map.empty) checkError( exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t PARTITION (a=1, b=2) SET SERDE 'org.apache.madoop' " + "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") }, - errorClass = "_LEGACY_ERROR_TEMP_1247", + condition = "_LEGACY_ERROR_TEMP_1247", parameters = Map.empty) // set serde properties only @@ -149,7 +149,7 @@ class AlterTableSetSerdeSuite extends AlterTableSetSerdeSuiteBase with CommandSu sql(s"ALTER TABLE $t PARTITION (a=1, b=2) " + "SET SERDEPROPERTIES ('k' = 'vvv', 'kay' = 'vee')") }, - errorClass = "_LEGACY_ERROR_TEMP_1247", + condition = "_LEGACY_ERROR_TEMP_1247", parameters = Map.empty) // set things without explicitly specifying database @@ -158,7 +158,7 @@ class AlterTableSetSerdeSuite extends AlterTableSetSerdeSuiteBase with CommandSu exception = intercept[AnalysisException] { sql(s"ALTER TABLE tbl PARTITION (a=1, b=2) SET SERDEPROPERTIES ('kay' = 'veee')") }, - errorClass = "_LEGACY_ERROR_TEMP_1247", + condition = "_LEGACY_ERROR_TEMP_1247", parameters = Map.empty) // table to alter does not exist diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala index 02cf1958b9499..eaf016ac2fa9f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala @@ -47,7 +47,7 @@ trait DescribeTableSuiteBase extends command.DescribeTableSuiteBase sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)") } checkError(e, - errorClass = "PARTITIONS_NOT_FOUND", + condition = "PARTITIONS_NOT_FOUND", parameters = Map("partitionList" -> "PARTITION (`id` = 1)", "tableName" -> "`ns`.`table`")) } @@ -63,7 +63,7 @@ trait DescribeTableSuiteBase extends command.DescribeTableSuiteBase exception = intercept[AnalysisException] { sql(s"DESC $tbl key1").collect() }, - errorClass = "COLUMN_NOT_FOUND", + condition = "COLUMN_NOT_FOUND", parameters = Map( "colName" -> "`key1`", "caseSensitiveConfig" -> "\"spark.sql.caseSensitive\"" @@ -89,7 +89,7 @@ trait DescribeTableSuiteBase extends command.DescribeTableSuiteBase exception = intercept[AnalysisException] { sql(s"DESC $tbl KEY").collect() }, - errorClass = "COLUMN_NOT_FOUND", + condition = "COLUMN_NOT_FOUND", parameters = Map( "colName" -> "`KEY`", "caseSensitiveConfig" -> "\"spark.sql.caseSensitive\"" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala index cec72b8855291..f3f9369ea062c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala @@ -39,7 +39,7 @@ trait DropNamespaceSuiteBase extends command.DropNamespaceSuiteBase exception = intercept[AnalysisException] { sql(s"DROP NAMESPACE default") }, - errorClass = "UNSUPPORTED_FEATURE.DROP_DATABASE", + condition = "UNSUPPORTED_FEATURE.DROP_DATABASE", parameters = Map("database" -> s"`$catalog`.`default`") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowColumnsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowColumnsSuite.scala index e9459a224486c..3e8ac98dbf767 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowColumnsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowColumnsSuite.scala @@ -38,7 +38,7 @@ trait ShowColumnsSuiteBase extends command.ShowColumnsSuiteBase { exception = intercept[AnalysisException] { sql("SHOW COLUMNS IN tbl FROM a.b.c") }, - errorClass = "REQUIRES_SINGLE_PART_NAMESPACE", + condition = "REQUIRES_SINGLE_PART_NAMESPACE", parameters = Map( "sessionCatalog" -> catalog, "namespace" -> "`a`.`b`.`c`" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowCreateTableSuite.scala index 18b5da0ca59fa..afbb943bf91f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowCreateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowCreateTableSuite.scala @@ -162,7 +162,7 @@ trait ShowCreateTableSuiteBase extends command.ShowCreateTableSuiteBase exception = intercept[AnalysisException] { getShowCreateDDL(t, true) }, - errorClass = "UNSUPPORTED_SHOW_CREATE_TABLE.ON_DATA_SOURCE_TABLE_WITH_AS_SERDE", + condition = "UNSUPPORTED_SHOW_CREATE_TABLE.ON_DATA_SOURCE_TABLE_WITH_AS_SERDE", sqlState = "0A000", parameters = Map("tableName" -> "`spark_catalog`.`ns1`.`tbl`") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala index 85a46cfb93233..30189b46db4ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala @@ -49,7 +49,7 @@ trait ShowNamespacesSuiteBase extends command.ShowNamespacesSuiteBase { sql("SHOW NAMESPACES in dummy") } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> s"`$catalog`.`dummy`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index 9863942c6ea19..0f64fa49f4862 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -57,7 +57,7 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { exception = intercept[AnalysisException] { sql(s"SHOW PARTITIONS $view") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`spark_catalog`.`default`.`view1`", "operation" -> "SHOW PARTITIONS" @@ -80,7 +80,7 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { exception = intercept[AnalysisException] { sql(s"SHOW PARTITIONS $viewName") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> "`test_view`", "operation" -> "SHOW PARTITIONS" @@ -124,7 +124,7 @@ class ShowPartitionsSuite extends ShowPartitionsSuiteBase with CommandSuiteBase exception = intercept[AnalysisException] { sql(s"SHOW PARTITIONS $viewName") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> "`test_view`", "operation" -> "SHOW PARTITIONS" @@ -163,7 +163,7 @@ class ShowPartitionsSuite extends ShowPartitionsSuiteBase with CommandSuiteBase exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", + condition = "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", parameters = Map("name" -> tableName)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala index 9be802b5f1fea..001267a37d382 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala @@ -56,7 +56,7 @@ trait ShowTablesSuiteBase extends command.ShowTablesSuiteBase with command.Tests exception = intercept[AnalysisException] { runShowTablesSql("SHOW TABLES FROM a.b", Seq()) }, - errorClass = "_LEGACY_ERROR_TEMP_1126", + condition = "_LEGACY_ERROR_TEMP_1126", parameters = Map("catalog" -> "a.b") ) } @@ -102,7 +102,7 @@ trait ShowTablesSuiteBase extends command.ShowTablesSuiteBase with command.Tests exception = intercept[AnalysisException] { sql(showTableCmd) }, - errorClass = "_LEGACY_ERROR_TEMP_1125", + condition = "_LEGACY_ERROR_TEMP_1125", parameters = Map.empty ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/TruncateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/TruncateTableSuite.scala index 5810a35ddcf8b..348b216aeb044 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/TruncateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/TruncateTableSuite.scala @@ -201,7 +201,7 @@ class TruncateTableSuite extends TruncateTableSuiteBase with CommandSuiteBase { exception = intercept[AnalysisException] { sql(s"TRUNCATE TABLE $t") }, - errorClass = "_LEGACY_ERROR_TEMP_1266", + condition = "_LEGACY_ERROR_TEMP_1266", parameters = Map("tableIdentWithDB" -> "`spark_catalog`.`ns`.`tbl`") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableAddPartitionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableAddPartitionSuite.scala index defa026c0e281..ac142e029681e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableAddPartitionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableAddPartitionSuite.scala @@ -43,7 +43,7 @@ class AlterTableAddPartitionSuite exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED", + condition = "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED", parameters = Map("name" -> tableName), context = ExpectedContext( fragment = t, @@ -126,7 +126,7 @@ class AlterTableAddPartitionSuite " PARTITION (id=2) LOCATION 'loc1'") } checkError(e, - errorClass = "PARTITIONS_ALREADY_EXIST", + condition = "PARTITIONS_ALREADY_EXIST", parameters = Map("partitionList" -> "PARTITION (`id` = 2)", "tableName" -> "`test_catalog`.`ns`.`tbl`")) @@ -146,7 +146,7 @@ class AlterTableAddPartitionSuite exception = intercept[SparkNumberFormatException] { sql(s"ALTER TABLE $t ADD PARTITION (p='aaa')") }, - errorClass = "CAST_INVALID_INPUT", + condition = "CAST_INVALID_INPUT", parameters = Map( "ansiConfig" -> "\"spark.sql.ansi.enabled\"", "expression" -> "'aaa'", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropColumnSuite.scala index d541f1286e598..6cd9ed2628dbb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropColumnSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropColumnSuite.scala @@ -45,7 +45,7 @@ class AlterTableDropColumnSuite exception = intercept[AnalysisException] { sql("ALTER TABLE does_not_exist DROP COLUMN id") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`does_not_exist`"), context = ExpectedContext(fragment = "does_not_exist", start = 12, stop = 25) ) @@ -127,7 +127,7 @@ class AlterTableDropColumnSuite exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`does_not_exist`", @@ -145,7 +145,7 @@ class AlterTableDropColumnSuite exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`point`.`does_not_exist`", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropPartitionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropPartitionSuite.scala index 2df7eebaecc81..35afb00ff0f38 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropPartitionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropPartitionSuite.scala @@ -43,7 +43,7 @@ class AlterTableDropPartitionSuite exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED", + condition = "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED", parameters = Map("name" -> tableName), context = ExpectedContext( fragment = t, @@ -61,7 +61,7 @@ class AlterTableDropPartitionSuite exception = intercept[SparkUnsupportedOperationException] { sql(s"ALTER TABLE $t DROP PARTITION (id=1) PURGE") }, - errorClass = "UNSUPPORTED_FEATURE.PURGE_PARTITION", + condition = "UNSUPPORTED_FEATURE.PURGE_PARTITION", parameters = Map.empty ) } finally { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRecoverPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRecoverPartitionsSuite.scala index ff6ff0df5306a..508b8e9d0339d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRecoverPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRecoverPartitionsSuite.scala @@ -35,7 +35,7 @@ class AlterTableRecoverPartitionsSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t RECOVER PARTITIONS") }, - errorClass = "NOT_SUPPORTED_COMMAND_FOR_V2_TABLE", + condition = "NOT_SUPPORTED_COMMAND_FOR_V2_TABLE", parameters = Map("cmd" -> "ALTER TABLE ... RECOVER PARTITIONS") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRenameColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRenameColumnSuite.scala index a2ab63d9ebd85..6edf9ee4a10d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRenameColumnSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRenameColumnSuite.scala @@ -45,7 +45,7 @@ class AlterTableRenameColumnSuite exception = intercept[AnalysisException] { sql("ALTER TABLE does_not_exist RENAME COLUMN col1 TO col3") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`does_not_exist`"), context = ExpectedContext(fragment = "does_not_exist", start = 12, stop = 25) ) @@ -153,7 +153,7 @@ class AlterTableRenameColumnSuite exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`does_not_exist`", @@ -171,7 +171,7 @@ class AlterTableRenameColumnSuite exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`point`.`does_not_exist`", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableReplaceColumnsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableReplaceColumnsSuite.scala index 599820d7622d4..4afe294549f45 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableReplaceColumnsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableReplaceColumnsSuite.scala @@ -35,7 +35,7 @@ class AlterTableReplaceColumnsSuite exception = intercept[ParseException] { sql(sql1) }, - errorClass = "UNSUPPORTED_DEFAULT_VALUE.WITHOUT_SUGGESTION", + condition = "UNSUPPORTED_DEFAULT_VALUE.WITHOUT_SUGGESTION", parameters = Map.empty, context = ExpectedContext(sql1, 0, 48) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetLocationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetLocationSuite.scala index 13f6b8d5b33bb..feb00ce0ec69f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetLocationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetLocationSuite.scala @@ -60,7 +60,7 @@ class AlterTableSetLocationSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t PARTITION(ds='2017-06-10') SET LOCATION 's3://bucket/path'") }, - errorClass = "_LEGACY_ERROR_TEMP_1045", + condition = "_LEGACY_ERROR_TEMP_1045", parameters = Map.empty ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetSerdeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetSerdeSuite.scala index d17bab99d01fe..971a5cd077bc1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetSerdeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetSerdeSuite.scala @@ -40,7 +40,7 @@ class AlterTableSetSerdeSuite extends command.AlterTableSetSerdeSuiteBase with C exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')") }, - errorClass = "NOT_SUPPORTED_COMMAND_FOR_V2_TABLE", + condition = "NOT_SUPPORTED_COMMAND_FOR_V2_TABLE", sqlState = "0A000", parameters = Map("cmd" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala index cfd26c09bf3e5..9cd7f0d8aade6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala @@ -106,7 +106,7 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase exception = intercept[AnalysisException] { sql(query).collect() }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`key1`", @@ -137,7 +137,7 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase exception = intercept[AnalysisException] { sql(query).collect() }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`KEY`", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropTableSuite.scala index 83bded7ab4f52..ffc2c6c679a8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropTableSuite.scala @@ -35,7 +35,7 @@ class DropTableSuite extends command.DropTableSuiteBase with CommandSuiteBase { exception = intercept[SparkUnsupportedOperationException] { sql(s"DROP TABLE $catalog.ns.tbl PURGE") }, - errorClass = "UNSUPPORTED_FEATURE.PURGE_TABLE", + condition = "UNSUPPORTED_FEATURE.PURGE_TABLE", parameters = Map.empty ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/MsckRepairTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/MsckRepairTableSuite.scala index 381e55b49393c..73764e88bffa9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/MsckRepairTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/MsckRepairTableSuite.scala @@ -36,7 +36,7 @@ class MsckRepairTableSuite exception = intercept[AnalysisException] { sql(s"MSCK REPAIR TABLE $t") }, - errorClass = "NOT_SUPPORTED_COMMAND_FOR_V2_TABLE", + condition = "NOT_SUPPORTED_COMMAND_FOR_V2_TABLE", parameters = Map("cmd" -> "MSCK REPAIR TABLE") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala index 8e1bb08162e3e..44a1bcad46a03 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala @@ -44,7 +44,7 @@ class ShowNamespacesSuite extends command.ShowNamespacesSuiteBase with CommandSu exception = intercept[AnalysisException] { sql("SHOW NAMESPACES") }, - errorClass = "_LEGACY_ERROR_TEMP_1184", + condition = "_LEGACY_ERROR_TEMP_1184", parameters = Map( "plugin" -> "testcat_no_namespace", "ability" -> "namespaces" @@ -58,7 +58,7 @@ class ShowNamespacesSuite extends command.ShowNamespacesSuiteBase with CommandSu exception = intercept[AnalysisException] { sql("SHOW NAMESPACES in testcat_no_namespace") }, - errorClass = "_LEGACY_ERROR_TEMP_1184", + condition = "_LEGACY_ERROR_TEMP_1184", parameters = Map( "plugin" -> "testcat_no_namespace", "ability" -> "namespaces" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 203ef4314ad25..1fb1c48890607 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -38,7 +38,7 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with CommandSu exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", + condition = "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", parameters = Map("name" -> tableName), context = ExpectedContext( fragment = t, @@ -61,7 +61,7 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with CommandSu exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED", + condition = "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED", parameters = Map("name" -> tableName), context = ExpectedContext( fragment = table, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/TruncateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/TruncateTableSuite.scala index 36b994c21a083..972511a470465 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/TruncateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/TruncateTableSuite.scala @@ -41,7 +41,7 @@ class TruncateTableSuite extends command.TruncateTableSuiteBase with CommandSuit exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED", + condition = "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED", parameters = Map("name" -> tableName), context = ExpectedContext( fragment = t, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala index 7f886940473de..fd9d31e7a594d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala @@ -128,7 +128,7 @@ class DataSourceSuite extends SharedSparkSession with PrivateMethodTester { enableGlobbing = true ) ), - errorClass = "PATH_NOT_FOUND", + condition = "PATH_NOT_FOUND", parameters = Map("path" -> nonExistentPath.toString) ) } @@ -173,7 +173,7 @@ class DataSourceSuite extends SharedSparkSession with PrivateMethodTester { new File(uuid, "file3").getAbsolutePath, uuid).rdd }, - errorClass = "PATH_NOT_FOUND", + condition = "PATH_NOT_FOUND", parameters = Map("path" -> "file:.*"), matchPVals = true ) @@ -187,7 +187,7 @@ class DataSourceSuite extends SharedSparkSession with PrivateMethodTester { exception = intercept[AnalysisException] { spark.read.format("text").load(s"$nonExistentBasePath/*") }, - errorClass = "PATH_NOT_FOUND", + condition = "PATH_NOT_FOUND", parameters = Map("path" -> s"file:$nonExistentBasePath/*") ) @@ -200,7 +200,7 @@ class DataSourceSuite extends SharedSparkSession with PrivateMethodTester { exception = intercept[AnalysisException] { spark.read.json(s"${baseDir.getAbsolutePath}/*/*-xyz.json").rdd }, - errorClass = "PATH_NOT_FOUND", + condition = "PATH_NOT_FOUND", parameters = Map("path" -> s"file:${baseDir.getAbsolutePath}/*/*-xyz.json") ) } finally { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala index 21623f94c8baf..31b7380889158 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala @@ -137,7 +137,7 @@ class FileIndexSuite extends SharedSparkSession { exception = intercept[SparkRuntimeException] { fileIndex.partitionSpec() }, - errorClass = "_LEGACY_ERROR_TEMP_2058", + condition = "_LEGACY_ERROR_TEMP_2058", parameters = Map("value" -> "foo", "dataType" -> "IntegerType", "columnName" -> "a") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala index 185a7cf5a6b40..880f1dd9af8f9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala @@ -243,7 +243,7 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select("name", METADATA_FILE_NAME).collect() }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`file_name`", "fields" -> "`id`, `university`"), context = ExpectedContext(fragment = "select", callSitePattern = getCurrentClassCallSitePattern)) @@ -309,7 +309,7 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.metadataColumn("foo") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`foo`", "proposal" -> "`_metadata`")) // Name exists, but does not reference a metadata column @@ -317,7 +317,7 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.metadataColumn("name") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`name`", "proposal" -> "`_metadata`")) } @@ -525,7 +525,7 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select("name", "_metadata.file_name").collect() }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`file_name`", "fields" -> "`id`, `university`"), context = ExpectedContext( fragment = "select", @@ -535,7 +535,7 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.select("name", "_METADATA.file_NAME").collect() }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map("fieldName" -> "`file_NAME`", "fields" -> "`id`, `university`"), context = ExpectedContext( fragment = "select", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala index 3762241719acd..26962d89452ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala @@ -101,7 +101,7 @@ class SaveIntoDataSourceCommandSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { dataSource.planForWriting(SaveMode.ErrorIfExists, df.logicalPlan) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map("columnName" -> "`col`", "columnType" -> s"\"${testCase._2}\"", "format" -> ".*JdbcRelationProvider.*"), matchPVals = true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala index 5c118ac12b72a..3dec1b9ff5cf2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala @@ -168,7 +168,7 @@ class BinaryFileFormatSuite extends QueryTest with SharedSparkSession { .format(BINARY_FILE) .save(s"$tmpDir/test_save") }, - errorClass = "_LEGACY_ERROR_TEMP_2075", + condition = "_LEGACY_ERROR_TEMP_2075", parameters = Map.empty) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index b1b5c882f4e97..e2d1d9b05c3c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -266,7 +266,7 @@ abstract class CSVSuite spark.read.format("csv").option("charset", "1-9588-osi") .load(testFile(carsFile8859)) }, - errorClass = "INVALID_PARAMETER_VALUE.CHARSET", + condition = "INVALID_PARAMETER_VALUE.CHARSET", parameters = Map( "charset" -> "1-9588-osi", "functionName" -> toSQLId("CSVOptions"), @@ -388,13 +388,13 @@ abstract class CSVSuite } checkErrorMatchPVals( exception = e1, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*$carsFile.*")) val e2 = e1.getCause.asInstanceOf[SparkException] assert(e2.getErrorClass == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION") checkError( exception = e2.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "MALFORMED_CSV_RECORD", + condition = "MALFORMED_CSV_RECORD", parameters = Map("badRecord" -> "2015,Chevy,Volt") ) } @@ -650,7 +650,7 @@ abstract class CSVSuite .csv(csvDir) } }, - errorClass = "INVALID_PARAMETER_VALUE.CHARSET", + condition = "INVALID_PARAMETER_VALUE.CHARSET", parameters = Map( "charset" -> "1-9588-osi", "functionName" -> toSQLId("CSVOptions"), @@ -1269,7 +1269,7 @@ abstract class CSVSuite } checkErrorMatchPVals( exception = ex, - errorClass = "TASK_WRITE_FAILED", + condition = "TASK_WRITE_FAILED", parameters = Map("path" -> s".*${path.getName}.*")) val msg = ex.getCause.getMessage assert( @@ -1509,7 +1509,7 @@ abstract class CSVSuite .csv(testFile(valueMalformedFile)) .collect() }, - errorClass = "_LEGACY_ERROR_TEMP_1097", + condition = "_LEGACY_ERROR_TEMP_1097", parameters = Map.empty ) } @@ -1523,7 +1523,7 @@ abstract class CSVSuite } checkErrorMatchPVals( exception = e, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*${inputFile.getName}.*") ) assert(e.getCause.isInstanceOf[EOFException]) @@ -1533,7 +1533,7 @@ abstract class CSVSuite } checkErrorMatchPVals( exception = e2, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*${inputFile.getName}.*") ) assert(e2.getCause.getCause.getCause.isInstanceOf[EOFException]) @@ -1557,7 +1557,7 @@ abstract class CSVSuite exception = intercept[SparkException] { df.collect() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> s".*$dir.*") ) } @@ -1705,7 +1705,7 @@ abstract class CSVSuite } checkError( exception = exception, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map("badRecord" -> "[null]", "failFastMode" -> "FAILFAST")) assert(exception.getCause.getMessage.contains("""input string: "10u12"""")) @@ -1794,7 +1794,7 @@ abstract class CSVSuite spark.read.schema(schema).csv(testFile(valueMalformedFile)) .select("_corrupt_record").collect() }, - errorClass = "UNSUPPORTED_FEATURE.QUERY_ONLY_CORRUPT_RECORD_COLUMN", + condition = "UNSUPPORTED_FEATURE.QUERY_ONLY_CORRUPT_RECORD_COLUMN", parameters = Map.empty ) // workaround @@ -2013,7 +2013,7 @@ abstract class CSVSuite } checkErrorMatchPVals( exception = exception, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*${path.getCanonicalPath}.*")) assert(exception.getCause.getMessage.contains("CSV header does not conform to the schema")) @@ -2029,7 +2029,7 @@ abstract class CSVSuite } checkErrorMatchPVals( exception = exceptionForShortSchema, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*${path.getCanonicalPath}.*")) assert(exceptionForShortSchema.getCause.getMessage.contains( "Number of column in CSV header is not equal to number of fields in the schema")) @@ -2050,7 +2050,7 @@ abstract class CSVSuite } checkErrorMatchPVals( exception = exceptionForLongSchema, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*${path.getCanonicalPath}.*")) assert(exceptionForLongSchema.getCause.getMessage.contains( "Header length: 2, schema size: 3")) @@ -2067,7 +2067,7 @@ abstract class CSVSuite } checkErrorMatchPVals( exception = caseSensitiveException, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*${path.getCanonicalPath}.*")) assert(caseSensitiveException.getCause.getMessage.contains( "CSV header does not conform to the schema")) @@ -2122,7 +2122,7 @@ abstract class CSVSuite exception = intercept[SparkIllegalArgumentException] { spark.read.schema(ischema).option("header", true).option("enforceSchema", false).csv(ds) }, - errorClass = "_LEGACY_ERROR_TEMP_3241", + condition = "_LEGACY_ERROR_TEMP_3241", parameters = Map("msg" -> """CSV header does not conform to the schema. | Header: columnA, columnB @@ -2161,7 +2161,7 @@ abstract class CSVSuite .schema(schema) .csv(Seq("col1,col2", "1.0,a").toDS()) }, - errorClass = "_LEGACY_ERROR_TEMP_3241", + condition = "_LEGACY_ERROR_TEMP_3241", parameters = Map("msg" -> """CSV header does not conform to the schema. | Header: col1, col2 @@ -2790,7 +2790,7 @@ abstract class CSVSuite exception = intercept[SparkUpgradeException] { csv.collect() }, - errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER", + condition = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER", parameters = Map( "datetime" -> "'2020-01-27T20:06:11.847-08000'", "config" -> "\"spark.sql.legacy.timeParserPolicy\"")) @@ -2850,7 +2850,7 @@ abstract class CSVSuite exception = intercept[AnalysisException] { readback.filter($"AAA" === 2 && $"bbb" === 3).collect() }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`AAA`", "proposal" -> "`BBB`, `aaa`"), context = ExpectedContext(fragment = "$", callSitePattern = getCurrentClassCallSitePattern)) @@ -3458,11 +3458,11 @@ class CSVv1Suite extends CSVSuite { } checkErrorMatchPVals( exception = ex, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*$carsFile")) checkError( exception = ex.getCause.asInstanceOf[SparkException], - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[2015,Chevy,Volt,null,null]", "failFastMode" -> "FAILFAST") @@ -3487,7 +3487,7 @@ class CSVv2Suite extends CSVSuite { .options(Map("header" -> "true", "mode" -> "failfast")) .load(testFile(carsFile)).collect() }, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*$carsFile"), matchPVals = true ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala index 486532028de9c..d3723881bfa24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala @@ -51,7 +51,7 @@ class JdbcUtilsSuite extends SparkFunSuite { } checkError( exception = duplicate, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`c1`")) // Throw ParseException @@ -59,14 +59,14 @@ class JdbcUtilsSuite extends SparkFunSuite { exception = intercept[ParseException]{ JdbcUtils.getCustomSchema(tableSchema, "c3 DATEE, C2 STRING", caseInsensitive) }, - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> "\"DATEE\"")) checkError( exception = intercept[ParseException]{ JdbcUtils.getCustomSchema(tableSchema, "c3 DATE. C2 STRING", caseInsensitive) }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'.'", "hint" -> "")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index e46460dc8d3cb..aea95f0af117a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -1072,7 +1072,7 @@ abstract class JsonSuite .option("mode", "FAILFAST") .json(corruptRecords) }, - errorClass = "_LEGACY_ERROR_TEMP_2165", + condition = "_LEGACY_ERROR_TEMP_2165", parameters = Map("failFastMode" -> "FAILFAST") ) @@ -1084,7 +1084,7 @@ abstract class JsonSuite .json(corruptRecords) .collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[null]", "failFastMode" -> "FAILFAST") @@ -1962,7 +1962,7 @@ abstract class JsonSuite } checkErrorMatchPVals( exception = e, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*${inputFile.getName}.*")) assert(e.getCause.isInstanceOf[EOFException]) assert(e.getCause.getMessage === "Unexpected end of input stream") @@ -1990,7 +1990,7 @@ abstract class JsonSuite exception = intercept[SparkException] { df.collect() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> s".*$dir.*") ) } @@ -2076,7 +2076,7 @@ abstract class JsonSuite .option("mode", "FAILFAST") .json(path) }, - errorClass = "_LEGACY_ERROR_TEMP_2167", + condition = "_LEGACY_ERROR_TEMP_2167", parameters = Map("failFastMode" -> "FAILFAST", "dataType" -> "string|bigint")) val ex = intercept[SparkException] { @@ -2089,11 +2089,11 @@ abstract class JsonSuite } checkErrorMatchPVals( exception = ex, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*$path.*")) checkError( exception = ex.getCause.asInstanceOf[SparkException], - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[null]", "failFastMode" -> "FAILFAST") @@ -2117,7 +2117,7 @@ abstract class JsonSuite .schema(schema) .json(corruptRecords) }, - errorClass = "_LEGACY_ERROR_TEMP_1097", + condition = "_LEGACY_ERROR_TEMP_1097", parameters = Map.empty ) @@ -2134,7 +2134,7 @@ abstract class JsonSuite .json(path) .collect() }, - errorClass = "_LEGACY_ERROR_TEMP_1097", + condition = "_LEGACY_ERROR_TEMP_1097", parameters = Map.empty ) } @@ -2182,7 +2182,7 @@ abstract class JsonSuite .json(Seq(lowerCasedJsonFieldValue._1).toDS()) .collect() }, - errorClass = "MALFORMED_RECORD_IN_PARSING.CANNOT_PARSE_STRING_AS_DATATYPE", + condition = "MALFORMED_RECORD_IN_PARSING.CANNOT_PARSE_STRING_AS_DATATYPE", parameters = Map( "failFastMode" -> "FAILFAST", "badRecord" -> lowerCasedJsonFieldValue._1, @@ -2210,7 +2210,7 @@ abstract class JsonSuite exception = intercept[AnalysisException] { spark.read.schema(schema).json(path).select("_corrupt_record").collect() }, - errorClass = "UNSUPPORTED_FEATURE.QUERY_ONLY_CORRUPT_RECORD_COLUMN", + condition = "UNSUPPORTED_FEATURE.QUERY_ONLY_CORRUPT_RECORD_COLUMN", parameters = Map.empty ) @@ -2378,7 +2378,7 @@ abstract class JsonSuite .json(testFile("test-data/utf16LE.json")) .count() }, - errorClass = "INVALID_PARAMETER_VALUE.CHARSET", + condition = "INVALID_PARAMETER_VALUE.CHARSET", parameters = Map( "charset" -> invalidCharset, "functionName" -> toSQLId("JSONOptionsInRead"), @@ -2412,11 +2412,11 @@ abstract class JsonSuite } checkErrorMatchPVals( exception = exception, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*$fileName.*")) checkError( exception = exception.getCause.asInstanceOf[SparkException], - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map("badRecord" -> "[empty row]", "failFastMode" -> "FAILFAST") ) } @@ -2474,7 +2474,7 @@ abstract class JsonSuite .json(path.getCanonicalPath) } }, - errorClass = "INVALID_PARAMETER_VALUE.CHARSET", + condition = "INVALID_PARAMETER_VALUE.CHARSET", parameters = Map( "charset" -> encoding, "functionName" -> toSQLId("JSONOptions"), @@ -2756,11 +2756,11 @@ abstract class JsonSuite val e = intercept[SparkException] { df.collect() } checkError( exception = e, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map("badRecord" -> "[null]", "failFastMode" -> "FAILFAST")) checkError( exception = e.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "EMPTY_JSON_FIELD_VALUE", + condition = "EMPTY_JSON_FIELD_VALUE", parameters = Map("dataType" -> toSQLType(dataType)) ) } @@ -2901,7 +2901,7 @@ abstract class JsonSuite exception = intercept[SparkUpgradeException] { json.collect() }, - errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER", + condition = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER", parameters = Map( "datetime" -> "'2020-01-27T20:06:11.847-08000'", "config" -> "\"spark.sql.legacy.timeParserPolicy\"")) @@ -3090,7 +3090,7 @@ abstract class JsonSuite } checkErrorMatchPVals( exception = err, - errorClass = "TASK_WRITE_FAILED", + condition = "TASK_WRITE_FAILED", parameters = Map("path" -> s".*${path.getName}.*")) val msg = err.getCause.getMessage @@ -3197,7 +3197,7 @@ abstract class JsonSuite exception = intercept[AnalysisException] { spark.read.json(path.getCanonicalPath).collect() }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`aaa`")) } withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { @@ -3208,7 +3208,7 @@ abstract class JsonSuite exception = intercept[AnalysisException] { readback.filter($"AAA" === 0 && $"bbb" === 1).collect() }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`AAA`", "proposal" -> "`BBB`, `aaa`"), context = ExpectedContext(fragment = "$", callSitePattern = getCurrentClassCallSitePattern)) @@ -3362,13 +3362,13 @@ abstract class JsonSuite checkError( exception = exception, - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map("badRecord" -> "[null]", "failFastMode" -> "FAILFAST") ) checkError( exception = ExceptionUtils.getRootCause(exception).asInstanceOf[SparkRuntimeException], - errorClass = "INVALID_JSON_ROOT_FIELD", + condition = "INVALID_JSON_ROOT_FIELD", parameters = Map.empty ) @@ -3852,7 +3852,7 @@ abstract class JsonSuite exception = intercept[AnalysisException]( spark.read.schema(jsonDataSchema).json(Seq(jsonData).toDS()).collect() ), - errorClass = "INVALID_JSON_SCHEMA_MAP_TYPE", + condition = "INVALID_JSON_SCHEMA_MAP_TYPE", parameters = Map("jsonSchema" -> toSQLType(jsonDataSchema))) val jsonDir = new File(dir, "json").getCanonicalPath @@ -3862,7 +3862,7 @@ abstract class JsonSuite exception = intercept[AnalysisException]( spark.read.schema(jsonDirSchema).json(jsonDir).collect() ), - errorClass = "INVALID_JSON_SCHEMA_MAP_TYPE", + condition = "INVALID_JSON_SCHEMA_MAP_TYPE", parameters = Map("jsonSchema" -> toSQLType(jsonDirSchema))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index f1431f2a81b8e..f13d66b76838f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -683,7 +683,7 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"select a from $tableName where a < 0").collect() }, - errorClass = "AMBIGUOUS_REFERENCE", + condition = "AMBIGUOUS_REFERENCE", parameters = Map( "name" -> "`a`", "referenceNames" -> ("[`spark_catalog`.`default`.`spark_32622`.`a`, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index 536db3dfe74b5..2e6413d998d12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -594,7 +594,7 @@ abstract class OrcQueryTest extends OrcTest { exception = intercept[AnalysisException] { testAllCorruptFiles() }, - errorClass = "UNABLE_TO_INFER_SCHEMA", + condition = "UNABLE_TO_INFER_SCHEMA", parameters = Map("format" -> "ORC") ) testAllCorruptFilesWithoutSchemaInfer() @@ -619,7 +619,7 @@ abstract class OrcQueryTest extends OrcTest { exception = intercept[SparkException] { testAllCorruptFiles() }, - errorClass = "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER", + condition = "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER", parameters = Map("path" -> "file:.*") ) val e4 = intercept[SparkException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 5d247c76b70be..9348d10711b35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -235,7 +235,7 @@ abstract class OrcSuite exception = intercept[SparkException] { testMergeSchemasInParallel(false, schemaReader) }.getCause.getCause.asInstanceOf[SparkException], - errorClass = "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER", + condition = "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER", parameters = Map("path" -> "file:.*") ) } @@ -481,7 +481,7 @@ abstract class OrcSuite exception = intercept[SparkException] { spark.read.orc(basePath).columns.length }.getCause.getCause.asInstanceOf[SparkException], - errorClass = "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER", + condition = "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER", parameters = Map("path" -> "file:.*") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala index 28644720d0436..359436ca23636 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala @@ -125,7 +125,7 @@ class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSpar exception = intercept[SparkIllegalArgumentException] { checkCompressionCodec("aa", isPartitioned) }, - errorClass = "CODEC_NOT_AVAILABLE.WITH_AVAILABLE_CODECS_SUGGESTION", + condition = "CODEC_NOT_AVAILABLE.WITH_AVAILABLE_CODECS_SUGGESTION", parameters = Map( "codecName" -> "aa", "availableCodecs" -> ("brotli, uncompressed, lzo, snappy, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala index b0995477030c9..ee283386b8eff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala @@ -65,7 +65,7 @@ abstract class ParquetFileFormatSuite exception = intercept[SparkException] { testReadFooters(false) }.getCause.asInstanceOf[SparkException], - errorClass = "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER", + condition = "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER", parameters = Map("path" -> "file:.*") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileMetadataStructRowIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileMetadataStructRowIndexSuite.scala index 5e59418f8f928..be8d41c75bfde 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileMetadataStructRowIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileMetadataStructRowIndexSuite.scala @@ -130,7 +130,7 @@ class ParquetFileMetadataStructRowIndexSuite extends QueryTest with SharedSparkS exception = intercept[AnalysisException] { df.select("*", s"${FileFormat.METADATA_NAME}.${ROW_INDEX}") }, - errorClass = "FIELD_NOT_FOUND", + condition = "FIELD_NOT_FOUND", parameters = Map( "fieldName" -> "`row_index`", "fields" -> ("`file_path`, `file_name`, `file_size`, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 02e1c70cc8cb7..6c2f5a2d134db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -1068,7 +1068,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession exception = intercept[SparkException] { spark.read.schema(readSchema).parquet(path).collect() }, - errorClass = "FAILED_READ_FILE.PARQUET_COLUMN_DATA_TYPE_MISMATCH", + condition = "FAILED_READ_FILE.PARQUET_COLUMN_DATA_TYPE_MISMATCH", parameters = Map( "path" -> ".*", "column" -> "\\[_1\\]", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 1484511a98b63..52d67a0954325 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -968,7 +968,7 @@ abstract class ParquetPartitionDiscoverySuite PartitionValues(Seq("b"), Seq(TypedPartValue("1", IntegerType)))) ) ), - errorClass = "CONFLICTING_PARTITION_COLUMN_NAMES", + condition = "CONFLICTING_PARTITION_COLUMN_NAMES", parameters = Map( "distinctPartColLists" -> "\n\tPartition column name list #0: a\n\tPartition column name list #1: b\n", @@ -985,7 +985,7 @@ abstract class ParquetPartitionDiscoverySuite PartitionValues(Seq("a"), Seq(TypedPartValue("1", IntegerType)))) ) ), - errorClass = "CONFLICTING_PARTITION_COLUMN_NAMES", + condition = "CONFLICTING_PARTITION_COLUMN_NAMES", parameters = Map( "distinctPartColLists" -> "\n\tPartition column name list #0: a\n", @@ -1003,7 +1003,7 @@ abstract class ParquetPartitionDiscoverySuite Seq(TypedPartValue("1", IntegerType), TypedPartValue("foo", StringType)))) ) ), - errorClass = "CONFLICTING_PARTITION_COLUMN_NAMES", + condition = "CONFLICTING_PARTITION_COLUMN_NAMES", parameters = Map( "distinctPartColLists" -> "\n\tPartition column name list #0: a\n\tPartition column name list #1: a, b\n", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 074781da830fe..0acb21f3e6fba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -989,7 +989,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { exception = intercept[SparkException] { spark.read.option("mergeSchema", "true").parquet(path) }, - errorClass = "CANNOT_MERGE_SCHEMAS", + condition = "CANNOT_MERGE_SCHEMAS", sqlState = "42KD9", parameters = Map( "left" -> toSQLType(df1.schema), @@ -1056,7 +1056,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { if (col(0).dataType == StringType) { checkErrorMatchPVals( exception = e, - errorClass = "FAILED_READ_FILE.PARQUET_COLUMN_DATA_TYPE_MISMATCH", + condition = "FAILED_READ_FILE.PARQUET_COLUMN_DATA_TYPE_MISMATCH", parameters = Map( "path" -> s".*${dir.getCanonicalPath}.*", "column" -> "\\[a\\]", @@ -1067,7 +1067,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { } else { checkErrorMatchPVals( exception = e, - errorClass = "FAILED_READ_FILE.PARQUET_COLUMN_DATA_TYPE_MISMATCH", + condition = "FAILED_READ_FILE.PARQUET_COLUMN_DATA_TYPE_MISMATCH", parameters = Map( "path" -> s".*${dir.getCanonicalPath}.*", "column" -> "\\[a\\]", @@ -1115,7 +1115,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { exception = intercept[AnalysisException] { spark.read.parquet(testDataPath).collect() }, - errorClass = "PARQUET_TYPE_ILLEGAL", + condition = "PARQUET_TYPE_ILLEGAL", parameters = Map("parquetType" -> "INT64 (TIMESTAMP(NANOS,true))") ) } @@ -1126,7 +1126,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { exception = intercept[AnalysisException] { spark.read.parquet(testDataPath).collect() }, - errorClass = "PARQUET_TYPE_NOT_SUPPORTED", + condition = "PARQUET_TYPE_NOT_SUPPORTED", parameters = Map("parquetType" -> "FIXED_LEN_BYTE_ARRAY (INTERVAL)") ) } @@ -1139,7 +1139,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { exception = intercept[AnalysisException] { spark.read.parquet(testDataPath).collect() }, - errorClass = "PARQUET_TYPE_NOT_RECOGNIZED", + condition = "PARQUET_TYPE_NOT_RECOGNIZED", parameters = Map("field" -> expectedParameter) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 25aa6def052b8..5c373a2de9738 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -111,7 +111,7 @@ abstract class TextSuite extends QueryTest with SharedSparkSession with CommonFi testDf.write.option("compression", "illegal").mode( SaveMode.Overwrite).text(dir.getAbsolutePath) }, - errorClass = "CODEC_NOT_AVAILABLE.WITH_AVAILABLE_CODECS_SUGGESTION", + condition = "CODEC_NOT_AVAILABLE.WITH_AVAILABLE_CODECS_SUGGESTION", parameters = Map( "codecName" -> "illegal", "availableCodecs" -> "bzip2, deflate, uncompressed, snappy, none, lz4, gzip") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index 50988e133005a..c88f51a6b7d06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -1173,7 +1173,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { exception = intercept[SparkUnsupportedOperationException] { catalog.alterNamespace(testNs, NamespaceChange.removeProperty(p)) }, - errorClass = "_LEGACY_ERROR_TEMP_2069", + condition = "_LEGACY_ERROR_TEMP_2069", parameters = Map("property" -> p)) } @@ -1184,7 +1184,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { val testIdent: IdentifierHelper = Identifier.of(Array("a", "b"), "c") checkError( exception = intercept[AnalysisException](testIdent.asTableIdentifier), - errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", + condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", parameters = Map("identifier" -> "`a`.`b`.`c`") ) } @@ -1193,7 +1193,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { val testIdent: MultipartIdentifierHelper = Seq("a", "b", "c") checkError( exception = intercept[AnalysisException](testIdent.asFunctionIdentifier), - errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", + condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", parameters = Map("identifier" -> "`a`.`b`.`c`") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DerbyTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DerbyTableCatalogSuite.scala index d793ef526c47b..6125777c7a426 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DerbyTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DerbyTableCatalogSuite.scala @@ -45,7 +45,7 @@ class DerbyTableCatalogSuite extends QueryTest with SharedSparkSession { checkError( exception = intercept[SparkUnsupportedOperationException]( sql(s"ALTER TABLE $n1t1 RENAME TO $n2t2")), - errorClass = "CANNOT_RENAME_ACROSS_SCHEMA", + condition = "CANNOT_RENAME_ACROSS_SCHEMA", parameters = Map("type" -> "table")) sql(s"ALTER TABLE $n1t1 RENAME TO $n1t2") checkAnswer(sql(s"SHOW TABLES IN derby.test1"), Row("test1", "TABLE2", false)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index 8e5fba3607b93..580034ff7b0e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -176,7 +176,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { sql("CREATE TABLE h2.bad_test.new_table(i INT, j STRING)") } checkError(exp, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`bad_test`")) } @@ -200,7 +200,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $tableName ADD COLUMNS (c3 DOUBLE)") }, - errorClass = "FIELD_ALREADY_EXISTS", + condition = "FIELD_ALREADY_EXISTS", parameters = Map( "op" -> "add", "fieldNames" -> "`c3`", @@ -239,7 +239,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $tableName RENAME COLUMN C TO C0") }, - errorClass = "FIELD_ALREADY_EXISTS", + condition = "FIELD_ALREADY_EXISTS", parameters = Map( "op" -> "rename", "fieldNames" -> "`C0`", @@ -279,7 +279,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`bad_column`", @@ -316,7 +316,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`bad_column`", @@ -327,7 +327,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[ParseException] { sql(s"ALTER TABLE $tableName ALTER COLUMN id TYPE bad_type") }, - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> "\"BAD_TYPE\""), context = ExpectedContext("bad_type", 51, 58)) } @@ -361,7 +361,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`bad_column`", @@ -393,7 +393,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("CREATE NAMESPACE h2.test_namespace LOCATION './samplepath'") }, - errorClass = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND", + condition = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND", sqlState = "0A000", parameters = Map("cmd" -> toSQLStmt("CREATE NAMESPACE ... LOCATION ..."))) } @@ -416,7 +416,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"ALTER NAMESPACE h2.test_namespace SET LOCATION '/tmp/loc_test_2'") }, - errorClass = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND_WITH_PROPERTY", + condition = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND_WITH_PROPERTY", sqlState = "0A000", parameters = Map( "cmd" -> toSQLStmt("SET NAMESPACE"), @@ -426,7 +426,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"ALTER NAMESPACE h2.test_namespace SET PROPERTIES('a'='b')") }, - errorClass = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND_WITH_PROPERTY", + condition = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND_WITH_PROPERTY", sqlState = "0A000", parameters = Map( "cmd" -> toSQLStmt("SET NAMESPACE"), @@ -444,7 +444,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"ALTER TABLE $tableName ALTER COLUMN ID COMMENT 'test'") }, - errorClass = "_LEGACY_ERROR_TEMP_1305", + condition = "_LEGACY_ERROR_TEMP_1305", parameters = Map("change" -> "org.apache.spark.sql.connector.catalog.TableChange\\$UpdateColumnComment.*"), matchPVals = true) @@ -454,7 +454,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`bad_column`", @@ -490,7 +490,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`C2`", @@ -513,7 +513,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`C3`", @@ -535,7 +535,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`C1`", @@ -557,7 +557,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`C1`", @@ -596,7 +596,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { sql("CREATE TABLE h2.test.new_table(i INT, j STRING)" + " TBLPROPERTIES('ENGINE'='tableEngineName')") }, - errorClass = "FAILED_JDBC.CREATE_TABLE", + condition = "FAILED_JDBC.CREATE_TABLE", parameters = Map( "url" -> "jdbc:.*", "tableName" -> "`test`.`new_table`")) @@ -615,7 +615,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException]{ sql("CREATE TABLE h2.test.new_table(c CHAR(1000000001))") }, - errorClass = "FAILED_JDBC.CREATE_TABLE", + condition = "FAILED_JDBC.CREATE_TABLE", parameters = Map( "url" -> "jdbc:.*", "tableName" -> "`test`.`new_table`")) @@ -626,7 +626,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkIllegalArgumentException]( sql("CREATE TABLE h2.test.new_table(c array)") ), - errorClass = "_LEGACY_ERROR_TEMP_2082", + condition = "_LEGACY_ERROR_TEMP_2082", parameters = Map("catalogString" -> "array") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala index ebead0b663486..91f21c4a2ed34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala @@ -255,7 +255,7 @@ class XmlSuite .option("mode", FailFastMode.name) .xml(inputFile) }, - errorClass = "_LEGACY_ERROR_TEMP_2165", + condition = "_LEGACY_ERROR_TEMP_2165", parameters = Map("failFastMode" -> "FAILFAST") ) val exceptionInParsing = intercept[SparkException] { @@ -268,11 +268,11 @@ class XmlSuite } checkErrorMatchPVals( exception = exceptionInParsing, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*$inputFile.*")) checkError( exception = exceptionInParsing.getCause.asInstanceOf[SparkException], - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[null]", "failFastMode" -> FailFastMode.name) @@ -288,7 +288,7 @@ class XmlSuite .option("mode", FailFastMode.name) .xml(inputFile) }, - errorClass = "_LEGACY_ERROR_TEMP_2165", + condition = "_LEGACY_ERROR_TEMP_2165", parameters = Map("failFastMode" -> "FAILFAST")) val exceptionInParsing = intercept[SparkException] { spark.read @@ -300,11 +300,11 @@ class XmlSuite } checkErrorMatchPVals( exception = exceptionInParsing, - errorClass = "FAILED_READ_FILE.NO_HINT", + condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*$inputFile.*")) checkError( exception = exceptionInParsing.getCause.asInstanceOf[SparkException], - errorClass = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", parameters = Map( "badRecord" -> "[null]", "failFastMode" -> FailFastMode.name) @@ -1315,7 +1315,7 @@ class XmlSuite spark.sql(s"""SELECT schema_of_xml('1', map('mode', 'DROPMALFORMED'))""") .collect() }, - errorClass = "_LEGACY_ERROR_TEMP_1099", + condition = "_LEGACY_ERROR_TEMP_1099", parameters = Map( "funcName" -> "schema_of_xml", "mode" -> "DROPMALFORMED", @@ -1330,7 +1330,7 @@ class XmlSuite spark.sql(s"""SELECT schema_of_xml('1', map('mode', 'FAILFAST'))""") .collect() }, - errorClass = "_LEGACY_ERROR_TEMP_2165", + condition = "_LEGACY_ERROR_TEMP_2165", parameters = Map( "failFastMode" -> FailFastMode.name) ) @@ -1394,7 +1394,7 @@ class XmlSuite exception = intercept[AnalysisException] { df.select(to_xml($"value")).collect() }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"to_xml(value)\"", "paramIndex" -> ordinalNumber(0), @@ -1777,14 +1777,14 @@ class XmlSuite exception = intercept[AnalysisException] { spark.read.xml("/this/file/does/not/exist") }, - errorClass = "PATH_NOT_FOUND", + condition = "PATH_NOT_FOUND", parameters = Map("path" -> "file:/this/file/does/not/exist") ) checkError( exception = intercept[AnalysisException] { spark.read.schema(buildSchema(field("dummy"))).xml("/this/file/does/not/exist") }, - errorClass = "PATH_NOT_FOUND", + condition = "PATH_NOT_FOUND", parameters = Map("path" -> "file:/this/file/does/not/exist") ) } @@ -2498,7 +2498,7 @@ class XmlSuite } checkErrorMatchPVals( exception = err, - errorClass = "TASK_WRITE_FAILED", + condition = "TASK_WRITE_FAILED", parameters = Map("path" -> s".*${path.getName}.*")) val msg = err.getCause.getMessage assert( @@ -2923,7 +2923,7 @@ class XmlSuite exception = intercept[SparkException] { df.collect() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> s".*$dir.*") ) } @@ -3020,7 +3020,7 @@ class XmlSuite } checkErrorMatchPVals( exception = e, - errorClass = "TASK_WRITE_FAILED", + condition = "TASK_WRITE_FAILED", parameters = Map("path" -> s".*${dir.getName}.*")) assert(e.getCause.isInstanceOf[XMLStreamException]) assert(e.getCause.getMessage.contains(errorMsg)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 079ab994736b2..e555033b53055 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -548,7 +548,7 @@ class HashedRelationSuite extends SharedSparkSession { exception = intercept[SparkException] { keyIterator.next() }, - errorClass = "_LEGACY_ERROR_TEMP_2104", + condition = "_LEGACY_ERROR_TEMP_2104", parameters = Map.empty ) assert(buffer.sortWith(_ < _) === randomArray) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala index b207afeae1068..dcebece29037f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala @@ -188,7 +188,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { spark.dataSource.registerPython(dataSourceName, dataSource) checkError( exception = intercept[AnalysisException](spark.read.format(dataSourceName).load()), - errorClass = "INVALID_SCHEMA.NON_STRUCT_TYPE", + condition = "INVALID_SCHEMA.NON_STRUCT_TYPE", parameters = Map("inputSchema" -> "INT", "dataType" -> "\"INT\"")) } @@ -309,7 +309,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { exception = intercept[AnalysisException] { spark.dataSource.registerPython(provider, dataSource) }, - errorClass = "DATA_SOURCE_ALREADY_EXISTS", + condition = "DATA_SOURCE_ALREADY_EXISTS", parameters = Map("provider" -> provider)) } } @@ -657,7 +657,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { exception = intercept[AnalysisException] { spark.range(1).write.format(dataSourceName).save() }, - errorClass = "UNSUPPORTED_DATA_SOURCE_SAVE_MODE", + condition = "UNSUPPORTED_DATA_SOURCE_SAVE_MODE", parameters = Map("source" -> "SimpleDataSource", "createMode" -> "\"ErrorIfExists\"")) } @@ -666,7 +666,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { exception = intercept[AnalysisException] { spark.range(1).write.format(dataSourceName).mode("ignore").save() }, - errorClass = "UNSUPPORTED_DATA_SOURCE_SAVE_MODE", + condition = "UNSUPPORTED_DATA_SOURCE_SAVE_MODE", parameters = Map("source" -> "SimpleDataSource", "createMode" -> "\"Ignore\"")) } @@ -675,7 +675,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { exception = intercept[AnalysisException] { spark.range(1).write.format(dataSourceName).mode("foo").save() }, - errorClass = "INVALID_SAVE_MODE", + condition = "INVALID_SAVE_MODE", parameters = Map("mode" -> "\"foo\"")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala index 8d62d4747198b..8d0e1c5f578fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala @@ -258,7 +258,7 @@ class PythonStreamingDataSourceSimpleSuite extends PythonDataSourceSuiteBase { } checkErrorMatchPVals( err, - errorClass = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", + condition = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", parameters = Map( "action" -> action, "msg" -> "(.|\\n)*" @@ -324,7 +324,7 @@ class PythonStreamingDataSourceSimpleSuite extends PythonDataSourceSuiteBase { } checkErrorMatchPVals( err, - errorClass = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", + condition = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", parameters = Map( "action" -> action, "msg" -> "(.|\\n)*" @@ -661,7 +661,7 @@ class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase { } checkErrorMatchPVals( err, - errorClass = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", + condition = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", parameters = Map( "action" -> action, "msg" -> "(.|\\n)*" @@ -723,7 +723,7 @@ class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase { } checkErrorMatchPVals( err, - errorClass = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", + condition = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", parameters = Map( "action" -> action, "msg" -> "(.|\\n)*" @@ -1035,7 +1035,7 @@ class PythonStreamingDataSourceWriteSuite extends PythonDataSourceSuiteBase { exception = intercept[AnalysisException] { runQuery("complete") }, - errorClass = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", + condition = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", sqlState = "42KDE", parameters = Map( "outputMode" -> "complete", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala index f82b544ecf120..0339f7461f0a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala @@ -96,7 +96,7 @@ class PythonUDFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.agg(testUdf(df("v"))).collect() }, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE", parameters = Map("sqlExpr" -> "\"pandas_udf(v)\"", "dataType" -> "VARIANT")) } @@ -110,7 +110,7 @@ class PythonUDFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.agg(testUdf(df("arr_v"))).collect() }, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE", parameters = Map("sqlExpr" -> "\"pandas_udf(arr_v)\"", "dataType" -> "ARRAY")) } @@ -123,7 +123,7 @@ class PythonUDFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.agg(testUdf(df("id"))).collect() }, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_OUTPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_OUTPUT_TYPE", parameters = Map("sqlExpr" -> "\"pandas_udf(id)\"", "dataType" -> "VARIANT")) } @@ -136,7 +136,7 @@ class PythonUDFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { df.agg(testUdf(df("id"))).collect() }, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_OUTPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_OUTPUT_TYPE", parameters = Map( "sqlExpr" -> "\"pandas_udf(id)\"", "dataType" -> "STRUCT>")) @@ -175,7 +175,7 @@ class PythonUDFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.range(1).select(transform(array("id"), x => pythonTestUDF(x))).collect() }, - errorClass = "UNSUPPORTED_FEATURE.LAMBDA_FUNCTION_WITH_PYTHON_UDF", + condition = "UNSUPPORTED_FEATURE.LAMBDA_FUNCTION_WITH_PYTHON_UDF", parameters = Map("funcName" -> "\"pyUDF(namedlambdavariable())\""), context = ExpectedContext( "transform", s".*${this.getClass.getSimpleName}.*")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala index 36a9c3c40e3e6..041bd143067a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala @@ -133,7 +133,7 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.sql("select udtf.* from t, lateral variantInputUDTF(v) udtf").collect() }, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE", parameters = Map( "sqlExpr" -> """"InputVariantUDTF\(outer\(v#\d+\)\)"""", "dataType" -> "VARIANT"), @@ -156,7 +156,7 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.sql("select udtf.* from t, lateral variantInputUDTF(map_v) udtf").collect() }, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE", parameters = Map( "sqlExpr" -> """"InputVariantUDTF\(outer\(map_v#\d+\)\)"""", "dataType" -> "MAP"), @@ -175,7 +175,7 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.sql("select * from variantOutUDTF()").collect() }, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_OUTPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_OUTPUT_TYPE", parameters = Map( "sqlExpr" -> "\"SimpleOutputVariantUDTF()\"", "dataType" -> "VARIANT"), @@ -192,7 +192,7 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.sql("select * from arrayOfVariantOutUDTF()").collect() }, - errorClass = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_OUTPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNSUPPORTED_UDF_OUTPUT_TYPE", parameters = Map( "sqlExpr" -> "\"OutputArrayOfVariantUDTF()\"", "dataType" -> "ARRAY"), @@ -488,7 +488,7 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { | WITH SINGLE PARTITION | ORDER BY device_id, data_ds) |""".stripMargin)), - errorClass = "_LEGACY_ERROR_TEMP_0064", + condition = "_LEGACY_ERROR_TEMP_0064", parameters = Map("msg" -> ("The table function call includes a table argument with an invalid " + "partitioning/ordering specification: the ORDER BY clause included multiple " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala index 42eb9fa17a210..808ffe036f89d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala @@ -247,7 +247,7 @@ class CompactibleFileStreamLogSuite extends SharedSparkSession { exception = intercept[SparkUnsupportedOperationException] { compactibleLog.purge(2) }, - errorClass = "_LEGACY_ERROR_TEMP_2260", + condition = "_LEGACY_ERROR_TEMP_2260", parameters = Map.empty) // Below line would fail with IllegalStateException if we don't prevent purge: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MergingSessionsIteratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MergingSessionsIteratorSuite.scala index e550d8ef46085..8ed63f5680b0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MergingSessionsIteratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MergingSessionsIteratorSuite.scala @@ -197,7 +197,7 @@ class MergingSessionsIteratorSuite extends SharedSparkSession { exception = intercept[SparkException] { iterator.next() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "Input iterator is not sorted based on session!")) // afterwards, calling either hasNext() or next() will throw IllegalStateException @@ -205,14 +205,14 @@ class MergingSessionsIteratorSuite extends SharedSparkSession { exception = intercept[SparkException] { iterator.hasNext }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "The iterator is already corrupted.")) checkError( exception = intercept[SparkException] { iterator.next() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "The iterator is already corrupted.")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/UpdatingSessionsIteratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/UpdatingSessionsIteratorSuite.scala index 88af5cfddb487..187eda5d36f61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/UpdatingSessionsIteratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/UpdatingSessionsIteratorSuite.scala @@ -270,7 +270,7 @@ class UpdatingSessionsIteratorSuite extends SharedSparkSession { exception = intercept[SparkException] { iterator.next() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "The iterator must be sorted by key and session start!")) // afterwards, calling either hasNext() or next() will throw IllegalStateException @@ -278,14 +278,14 @@ class UpdatingSessionsIteratorSuite extends SharedSparkSession { exception = intercept[SparkException] { iterator.hasNext }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "The iterator is already corrupted.")) checkError( exception = intercept[SparkException] { iterator.next() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "The iterator is already corrupted.")) } @@ -312,7 +312,7 @@ class UpdatingSessionsIteratorSuite extends SharedSparkSession { exception = intercept[SparkException] { iterator.next() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "The iterator must be sorted by key and session start!")) // afterwards, calling either hasNext() or next() will throw IllegalStateException @@ -320,14 +320,14 @@ class UpdatingSessionsIteratorSuite extends SharedSparkSession { exception = intercept[SparkException] { iterator.hasNext }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "The iterator is already corrupted.")) checkError( exception = intercept[SparkException] { iterator.next() }, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map("message" -> "The iterator is already corrupted.")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala index 4cf82cebeb812..32f92ce276a06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala @@ -310,7 +310,7 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA val errorEvent = allEvents(0)(1).asInstanceOf[ForeachWriterSuite.Close] checkError( exception = errorEvent.error.get.asInstanceOf[SparkException], - errorClass = "_LEGACY_ERROR_TEMP_2256", + condition = "_LEGACY_ERROR_TEMP_2256", parameters = Map.empty ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RatePerMicroBatchProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RatePerMicroBatchProviderSuite.scala index 128b59b26b823..aa03545625ec9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RatePerMicroBatchProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RatePerMicroBatchProviderSuite.scala @@ -202,7 +202,7 @@ class RatePerMicroBatchProviderSuite extends StreamTest { .schema(spark.range(1).schema) .load() }, - errorClass = "_LEGACY_ERROR_TEMP_2242", + condition = "_LEGACY_ERROR_TEMP_2242", parameters = Map("provider" -> "RatePerMicroBatchProvider")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala index 0732e126a0131..aeb1bba31410d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala @@ -208,7 +208,7 @@ class RateStreamProviderSuite extends StreamTest { checkError( exception = e, - errorClass = "INCORRECT_RAMP_UP_RATE", + condition = "INCORRECT_RAMP_UP_RATE", parameters = Map( "rowsPerSecond" -> Long.MaxValue.toString, "maxSeconds" -> "1", @@ -229,7 +229,7 @@ class RateStreamProviderSuite extends StreamTest { checkError( exception = e, - errorClass = "INTERNAL_ERROR", + condition = "INTERNAL_ERROR", parameters = Map( ("message" -> ("Max offset with 100 rowsPerSecond is 92233720368547758, " + @@ -352,7 +352,7 @@ class RateStreamProviderSuite extends StreamTest { .schema(spark.range(1).schema) .load() }, - errorClass = "_LEGACY_ERROR_TEMP_2242", + condition = "_LEGACY_ERROR_TEMP_2242", parameters = Map("provider" -> "RateStreamProvider")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala index 87e34601dc098..2c17d75624d38 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala @@ -198,7 +198,7 @@ class TextSocketStreamSuite extends StreamTest with SharedSparkSession { exception = intercept[SparkUnsupportedOperationException] { spark.readStream.schema(userSpecifiedSchema).format("socket").options(params).load() }, - errorClass = "_LEGACY_ERROR_TEMP_2242", + condition = "_LEGACY_ERROR_TEMP_2242", parameters = Map("provider" -> "TextSocketSourceProvider")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala index 48816486cbd00..add12f7e15352 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala @@ -49,7 +49,7 @@ class ListStateSuite extends StateVariableSuiteBase { checkError( exception = e, - errorClass = "ILLEGAL_STATE_STORE_VALUE.NULL_VALUE", + condition = "ILLEGAL_STATE_STORE_VALUE.NULL_VALUE", sqlState = Some("42601"), parameters = Map("stateName" -> "listState") ) @@ -234,7 +234,7 @@ class ListStateSuite extends StateVariableSuiteBase { checkError( ex, - errorClass = "STATEFUL_PROCESSOR_TTL_DURATION_MUST_BE_POSITIVE", + condition = "STATEFUL_PROCESSOR_TTL_DURATION_MUST_BE_POSITIVE", parameters = Map( "operationType" -> "update", "stateName" -> "testState" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala index b5ba25518a5ea..9c322b201da8c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala @@ -245,7 +245,7 @@ class MapStateSuite extends StateVariableSuiteBase { checkError( ex, - errorClass = "STATEFUL_PROCESSOR_TTL_DURATION_MUST_BE_POSITIVE", + condition = "STATEFUL_PROCESSOR_TTL_DURATION_MUST_BE_POSITIVE", parameters = Map( "operationType" -> "update", "stateName" -> "testState" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala index 0bd86068ca3f6..07130f94ede62 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala @@ -174,7 +174,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } checkError( ex1, - errorClass = "STATE_STORE_INCORRECT_NUM_ORDERING_COLS_FOR_RANGE_SCAN", + condition = "STATE_STORE_INCORRECT_NUM_ORDERING_COLS_FOR_RANGE_SCAN", parameters = Map( "numOrderingCols" -> "0" ), @@ -193,7 +193,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } checkError( ex2, - errorClass = "STATE_STORE_INCORRECT_NUM_ORDERING_COLS_FOR_RANGE_SCAN", + condition = "STATE_STORE_INCORRECT_NUM_ORDERING_COLS_FOR_RANGE_SCAN", parameters = Map( "numOrderingCols" -> (keySchemaWithRangeScan.length + 1).toString ), @@ -215,7 +215,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } checkError( ex, - errorClass = "STATE_STORE_VARIABLE_SIZE_ORDERING_COLS_NOT_SUPPORTED", + condition = "STATE_STORE_VARIABLE_SIZE_ORDERING_COLS_NOT_SUPPORTED", parameters = Map( "fieldName" -> keySchemaWithVariableSizeCols.fields(0).name, "index" -> "0" @@ -253,7 +253,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } checkError( ex, - errorClass = "STATE_STORE_VARIABLE_SIZE_ORDERING_COLS_NOT_SUPPORTED", + condition = "STATE_STORE_VARIABLE_SIZE_ORDERING_COLS_NOT_SUPPORTED", parameters = Map( "fieldName" -> field.name, "index" -> index.toString @@ -278,7 +278,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } checkError( ex, - errorClass = "STATE_STORE_NULL_TYPE_ORDERING_COLS_NOT_SUPPORTED", + condition = "STATE_STORE_NULL_TYPE_ORDERING_COLS_NOT_SUPPORTED", parameters = Map( "fieldName" -> keySchemaWithNullTypeCols.fields(0).name, "index" -> "0" @@ -934,7 +934,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (!colFamiliesEnabled) { checkError( ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + condition = "STATE_STORE_UNSUPPORTED_OPERATION", parameters = Map( "operationType" -> "create_col_family", "entity" -> "multiple column families is disabled in RocksDBStateStoreProvider" @@ -944,7 +944,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } else { checkError( ex, - errorClass = "STATE_STORE_CANNOT_USE_COLUMN_FAMILY_WITH_INVALID_NAME", + condition = "STATE_STORE_CANNOT_USE_COLUMN_FAMILY_WITH_INVALID_NAME", parameters = Map( "operationName" -> "create_col_family", "colFamilyName" -> colFamilyName @@ -971,7 +971,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (!colFamiliesEnabled) { checkError( ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + condition = "STATE_STORE_UNSUPPORTED_OPERATION", parameters = Map( "operationType" -> "create_col_family", "entity" -> "multiple column families is disabled in RocksDBStateStoreProvider" @@ -981,7 +981,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } else { checkError( ex, - errorClass = "STATE_STORE_CANNOT_CREATE_COLUMN_FAMILY_WITH_RESERVED_CHARS", + condition = "STATE_STORE_CANNOT_CREATE_COLUMN_FAMILY_WITH_RESERVED_CHARS", parameters = Map( "colFamilyName" -> colFamilyName ), @@ -1073,7 +1073,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } checkError( exception = e.asInstanceOf[StateStoreUnsupportedOperationOnMissingColumnFamily], - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", + condition = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", sqlState = Some("42802"), parameters = Map("operationType" -> "get", "colFamilyName" -> colFamily1) ) @@ -1221,7 +1221,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid checkError( exception = e.asInstanceOf[StateStoreUnsupportedOperationOnMissingColumnFamily], - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", + condition = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", sqlState = Some("42802"), parameters = Map("operationType" -> "iterator", "colFamilyName" -> cfName) ) @@ -1241,7 +1241,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (!colFamiliesEnabled) { checkError( ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + condition = "STATE_STORE_UNSUPPORTED_OPERATION", parameters = Map( "operationType" -> operationName, "entity" -> "multiple column families is disabled in RocksDBStateStoreProvider" @@ -1251,7 +1251,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } else { checkError( ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", + condition = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", parameters = Map( "operationType" -> operationName, "colFamilyName" -> colFamilyName diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index d07ce07c41e5c..7ac574db98d45 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -202,7 +202,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } checkError( ex, - errorClass = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", + condition = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", parameters = Map("version" -> "-1") ) ex = intercept[SparkException] { @@ -210,7 +210,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } checkError( ex, - errorClass = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", + condition = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", parameters = Map("version" -> "-1") ) @@ -222,7 +222,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } checkError( ex, - errorClass = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_STREAMING_STATE_FILE", + condition = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_STREAMING_STATE_FILE", parameters = Map( "fileToRead" -> s"$remoteDir/1.changelog" ) @@ -1107,7 +1107,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } checkError( ex, - errorClass = "CANNOT_LOAD_STATE_STORE.UNRELEASED_THREAD_ERROR", + condition = "CANNOT_LOAD_STATE_STORE.UNRELEASED_THREAD_ERROR", parameters = Map( "loggingId" -> "\\[Thread-\\d+\\]", "operationType" -> "load_store", @@ -1135,7 +1135,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } checkError( ex, - errorClass = "CANNOT_LOAD_STATE_STORE.UNRELEASED_THREAD_ERROR", + condition = "CANNOT_LOAD_STATE_STORE.UNRELEASED_THREAD_ERROR", parameters = Map( "loggingId" -> "\\[Thread-\\d+\\]", "operationType" -> "load_store", @@ -1187,7 +1187,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } checkError( e, - errorClass = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_CHECKPOINT", + condition = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_CHECKPOINT", parameters = Map( "expectedVersion" -> "v2", "actualVersion" -> "v1" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index b41fb91dd5d01..8bbc7a31760d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -175,7 +175,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] } checkError( ex, - errorClass = "UNSUPPORTED_FEATURE.STATE_STORE_MULTIPLE_COLUMN_FAMILIES", + condition = "UNSUPPORTED_FEATURE.STATE_STORE_MULTIPLE_COLUMN_FAMILIES", parameters = Map( "stateStoreProvider" -> "HDFSBackedStateStoreProvider" ), @@ -187,7 +187,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] } checkError( ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + condition = "STATE_STORE_UNSUPPORTED_OPERATION", parameters = Map( "operationType" -> operationName, "entity" -> "HDFSBackedStateStoreProvider" @@ -241,7 +241,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] } checkError( ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + condition = "STATE_STORE_UNSUPPORTED_OPERATION", parameters = Map( "operationType" -> "Range scan", "entity" -> "HDFSBackedStateStoreProvider" @@ -373,7 +373,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] } checkError( e, - errorClass = "CANNOT_LOAD_STATE_STORE.UNCATEGORIZED", + condition = "CANNOT_LOAD_STATE_STORE.UNCATEGORIZED", parameters = Map.empty ) @@ -385,7 +385,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] } checkError( e, - errorClass = "CANNOT_LOAD_STATE_STORE.UNCATEGORIZED", + condition = "CANNOT_LOAD_STATE_STORE.UNCATEGORIZED", parameters = Map.empty ) @@ -396,7 +396,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] } checkError( e, - errorClass = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_DELTA_FILE_NOT_EXISTS", + condition = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_DELTA_FILE_NOT_EXISTS", parameters = Map( "fileToRead" -> s"${provider.stateStoreId.storeCheckpointLocation()}/1.delta", "clazz" -> s"${provider.toString()}" @@ -1273,21 +1273,21 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] if (version < 0) { checkError( e, - errorClass = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", + condition = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", parameters = Map("version" -> version.toString) ) } else { if (isHDFSBackedStoreProvider) { checkError( e, - errorClass = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_DELTA_FILE_NOT_EXISTS", + condition = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_DELTA_FILE_NOT_EXISTS", parameters = Map("fileToRead" -> ".*", "clazz" -> ".*"), matchPVals = true ) } else { checkError( e, - errorClass = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_STREAMING_STATE_FILE", + condition = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_STREAMING_STATE_FILE", parameters = Map("fileToRead" -> ".*"), matchPVals = true ) @@ -1478,7 +1478,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] } checkError( e, - errorClass = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", + condition = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", parameters = Map( "version" -> "-1" ) @@ -1493,7 +1493,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] } checkError( e, - errorClass = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_DELTA_FILE_NOT_EXISTS", + condition = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_DELTA_FILE_NOT_EXISTS", parameters = Map( "fileToRead" -> s"$dir/0/0/1.delta", "clazz" -> "HDFSStateStoreProvider\\[.+\\]" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala index 52bdb0213c7e5..e2940497e911e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala @@ -68,7 +68,7 @@ class StatefulProcessorHandleSuite extends StateVariableSuiteBase { } checkError( ex, - errorClass = "STATEFUL_PROCESSOR_CANNOT_PERFORM_OPERATION_WITH_INVALID_HANDLE_STATE", + condition = "STATEFUL_PROCESSOR_CANNOT_PERFORM_OPERATION_WITH_INVALID_HANDLE_STATE", parameters = Map( "operationType" -> operationType, "handleState" -> handleState.toString @@ -116,7 +116,7 @@ class StatefulProcessorHandleSuite extends StateVariableSuiteBase { checkError( ex, - errorClass = "STATEFUL_PROCESSOR_CANNOT_PERFORM_OPERATION_WITH_INVALID_TIME_MODE", + condition = "STATEFUL_PROCESSOR_CANNOT_PERFORM_OPERATION_WITH_INVALID_TIME_MODE", parameters = Map( "operationType" -> "register_timer", "timeMode" -> TimeMode.None().toString @@ -130,7 +130,7 @@ class StatefulProcessorHandleSuite extends StateVariableSuiteBase { checkError( ex2, - errorClass = "STATEFUL_PROCESSOR_CANNOT_PERFORM_OPERATION_WITH_INVALID_TIME_MODE", + condition = "STATEFUL_PROCESSOR_CANNOT_PERFORM_OPERATION_WITH_INVALID_TIME_MODE", parameters = Map( "operationType" -> "delete_timer", "timeMode" -> TimeMode.None().toString diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala index 7d5b3e4a6bdc9..e9136623bd8e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala @@ -61,7 +61,7 @@ class ValueStateSuite extends StateVariableSuiteBase { assert(ex.isInstanceOf[SparkException]) checkError( ex.asInstanceOf[SparkException], - errorClass = "INTERNAL_ERROR_TWS", + condition = "INTERNAL_ERROR_TWS", parameters = Map( "message" -> s"Implicit key not found in state store for stateName=$stateName" ), @@ -80,7 +80,7 @@ class ValueStateSuite extends StateVariableSuiteBase { } checkError( ex1.asInstanceOf[SparkException], - errorClass = "INTERNAL_ERROR_TWS", + condition = "INTERNAL_ERROR_TWS", parameters = Map( "message" -> s"Implicit key not found in state store for stateName=$stateName" ), @@ -172,7 +172,7 @@ class ValueStateSuite extends StateVariableSuiteBase { } checkError( ex, - errorClass = "STATE_STORE_CANNOT_CREATE_COLUMN_FAMILY_WITH_RESERVED_CHARS", + condition = "STATE_STORE_CANNOT_CREATE_COLUMN_FAMILY_WITH_RESERVED_CHARS", parameters = Map( "colFamilyName" -> cfName ), @@ -192,7 +192,7 @@ class ValueStateSuite extends StateVariableSuiteBase { } checkError( ex, - errorClass = "UNSUPPORTED_FEATURE.STATE_STORE_MULTIPLE_COLUMN_FAMILIES", + condition = "UNSUPPORTED_FEATURE.STATE_STORE_MULTIPLE_COLUMN_FAMILIES", parameters = Map( "stateStoreProvider" -> "HDFSBackedStateStoreProvider" ), @@ -377,7 +377,7 @@ class ValueStateSuite extends StateVariableSuiteBase { checkError( ex, - errorClass = "STATEFUL_PROCESSOR_TTL_DURATION_MUST_BE_POSITIVE", + condition = "STATEFUL_PROCESSOR_TTL_DURATION_MUST_BE_POSITIVE", parameters = Map( "operationType" -> "update", "stateName" -> "testState" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala index 111e233c04e32..8c10d646e935c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala @@ -127,7 +127,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser { exception = intercept[SparkRuntimeException] { spark.sql(s"SELECT raise_error('$errorMsg')").collect() }, - errorClass = "USER_RAISED_EXCEPTION", + condition = "USER_RAISED_EXCEPTION", parameters = Map("errorMessage" -> escape)) eventually(timeout(10.seconds), interval(100.milliseconds)) { val summary = findErrorSummaryOnSQLUI() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala index e92428f371e05..898aeec22ad17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala @@ -67,7 +67,7 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { new ExpressionInfo( "testClass", null, "testName", null, "", "", "", invalidGroupName, "", "", "") }, - errorClass = "_LEGACY_ERROR_TEMP_3202", + condition = "_LEGACY_ERROR_TEMP_3202", parameters = Map( "exprName" -> "testName", "group" -> invalidGroupName, @@ -91,7 +91,7 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { new ExpressionInfo( "testClass", null, "testName", null, "", "", "", "", "", "", invalidSource) }, - errorClass = "_LEGACY_ERROR_TEMP_3203", + condition = "_LEGACY_ERROR_TEMP_3203", parameters = Map( "exprName" -> "testName", "source" -> invalidSource, @@ -104,7 +104,7 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { exception = intercept[SparkIllegalArgumentException] { new ExpressionInfo("testClass", null, "testName", null, "", "", invalidNote, "", "", "", "") }, - errorClass = "_LEGACY_ERROR_TEMP_3201", + condition = "_LEGACY_ERROR_TEMP_3201", parameters = Map("exprName" -> "testName", "note" -> invalidNote)) val invalidSince = "-3.0.0" @@ -113,7 +113,7 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { new ExpressionInfo( "testClass", null, "testName", null, "", "", "", "", invalidSince, "", "") }, - errorClass = "_LEGACY_ERROR_TEMP_3204", + condition = "_LEGACY_ERROR_TEMP_3204", parameters = Map("since" -> invalidSince, "exprName" -> "testName")) val invalidDeprecated = " invalid deprecated" @@ -122,7 +122,7 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { new ExpressionInfo( "testClass", null, "testName", null, "", "", "", "", "", invalidDeprecated, "") }, - errorClass = "_LEGACY_ERROR_TEMP_3205", + condition = "_LEGACY_ERROR_TEMP_3205", parameters = Map("exprName" -> "testName", "deprecated" -> invalidDeprecated)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index 7c929b5da872a..3d4f7c8df3bae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -778,7 +778,7 @@ class CatalogSuite extends SharedSparkSession with AnalysisTest with BeforeAndAf exception = intercept[AnalysisException] { spark.catalog.recoverPartitions("my_temp_table") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> "`my_temp_table`", "operation" -> "recoverPartitions()") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 05cd9800bdf21..d0d4dc6b344fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -204,7 +204,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { // spark core conf w/ entry registered checkError( exception = intercept[AnalysisException](sql("RESET spark.executor.cores")), - errorClass = "CANNOT_MODIFY_CONFIG", + condition = "CANNOT_MODIFY_CONFIG", parameters = Map("key" -> "\"spark.executor.cores\"", "docroot" -> SPARK_DOC_ROOT) ) @@ -233,7 +233,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { // static sql configs checkError( exception = intercept[AnalysisException](sql(s"RESET ${StaticSQLConf.WAREHOUSE_PATH.key}")), - errorClass = "_LEGACY_ERROR_TEMP_1325", + condition = "_LEGACY_ERROR_TEMP_1325", parameters = Map("key" -> "spark.sql.warehouse.dir")) } @@ -450,7 +450,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkIllegalArgumentException] { spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, invalidTz) }, - errorClass = "INVALID_CONF_VALUE.TIME_ZONE", + condition = "INVALID_CONF_VALUE.TIME_ZONE", parameters = Map( "confValue" -> invalidTz, "confName" -> SQLConf.SESSION_LOCAL_TIMEZONE.key)) @@ -469,7 +469,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkIllegalArgumentException] { sql(s"SET TIME ZONE '$tz'").collect() }, - errorClass = "INVALID_CONF_VALUE.TIME_ZONE", + condition = "INVALID_CONF_VALUE.TIME_ZONE", parameters = Map( "confValue" -> tz, "confName" -> SQLConf.SESSION_LOCAL_TIMEZONE.key)) @@ -486,7 +486,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { val sqlText = "set time zone interval 19 hours" checkError( exception = intercept[ParseException](sql(sqlText)), - errorClass = "_LEGACY_ERROR_TEMP_0044", + condition = "_LEGACY_ERROR_TEMP_0044", parameters = Map.empty, context = ExpectedContext(sqlText, 0, 30)) } @@ -511,7 +511,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkIllegalArgumentException] { sql(s"SET COLLATION unicode_c").collect() }, - errorClass = "INVALID_CONF_VALUE.DEFAULT_COLLATION", + condition = "INVALID_CONF_VALUE.DEFAULT_COLLATION", parameters = Map( "confValue" -> "UNICODE_C", "confName" -> "spark.sql.session.collation.default", @@ -522,7 +522,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { test("SPARK-43028: config not found error") { checkError( exception = intercept[SparkNoSuchElementException](spark.conf.get("some.conf")), - errorClass = "SQL_CONF_NOT_FOUND", + condition = "SQL_CONF_NOT_FOUND", parameters = Map("sqlConf" -> "\"some.conf\"")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala index d3154d0125af8..b323c6366f520 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala @@ -61,7 +61,7 @@ class SharedStateSuite extends SharedSparkSession { exception = intercept[SparkException] { spark.sharedState.externalCatalog }, - errorClass = "DEFAULT_DATABASE_NOT_EXISTS", + condition = "DEFAULT_DATABASE_NOT_EXISTS", parameters = Map("defaultDatabase" -> "default_database_not_exists") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 77997f95188c0..e8a8a0ae47bfd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -1520,7 +1520,7 @@ class JDBCSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkSQLException] { spark.read.jdbc(urlWithUserAndPass, "TEST.ARRAY_TABLE", new Properties()).collect() }, - errorClass = "UNRECOGNIZED_SQL_TYPE", + condition = "UNRECOGNIZED_SQL_TYPE", parameters = Map("typeName" -> "INTEGER ARRAY", "jdbcType" -> "ARRAY")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala index 780cc86bb6a61..054c7e644ff55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala @@ -399,7 +399,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel exception = intercept[AnalysisException] { df.collect() }, - errorClass = "NULL_DATA_SOURCE_OPTION", + condition = "NULL_DATA_SOURCE_OPTION", parameters = Map( "option" -> "pushDownOffset") ) @@ -2943,7 +2943,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel exception = intercept[AnalysisException] { checkAnswer(sql("SELECT h2.test.my_avg2(id) FROM h2.test.people"), Seq.empty) }, - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`h2`.`test`.`my_avg2`", "searchPath" -> "[`system`.`builtin`, `system`.`session`, `h2`.`default`]"), @@ -2955,7 +2955,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel exception = intercept[AnalysisException] { checkAnswer(sql("SELECT h2.my_avg2(id) FROM h2.test.people"), Seq.empty) }, - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`h2`.`my_avg2`", "searchPath" -> "[`system`.`builtin`, `system`.`session`, `h2`.`default`]"), @@ -3038,7 +3038,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel exception = intercept[IndexAlreadyExistsException] { sql(s"CREATE INDEX people_index ON TABLE h2.test.people (id)") }, - errorClass = "INDEX_ALREADY_EXISTS", + condition = "INDEX_ALREADY_EXISTS", parameters = Map( "indexName" -> "`people_index`", "tableName" -> "`test`.`people`" @@ -3056,7 +3056,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel exception = intercept[NoSuchIndexException] { sql(s"DROP INDEX people_index ON TABLE h2.test.people") }, - errorClass = "INDEX_NOT_FOUND", + condition = "INDEX_NOT_FOUND", parameters = Map("indexName" -> "`people_index`", "tableName" -> "`test`.`people`") ) assert(jdbcTable.indexExists("people_index") == false) @@ -3073,7 +3073,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel exception = intercept[AnalysisException] { sql("SELECT * FROM h2.test.people where h2.db_name.schema_name.function_name()") }, - errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", + condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", sqlState = "42601", parameters = Map("identifier" -> "`db_name`.`schema_name`.`function_name`") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 76a092b552f98..e7044ea50f54f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -191,7 +191,7 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { exception = intercept[AnalysisException] { df2.write.mode(SaveMode.Append).jdbc(url, "TEST.APPENDTEST", new Properties()) }, - errorClass = "_LEGACY_ERROR_TEMP_1156", + condition = "_LEGACY_ERROR_TEMP_1156", parameters = Map( "colName" -> "NAME", "tableSchema" -> @@ -224,7 +224,7 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { df3.write.mode(SaveMode.Overwrite).option("truncate", true) .jdbc(url1, "TEST.TRUNCATETEST", properties) }, - errorClass = "_LEGACY_ERROR_TEMP_1156", + condition = "_LEGACY_ERROR_TEMP_1156", parameters = Map( "colName" -> "seq", "tableSchema" -> @@ -256,7 +256,7 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { exception = intercept[AnalysisException] { df2.write.mode(SaveMode.Append).jdbc(url, "TEST.INCOMPATIBLETEST", new Properties()) }, - errorClass = "_LEGACY_ERROR_TEMP_1156", + condition = "_LEGACY_ERROR_TEMP_1156", parameters = Map( "colName" -> "seq", "tableSchema" -> @@ -507,7 +507,7 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { .option("createTableColumnTypes", "name CLOB(2000)") .jdbc(url1, "TEST.USERDBTYPETEST", properties) }, - errorClass = "UNSUPPORTED_DATATYPE", + condition = "UNSUPPORTED_DATATYPE", parameters = Map("typeName" -> "\"CLOB(2000)\"")) } @@ -519,7 +519,7 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { .option("createTableColumnTypes", "`name char(20)") // incorrectly quoted column .jdbc(url1, "TEST.USERDBTYPETEST", properties) }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'`'", "hint" -> "")) } @@ -533,7 +533,7 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { } checkError( exception = e, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`name`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala index 4b47529591c04..a45cd0bf010b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala @@ -190,7 +190,7 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { } checkError( exception = e, - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", sqlState = "42703", parameters = Map("objectName" -> s"`$varName`"), context = ExpectedContext( @@ -382,7 +382,7 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { exception = intercept[SqlScriptingException] ( runSqlScript(commands) ), - errorClass = "INVALID_BOOLEAN_STATEMENT", + condition = "INVALID_BOOLEAN_STATEMENT", parameters = Map("invalidStatement" -> "1") ) } @@ -404,7 +404,7 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { exception = intercept[SqlScriptingException] ( runSqlScript(commands1) ), - errorClass = "BOOLEAN_STATEMENT_WITH_EMPTY_ROW", + condition = "BOOLEAN_STATEMENT_WITH_EMPTY_ROW", parameters = Map("invalidStatement" -> "(SELECT * FROM T1)") ) @@ -424,7 +424,7 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { exception = intercept[SparkException] ( runSqlScript(commands2) ), - errorClass = "SCALAR_SUBQUERY_TOO_MANY_ROWS", + condition = "SCALAR_SUBQUERY_TOO_MANY_ROWS", parameters = Map.empty, context = ExpectedContext(fragment = "(SELECT * FROM t2)", start = 121, stop = 138) ) @@ -576,7 +576,7 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { exception = intercept[SqlScriptingException] { runSqlScript(sqlScriptText) }, - errorClass = "INVALID_LABEL_USAGE.ITERATE_IN_COMPOUND", + condition = "INVALID_LABEL_USAGE.ITERATE_IN_COMPOUND", parameters = Map("labelName" -> "LBL")) } @@ -615,7 +615,7 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { exception = intercept[SqlScriptingException] { runSqlScript(sqlScriptText) }, - errorClass = "INVALID_LABEL_USAGE.DOES_NOT_EXIST", + condition = "INVALID_LABEL_USAGE.DOES_NOT_EXIST", parameters = Map("labelName" -> "RANDOMLBL", "statementType" -> "LEAVE")) } @@ -630,7 +630,7 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { exception = intercept[SqlScriptingException] { runSqlScript(sqlScriptText) }, - errorClass = "INVALID_LABEL_USAGE.DOES_NOT_EXIST", + condition = "INVALID_LABEL_USAGE.DOES_NOT_EXIST", parameters = Map("labelName" -> "RANDOMLBL", "statementType" -> "ITERATE")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index 4f1b7d363a124..b473716b33fca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -91,7 +91,7 @@ abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { df.write.sortBy("j").saveAsTable("tt") }, - errorClass = "SORT_BY_WITHOUT_BUCKETING", + condition = "SORT_BY_WITHOUT_BUCKETING", parameters = Map.empty) } @@ -106,7 +106,7 @@ abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { df.write.bucketBy(2, "i").parquet("/tmp/path") }, - errorClass = "_LEGACY_ERROR_TEMP_1312", + condition = "_LEGACY_ERROR_TEMP_1312", parameters = Map("operation" -> "save")) } @@ -116,7 +116,7 @@ abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { df.write.bucketBy(2, "i").sortBy("i").parquet("/tmp/path") }, - errorClass = "_LEGACY_ERROR_TEMP_1313", + condition = "_LEGACY_ERROR_TEMP_1313", parameters = Map("operation" -> "save")) } @@ -126,7 +126,7 @@ abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { df.write.bucketBy(2, "i").insertInto("tt") }, - errorClass = "_LEGACY_ERROR_TEMP_1312", + condition = "_LEGACY_ERROR_TEMP_1312", parameters = Map("operation" -> "insertInto")) } @@ -136,7 +136,7 @@ abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException] { df.write.bucketBy(2, "i").sortBy("i").insertInto("tt") }, - errorClass = "_LEGACY_ERROR_TEMP_1313", + condition = "_LEGACY_ERROR_TEMP_1313", parameters = Map("operation" -> "insertInto")) } @@ -252,7 +252,7 @@ abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { .bucketBy(8, "j", "k") .sortBy("k") .saveAsTable("bucketed_table")), - errorClass = "_LEGACY_ERROR_TEMP_1166", + condition = "_LEGACY_ERROR_TEMP_1166", parameters = Map("bucketCol" -> "j", "normalizedPartCols" -> "i, j")) checkError( @@ -261,7 +261,7 @@ abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { .bucketBy(8, "k") .sortBy("i") .saveAsTable("bucketed_table")), - errorClass = "_LEGACY_ERROR_TEMP_1167", + condition = "_LEGACY_ERROR_TEMP_1167", parameters = Map("sortCol" -> "i", "normalizedPartCols" -> "i, j")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 6f897a9c0b7b0..95c2fcbd7b5d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -164,7 +164,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSparkSession { exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map( "message" -> "CREATE TEMPORARY TABLE ... AS ..., use CREATE TEMPORARY VIEW instead"), context = ExpectedContext( @@ -291,7 +291,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSparkSession { exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map( "message" -> "Schema may not be specified in a Create Table As Select (CTAS) statement"), context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala index b6fb83fa5b876..43dfed277cbe7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala @@ -30,7 +30,7 @@ class DDLSourceLoadSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.read.format("Fluet da Bomb").load() }, - errorClass = "_LEGACY_ERROR_TEMP_1141", + condition = "_LEGACY_ERROR_TEMP_1141", parameters = Map( "provider" -> "Fluet da Bomb", "sourceNames" -> ("org.apache.spark.sql.sources.FakeSourceOne, " + @@ -49,7 +49,7 @@ class DDLSourceLoadSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.read.format("Fake external source").load() }, - errorClass = "_LEGACY_ERROR_TEMP_1141", + condition = "_LEGACY_ERROR_TEMP_1141", parameters = Map( "provider" -> "Fake external source", "sourceNames" -> ("org.apache.fakesource.FakeExternalSourceOne, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index e3e385e9d1810..57655a58a694d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -117,7 +117,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT INTO TABLE t1 SELECT a FROM t2") }, - errorClass = "UNSUPPORTED_INSERT.NOT_ALLOWED", + condition = "UNSUPPORTED_INSERT.NOT_ALLOWED", parameters = Map("relationId" -> "`SimpleScan(1,10)`") ) } @@ -131,7 +131,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT INTO TABLE t1 SELECT a FROM t1") }, - errorClass = "UNSUPPORTED_INSERT.RDD_BASED", + condition = "UNSUPPORTED_INSERT.RDD_BASED", parameters = Map.empty ) } @@ -151,7 +151,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT INTO TABLE t1 SELECT * FROM t1") }, - errorClass = "UNSUPPORTED_INSERT.READ_FROM", + condition = "UNSUPPORTED_INSERT.READ_FROM", parameters = Map("relationId" -> "`SimpleScan(1,10)`") ) } @@ -293,7 +293,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jsonTable") }, - errorClass = "UNSUPPORTED_OVERWRITE.PATH", + condition = "UNSUPPORTED_OVERWRITE.PATH", parameters = Map("path" -> ".*")) } @@ -338,7 +338,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { |SELECT i + 1, part2 FROM insertTable """.stripMargin) }, - errorClass = "UNSUPPORTED_OVERWRITE.TABLE", + condition = "UNSUPPORTED_OVERWRITE.TABLE", parameters = Map("table" -> "`spark_catalog`.`default`.`inserttable`")) } } @@ -418,7 +418,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT OVERWRITE TABLE oneToTen SELECT CAST(a AS INT) FROM jt") }, - errorClass = "UNSUPPORTED_INSERT.NOT_ALLOWED", + condition = "UNSUPPORTED_INSERT.NOT_ALLOWED", parameters = Map("relationId" -> "`SimpleScan(1,10)`")) spark.catalog.dropTempView("oneToTen") @@ -527,7 +527,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { |SELECT 1, 2 """.stripMargin) }, - errorClass = "NOT_SUPPORTED_COMMAND_WITHOUT_HIVE_SUPPORT", + condition = "NOT_SUPPORTED_COMMAND_WITHOUT_HIVE_SUPPORT", parameters = Map("cmd" -> "INSERT OVERWRITE DIRECTORY with the Hive format") ) } @@ -548,7 +548,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[SparkException] { spark.sql(v1) }, - errorClass = "_LEGACY_ERROR_TEMP_2233", + condition = "_LEGACY_ERROR_TEMP_2233", parameters = Map( "providingClass" -> ("class org.apache.spark.sql.execution.datasources." + "jdbc.JdbcRelationProvider")) @@ -658,7 +658,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t select 1L, 2") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`i`", @@ -670,7 +670,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t select 1, 2.0") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`d`", @@ -682,7 +682,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t select 1, 2.0D, 3") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "tableColumns" -> "`i`, `d`", @@ -705,7 +705,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t values('a', 'b')") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`i`", @@ -716,7 +716,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t values(now(), now())") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`i`", @@ -727,7 +727,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t values(true, false)") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`i`", @@ -775,7 +775,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[SparkArithmeticException] { sql(s"insert into t values($outOfRangeValue1)") }, - errorClass = "CAST_OVERFLOW_IN_TABLE_INSERT", + condition = "CAST_OVERFLOW_IN_TABLE_INSERT", parameters = Map( "sourceType" -> "\"BIGINT\"", "targetType" -> "\"INT\"", @@ -786,7 +786,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[SparkArithmeticException] { sql(s"insert into t values($outOfRangeValue2)") }, - errorClass = "CAST_OVERFLOW_IN_TABLE_INSERT", + condition = "CAST_OVERFLOW_IN_TABLE_INSERT", parameters = Map( "sourceType" -> "\"BIGINT\"", "targetType" -> "\"INT\"", @@ -806,7 +806,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[SparkArithmeticException] { sql(s"insert into t values(${outOfRangeValue1}D)") }, - errorClass = "CAST_OVERFLOW_IN_TABLE_INSERT", + condition = "CAST_OVERFLOW_IN_TABLE_INSERT", parameters = Map( "sourceType" -> "\"DOUBLE\"", "targetType" -> "\"BIGINT\"", @@ -817,7 +817,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[SparkArithmeticException] { sql(s"insert into t values(${outOfRangeValue2}D)") }, - errorClass = "CAST_OVERFLOW_IN_TABLE_INSERT", + condition = "CAST_OVERFLOW_IN_TABLE_INSERT", parameters = Map( "sourceType" -> "\"DOUBLE\"", "targetType" -> "\"BIGINT\"", @@ -836,7 +836,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[SparkArithmeticException] { sql(s"insert into t values($outOfRangeValue)") }, - errorClass = "CAST_OVERFLOW_IN_TABLE_INSERT", + condition = "CAST_OVERFLOW_IN_TABLE_INSERT", parameters = Map( "sourceType" -> "\"DECIMAL(5,2)\"", "targetType" -> "\"DECIMAL(3,2)\"", @@ -854,7 +854,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT INTO t VALUES (TIMESTAMP('2010-09-02 14:10:10'), 1)") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`i`", @@ -869,7 +869,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT INTO t VALUES (date('2010-09-02'), 1)") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`i`", @@ -884,7 +884,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT INTO t VALUES (TIMESTAMP('2010-09-02 14:10:10'), true)") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`b`", @@ -899,7 +899,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT INTO t VALUES (date('2010-09-02'), true)") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`b`", @@ -971,7 +971,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT OVERWRITE TABLE jsonTable SELECT a FROM jt") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`unknown`", "tableColumns" -> "`a`, `b`", @@ -994,7 +994,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT OVERWRITE TABLE jsonTable SELECT a FROM jt") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`unknown`", "tableColumns" -> "`a`, `b`", @@ -1170,7 +1170,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("create table t(i boolean, s bigint default badvalue) using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.NOT_CONSTANT", + condition = "INVALID_DEFAULT_VALUE.NOT_CONSTANT", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`s`", @@ -1186,7 +1186,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { }, // V1 command still use the fake Analyzer which can't resolve session variables and we // can only report UNRESOLVED_EXPRESSION error. - errorClass = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`s`", @@ -1199,7 +1199,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { }, // V2 command uses the actual analyzer and can resolve session variables. We can report // a more accurate NOT_CONSTANT error. - errorClass = "INVALID_DEFAULT_VALUE.NOT_CONSTANT", + condition = "INVALID_DEFAULT_VALUE.NOT_CONSTANT", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`j`", @@ -1216,7 +1216,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { sql("create table t(i boolean, s bigint default (select min(x) from badtable)) " + "using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`s`", @@ -1230,7 +1230,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { sql("create table t(i boolean, s bigint default (select min(x) from other)) " + "using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`s`", @@ -1243,7 +1243,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("create table t(i boolean default (select false as alias), s bigint) using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`i`", @@ -1256,7 +1256,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t values(false, default + 1)") }, - errorClass = "DEFAULT_PLACEMENT_INVALID", + condition = "DEFAULT_PLACEMENT_INVALID", parameters = Map.empty ) } @@ -1267,7 +1267,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t select false, default + 1") }, - errorClass = "DEFAULT_PLACEMENT_INVALID", + condition = "DEFAULT_PLACEMENT_INVALID", parameters = Map.empty ) } @@ -1277,7 +1277,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t values(false, default)") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`t`"), context = ExpectedContext("t", 12, 12) ) @@ -1288,7 +1288,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("create table t(i boolean, s bigint default false) using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`s`", @@ -1306,7 +1306,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { sql("insert into t select t1.id, t2.id, t1.val, t2.val, t1.val * t2.val " + "from num_data t1, num_data t2") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "tableColumns" -> "`id1`, `int2`, `result`", @@ -1319,7 +1319,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[ParseException] { sql("create table t(i boolean, s bigint default 42L) using parquet") }, - errorClass = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION", + condition = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION", parameters = Map.empty, context = ExpectedContext("s bigint default 42L", 26, 45) ) @@ -1333,7 +1333,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[ParseException] { sql("insert into t partition(i=default) values(5, default)") }, - errorClass = "REF_DEFAULT_VALUE_IS_NOT_ALLOWED_IN_PARTITION", + condition = "REF_DEFAULT_VALUE_IS_NOT_ALLOWED_IN_PARTITION", parameters = Map.empty, context = ExpectedContext( fragment = "partition(i=default)", @@ -1349,7 +1349,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t values(true)") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "tableColumns" -> "`i`, `s`", @@ -1423,7 +1423,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t (i, q) select true from (select 1)") }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "tableColumns" -> "`i`, `q`", @@ -1439,7 +1439,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t (i) values (true)") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`s`")) @@ -1450,7 +1450,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t (i) values (default)") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`s`")) @@ -1461,7 +1461,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t (s) values (default)") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`i`")) @@ -1472,7 +1472,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t partition(i='true') (s) values(5)") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`q`")) @@ -1483,7 +1483,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t partition(i='false') (q) select 43") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`s`")) @@ -1494,7 +1494,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("insert into t partition(i='false') (q) select default") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`s`")) @@ -1508,7 +1508,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { checkError( exception = intercept[AnalysisException](sql("insert into t (I) select true from (select 1)")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`I`", "proposal" -> "`i`, `s`"), context = ExpectedContext( @@ -1640,7 +1640,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t add column s bigint default badvalue") }, - errorClass = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", parameters = Map( "statement" -> "ALTER TABLE ADD COLUMNS", "colName" -> "`s`", @@ -1653,7 +1653,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t add column s bigint default (select min(x) from badtable)") }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "ALTER TABLE ADD COLUMNS", "colName" -> "`s`", @@ -1667,7 +1667,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t add column s bigint default (select min(x) from other)") }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "ALTER TABLE ADD COLUMNS", "colName" -> "`s`", @@ -1680,7 +1680,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t add column s bigint default false") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "ALTER TABLE ADD COLUMNS", "colName" -> "`s`", @@ -1696,7 +1696,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[ParseException] { sql("alter table t add column s bigint default 42L") }, - errorClass = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION", + condition = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION", parameters = Map.empty, context = ExpectedContext( fragment = "s bigint default 42L", @@ -1740,7 +1740,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t alter column s set default badvalue") }, - errorClass = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION", parameters = Map( "statement" -> "ALTER TABLE ALTER COLUMN", "colName" -> "`s`", @@ -1750,7 +1750,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t alter column s set default (select min(x) from badtable)") }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "ALTER TABLE ALTER COLUMN", "colName" -> "`s`", @@ -1761,7 +1761,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t alter column s set default (select 42 as alias)") }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "ALTER TABLE ALTER COLUMN", "colName" -> "`s`", @@ -1771,7 +1771,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t alter column s set default false") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "ALTER TABLE ALTER COLUMN", "colName" -> "`s`", @@ -1785,7 +1785,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION", + condition = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION", parameters = Map.empty, context = ExpectedContext( fragment = sqlText, @@ -1800,7 +1800,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t alter column i set default false") }, - errorClass = "CANNOT_ALTER_PARTITION_COLUMN", + condition = "CANNOT_ALTER_PARTITION_COLUMN", parameters = Map("tableName" -> "`spark_catalog`.`default`.`t`", "columnName" -> "`i`") ) } @@ -1964,7 +1964,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq("xyz").toDF().select("value", "default").write.insertInto("t") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`default`", "proposal" -> "`value`"), context = ExpectedContext(fragment = "select", callSitePattern = getCurrentClassCallSitePattern)) @@ -1998,7 +1998,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(s"create table t(a string default 'abc') using parquet") }, - errorClass = "_LEGACY_ERROR_TEMP_1345", + condition = "_LEGACY_ERROR_TEMP_1345", parameters = Map("statementType" -> "CREATE TABLE", "dataSource" -> "parquet")) withTable("t") { sql(s"create table t(a string, b int) using parquet") @@ -2006,7 +2006,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t add column s bigint default 42") }, - errorClass = "_LEGACY_ERROR_TEMP_1345", + condition = "_LEGACY_ERROR_TEMP_1345", parameters = Map( "statementType" -> "ALTER TABLE ADD COLUMNS", "dataSource" -> "parquet")) @@ -2065,7 +2065,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t add column s array default array('abc', 'def')") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "ALTER TABLE ADD COLUMNS", "colName" -> "`s`", @@ -2128,7 +2128,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t add column s struct default struct(42, 56)") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "ALTER TABLE ADD COLUMNS", "colName" -> "`s`", @@ -2248,7 +2248,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("alter table t add column s map default map(42, 56)") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + condition = "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "ALTER TABLE ADD COLUMNS", "colName" -> "`s`", @@ -2264,7 +2264,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("create table t(a string default (select 'abc')) using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`a`", @@ -2273,7 +2273,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("create table t(a string default exists(select 42 where true)) using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`a`", @@ -2282,7 +2282,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("create table t(a string default 1 in (select 1 union all select 2)) using parquet") }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + condition = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`a`", @@ -2314,7 +2314,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { // provider is now in the denylist. sql(s"alter table t1 add column (b string default 'abc')") }, - errorClass = "_LEGACY_ERROR_TEMP_1346", + condition = "_LEGACY_ERROR_TEMP_1346", parameters = Map( "statementType" -> "ALTER TABLE ADD COLUMNS", "dataSource" -> provider)) @@ -2389,7 +2389,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } else { checkError( exception = err, - errorClass = "TASK_WRITE_FAILED", + condition = "TASK_WRITE_FAILED", parameters = Map("path" -> s".*$tableName"), matchPVals = true ) @@ -2419,7 +2419,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[ParseException] { sql("insert overwrite local directory 'hdfs:/abcd' using parquet select 1") }, - errorClass = "LOCAL_MUST_WITH_SCHEMA_FILE", + condition = "LOCAL_MUST_WITH_SCHEMA_FILE", parameters = Map("actualSchema" -> "hdfs"), context = ExpectedContext( fragment = "insert overwrite local directory 'hdfs:/abcd' using parquet", @@ -2439,7 +2439,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT INTO TABLE insertTable PARTITION(part1=1, part2='') SELECT 1") }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> ("The spec ([part1=Some(1), part2=Some()]) " + "contains an empty partition column value")) @@ -2448,7 +2448,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql("INSERT INTO TABLE insertTable PARTITION(part1='', part2) SELECT 1 ,'' AS part2") }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> ("The spec ([part1=Some(), part2=None]) " + "contains an empty partition column value")) @@ -2475,7 +2475,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { |) """.stripMargin) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`c3`", @@ -2705,7 +2705,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { spark.table(tableName).write.mode(SaveMode.Overwrite).saveAsTable(tableName) }, - errorClass = "UNSUPPORTED_OVERWRITE.TABLE", + condition = "UNSUPPORTED_OVERWRITE.TABLE", parameters = Map("table" -> s"`spark_catalog`.`default`.`$tableName`") ) } @@ -2726,7 +2726,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException] { sql(insertDirSql) }, - errorClass = "UNSUPPORTED_OVERWRITE.PATH", + condition = "UNSUPPORTED_OVERWRITE.PATH", parameters = Map("path" -> ("file:" + path))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala index 730b63850d99a..f3849fe34ec29 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala @@ -168,7 +168,7 @@ class PartitionedWriteSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException] { Seq((3, 2)).toDF("a", "b").write.partitionBy("b", "b").csv(f.getAbsolutePath) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`b`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala index 6067efc1d1c1c..ee40d70c88291 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala @@ -84,7 +84,7 @@ class ResolvedDataSourceSuite extends SharedSparkSession { exception = intercept[AnalysisException] { getProvidingClass(provider) }, - errorClass = "_LEGACY_ERROR_TEMP_1139", + condition = "_LEGACY_ERROR_TEMP_1139", parameters = Map("provider" -> provider) ) } @@ -95,7 +95,7 @@ class ResolvedDataSourceSuite extends SharedSparkSession { exception = intercept[AnalysisException] { getProvidingClass("kafka") }, - errorClass = "_LEGACY_ERROR_TEMP_1140", + condition = "_LEGACY_ERROR_TEMP_1140", parameters = Map("provider" -> "kafka") ) } @@ -106,7 +106,7 @@ class ResolvedDataSourceSuite extends SharedSparkSession { } checkError( exception = error, - errorClass = "DATA_SOURCE_NOT_FOUND", + condition = "DATA_SOURCE_NOT_FOUND", parameters = Map("provider" -> "asfdwefasdfasdf") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index d1fe601838cb6..168b6b8629926 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -280,7 +280,7 @@ abstract class FileStreamSinkSuite extends StreamTest { exception = intercept[AnalysisException] { df.writeStream.format("parquet").outputMode(mode).start(dir.getCanonicalPath) }, - errorClass = "STREAMING_OUTPUT_MODE.UNSUPPORTED_DATASOURCE", + condition = "STREAMING_OUTPUT_MODE.UNSUPPORTED_DATASOURCE", sqlState = "42KDE", parameters = Map("className" -> "parquet", "outputMode" -> mode)) } @@ -378,7 +378,7 @@ abstract class FileStreamSinkSuite extends StreamTest { exception = intercept[AnalysisException] { spark.read.schema(s"$c0 INT, $c1 INT").json(outputDir).as[(Int, Int)] }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c1.toLowerCase(Locale.ROOT)}`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 634311b669a85..56c4aecb23770 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -419,7 +419,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { createFileStreamSourceAndGetSchema( format = Some("json"), path = Some(src.getCanonicalPath), schema = None) }, - errorClass = "UNABLE_TO_INFER_SCHEMA", + condition = "UNABLE_TO_INFER_SCHEMA", parameters = Map("format" -> "JSON") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala index 050c1a2d7d978..80c87d3297b01 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala @@ -302,13 +302,13 @@ class GroupStateSuite extends SparkFunSuite { TestGroupState.create[Int]( Optional.of(5), NoTimeout, 100L, Optional.empty[Long], hasTimedOut = true) }, - errorClass = "_LEGACY_ERROR_TEMP_3168", + condition = "_LEGACY_ERROR_TEMP_3168", parameters = Map.empty) checkError( exception = intercept[SparkUnsupportedOperationException] { GroupStateImpl.createForStreaming[Int](Some(5), 100L, NO_TIMESTAMP, NoTimeout, true, false) }, - errorClass = "_LEGACY_ERROR_TEMP_3168", + condition = "_LEGACY_ERROR_TEMP_3168", parameters = Map.empty) } @@ -349,7 +349,7 @@ class GroupStateSuite extends SparkFunSuite { def assertWrongTimeoutError(test: => Unit): Unit = { checkError( exception = intercept[SparkUnsupportedOperationException] { test }, - errorClass = "_LEGACY_ERROR_TEMP_2204", + condition = "_LEGACY_ERROR_TEMP_2204", parameters = Map.empty) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index 854893b1f033e..ab9df9a1e5a6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -521,7 +521,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { // verify that the key schema not compatible error is thrown checkError( ex.getCause.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE", + condition = "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE", parameters = Map("storedKeySchema" -> ".*", "newKeySchema" -> ".*"), matchPVals = true @@ -567,7 +567,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { checkError( ex.getCause.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY", + condition = "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY", parameters = Map( "schema" -> ".+\"str\":\"spark.UTF8_LCASE\".+" ), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index 5e9bdad8fd825..a733d54d275d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -752,7 +752,7 @@ class StreamingInnerJoinSuite extends StreamingJoinSuite { checkError( ex.getCause.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE", + condition = "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE", parameters = Map("storedKeySchema" -> ".*", "newKeySchema" -> ".*"), matchPVals = true @@ -822,7 +822,7 @@ class StreamingInnerJoinSuite extends StreamingJoinSuite { checkError( ex.getCause.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATE_STORE_VALUE_SCHEMA_NOT_COMPATIBLE", + condition = "STATE_STORE_VALUE_SCHEMA_NOT_COMPATIBLE", parameters = Map("storedValueSchema" -> ".*", "newValueSchema" -> ".*"), matchPVals = true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 2767f2dd46b2e..8471995cb1e50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -1422,7 +1422,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } checkError( ex.getCause.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY", + condition = "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY", parameters = Map( "schema" -> ".+\"c1\":\"spark.UTF8_LCASE\".+" ), @@ -1457,7 +1457,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi .option("checkpointLocation", checkpointDir.getCanonicalPath) .start(outputDir.getCanonicalPath) }, - errorClass = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", + condition = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", sqlState = "42KDE", parameters = Map( "outputMode" -> "append", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala index bf4e3f0a4e4aa..ec3c145af686c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala @@ -607,7 +607,7 @@ class StreamingSessionWindowSuite extends StreamTest CheckAnswer() // this is just to trigger the exception ) }, - errorClass = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", + condition = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", sqlState = "42KDE", parameters = Map( "outputMode" -> OutputMode.Update().toString.toLowerCase(Locale.ROOT), @@ -625,7 +625,7 @@ class StreamingSessionWindowSuite extends StreamTest CheckAnswer() // this is just to trigger the exception ) }, - errorClass = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", + condition = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", sqlState = "42KDE", parameters = Map( "outputMode" -> OutputMode.Update().toString.toLowerCase(Locale.ROOT), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala index 0f394aac8f782..fe88fbaa91cb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala @@ -103,7 +103,7 @@ class TransformWithMapStateSuite extends StreamTest ExpectFailure[SparkIllegalArgumentException] { e => { checkError( exception = e.asInstanceOf[SparkIllegalArgumentException], - errorClass = "ILLEGAL_STATE_STORE_VALUE.NULL_VALUE", + condition = "ILLEGAL_STATE_STORE_VALUE.NULL_VALUE", sqlState = Some("42601"), parameters = Map("stateName" -> "sessionState") ) @@ -152,7 +152,7 @@ class TransformWithMapStateSuite extends StreamTest ExpectFailure[SparkIllegalArgumentException] { e => { checkError( exception = e.asInstanceOf[SparkIllegalArgumentException], - errorClass = "ILLEGAL_STATE_STORE_VALUE.NULL_VALUE", + condition = "ILLEGAL_STATE_STORE_VALUE.NULL_VALUE", sqlState = Some("42601"), parameters = Map("stateName" -> "sessionState")) }} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala index b32d3c7e52013..b1025d9d89494 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala @@ -191,7 +191,7 @@ class TransformWithStateChainingSuite extends StreamTest { StartStream() ) }, - errorClass = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", + condition = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", sqlState = "42KDE", parameters = Map( "outputMode" -> "append", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala index 54cff6fc44c08..d141407b4fcd0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala @@ -418,7 +418,7 @@ class TransformWithStateInitialStateSuite extends StateStoreMetricsTest } checkError( exception = e.getCause.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATEFUL_PROCESSOR_CANNOT_REINITIALIZE_STATE_ON_KEY", + condition = "STATEFUL_PROCESSOR_CANNOT_REINITIALIZE_STATE_ON_KEY", sqlState = Some("42802"), parameters = Map("groupingKey" -> "init_1") ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index 9eeedd8598092..a0100d19c1b31 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -708,7 +708,7 @@ class TransformWithStateSuite extends StateStoreMetricsTest } checkError( ex.asInstanceOf[SparkRuntimeException], - errorClass = "STATE_STORE_HANDLE_NOT_INITIALIZED", + condition = "STATE_STORE_HANDLE_NOT_INITIALIZED", parameters = Map.empty ) } @@ -1151,7 +1151,7 @@ class TransformWithStateSuite extends StateStoreMetricsTest ExpectFailure[StateStoreInvalidConfigAfterRestart] { e => checkError( e.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATE_STORE_INVALID_CONFIG_AFTER_RESTART", + condition = "STATE_STORE_INVALID_CONFIG_AFTER_RESTART", parameters = Map( "configName" -> "outputMode", "oldConfig" -> "Update", @@ -1193,7 +1193,7 @@ class TransformWithStateSuite extends StateStoreMetricsTest ExpectFailure[StateStoreInvalidVariableTypeChange] { t => checkError( t.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATE_STORE_INVALID_VARIABLE_TYPE_CHANGE", + condition = "STATE_STORE_INVALID_VARIABLE_TYPE_CHANGE", parameters = Map( "stateVarName" -> "countState", "newType" -> "ListState", @@ -1240,7 +1240,7 @@ class TransformWithStateSuite extends StateStoreMetricsTest ExpectFailure[StateStoreInvalidConfigAfterRestart] { e => checkError( e.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATE_STORE_INVALID_CONFIG_AFTER_RESTART", + condition = "STATE_STORE_INVALID_CONFIG_AFTER_RESTART", parameters = Map( "configName" -> "timeMode", "oldConfig" -> "NoTime", @@ -1292,7 +1292,7 @@ class TransformWithStateSuite extends StateStoreMetricsTest ExpectFailure[StateStoreValueSchemaNotCompatible] { t => checkError( t.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATE_STORE_VALUE_SCHEMA_NOT_COMPATIBLE", + condition = "STATE_STORE_VALUE_SCHEMA_NOT_COMPATIBLE", parameters = Map( "storedValueSchema" -> "StructType(StructField(value,LongType,false))", "newValueSchema" -> diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index e77ba92fe2981..64b0e96845d68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -339,7 +339,7 @@ class StreamingDataSourceV2Suite extends StreamTest { assert(query.exception.get.cause != null) checkErrorMatchPVals( exception = query.exception.get.cause.asInstanceOf[SparkUnsupportedOperationException], - errorClass = errorClass, + condition = errorClass, parameters = parameters ) } @@ -436,7 +436,7 @@ class StreamingDataSourceV2Suite extends StreamTest { exception = intercept[SparkUnsupportedOperationException] { testCase(read, write, trigger) }, - errorClass = "_LEGACY_ERROR_TEMP_2049", + condition = "_LEGACY_ERROR_TEMP_2049", parameters = Map( "className" -> "fake-read-neither-mode", "operator" -> "reading" @@ -449,7 +449,7 @@ class StreamingDataSourceV2Suite extends StreamTest { exception = intercept[SparkUnsupportedOperationException] { testCase(read, write, trigger) }, - errorClass = "_LEGACY_ERROR_TEMP_2049", + condition = "_LEGACY_ERROR_TEMP_2049", parameters = Map( "className" -> "fake-write-neither-mode", "operator" -> "writing" @@ -466,7 +466,7 @@ class StreamingDataSourceV2Suite extends StreamTest { exception = intercept[SparkUnsupportedOperationException] { testCase(read, write, trigger) }, - errorClass = "_LEGACY_ERROR_TEMP_2253", + condition = "_LEGACY_ERROR_TEMP_2253", parameters = Map("sourceName" -> "fake-read-microbatch-only") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index c4ec0af80b725..e74627f3f51e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -133,7 +133,7 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { .write .save() }, - errorClass = "CALL_ON_STREAMING_DATASET_UNSUPPORTED", + condition = "CALL_ON_STREAMING_DATASET_UNSUPPORTED", parameters = Map("methodName" -> "`write`")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index 5ae7b3eec37e7..86c4e49f6f66f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -117,7 +117,7 @@ class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter { exception = intercept[AnalysisException] { spark.readStream.table(tableIdentifier) }, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", parameters = Map( "tableName" -> "`testcat`.`table_name`", "operation" -> "either micro-batch or continuous scan" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index b9e5c176f93e0..f3d21e384ed42 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -165,7 +165,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with } checkError( exception = e, - errorClass = "WRITE_STREAM_NOT_ALLOWED", + condition = "WRITE_STREAM_NOT_ALLOWED", parameters = Map.empty ) } @@ -306,7 +306,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with exception = intercept[AnalysisException] { df.write.mode("append").clusterBy("a").saveAsTable("clusteredTable") }, - errorClass = "CLUSTERING_COLUMNS_MISMATCH", + condition = "CLUSTERING_COLUMNS_MISMATCH", parameters = Map( "tableName" -> "spark_catalog.default.clusteredtable", "specifiedClusteringString" -> """[["a"]]""", @@ -317,7 +317,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with exception = intercept[AnalysisException] { df.write.mode("append").clusterBy("b", "a").saveAsTable("clusteredTable") }, - errorClass = "CLUSTERING_COLUMNS_MISMATCH", + condition = "CLUSTERING_COLUMNS_MISMATCH", parameters = Map( "tableName" -> "spark_catalog.default.clusteredtable", "specifiedClusteringString" -> """[["b"],["a"]]""", @@ -328,7 +328,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with exception = intercept[AnalysisException] { df.write.mode("append").saveAsTable("clusteredTable") }, - errorClass = "CLUSTERING_COLUMNS_MISMATCH", + condition = "CLUSTERING_COLUMNS_MISMATCH", parameters = Map( "tableName" -> "spark_catalog.default.clusteredtable", "specifiedClusteringString" -> "", "existingClusteringString" -> """[["a"],["b"]]""") @@ -455,7 +455,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with exception = intercept[AnalysisException] { Seq((1L, 2.0)).toDF("i", "d").write.mode("append").saveAsTable("t") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`i`", @@ -483,7 +483,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with exception = intercept[AnalysisException] { Seq(("a", "b")).toDF("i", "d").write.mode("append").saveAsTable("t") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`i`", @@ -495,7 +495,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with exception = intercept[AnalysisException] { Seq((true, false)).toDF("i", "d").write.mode("append").saveAsTable("t") }, - errorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + condition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t`", "colName" -> "`i`", @@ -728,7 +728,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with exception = intercept[AnalysisException] { testRead(spark.read.csv(), Seq.empty, schema) }, - errorClass = "UNABLE_TO_INFER_SCHEMA", + condition = "UNABLE_TO_INFER_SCHEMA", parameters = Map("format" -> "CSV") ) @@ -1066,13 +1066,13 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with exception = intercept[AnalysisException] { Seq((1, 1)).toDF("col", c0).write.bucketBy(2, c0, c1).saveAsTable("t") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c1.toLowerCase(Locale.ROOT)}`")) checkError( exception = intercept[AnalysisException] { Seq((1, 1)).toDF("col", c0).write.bucketBy(2, "col").sortBy(c0, c1).saveAsTable("t") }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c1.toLowerCase(Locale.ROOT)}`")) } } @@ -1086,7 +1086,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with Seq((1, 1)).toDF(colName0, colName1).write.format(format).mode("overwrite") .save(tempDir.getAbsolutePath) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${colName1.toLowerCase(Locale.ROOT)}`")) } @@ -1099,7 +1099,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with spark.read.format(format).schema(s"$colName0 INT, $colName1 INT") .load(testDir.getAbsolutePath) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${colName1.toLowerCase(Locale.ROOT)}`")) } @@ -1112,7 +1112,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with exception = intercept[AnalysisException] { spark.read.format(format).load(testDir.getAbsolutePath) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${colName1.toLowerCase(Locale.ROOT)}`")) } @@ -1142,7 +1142,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with exception = intercept[AnalysisException] { spark.read.format("json").option("inferSchema", true).load(testDir.getAbsolutePath) }, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> s"`${c1.toLowerCase(Locale.ROOT)}`")) checkReadPartitionColumnDuplication("json", c0, c1, src) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala index 600fddd797ca4..2a15d5b4dcb45 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala @@ -59,7 +59,7 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi exception = intercept[SparkException] { sql("select count(*) from view_refresh").first() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> ".*") ) @@ -102,7 +102,7 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi exception = intercept[SparkException] { sql("select * from test").count() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> ".*") ) @@ -120,7 +120,7 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi exception = intercept[SparkException] { sql("select * from test").count() }, - errorClass = "FAILED_READ_FILE.FILE_NOT_EXIST", + condition = "FAILED_READ_FILE.FILE_NOT_EXIST", parameters = Map("path" -> ".*") ) spark.catalog.refreshByPath(dir.getAbsolutePath) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index f9a24f44b76c0..72c570d1f9097 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -358,7 +358,7 @@ class DataSourceWithHiveMetastoreCatalogSuite |""".stripMargin) checkError( exception = intercept[AnalysisException](spark.table("non_partition_table")), - errorClass = "_LEGACY_ERROR_TEMP_3096", + condition = "_LEGACY_ERROR_TEMP_3096", parameters = Map( "resLen" -> "2", "relLen" -> "1", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index 2152a7e300021..6d7248a7dd67f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -128,7 +128,7 @@ class HiveParquetSuite extends QueryTest } checkError( exception = ex, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map("objectName" -> "`c3`", "proposal" -> "`c1`, `c2`"), context = ExpectedContext( fragment = "c3", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSQLInsertTestSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSQLInsertTestSuite.scala index d6ba38359f496..4109c0a127065 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSQLInsertTestSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSQLInsertTestSuite.scala @@ -45,7 +45,7 @@ class HiveSQLInsertTestSuite extends SQLInsertTestSuite with TestHiveSingleton { v2ErrorClass: String, v1Parameters: Map[String, String], v2Parameters: Map[String, String]): Unit = { - checkError(exception = exception, sqlState = None, errorClass = v1ErrorClass, + checkError(exception = exception, sqlState = None, condition = v1ErrorClass, parameters = v1Parameters) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index ea43f1d2c6729..cc7bb193731f5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -350,7 +350,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter exception = intercept[AnalysisException] { Seq((1, 2, 3, 4)).toDF("a", "b", "c", "d").write.partitionBy("b", "c").insertInto(tableName) }, - errorClass = "_LEGACY_ERROR_TEMP_1309", + condition = "_LEGACY_ERROR_TEMP_1309", parameters = Map.empty ) } @@ -362,7 +362,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter exception = intercept[AnalysisException] { sql(s"INSERT INTO TABLE $tableName PARTITION(b=1, c=2) SELECT 1, 2, 3") }, - errorClass = "INSERT_PARTITION_COLUMN_ARITY_MISMATCH", + condition = "INSERT_PARTITION_COLUMN_ARITY_MISMATCH", parameters = Map( "staticPartCols" -> "`b`, `c`", "tableColumns" -> "`a`, `d`, `b`, `c`", @@ -720,7 +720,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' |SELECT * FROM test_insert_table""".stripMargin) }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'ROW'", "hint" -> "")) } } @@ -740,7 +740,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' |SELECT * FROM test_insert_table""".stripMargin) }, - errorClass = "PARSE_SYNTAX_ERROR", + condition = "PARSE_SYNTAX_ERROR", parameters = Map("error" -> "'ROW'", "hint" -> "")) } } @@ -809,7 +809,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } checkError( exception = e, - errorClass = "COLUMN_ALREADY_EXISTS", + condition = "COLUMN_ALREADY_EXISTS", parameters = Map("columnName" -> "`id`")) } } @@ -858,7 +858,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter |SELECT 1 """.stripMargin) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> "The spec ([d=Some()]) contains an empty partition column value") ) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index cde0da67e83e9..f2cab33dea76a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -575,7 +575,7 @@ class MetastoreDataSourcesSuite extends QueryTest table("createdJsonTable") }, - errorClass = "UNABLE_TO_INFER_SCHEMA", + condition = "UNABLE_TO_INFER_SCHEMA", parameters = Map("format" -> "JSON") ) @@ -925,7 +925,7 @@ class MetastoreDataSourcesSuite extends QueryTest createDF(10, 19).write.mode(SaveMode.Append).format("orc"). saveAsTable("appendOrcToParquet") }, - errorClass = "_LEGACY_ERROR_TEMP_1159", + condition = "_LEGACY_ERROR_TEMP_1159", parameters = Map( "tableName" -> s"$SESSION_CATALOG_NAME.default.appendorctoparquet", "existingProvider" -> "ParquetDataSourceV2", @@ -941,7 +941,7 @@ class MetastoreDataSourcesSuite extends QueryTest createDF(10, 19).write.mode(SaveMode.Append).format("parquet") .saveAsTable("appendParquetToJson") }, - errorClass = "_LEGACY_ERROR_TEMP_1159", + condition = "_LEGACY_ERROR_TEMP_1159", parameters = Map( "tableName" -> s"$SESSION_CATALOG_NAME.default.appendparquettojson", "existingProvider" -> "JsonDataSourceV2", @@ -957,7 +957,7 @@ class MetastoreDataSourcesSuite extends QueryTest createDF(10, 19).write.mode(SaveMode.Append).format("text") .saveAsTable("appendTextToJson") }, - errorClass = "_LEGACY_ERROR_TEMP_1159", + condition = "_LEGACY_ERROR_TEMP_1159", // The format of the existing table can be JsonDataSourceV2 or JsonFileFormat. parameters = Map( "tableName" -> s"$SESSION_CATALOG_NAME.default.appendtexttojson", @@ -1232,7 +1232,7 @@ class MetastoreDataSourcesSuite extends QueryTest Seq((3, 4)).toDF("i", "k") .write.mode("append").saveAsTable("saveAsTable_mismatch_column_names") }, - errorClass = "_LEGACY_ERROR_TEMP_1162", + condition = "_LEGACY_ERROR_TEMP_1162", parameters = Map("col" -> "j", "inputColumns" -> "i, k")) } } @@ -1245,7 +1245,7 @@ class MetastoreDataSourcesSuite extends QueryTest Seq((3, 4, 5)).toDF("i", "j", "k") .write.mode("append").saveAsTable("saveAsTable_too_many_columns") }, - errorClass = "_LEGACY_ERROR_TEMP_1161", + condition = "_LEGACY_ERROR_TEMP_1161", parameters = Map( "tableName" -> "spark_catalog.default.saveastable_too_many_columns", "existingTableSchema" -> "struct", @@ -1265,7 +1265,7 @@ class MetastoreDataSourcesSuite extends QueryTest |USING hive """.stripMargin) }, - errorClass = "_LEGACY_ERROR_TEMP_1293", + condition = "_LEGACY_ERROR_TEMP_1293", parameters = Map.empty ) } @@ -1288,7 +1288,7 @@ class MetastoreDataSourcesSuite extends QueryTest exception = intercept[AnalysisException] { table(tableName).write.mode(SaveMode.Overwrite).saveAsTable(tableName) }, - errorClass = "UNSUPPORTED_OVERWRITE.TABLE", + condition = "UNSUPPORTED_OVERWRITE.TABLE", parameters = Map("table" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`") ) @@ -1296,7 +1296,7 @@ class MetastoreDataSourcesSuite extends QueryTest exception = intercept[AnalysisException] { table(tableName).write.mode(SaveMode.ErrorIfExists).saveAsTable(tableName) }, - errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`") ) } @@ -1326,7 +1326,7 @@ class MetastoreDataSourcesSuite extends QueryTest exception = intercept[AnalysisException] { table(tableName).write.mode(SaveMode.Overwrite).insertInto(tableName) }, - errorClass = "UNSUPPORTED_OVERWRITE.TABLE", + condition = "UNSUPPORTED_OVERWRITE.TABLE", parameters = Map("table" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`") ) } @@ -1339,7 +1339,7 @@ class MetastoreDataSourcesSuite extends QueryTest exception = intercept[AnalysisException] { Seq(4).toDF("j").write.mode("append").saveAsTable("saveAsTable_less_columns") }, - errorClass = "_LEGACY_ERROR_TEMP_1161", + condition = "_LEGACY_ERROR_TEMP_1161", parameters = Map( "tableName" -> "spark_catalog.default.saveastable_less_columns", "existingTableSchema" -> "struct", @@ -1396,7 +1396,7 @@ class MetastoreDataSourcesSuite extends QueryTest exception = intercept[AnalysisException] { sharedState.externalCatalog.getTable("default", "t") }, - errorClass = "INSUFFICIENT_TABLE_PROPERTY.MISSING_KEY", + condition = "INSUFFICIENT_TABLE_PROPERTY.MISSING_KEY", parameters = Map("key" -> toSQLConf("spark.sql.sources.schema")) ) @@ -1417,7 +1417,7 @@ class MetastoreDataSourcesSuite extends QueryTest exception = intercept[AnalysisException] { sharedState.externalCatalog.getTable("default", "t2") }, - errorClass = "INSUFFICIENT_TABLE_PROPERTY.MISSING_KEY_PART", + condition = "INSUFFICIENT_TABLE_PROPERTY.MISSING_KEY_PART", parameters = Map( "key" -> toSQLConf("spark.sql.sources.schema.part.1"), "totalAmountOfParts" -> "3") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 55be6102a8535..0b10829f66910 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -272,7 +272,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.format("parquet").saveAsTable("`d:b`.`t:a`") } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`spark_catalog`.`d:b`")) } @@ -281,7 +281,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.format("parquet").saveAsTable("`d:b`.`table`") } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`spark_catalog`.`d:b`")) } @@ -297,7 +297,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle |) """.stripMargin) } - checkError(e, errorClass = "INVALID_SCHEMA_OR_RELATION_NAME", + checkError(e, condition = "INVALID_SCHEMA_OR_RELATION_NAME", parameters = Map("name" -> "`t:a`")) } @@ -313,7 +313,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle """.stripMargin) } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`spark_catalog`.`d:b`")) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 7dc7fc41dc708..9c2f4461ff263 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -668,7 +668,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto exception = intercept[AnalysisException] { sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS fakeColumn") }, - errorClass = "COLUMN_NOT_FOUND", + condition = "COLUMN_NOT_FOUND", parameters = Map( "colName" -> "`fakeColumn`", "caseSensitiveConfig" -> "\"spark.sql.caseSensitive\"" @@ -1706,7 +1706,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto exception = intercept[AnalysisException] { sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS value") }, - errorClass = "UNSUPPORTED_FEATURE.ANALYZE_UNSUPPORTED_COLUMN_TYPE", + condition = "UNSUPPORTED_FEATURE.ANALYZE_UNSUPPORTED_COLUMN_TYPE", parameters = Map( "columnType" -> "\"MAP\"", "columnName" -> "`value`", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala index b60adfb6f4cf1..07f212d2dcabb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala @@ -186,7 +186,7 @@ class HiveClientSuite(version: String) extends HiveVersionSuite(version) { assert(false, "dropDatabase should throw HiveException") } checkError(ex, - errorClass = "SCHEMA_NOT_EMPTY", + condition = "SCHEMA_NOT_EMPTY", parameters = Map("schemaName" -> "`temporary`")) client.dropDatabase("temporary", ignoreIfNotExists = false, cascade = true) @@ -485,7 +485,7 @@ class HiveClientSuite(version: String) extends HiveVersionSuite(version) { client.createPartitions("default", "src_part", partitions, ignoreIfExists = false) } checkError(e, - errorClass = "PARTITIONS_ALREADY_EXIST", + condition = "PARTITIONS_ALREADY_EXIST", parameters = Map("partitionList" -> "PARTITION (`key1` = 101, `key2` = 102)", "tableName" -> "`default`.`src_part`")) } finally { @@ -577,7 +577,7 @@ class HiveClientSuite(version: String) extends HiveVersionSuite(version) { exception = intercept[AnalysisException] { versionSpark.table("mv1").collect() }, - errorClass = "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE", + condition = "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE", parameters = Map( "tableName" -> "`mv1`", "tableType" -> "materialized view" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 4b000fff0eb92..7c9b0b7781427 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -526,7 +526,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te |GROUP BY key """.stripMargin) }, - errorClass = "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION", + condition = "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION", parameters = Map("sqlExpr" -> "\"mydoublesum(((value + (1.5 * key)) + rand()))\""), context = ExpectedContext( fragment = "value + 1.5 * key + rand()", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 15dbd6aaa5b06..cb2799ac8bc6f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -399,14 +399,14 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("CREATE TABLE tab1 USING hive") }, - errorClass = "_LEGACY_ERROR_TEMP_3083", + condition = "_LEGACY_ERROR_TEMP_3083", parameters = Map("tableName" -> "`spark_catalog`.`default`.`tab1`") ) checkError( exception = intercept[AnalysisException] { sql(s"CREATE TABLE tab2 USING hive location '${tempDir.getCanonicalPath}'") }, - errorClass = "_LEGACY_ERROR_TEMP_3083", + condition = "_LEGACY_ERROR_TEMP_3083", parameters = Map("tableName" -> "`spark_catalog`.`default`.`tab2`") ) } @@ -542,7 +542,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "COLUMN_NOT_DEFINED_IN_TABLE", + condition = "COLUMN_NOT_DEFINED_IN_TABLE", parameters = Map( "colType" -> "partition", "colName" -> "`b`", @@ -605,7 +605,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "_LEGACY_ERROR_TEMP_1076", + condition = "_LEGACY_ERROR_TEMP_1076", parameters = Map( "details" -> "The spec ([partCol1=]) contains an empty partition column value") ) @@ -657,7 +657,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $externalTab DROP PARTITION (ds='2008-04-09', unknownCol='12')") }, - errorClass = "_LEGACY_ERROR_TEMP_1231", + condition = "_LEGACY_ERROR_TEMP_1231", parameters = Map( "key" -> "unknownCol", "tblName" -> s"`$SESSION_CATALOG_NAME`.`default`.`exttable_with_partitions`") @@ -776,7 +776,7 @@ class HiveDDLSuite parameters: Map[String, String]): Unit = { val e = intercept[AnalysisException](sql(sqlText)) checkError(e, - errorClass = errorClass, parameters = parameters) + condition = errorClass, parameters = parameters) } test("create table - SET TBLPROPERTIES EXTERNAL to TRUE") { @@ -787,7 +787,7 @@ class HiveDDLSuite sql(s"CREATE TABLE $tabName (height INT, length INT) " + s"TBLPROPERTIES('EXTERNAL'='TRUE')") }, - errorClass = "_LEGACY_ERROR_TEMP_3087", + condition = "_LEGACY_ERROR_TEMP_3087", parameters = Map.empty ) } @@ -804,7 +804,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $tabName SET TBLPROPERTIES ('EXTERNAL' = 'TRUE')") }, - errorClass = "_LEGACY_ERROR_TEMP_3087", + condition = "_LEGACY_ERROR_TEMP_3087", parameters = Map.empty ) // The table type is not changed to external @@ -836,7 +836,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER VIEW $tabName RENAME TO $newViewName") }, - errorClass = "_LEGACY_ERROR_TEMP_1253", + condition = "_LEGACY_ERROR_TEMP_1253", parameters = Map.empty ) @@ -844,7 +844,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName RENAME TO $newViewName") }, - errorClass = "_LEGACY_ERROR_TEMP_1252", + condition = "_LEGACY_ERROR_TEMP_1252", parameters = Map.empty ) @@ -852,7 +852,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER VIEW $tabName SET TBLPROPERTIES ('p' = 'an')") }, - errorClass = "EXPECT_VIEW_NOT_TABLE.USE_ALTER_TABLE", + condition = "EXPECT_VIEW_NOT_TABLE.USE_ALTER_TABLE", parameters = Map( "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$tabName`", "operation" -> "ALTER VIEW ... SET TBLPROPERTIES"), @@ -863,7 +863,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName SET TBLPROPERTIES ('p' = 'an')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", + condition = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`", "operation" -> "ALTER TABLE ... SET TBLPROPERTIES"), @@ -874,7 +874,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER VIEW $tabName UNSET TBLPROPERTIES ('p')") }, - errorClass = "EXPECT_VIEW_NOT_TABLE.USE_ALTER_TABLE", + condition = "EXPECT_VIEW_NOT_TABLE.USE_ALTER_TABLE", parameters = Map( "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$tabName`", "operation" -> "ALTER VIEW ... UNSET TBLPROPERTIES"), @@ -885,7 +885,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName UNSET TBLPROPERTIES ('p')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", + condition = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`", "operation" -> "ALTER TABLE ... UNSET TBLPROPERTIES"), @@ -896,7 +896,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName SET LOCATION '/path/to/home'") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`", "operation" -> "ALTER TABLE ... SET LOCATION ..."), @@ -907,7 +907,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName SET SERDE 'whatever'") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", + condition = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`", "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), @@ -918,7 +918,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName SET SERDEPROPERTIES ('x' = 'y')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", + condition = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`", "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), @@ -929,7 +929,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", + condition = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`", "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), @@ -940,7 +940,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName RECOVER PARTITIONS") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`", "operation" -> "ALTER TABLE ... RECOVER PARTITIONS"), @@ -951,7 +951,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName PARTITION (a='1') RENAME TO PARTITION (a='100')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`", "operation" -> "ALTER TABLE ... RENAME TO PARTITION"), @@ -962,7 +962,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName ADD IF NOT EXISTS PARTITION (a='4', b='8')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`", "operation" -> "ALTER TABLE ... ADD PARTITION ..."), @@ -973,7 +973,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName DROP IF EXISTS PARTITION (a='2')") }, - errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + condition = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`", "operation" -> "ALTER TABLE ... DROP PARTITION ..."), @@ -1159,7 +1159,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("CREATE VIEW view1 (col1, col3) AS SELECT * FROM tab1") }, - errorClass = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`", "viewColumns" -> "`col1`, `col3`", @@ -1175,7 +1175,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("CREATE VIEW view2 (col1, col3) AS SELECT * FROM tab2") }, - errorClass = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + condition = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", parameters = Map( "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`view2`", "viewColumns" -> "`col1`, `col3`", @@ -1358,7 +1358,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("DROP DATABASE default") }, - errorClass = "UNSUPPORTED_FEATURE.DROP_DATABASE", + condition = "UNSUPPORTED_FEATURE.DROP_DATABASE", parameters = Map("database" -> "`spark_catalog`.`default`") ) @@ -1368,7 +1368,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("DROP DATABASE DeFault") }, - errorClass = caseSensitive match { + condition = caseSensitive match { case "false" => "UNSUPPORTED_FEATURE.DROP_DATABASE" case _ => "_LEGACY_ERROR_TEMP_3065" }, @@ -1764,7 +1764,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { spark.catalog.getTable("default", indexTabName) }, - errorClass = "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE", + condition = "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE", parameters = Map( "tableName" -> s"`$indexTabName`", "tableType" -> "index table") @@ -1774,7 +1774,7 @@ class HiveDDLSuite exception = intercept[TableAlreadyExistsException] { sql(s"CREATE TABLE $indexTabName(b int) USING hive") }, - errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> s"`default`.`$indexTabName`") ) @@ -1782,7 +1782,7 @@ class HiveDDLSuite exception = intercept[TableAlreadyExistsException] { sql(s"ALTER TABLE $tabName RENAME TO $indexTabName") }, - errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> s"`default`.`$indexTabName`") ) @@ -1791,7 +1791,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"DESCRIBE $indexTabName") }, - errorClass = "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE", + condition = "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE", parameters = Map( "tableName" -> s"`$indexTabName`", "tableType" -> "index table") @@ -1869,7 +1869,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE tbl SET TBLPROPERTIES ('${forbiddenPrefix}foo' = 'loser')") }, - errorClass = "_LEGACY_ERROR_TEMP_3086", + condition = "_LEGACY_ERROR_TEMP_3086", parameters = Map( "tableName" -> "spark_catalog.default.tbl", "invalidKeys" -> s"[${forbiddenPrefix}foo]") @@ -1878,7 +1878,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"CREATE TABLE tbl2 (a INT) TBLPROPERTIES ('${forbiddenPrefix}foo'='anything')") }, - errorClass = "_LEGACY_ERROR_TEMP_3086", + condition = "_LEGACY_ERROR_TEMP_3086", parameters = Map( "tableName" -> "spark_catalog.default.tbl2", "invalidKeys" -> s"[${forbiddenPrefix}foo]") @@ -1987,7 +1987,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { spark.table("t").write.format("hive").mode("overwrite").saveAsTable("t") }, - errorClass = "UNSUPPORTED_OVERWRITE.TABLE", + condition = "UNSUPPORTED_OVERWRITE.TABLE", parameters = Map("table" -> s"`$SESSION_CATALOG_NAME`.`default`.`t`")) } } @@ -2380,7 +2380,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("INSERT INTO TABLE t SELECT 1") }, - errorClass = "_LEGACY_ERROR_TEMP_3065", + condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( "clazz" -> "java.lang.IllegalArgumentException", "msg" -> "java.net.URISyntaxException: Relative path in absolute URI: a:b") @@ -2427,7 +2427,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("INSERT INTO TABLE t1 PARTITION(b=2) SELECT 1") }, - errorClass = "_LEGACY_ERROR_TEMP_3065", + condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( "clazz" -> "java.lang.IllegalArgumentException", "msg" -> "java.net.URISyntaxException: Relative path in absolute URI: a:b") @@ -2437,7 +2437,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("INSERT INTO TABLE t1 PARTITION(b='2017-03-03 12:13%3A14') SELECT 1") }, - errorClass = "_LEGACY_ERROR_TEMP_3065", + condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( "clazz" -> "java.lang.IllegalArgumentException", "msg" -> "java.net.URISyntaxException: Relative path in absolute URI: a:b") @@ -2543,7 +2543,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("ALTER TABLE tab ADD COLUMNS (C2 string)") }, - errorClass = "_LEGACY_ERROR_TEMP_3065", + condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", "msg" -> "Partition column name c2 conflicts with table columns.") @@ -2555,7 +2555,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("ALTER TABLE tab ADD COLUMNS (C1 string)") }, - errorClass = "_LEGACY_ERROR_TEMP_3065", + condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", "msg" -> "Duplicate column name c1 in the table definition.") @@ -2573,7 +2573,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("CREATE TABLE t1 USING PARQUET AS SELECT NULL AS null_col") }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`null_col`", "columnType" -> "\"VOID\"", @@ -2584,7 +2584,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("CREATE TABLE t2 STORED AS PARQUET AS SELECT null as null_col") }, - errorClass = "_LEGACY_ERROR_TEMP_3065", + condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", "msg" -> "java.lang.UnsupportedOperationException: Unknown field type: void") @@ -2600,7 +2600,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("CREATE TABLE t1 (v VOID) USING PARQUET") }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`v`", "columnType" -> "\"VOID\"", @@ -2610,7 +2610,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("CREATE TABLE t2 (v VOID) STORED AS PARQUET") }, - errorClass = "_LEGACY_ERROR_TEMP_3065", + condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", "msg" -> "java.lang.UnsupportedOperationException: Unknown field type: void") @@ -2818,7 +2818,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("load data inpath '/doesnotexist.csv' into table tbl") }, - errorClass = "LOAD_DATA_PATH_NOT_EXISTS", + condition = "LOAD_DATA_PATH_NOT_EXISTS", parameters = Map("path" -> "/doesnotexist.csv") ) } @@ -2860,7 +2860,7 @@ class HiveDDLSuite exception = intercept[SparkException] { sql(s"CREATE TABLE t (a $typ) USING hive") }, - errorClass = "CANNOT_RECOGNIZE_HIVE_TYPE", + condition = "CANNOT_RECOGNIZE_HIVE_TYPE", parameters = Map( "fieldType" -> toSQLType(replaced), "fieldName" -> "`a`") @@ -2878,7 +2878,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"CREATE TABLE t (a $typ) USING hive") }, - errorClass = "_LEGACY_ERROR_TEMP_3065", + condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", "msg" -> msg) @@ -2917,7 +2917,7 @@ class HiveDDLSuite |AS SELECT 1 as a, "a" as b""".stripMargin checkError( exception = intercept[ParseException](sql(sql1)), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map( "message" -> "Schema may not be specified in a Create Table As Select (CTAS) statement"), context = ExpectedContext(sql1, 0, 92)) @@ -2929,7 +2929,7 @@ class HiveDDLSuite |AS SELECT 1 as a, "a" as b""".stripMargin checkError( exception = intercept[ParseException](sql(sql2)), - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map( "message" -> "Partition column types may not be specified in Create Table As Select (CTAS)"), @@ -3020,7 +3020,7 @@ class HiveDDLSuite exception = intercept[ParseException] { sql(sql1) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map( "operation" -> ("CREATE TABLE LIKE ... USING ... ROW FORMAT SERDE " + "ORG.APACHE.HADOOP.HIVE.SERDE2.LAZY.LAZYSIMPLESERDE")), @@ -3036,7 +3036,7 @@ class HiveDDLSuite exception = intercept[ParseException] { sql(sql2) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map( "operation" -> ("CREATE TABLE LIKE ... USING ... ROW FORMAT SERDE " + "ORG.APACHE.HADOOP.HIVE.SERDE2.LAZY.LAZYSIMPLESERDE")), @@ -3052,7 +3052,7 @@ class HiveDDLSuite exception = intercept[ParseException] { sql(sql3) }, - errorClass = "_LEGACY_ERROR_TEMP_0047", + condition = "_LEGACY_ERROR_TEMP_0047", parameters = Map.empty, context = ExpectedContext(fragment = sql3, start = 0, stop = 153) ) @@ -3066,7 +3066,7 @@ class HiveDDLSuite exception = intercept[ParseException] { sql(sql4) }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map( "operation" -> ("CREATE TABLE LIKE ... USING ... STORED AS " + "INPUTFORMAT INFORMAT OUTPUTFORMAT OUTFORMAT ROW FORMAT " + @@ -3140,7 +3140,7 @@ class HiveDDLSuite exception = intercept[ParseException] { sql(sql1) }, - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map( "message" -> (s"ROW FORMAT SERDE is incompatible with format " + s"'${format.toLowerCase(Locale.ROOT)}', which also specifies a serde")), @@ -3179,7 +3179,7 @@ class HiveDDLSuite exception = intercept[ParseException] { sql(sql1) }, - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map( "message" -> ("ROW FORMAT DELIMITED is only compatible " + "with 'textfile', not 'parquet'")), @@ -3226,7 +3226,7 @@ class HiveDDLSuite spark.sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path.getCanonicalPath}' " + s"STORED AS $format SELECT ID, if(1=1, 1, 0), abs(id), '^-' FROM v") }.getCause.asInstanceOf[AnalysisException], - errorClass = "INVALID_COLUMN_NAME_AS_PATH", + condition = "INVALID_COLUMN_NAME_AS_PATH", parameters = Map( "datasource" -> "HiveFileFormat", "columnName" -> "`(IF((1 = 1), 1, 0))`" @@ -3254,7 +3254,7 @@ class HiveDDLSuite |FROM v """.stripMargin) }.getCause.asInstanceOf[AnalysisException], - errorClass = "INVALID_COLUMN_NAME_AS_PATH", + condition = "INVALID_COLUMN_NAME_AS_PATH", parameters = Map("datasource" -> "HiveFileFormat", "columnName" -> "`IF(ID=1,ID,0)`") ) } @@ -3276,7 +3276,7 @@ class HiveDDLSuite s"'org.apache.hadoop.hive.ql.udf.UDFUUID' USING JAR '$jar'") } checkError(e, - errorClass = "ROUTINE_ALREADY_EXISTS", + condition = "ROUTINE_ALREADY_EXISTS", parameters = Map("routineName" -> "`f1`", "newRoutineType" -> "routine", "existingRoutineType" -> "routine")) @@ -3305,7 +3305,7 @@ class HiveDDLSuite exception = intercept[SparkUnsupportedOperationException] { sql(sqlCmd) }, - errorClass = "UNSUPPORTED_FEATURE.HIVE_WITH_ANSI_INTERVALS", + condition = "UNSUPPORTED_FEATURE.HIVE_WITH_ANSI_INTERVALS", parameters = Map("tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$tbl`") ) } @@ -3356,7 +3356,7 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql("CREATE TABLE tab (c1 int) PARTITIONED BY (c1) STORED AS PARQUET") }, - errorClass = "ALL_PARTITION_COLUMNS_NOT_ALLOWED", + condition = "ALL_PARTITION_COLUMNS_NOT_ALLOWED", parameters = Map.empty ) } @@ -3369,7 +3369,7 @@ class HiveDDLSuite sql(s"DELETE FROM $tbl WHERE c1 = 1") } checkError(e, - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", parameters = Map( "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$tbl`", "operation" -> "DELETE") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index e5e180e7c135c..42fc50e5b163b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -78,7 +78,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd exception = intercept[ParseException] { body }, - errorClass = "INVALID_STATEMENT_OR_CLAUSE", + condition = "INVALID_STATEMENT_OR_CLAUSE", parameters = Map("operation" -> operation), context = expectedContext) } @@ -683,7 +683,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd exception = intercept[AnalysisException] { sql("SELECT (CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END) FROM src").collect() }, - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"CASE WHEN (key > 2) THEN 3 WHEN 1 THEN 2 ELSE 0 END\"", "paramIndex" -> "second", @@ -819,7 +819,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' |WITH serdeproperties('s1'='9')""".stripMargin) }, - errorClass = "_LEGACY_ERROR_TEMP_3065", + condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", "msg" -> "at least one column must be specified for the table")) @@ -1251,7 +1251,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd """INSERT INTO TABLE dp_test PARTITION(dp) |SELECT key, value, key % 5 FROM src""".stripMargin) }, - errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + condition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`dp_test`", "tableColumns" -> "`key`, `value`, `dp`, `sp`", @@ -1265,7 +1265,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd """INSERT INTO TABLE dp_test PARTITION(dp, sp = 1) |SELECT key, value, key % 5 FROM src""".stripMargin) }, - errorClass = "_LEGACY_ERROR_TEMP_3079", + condition = "_LEGACY_ERROR_TEMP_3079", parameters = Map.empty) } } @@ -1368,7 +1368,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd exception = intercept[AnalysisException] { sql("select * from test_b") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`test_b`"), context = ExpectedContext( fragment = "test_b", @@ -1382,7 +1382,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd exception = intercept[AnalysisException] { s2.sql("select * from test_a") }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", + condition = "TABLE_OR_VIEW_NOT_FOUND", parameters = Map("relationName" -> "`test_a`"), context = ExpectedContext( fragment = "test_a", @@ -1408,7 +1408,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd exception = intercept[AnalysisException] { sql("USE not_existing_db") }, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`spark_catalog`.`not_existing_db`") ) } @@ -1420,7 +1420,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd exception = intercept[AnalysisException] { range(1).selectExpr("not_a_udf()") }, - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", sqlState = None, parameters = Map( "routineName" -> "`not_a_udf`", @@ -1437,7 +1437,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd exception = intercept[AnalysisException] { range(1).selectExpr("not_a_udf()") }, - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", sqlState = None, parameters = Map( "routineName" -> "`not_a_udf`", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index d7d859f57e5b6..df6ef57a581d0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -48,7 +48,7 @@ class HiveResolutionSuite extends HiveComparisonTest { exception = intercept[AnalysisException] { sql("SELECT a[0].b from nested").queryExecution.analyzed }, - errorClass = "AMBIGUOUS_REFERENCE_TO_FIELDS", + condition = "AMBIGUOUS_REFERENCE_TO_FIELDS", sqlState = "42000", parameters = Map("field" -> "`b`", "count" -> "2") ) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala index 9d86c72f86afd..8ec3dd6dffa14 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala @@ -87,7 +87,7 @@ class HiveSQLViewSuite extends SQLViewSuite with TestHiveSingleton { } checkError( exception = e, - errorClass = "INVALID_TEMP_OBJ_REFERENCE", + condition = "INVALID_TEMP_OBJ_REFERENCE", parameters = Map( "obj" -> "VIEW", "objName" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`", @@ -213,7 +213,7 @@ class HiveSQLViewSuite extends SQLViewSuite with TestHiveSingleton { exception = intercept[AnalysisException] { sql("SHOW CREATE TABLE v1") }, - errorClass = "UNSUPPORTED_SHOW_CREATE_TABLE.WITH_UNSUPPORTED_FEATURE", + condition = "UNSUPPORTED_SHOW_CREATE_TABLE.WITH_UNSUPPORTED_FEATURE", sqlState = "0A000", parameters = Map( "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`v1`", @@ -224,7 +224,7 @@ class HiveSQLViewSuite extends SQLViewSuite with TestHiveSingleton { exception = intercept[AnalysisException] { sql("SHOW CREATE TABLE v1 AS SERDE") }, - errorClass = "UNSUPPORTED_SHOW_CREATE_TABLE.WITH_UNSUPPORTED_FEATURE", + condition = "UNSUPPORTED_SHOW_CREATE_TABLE.WITH_UNSUPPORTED_FEATURE", sqlState = "0A000", parameters = Map( "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`v1`", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala index 4aadd710b42a7..9bf84687c8f51 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala @@ -151,7 +151,7 @@ class HiveUDAFSuite extends QueryTest exception = intercept[AnalysisException] { sql("SELECT testUDAFPercentile(x, rand()) from view1 group by y") }, - errorClass = "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION", + condition = "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION", parameters = Map("sqlExpr" -> "\"testUDAFPercentile( x, rand())\""), context = ExpectedContext( fragment = "rand()", @@ -181,7 +181,7 @@ class HiveUDAFSuite extends QueryTest exception = intercept[AnalysisException] { sql(s"SELECT $functionName(100)") }, - errorClass = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + condition = "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", parameters = Map( "functionName" -> toSQLId("longProductSum"), "expectedNum" -> "2", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index 2e88b13f0963d..6604fe2a9d61e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -591,7 +591,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { exception = intercept[AnalysisException] { sql("SELECT dAtABaSe1.unknownFunc(1)") }, - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`dAtABaSe1`.`unknownFunc`", "searchPath" -> @@ -790,7 +790,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { checkError( exception = intercept[SparkException](df.collect()), - errorClass = "FAILED_EXECUTE_UDF", + condition = "FAILED_EXECUTE_UDF", parameters = Map( "functionName" -> "`org`.`apache`.`spark`.`sql`.`hive`.`execution`.`SimpleUDFAssertTrue`", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala index 98801e0b0273a..0c54381551bf8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala @@ -107,7 +107,7 @@ class Hive_2_1_DDLSuite extends SparkFunSuite with TestHiveSingleton { "CREATE TABLE t1 (c1 string) USING parquet", StructType(Array(StructField("c2", IntegerType)))) }, - errorClass = "_LEGACY_ERROR_TEMP_3065", + condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", "msg" -> ("Unable to alter table. " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 05b73e31d1156..3deb355e0e4a9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -221,7 +221,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi val sqlText = "describe functioN abcadf" checkError( exception = intercept[AnalysisException](sql(sqlText)), - errorClass = "UNRESOLVED_ROUTINE", + condition = "UNRESOLVED_ROUTINE", parameters = Map( "routineName" -> "`abcadf`", "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"), @@ -1356,7 +1356,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi exception = intercept[AnalysisException] { sql(s"select id from parquet.`invalid_path`") }, - errorClass = "PATH_NOT_FOUND", + condition = "PATH_NOT_FOUND", parameters = Map("path" -> "file.*invalid_path"), matchPVals = true ) @@ -1413,7 +1413,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi exception = intercept[AnalysisException] { sql(s"select id from hive.`${f.getCanonicalPath}`") }, - errorClass = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", + condition = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", parameters = Map("dataSourceType" -> "hive"), context = ExpectedContext(s"hive.`${f.getCanonicalPath}`", 15, 21 + f.getCanonicalPath.length) @@ -1424,7 +1424,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi exception = intercept[AnalysisException] { sql(s"select id from HIVE.`${f.getCanonicalPath}`") }, - errorClass = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", + condition = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", parameters = Map("dataSourceType" -> "HIVE"), context = ExpectedContext(s"HIVE.`${f.getCanonicalPath}`", 15, 21 + f.getCanonicalPath.length) @@ -1782,7 +1782,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi |AS SELECT 1 AS a, 2 AS b """.stripMargin) }, - errorClass = "_LEGACY_ERROR_TEMP_0035", + condition = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "Column ordering must be ASC, was 'DESC'"), context = ExpectedContext( fragment = "CLUSTERED BY (a) SORTED BY (b DESC) INTO 2 BUCKETS", @@ -2638,7 +2638,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi sql("CREATE TABLE t (a STRING)") checkError( exception = intercept[AnalysisException](sql("INSERT INTO t SELECT a*2 FROM t where b=1")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`b`", "proposal" -> "`a`"), context = ExpectedContext( @@ -2648,7 +2648,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi checkError( exception = intercept[AnalysisException]( sql("INSERT INTO t SELECT cast(a as short) FROM t where b=1")), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = None, parameters = Map("objectName" -> "`b`", "proposal" -> "`a`"), context = ExpectedContext( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala index 2eff462faa8dc..ce1b41ecc6dd7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala @@ -234,7 +234,7 @@ abstract class UDAQuerySuite extends QueryTest with SQLTestUtils with TestHiveSi |GROUP BY key """.stripMargin) }, - errorClass = "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION", + condition = "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION", parameters = Map("sqlExpr" -> "\"mydoublesum(((value + (1.5 * key)) + rand()))\""), context = ExpectedContext( fragment = "value + 1.5 * key + rand()", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala index 1dbe405b217e5..232916b6e05b2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala @@ -36,7 +36,7 @@ class AlterNamespaceSetLocationSuite extends v1.AlterNamespaceSetLocationSuiteBa exception = intercept[AnalysisException] { sql(s"ALTER DATABASE $ns SET LOCATION 'loc'") }, - errorClass = "_LEGACY_ERROR_TEMP_1219", + condition = "_LEGACY_ERROR_TEMP_1219", parameters = Map.empty ) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableAddColumnsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableAddColumnsSuite.scala index 3ae2ff562d102..521ad759c302d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableAddColumnsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableAddColumnsSuite.scala @@ -35,7 +35,7 @@ class AlterTableAddColumnsSuite exception = intercept[SparkUnsupportedOperationException] { sql(s"ALTER TABLE $tbl ADD COLUMNS (ym INTERVAL YEAR)") }, - errorClass = "UNSUPPORTED_FEATURE.HIVE_WITH_ANSI_INTERVALS", + condition = "UNSUPPORTED_FEATURE.HIVE_WITH_ANSI_INTERVALS", parameters = Map("tableName" -> toSQLId(tbl)) ) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/CreateNamespaceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/CreateNamespaceSuite.scala index 12e41a569b346..cc54469a52f3f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/CreateNamespaceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/CreateNamespaceSuite.scala @@ -33,7 +33,7 @@ class CreateNamespaceSuite extends v1.CreateNamespaceSuiteBase with CommandSuite exception = intercept[AnalysisException] { sql(s"CREATE NAMESPACE $catalog.$namespace") }, - errorClass = "REQUIRES_SINGLE_PART_NAMESPACE", + condition = "REQUIRES_SINGLE_PART_NAMESPACE", parameters = Map( "sessionCatalog" -> catalog, "namespace" -> "`ns1`.`ns2`" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala index 4c6252128094f..8e654d28cd033 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala @@ -365,7 +365,7 @@ class ShowCreateTableSuite extends v1.ShowCreateTableSuiteBase with CommandSuite exception = intercept[AnalysisException] { checkCreateSparkTableAsHive("t1") }, - errorClass = "UNSUPPORTED_SHOW_CREATE_TABLE.WITH_UNSUPPORTED_SERDE_CONFIGURATION", + condition = "UNSUPPORTED_SHOW_CREATE_TABLE.WITH_UNSUPPORTED_SERDE_CONFIGURATION", sqlState = "0A000", parameters = Map( "tableName" -> "`spark_catalog`.`default`.`t1`", @@ -438,7 +438,7 @@ class ShowCreateTableSuite extends v1.ShowCreateTableSuiteBase with CommandSuite exception = intercept[AnalysisException] { sql("SHOW CREATE TABLE t1") }, - errorClass = "UNSUPPORTED_SHOW_CREATE_TABLE.ON_TRANSACTIONAL_HIVE_TABLE", + condition = "UNSUPPORTED_SHOW_CREATE_TABLE.ON_TRANSACTIONAL_HIVE_TABLE", sqlState = "0A000", parameters = Map("tableName" -> "`spark_catalog`.`default`.`t1`") ) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala index 64a7731a3bf84..4c6218c6366c8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala @@ -58,7 +58,7 @@ class HiveOrcQuerySuite extends OrcQueryTest with TestHiveSingleton { exception = intercept[AnalysisException] { spark.read.orc(path) }, - errorClass = "UNABLE_TO_INFER_SCHEMA", + condition = "UNABLE_TO_INFER_SCHEMA", parameters = Map("format" -> "ORC") ) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala index bac48f6c0c018..c1084dd4ee7ff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala @@ -117,7 +117,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { exception = intercept[AnalysisException] { sql("select interval 1 days").write.mode("overwrite").orc(orcDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`INTERVAL '1' DAY`", "columnType" -> "\"INTERVAL DAY\"", @@ -128,7 +128,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { exception = intercept[AnalysisException] { sql("select null").write.mode("overwrite").orc(orcDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`NULL`", "columnType" -> "\"VOID\"", @@ -140,7 +140,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { spark.udf.register("testType", () => new IntervalData()) sql("select testType()").write.mode("overwrite").orc(orcDir) }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`testType()`", "columnType" -> "UDT(\"INTERVAL\")", @@ -154,7 +154,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { spark.range(1).write.mode("overwrite").orc(orcDir) spark.read.schema(schema).orc(orcDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "\"INTERVAL\"", @@ -167,7 +167,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { spark.range(1).write.mode("overwrite").orc(orcDir) spark.read.schema(schema).orc(orcDir).collect() }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", parameters = Map( "columnName" -> "`a`", "columnType" -> "UDT(\"INTERVAL\")", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 86401bf923927..56f835b53a75d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -246,7 +246,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes testDF.write.format(dataSourceName) .mode(SaveMode.ErrorIfExists).save(file.getCanonicalPath) }, - errorClass = "PATH_ALREADY_EXISTS", + condition = "PATH_ALREADY_EXISTS", parameters = Map("outputPath" -> "file:.*"), matchPVals = true ) @@ -354,7 +354,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes .partitionBy("p1", "p2") .save(file.getCanonicalPath) }, - errorClass = "PATH_ALREADY_EXISTS", + condition = "PATH_ALREADY_EXISTS", parameters = Map("outputPath" -> "file:.*"), matchPVals = true ) From b00a36e998748845656c767652abb3d9e3211595 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 08:16:03 +0200 Subject: [PATCH 02/14] Rename errorClass to condition in checkErrorInExpression --- .../CollectionExpressionsSuite.scala | 22 +++++++++---------- .../expressions/ComplexTypeSuite.scala | 6 ++--- .../expressions/DateExpressionsSuite.scala | 4 ++-- .../expressions/ExpressionEvalHelper.scala | 12 +++++----- .../HigherOrderFunctionsSuite.scala | 4 ++-- .../expressions/ObjectExpressionsSuite.scala | 2 +- .../expressions/StringExpressionsSuite.scala | 2 +- .../catalyst/expressions/TryCastSuite.scala | 2 +- 8 files changed, 27 insertions(+), 27 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 40490a423b781..e9de59b3ec48c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -266,7 +266,7 @@ class CollectionExpressionsSuite checkErrorInExpression[SparkRuntimeException]( MapConcat(Seq(m0, m1)), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> "a", "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") @@ -426,7 +426,7 @@ class CollectionExpressionsSuite checkErrorInExpression[SparkRuntimeException]( MapFromEntries(ai4), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> "1", "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") @@ -458,7 +458,7 @@ class CollectionExpressionsSuite checkErrorInExpression[SparkRuntimeException]( MapFromEntries(as4), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> "a", "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") @@ -720,7 +720,7 @@ class CollectionExpressionsSuite checkEvaluation(Slice(a0, Literal(-1), Literal(2)), Seq(6)) checkErrorInExpression[SparkRuntimeException]( expression = Slice(a0, Literal(1), Literal(-1)), - errorClass = "INVALID_PARAMETER_VALUE.LENGTH", + condition = "INVALID_PARAMETER_VALUE.LENGTH", parameters = Map( "parameter" -> toSQLId("length"), "length" -> (-1).toString, @@ -728,7 +728,7 @@ class CollectionExpressionsSuite )) checkErrorInExpression[SparkRuntimeException]( expression = Slice(a0, Literal(0), Literal(1)), - errorClass = "INVALID_PARAMETER_VALUE.START", + condition = "INVALID_PARAMETER_VALUE.START", parameters = Map( "parameter" -> toSQLId("start"), "functionName" -> toSQLId("slice") @@ -910,7 +910,7 @@ class CollectionExpressionsSuite // SPARK-43393: test Sequence overflow checking checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", parameters = Map( "numberOfElements" -> (BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1).toString, "functionName" -> toSQLId("sequence"), @@ -918,7 +918,7 @@ class CollectionExpressionsSuite "parameter" -> toSQLId("count"))) checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(0L), Literal(Long.MaxValue), Literal(1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", parameters = Map( "numberOfElements" -> (BigInt(Long.MaxValue) + 1).toString, "functionName" -> toSQLId("sequence"), @@ -926,7 +926,7 @@ class CollectionExpressionsSuite "parameter" -> toSQLId("count"))) checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(0L), Literal(Long.MinValue), Literal(-1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", parameters = Map( "numberOfElements" -> ((0 - BigInt(Long.MinValue)) + 1).toString(), "functionName" -> toSQLId("sequence"), @@ -934,7 +934,7 @@ class CollectionExpressionsSuite "parameter" -> toSQLId("count"))) checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(Long.MinValue), Literal(Long.MaxValue), Literal(1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", parameters = Map( "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, "functionName" -> toSQLId("sequence"), @@ -942,7 +942,7 @@ class CollectionExpressionsSuite "parameter" -> toSQLId("count"))) checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(Long.MaxValue), Literal(Long.MinValue), Literal(-1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", parameters = Map( "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, "functionName" -> toSQLId("sequence"), @@ -950,7 +950,7 @@ class CollectionExpressionsSuite "parameter" -> toSQLId("count"))) checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(Long.MaxValue), Literal(-1L), Literal(-1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + condition = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", parameters = Map( "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { -1L } + 1).toString, "functionName" -> toSQLId("sequence"), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index b92b7138c153f..7baad5ea92a00 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -285,7 +285,7 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { checkErrorInExpression[SparkRuntimeException]( CreateMap(Seq(Literal(1), Literal(2), Literal(1), Literal(3))), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> "1", "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") @@ -430,7 +430,7 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { MapFromArrays( Literal.create(Seq(1, 1), ArrayType(IntegerType)), Literal.create(Seq(2, 3), ArrayType(IntegerType))), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> "1", "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") @@ -556,7 +556,7 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { checkErrorInExpression[SparkRuntimeException]( new StringToMap(Literal("a:1,b:2,a:3")), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> "a", "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 1e10da6f93469..21ae35146282b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -2033,12 +2033,12 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkErrorInExpression[SparkArithmeticException](TimestampAdd("DAY", Literal(106751992), Literal(0L, TimestampType)), - errorClass = "DATETIME_OVERFLOW", + condition = "DATETIME_OVERFLOW", parameters = Map("operation" -> "add 106751992 DAY to TIMESTAMP '1970-01-01 00:00:00'")) checkErrorInExpression[SparkArithmeticException](TimestampAdd("QUARTER", Literal(1431655764), Literal(0L, TimestampType)), - errorClass = "DATETIME_OVERFLOW", + condition = "DATETIME_OVERFLOW", parameters = Map("operation" -> "add 1431655764 QUARTER to TIMESTAMP '1970-01-01 00:00:00'")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index 87d1208257070..02c7ed727a530 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -154,22 +154,22 @@ trait ExpressionEvalHelper extends ScalaCheckDrivenPropertyChecks with PlanTestB protected def checkErrorInExpression[T <: SparkThrowable : ClassTag]( expression: => Expression, - errorClass: String, + condition: String, parameters: Map[String, String] = Map.empty): Unit = { - checkErrorInExpression[T](expression, InternalRow.empty, errorClass, parameters) + checkErrorInExpression[T](expression, InternalRow.empty, condition, parameters) } protected def checkErrorInExpression[T <: SparkThrowable : ClassTag]( expression: => Expression, inputRow: InternalRow, - errorClass: String): Unit = { - checkErrorInExpression[T](expression, inputRow, errorClass, Map.empty[String, String]) + condition: String): Unit = { + checkErrorInExpression[T](expression, inputRow, condition, Map.empty[String, String]) } protected def checkErrorInExpression[T <: SparkThrowable : ClassTag]( expression: => Expression, inputRow: InternalRow, - errorClass: String, + condition: String, parameters: Map[String, String]): Unit = { def checkException(eval: => Unit, testMode: String): Unit = { @@ -179,7 +179,7 @@ trait ExpressionEvalHelper extends ScalaCheckDrivenPropertyChecks with PlanTestB withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> fallbackMode.toString) { checkError( exception = intercept[T](eval), - condition = errorClass, + condition = condition, parameters = parameters ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala index c06705606567d..cc36cd73d6d77 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala @@ -471,7 +471,7 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper checkErrorInExpression[SparkRuntimeException]( transformKeys(ai0, modKey), - errorClass = "DUPLICATED_MAP_KEY", + condition = "DUPLICATED_MAP_KEY", parameters = Map( "key" -> "1", "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"") @@ -858,7 +858,7 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper SQLConf.LEGACY_ALLOW_NULL_COMPARISON_RESULT_IN_ARRAY_SORT.key -> "false") { checkErrorInExpression[SparkException]( expression = arraySort(Literal.create(Seq(3, 1, 1, 2)), comparator), - errorClass = "COMPARATOR_RETURNS_NULL", + condition = "COMPARATOR_RETURNS_NULL", parameters = Map("firstValue" -> "1", "secondValue" -> "1") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index d37a6928189ff..762a4e9166d51 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -588,7 +588,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { DoubleType, DoubleType), inputRow = InternalRow.fromSeq(Seq(Row(1))), - errorClass = "INVALID_EXTERNAL_TYPE", + condition = "INVALID_EXTERNAL_TYPE", parameters = Map[String, String]( "externalType" -> "java.lang.Integer", "type" -> "\"DOUBLE\"", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 69988ca86f17d..beefabd981089 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -1505,7 +1505,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkErrorInExpression[SparkIllegalArgumentException]( toNumberExpr, - errorClass = "INVALID_FORMAT.MISMATCH_INPUT", + condition = "INVALID_FORMAT.MISMATCH_INPUT", parameters = Map( "inputType" -> "\"STRING\"", "input" -> str, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala index f138d9642d1e1..446514de91d69 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala @@ -47,7 +47,7 @@ class TryCastSuite extends CastWithAnsiOnSuite { override def checkErrorInExpression[T <: SparkThrowable : ClassTag]( expression: => Expression, inputRow: InternalRow, - errorClass: String, + condition: String, parameters: Map[String, String]): Unit = { checkEvaluation(expression, null, inputRow) } From db4bfbaea655b3811d1ec36538142faa10f51cc9 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 08:20:44 +0200 Subject: [PATCH 03/14] Fix integration tests --- .../jdbc/MsSqlServerIntegrationSuite.scala | 2 +- .../sql/jdbc/v2/DB2IntegrationSuite.scala | 2 +- .../jdbc/v2/MsSqlServerIntegrationSuite.scala | 4 ++-- .../sql/jdbc/v2/MySQLIntegrationSuite.scala | 4 ++-- .../sql/jdbc/v2/MySQLNamespaceSuite.scala | 8 ++++---- .../sql/jdbc/v2/OracleIntegrationSuite.scala | 4 ++-- .../jdbc/v2/PostgresIntegrationSuite.scala | 4 ++-- .../sql/jdbc/v2/V2JDBCNamespaceTest.scala | 2 +- .../apache/spark/sql/jdbc/v2/V2JDBCTest.scala | 20 +++++++++---------- 9 files changed, 25 insertions(+), 25 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 30009c03c49fd..90cd68e6e1d24 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -490,7 +490,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { .option("query", "SELECT @myvariant1 as variant1, @myvariant2 as variant2") .load() }, - errorClass = "UNRECOGNIZED_SQL_TYPE", + condition = "UNRECOGNIZED_SQL_TYPE", parameters = Map("typeName" -> "sql_variant", "jdbcType" -> "-156")) } } diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2IntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2IntegrationSuite.scala index b337eb2fc9b3b..91a82075a3607 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2IntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2IntegrationSuite.scala @@ -87,7 +87,7 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest { exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"DOUBLE\"", "newType" -> "\"VARCHAR(10)\"", diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala index 27ec98e9ac451..e5fd453cb057c 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala @@ -97,7 +97,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JD exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"STRING\"", "newType" -> "\"INT\"", @@ -115,7 +115,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JD exception = intercept[SparkSQLFeatureNotSupportedException] { sql(s"ALTER TABLE $tbl ALTER COLUMN ID DROP NOT NULL") }, - errorClass = "_LEGACY_ERROR_TEMP_2271") + condition = "_LEGACY_ERROR_TEMP_2271") } test("SPARK-47440: SQLServer does not support boolean expression in binary comparison") { diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala index 81aacf2c14d7a..60685f5c0c6b9 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala @@ -98,7 +98,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"STRING\"", "newType" -> "\"INT\"", @@ -131,7 +131,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest exception = intercept[SparkSQLFeatureNotSupportedException] { sql(s"ALTER TABLE $tbl ALTER COLUMN ID DROP NOT NULL") }, - errorClass = "_LEGACY_ERROR_TEMP_2271") + condition = "_LEGACY_ERROR_TEMP_2271") } override def testCreateTableWithProperty(tbl: String): Unit = { diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala index 42d82233b421b..5e40f0bbc4554 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala @@ -62,7 +62,7 @@ class MySQLNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespac exception = intercept[SparkSQLFeatureNotSupportedException] { catalog.createNamespace(Array("foo"), Map("comment" -> "test comment").asJava) }, - errorClass = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE", + condition = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE", parameters = Map("namespace" -> "`foo`") ) assert(catalog.namespaceExists(Array("foo")) === false) @@ -74,7 +74,7 @@ class MySQLNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespac Array("foo"), NamespaceChange.setProperty("comment", "comment for foo")) }, - errorClass = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE", + condition = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE", parameters = Map("namespace" -> "`foo`") ) @@ -82,7 +82,7 @@ class MySQLNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespac exception = intercept[SparkSQLFeatureNotSupportedException] { catalog.alterNamespace(Array("foo"), NamespaceChange.removeProperty("comment")) }, - errorClass = "UNSUPPORTED_FEATURE.REMOVE_NAMESPACE_COMMENT", + condition = "UNSUPPORTED_FEATURE.REMOVE_NAMESPACE_COMMENT", parameters = Map("namespace" -> "`foo`") ) @@ -90,7 +90,7 @@ class MySQLNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespac exception = intercept[SparkSQLFeatureNotSupportedException] { catalog.dropNamespace(Array("foo"), cascade = false) }, - errorClass = "UNSUPPORTED_FEATURE.DROP_NAMESPACE", + condition = "UNSUPPORTED_FEATURE.DROP_NAMESPACE", parameters = Map("namespace" -> "`foo`") ) catalog.dropNamespace(Array("foo"), cascade = true) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala index 342fb4bb38e60..2c97a588670a8 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala @@ -118,7 +118,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTes exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"DECIMAL(19,0)\"", "newType" -> "\"INT\"", @@ -139,7 +139,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTes exception = intercept[SparkRuntimeException] { sql(s"INSERT INTO $tableName SELECT rpad('hi', 256, 'spark')") }, - errorClass = "EXCEED_LIMIT_LENGTH", + condition = "EXCEED_LIMIT_LENGTH", parameters = Map("limit" -> "255") ) } diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala index e22136a09a56c..850391e8dc33c 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala @@ -84,7 +84,7 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT exception = intercept[AnalysisException] { sql(sql1) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + condition = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( "originType" -> "\"STRING\"", "newType" -> "\"INT\"", @@ -118,7 +118,7 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT sql(s"CREATE TABLE $t2(c int)") checkError( exception = intercept[TableAlreadyExistsException](sql(s"ALTER TABLE $t1 RENAME TO t2")), - errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> "`t2`") ) } diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala index e4cc88cec0f5e..3b1a457214be7 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala @@ -92,7 +92,7 @@ private[v2] trait V2JDBCNamespaceTest extends SharedSparkSession with DockerInte catalog.listNamespaces(Array("foo")) } checkError(e, - errorClass = "SCHEMA_NOT_FOUND", + condition = "SCHEMA_NOT_FOUND", parameters = Map("schemaName" -> "`foo`")) } diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala index b0ab614b27d1f..d3629d871cd42 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala @@ -71,7 +71,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`bad_column`", @@ -92,11 +92,11 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu private def checkErrorFailedJDBC( e: AnalysisException, - errorClass: String, + condition: String, tbl: String): Unit = { checkErrorMatchPVals( exception = e, - errorClass = errorClass, + condition = condition, parameters = Map( "url" -> "jdbc:.*", "tableName" -> s"`$tbl`") @@ -126,7 +126,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu exception = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.alt_table ADD COLUMNS (C3 DOUBLE)") }, - errorClass = "FIELD_ALREADY_EXISTS", + condition = "FIELD_ALREADY_EXISTS", parameters = Map( "op" -> "add", "fieldNames" -> "`C3`", @@ -159,7 +159,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`bad_column`", @@ -182,7 +182,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu exception = intercept[AnalysisException] { sql(sqlText) }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", sqlState = "42703", parameters = Map( "objectName" -> "`bad_column`", @@ -206,7 +206,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu exception = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.alt_table RENAME COLUMN ID1 TO ID2") }, - errorClass = "FIELD_ALREADY_EXISTS", + condition = "FIELD_ALREADY_EXISTS", parameters = Map( "op" -> "rename", "fieldNames" -> "`ID2`", @@ -308,7 +308,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu exception = intercept[IndexAlreadyExistsException] { sql(s"CREATE index i1 ON $catalogName.new_table (col1)") }, - errorClass = "INDEX_ALREADY_EXISTS", + condition = "INDEX_ALREADY_EXISTS", parameters = Map("indexName" -> "`i1`", "tableName" -> "`new_table`") ) @@ -333,7 +333,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu exception = intercept[NoSuchIndexException] { sql(s"DROP index i1 ON $catalogName.new_table") }, - errorClass = "INDEX_NOT_FOUND", + condition = "INDEX_NOT_FOUND", parameters = Map("indexName" -> "`i1`", "tableName" -> "`new_table`") ) } @@ -975,7 +975,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu exception = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.tbl2 RENAME TO tbl1") }, - errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + condition = "TABLE_OR_VIEW_ALREADY_EXISTS", parameters = Map("relationName" -> "`tbl1`") ) } From 83b1edcb8e2866fe0a8959ca087b27a3c23ba041 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 16:29:42 +0200 Subject: [PATCH 04/14] Fix docs --- core/src/test/scala/org/apache/spark/SparkFunSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index e1656583dce99..4aca98c1413ab 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -327,9 +327,9 @@ abstract class SparkFunSuite } /** - * Checks an exception with an error class against expected results. + * Checks an exception with an error condition against expected results. * @param exception The exception to check - * @param condition The expected error class identifying the error + * @param condition The expected error condition identifying the error * @param sqlState Optional the expected SQLSTATE, not verified if not supplied * @param parameters A map of parameter names and values. The names are as defined * in the error-classes file. From 124dea5c039c1f1074e45011a8d2e678f75a3551 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 16:31:40 +0200 Subject: [PATCH 05/14] Rename errorClass to condition in ProtobufSerdeSuite --- .../sql/protobuf/ProtobufSerdeSuite.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala index d71aa400c0ca7..2737bb9feb3ad 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala @@ -95,7 +95,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { protoFile, Deserializer, fieldMatch, - errorClass = "CANNOT_CONVERT_PROTOBUF_MESSAGE_TYPE_TO_SQL_TYPE", + condition = "CANNOT_CONVERT_PROTOBUF_MESSAGE_TYPE_TO_SQL_TYPE", params = Map( "protobufType" -> "MissMatchTypeInRoot", "toType" -> toSQLType(CATALYST_STRUCT))) @@ -104,7 +104,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { protoFile, Serializer, fieldMatch, - errorClass = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", + condition = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", params = Map( "protobufType" -> "MissMatchTypeInRoot", "toType" -> toSQLType(CATALYST_STRUCT))) @@ -122,7 +122,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { protoFile, Serializer, BY_NAME, - errorClass = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", + condition = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", params = Map( "protobufType" -> "FieldMissingInProto", "toType" -> toSQLType(CATALYST_STRUCT))) @@ -132,7 +132,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { Serializer, BY_NAME, nonnullCatalyst, - errorClass = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", + condition = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", params = Map( "protobufType" -> "FieldMissingInProto", "toType" -> toSQLType(nonnullCatalyst))) @@ -150,7 +150,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { Deserializer, fieldMatch, catalyst, - errorClass = "CANNOT_CONVERT_PROTOBUF_MESSAGE_TYPE_TO_SQL_TYPE", + condition = "CANNOT_CONVERT_PROTOBUF_MESSAGE_TYPE_TO_SQL_TYPE", params = Map( "protobufType" -> "MissMatchTypeInDeepNested", "toType" -> toSQLType(catalyst))) @@ -160,7 +160,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { Serializer, fieldMatch, catalyst, - errorClass = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", + condition = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", params = Map( "protobufType" -> "MissMatchTypeInDeepNested", "toType" -> toSQLType(catalyst))) @@ -177,7 +177,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { Serializer, BY_NAME, catalystSchema = foobarSQLType, - errorClass = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", + condition = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", params = Map( "protobufType" -> "FoobarWithRequiredFieldBar", "toType" -> toSQLType(foobarSQLType))) @@ -199,7 +199,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { Serializer, BY_NAME, catalystSchema = nestedFoobarSQLType, - errorClass = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", + condition = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", params = Map( "protobufType" -> "NestedFoobarWithRequiredFieldBar", "toType" -> toSQLType(nestedFoobarSQLType))) @@ -254,7 +254,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { serdeFactory: SerdeFactory[_], fieldMatchType: MatchType, catalystSchema: StructType = CATALYST_STRUCT, - errorClass: String, + condition: String, params: Map[String, String]): Unit = { val e = intercept[AnalysisException] { @@ -274,7 +274,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { assert(e.getMessage === expectMsg) checkError( exception = e, - condition = errorClass, + condition = condition, parameters = params) } From 0156a30e09b91bc6fe3870119c45fa4d6732fc76 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 16:32:39 +0200 Subject: [PATCH 06/14] Rename a val --- .../src/test/scala/org/apache/spark/sql/VariantSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala index 5c215a7875b59..18a3e8f5a1da1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala @@ -290,7 +290,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval (s"named_struct('value', $v, 'metadata', cast(null as binary))", "INVALID_VARIANT_FROM_PARQUET.NULLABLE_OR_NOT_BINARY_FIELD", Map("field" -> "metadata")) ) - cases.foreach { case (structDef, errorClass, parameters) => + cases.foreach { case (structDef, condition, parameters) => Seq(false, true).foreach { vectorizedReader => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorizedReader.toString) { withTempDir { dir => @@ -302,7 +302,7 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval val e = intercept[org.apache.spark.SparkException](result.collect()) checkError( exception = e.getCause.asInstanceOf[AnalysisException], - condition = errorClass, + condition = condition, parameters = parameters ) } From eb3bb17996b77564119bb7e6cb13b362731b3de6 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 16:34:20 +0200 Subject: [PATCH 07/14] Rename errorClass to condition in LateralColumnAliasSuite --- .../apache/spark/sql/LateralColumnAliasSuite.scala | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala index 8ea2b2a1edb7a..9afba65183974 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala @@ -209,11 +209,14 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { } private def checkSameError( - q1: String, q2: String, errorClass: String, errorParams: Map[String, String]): Unit = { + q1: String, + q2: String, + condition: String, + errorParams: Map[String, String]): Unit = { val e1 = intercept[AnalysisException] { sql(q1) } val e2 = intercept[AnalysisException] { sql(q2) } - assert(e1.getErrorClass == errorClass) - assert(e2.getErrorClass == errorClass) + assert(e1.getErrorClass == condition) + assert(e2.getErrorClass == condition) errorParams.foreach { case (k, v) => assert(e1.messageParameters.get(k).exists(_ == v)) assert(e2.messageParameters.get(k).exists(_ == v)) @@ -1188,7 +1191,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { s"from $testTable", s"select dept as d, d, rank() over (partition by dept order by avg(salary)) " + s"from $testTable", - errorClass = "MISSING_GROUP_BY", + condition = "MISSING_GROUP_BY", errorParams = Map.empty ) checkSameError( @@ -1196,7 +1199,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { s"from $testTable", "select salary as s, s, sum(sum(salary)) over (partition by dept order by salary) " + s"from $testTable", - errorClass = "MISSING_GROUP_BY", + condition = "MISSING_GROUP_BY", errorParams = Map.empty ) From 4ef508d38262a09a3a0dcd6d62360304988cd93b Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 16:36:23 +0200 Subject: [PATCH 08/14] Rename errorClass to condition in StreamingDataSourceV2Suite --- .../sql/streaming/sources/StreamingDataSourceV2Suite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index 64b0e96845d68..544f910333bfb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -324,7 +324,7 @@ class StreamingDataSourceV2Suite extends StreamTest { readFormat: String, writeFormat: String, trigger: Trigger, - errorClass: String, + condition: String, parameters: Map[String, String]) = { val query = spark.readStream .format(readFormat) @@ -339,7 +339,7 @@ class StreamingDataSourceV2Suite extends StreamTest { assert(query.exception.get.cause != null) checkErrorMatchPVals( exception = query.exception.get.cause.asInstanceOf[SparkUnsupportedOperationException], - condition = errorClass, + condition = condition, parameters = parameters ) } @@ -478,7 +478,7 @@ class StreamingDataSourceV2Suite extends StreamTest { } else { // Invalid - trigger is microbatch but reader is not testPostCreationNegativeCase(read, write, trigger, - errorClass = "_LEGACY_ERROR_TEMP_2209", + condition = "_LEGACY_ERROR_TEMP_2209", parameters = Map( "srcName" -> read, "disabledSources" -> "", From e9b3f1d7695c0e9a2beff8334b991d80291c3eb9 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 16:38:06 +0200 Subject: [PATCH 09/14] Fix DatasetSuite --- .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 39f30b141df03..84df437305966 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -2051,17 +2051,17 @@ class DatasetSuite extends QueryTest test("SPARK-22472: add null check for top-level primitive values") { // If the primitive values are from Option, we need to do runtime null check. val ds = Seq(Some(1), None).toDS().as[Int] - val errorClass = "NOT_NULL_ASSERT_VIOLATION" + val condition = "NOT_NULL_ASSERT_VIOLATION" val sqlState = "42000" val parameters = Map("walkedTypePath" -> "\n- root class: \"int\"\n") checkError( exception = intercept[SparkRuntimeException](ds.collect()), - condition = errorClass, + condition = condition, sqlState = sqlState, parameters = parameters) checkError( exception = intercept[SparkRuntimeException](ds.map(_ * 2).collect()), - condition = errorClass, + condition = condition, sqlState = sqlState, parameters = parameters) @@ -2071,12 +2071,12 @@ class DatasetSuite extends QueryTest val ds = spark.read.parquet(path.getCanonicalPath).as[Int] checkError( exception = intercept[SparkRuntimeException](ds.collect()), - condition = errorClass, + condition = condition, sqlState = sqlState, parameters = parameters) checkError( exception = intercept[SparkRuntimeException](ds.map(_ * 2).collect()), - condition = errorClass, + condition = condition, sqlState = sqlState, parameters = parameters) } From c764d2a635fb97174b76a75697361ea3a32802ef Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 16:40:52 +0200 Subject: [PATCH 10/14] Fix names in SQLViewSuite --- .../org/apache/spark/sql/execution/SQLViewSuite.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 9a45d43656b35..b26cdfaeb756a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -389,12 +389,13 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } } - private def assertAnalysisErrorClass(query: String, - errorClass: String, + private def assertAnalysisErrorCondition( + query: String, + condition: String, parameters: Map[String, String], context: ExpectedContext): Unit = { val e = intercept[AnalysisException](sql(query)) - checkError(e, condition = errorClass, parameters = parameters, context = context) + checkError(e, condition = condition, parameters = parameters, context = context) } test("error handling: insert/load table commands against a view") { @@ -488,7 +489,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { test("error handling: fail if the temp view sql itself is invalid") { // A database that does not exist - assertAnalysisErrorClass( + assertAnalysisErrorCondition( "CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM db_not_exist234.jt", "TABLE_OR_VIEW_NOT_FOUND", Map("relationName" -> "`db_not_exist234`.`jt`"), From 82c6f1bc3ffccf84b1fab6b2fe0fb55c85b4b949 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 16:54:31 +0200 Subject: [PATCH 11/14] Rename errorClass in AnalysisTest --- .../analysis/AnalysisErrorSuite.scala | 279 +++++++++--------- .../AnalysisExceptionPositionSuite.scala | 4 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 126 ++++---- .../sql/catalyst/analysis/AnalysisTest.scala | 13 +- ...eateTablePartitioningValidationSuite.scala | 16 +- .../analysis/ResolveNaturalJoinSuite.scala | 16 +- .../analysis/ResolveSubquerySuite.scala | 34 +-- .../analysis/V2WriteAnalysisSuite.scala | 140 ++++----- .../V2CommandsCaseSensitivitySuite.scala | 24 +- 9 files changed, 327 insertions(+), 325 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 4c66c52258cbd..70cc50a23a6a2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -110,14 +110,15 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { } } - def errorClassTest( + def errorConditionTest( name: String, plan: LogicalPlan, - errorClass: String, + condition: String, messageParameters: Map[String, String], caseSensitive: Boolean = true): Unit = { test(name) { - assertAnalysisErrorClass(plan, errorClass, messageParameters, caseSensitive = caseSensitive) + assertAnalysisErrorCondition( + plan, condition, messageParameters, caseSensitive = caseSensitive) } } @@ -134,10 +135,10 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { testRelation.select(ScalarSubquery(LocalRelation()).as("a")), "Scalar subquery must return only one column, but got 0" :: Nil) - errorClassTest( + errorConditionTest( "single invalid type, single arg", testRelation.select(TestFunction(dateLit :: Nil, IntegerType :: Nil).as("a")), - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", messageParameters = Map( "sqlExpr" -> "\"testfunction(NULL)\"", "paramIndex" -> "first", @@ -145,11 +146,11 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { "inputType" -> "\"DATE\"", "requiredType" -> "\"INT\"")) - errorClassTest( + errorConditionTest( "single invalid type, second arg", testRelation.select( TestFunction(dateLit :: dateLit :: Nil, DateType :: IntegerType :: Nil).as("a")), - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", messageParameters = Map( "sqlExpr" -> "\"testfunction(NULL, NULL)\"", "paramIndex" -> "second", @@ -157,11 +158,11 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { "inputType" -> "\"DATE\"", "requiredType" -> "\"INT\"")) - errorClassTest( + errorConditionTest( "multiple invalid type", testRelation.select( TestFunction(dateLit :: dateLit :: Nil, IntegerType :: IntegerType :: Nil).as("a")), - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", messageParameters = Map( "sqlExpr" -> "\"testfunction(NULL, NULL)\"", "paramIndex" -> "first", @@ -169,17 +170,17 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { "inputType" -> "\"DATE\"", "requiredType" -> "\"INT\"")) - errorClassTest( + errorConditionTest( "SPARK-44477: type check failure", testRelation.select( TestFunctionWithTypeCheckFailure(dateLit :: Nil, BinaryType :: Nil).as("a")), - errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", messageParameters = Map( "sqlExpr" -> "\"testfunctionwithtypecheckfailure(NULL)\"", "msg" -> "Expression must be a binary", "hint" -> "")) - errorClassTest( + errorConditionTest( "invalid window function", testRelation2.select( WindowExpression( @@ -188,10 +189,10 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { UnresolvedAttribute("a") :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, UnspecifiedFrame)).as("window")), - errorClass = "UNSUPPORTED_EXPR_FOR_WINDOW", + condition = "UNSUPPORTED_EXPR_FOR_WINDOW", messageParameters = Map("sqlExpr" -> "\"0\"")) - errorClassTest( + errorConditionTest( "distinct aggregate function in window", testRelation2.select( WindowExpression( @@ -200,7 +201,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { UnresolvedAttribute("a") :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, UnspecifiedFrame)).as("window")), - errorClass = "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED", + condition = "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED", messageParameters = Map("windowExpr" -> s""" |"count(DISTINCT b) OVER (PARTITION BY a ORDER BY b ASC NULLS FIRST @@ -221,9 +222,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) test("distinct function") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CatalystSqlParser.parsePlan("SELECT hex(DISTINCT a) FROM TaBlE"), - expectedErrorClass = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + expectedErrorCondition = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", expectedMessageParameters = Map( "prettyName" -> toSQLId("hex"), "syntax" -> toSQLStmt("DISTINCT")), @@ -231,9 +232,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { } test("non aggregate function with filter predicate") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CatalystSqlParser.parsePlan("SELECT hex(a) FILTER (WHERE c = 1) FROM TaBlE2"), - expectedErrorClass = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + expectedErrorCondition = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", expectedMessageParameters = Map( "prettyName" -> toSQLId("hex"), "syntax" -> toSQLStmt("FILTER CLAUSE")), @@ -241,9 +242,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { } test("distinct window function") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CatalystSqlParser.parsePlan("SELECT percent_rank(DISTINCT a) OVER () FROM TaBlE"), - expectedErrorClass = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + expectedErrorCondition = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", expectedMessageParameters = Map( "prettyName" -> toSQLId("percent_rank"), "syntax" -> toSQLStmt("DISTINCT")), @@ -251,10 +252,10 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { } test("window function with filter predicate") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CatalystSqlParser.parsePlan( "SELECT percent_rank(a) FILTER (WHERE c > 1) OVER () FROM TaBlE2"), - expectedErrorClass = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + expectedErrorCondition = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", expectedMessageParameters = Map( "prettyName" -> toSQLId("percent_rank"), "syntax" -> toSQLStmt("FILTER CLAUSE")), @@ -262,7 +263,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { } test("window specification error") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = CatalystSqlParser.parsePlan( """ |WITH sample_data AS ( @@ -274,17 +275,17 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { |FROM sample_data |GROUP BY a, b; |""".stripMargin), - expectedErrorClass = "MISSING_WINDOW_SPECIFICATION", + expectedErrorCondition = "MISSING_WINDOW_SPECIFICATION", expectedMessageParameters = Map( "windowName" -> "b", "docroot" -> SPARK_DOC_ROOT)) } test("higher order function with filter predicate") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CatalystSqlParser.parsePlan("SELECT aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x) " + "FILTER (WHERE c > 1)"), - expectedErrorClass = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + expectedErrorCondition = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", expectedMessageParameters = Map( "prettyName" -> toSQLId("aggregate"), "syntax" -> toSQLStmt("FILTER CLAUSE")), @@ -293,9 +294,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { } test("function don't support ignore nulls") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CatalystSqlParser.parsePlan("SELECT hex(a) IGNORE NULLS FROM TaBlE2"), - expectedErrorClass = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + expectedErrorCondition = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", expectedMessageParameters = Map( "prettyName" -> toSQLId("hex"), "syntax" -> toSQLStmt("IGNORE NULLS")), @@ -303,9 +304,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { } test("some window function don't support ignore nulls") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CatalystSqlParser.parsePlan("SELECT percent_rank(a) IGNORE NULLS FROM TaBlE2"), - expectedErrorClass = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + expectedErrorCondition = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", expectedMessageParameters = Map( "prettyName" -> toSQLId("percent_rank"), "syntax" -> toSQLStmt("IGNORE NULLS")), @@ -313,9 +314,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { } test("aggregate function don't support ignore nulls") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CatalystSqlParser.parsePlan("SELECT count(a) IGNORE NULLS FROM TaBlE2"), - expectedErrorClass = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + expectedErrorCondition = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", expectedMessageParameters = Map( "prettyName" -> toSQLId("count"), "syntax" -> toSQLStmt("IGNORE NULLS")), @@ -323,10 +324,10 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { } test("higher order function don't support ignore nulls") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CatalystSqlParser.parsePlan( "SELECT aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x) IGNORE NULLS"), - expectedErrorClass = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + expectedErrorCondition = "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", expectedMessageParameters = Map( "prettyName" -> toSQLId("aggregate"), "syntax" -> toSQLStmt("IGNORE NULLS")), @@ -334,11 +335,11 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { "aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x) IGNORE NULLS", 7, 68))) } - errorClassTest( + errorConditionTest( name = "nested aggregate functions", testRelation.groupBy($"a")( Max(Count(Literal(1)).toAggregateExpression()).toAggregateExpression()), - errorClass = "NESTED_AGGREGATE_FUNCTION", + condition = "NESTED_AGGREGATE_FUNCTION", messageParameters = Map.empty ) @@ -353,7 +354,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { SpecifiedWindowFrame(RangeFrame, Literal(1), Literal(2)))).as("window")), "Cannot specify window frame for lead function" :: Nil) - errorClassTest( + errorConditionTest( "the offset of nth_value window function is negative or zero", testRelation2.select( WindowExpression( @@ -362,14 +363,14 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { UnresolvedAttribute("a") :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, SpecifiedWindowFrame(RowFrame, Literal(0), Literal(0)))).as("window")), - errorClass = "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + condition = "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", messageParameters = Map( "sqlExpr" -> "\"nth_value(b, 0)\"", "exprName" -> "offset", "valueRange" -> "(0, 9223372036854775807]", "currentValue" -> "0L")) - errorClassTest( + errorConditionTest( "the offset of nth_value window function is not int literal", testRelation2.select( WindowExpression( @@ -378,7 +379,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { UnresolvedAttribute("a") :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, SpecifiedWindowFrame(RowFrame, Literal(0), Literal(0)))).as("window")), - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", messageParameters = Map( "sqlExpr" -> "\"nth_value(b, true)\"", "paramIndex" -> "second", @@ -386,7 +387,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { "inputType" -> "\"BOOLEAN\"", "requiredType" -> "\"INT\"")) - errorClassTest( + errorConditionTest( "the buckets of ntile window function is not foldable", testRelation2.select( WindowExpression( @@ -395,7 +396,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { UnresolvedAttribute("a") :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, UnspecifiedFrame)).as("window")), - errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", messageParameters = Map( "sqlExpr" -> "\"ntile(99.9)\"", "paramIndex" -> "first", @@ -404,7 +405,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { "requiredType" -> "\"INT\"")) - errorClassTest( + errorConditionTest( "the buckets of ntile window function is not int literal", testRelation2.select( WindowExpression( @@ -413,20 +414,20 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { UnresolvedAttribute("a") :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, UnspecifiedFrame)).as("window")), - errorClass = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + condition = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", messageParameters = Map( "sqlExpr" -> "\"ntile(b)\"", "inputName" -> "`buckets`", "inputExpr" -> "\"b\"", "inputType" -> "\"INT\"")) - errorClassTest( + errorConditionTest( "unresolved attributes", testRelation.select($"abcd"), "UNRESOLVED_COLUMN.WITH_SUGGESTION", Map("objectName" -> "`abcd`", "proposal" -> "`a`")) - errorClassTest( + errorConditionTest( "unresolved attributes with a generated name", testRelation2.groupBy($"a")(max($"b")) .where(sum($"b") > 0) @@ -434,41 +435,41 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { "UNRESOLVED_COLUMN.WITH_SUGGESTION", Map("objectName" -> "`havingCondition`", "proposal" -> "`max(b)`")) - errorClassTest( + errorConditionTest( "unresolved star expansion in max", testRelation2.groupBy($"a")(sum(UnresolvedStar(None))), - errorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + condition = "INVALID_USAGE_OF_STAR_OR_REGEX", messageParameters = Map("elem" -> "'*'", "prettyName" -> "expression `sum`") ) - errorClassTest( + errorConditionTest( "sorting by unsupported column types", mapRelation.orderBy($"map".asc), - errorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", messageParameters = Map( "sqlExpr" -> "\"map ASC NULLS FIRST\"", "functionName" -> "`sortorder`", "dataType" -> "\"MAP\"")) - errorClassTest( + errorConditionTest( "sorting by attributes are not from grouping expressions", testRelation2.groupBy($"a", $"c")($"a", $"c", count($"a").as("a3")).orderBy($"b".asc), "UNRESOLVED_COLUMN.WITH_SUGGESTION", Map("objectName" -> "`b`", "proposal" -> "`a`, `c`, `a3`")) - errorClassTest( + errorConditionTest( "non-boolean filters", testRelation.where(Literal(1)), - errorClass = "DATATYPE_MISMATCH.FILTER_NOT_BOOLEAN", + condition = "DATATYPE_MISMATCH.FILTER_NOT_BOOLEAN", messageParameters = Map("sqlExpr" -> "\"1\"", "filter" -> "\"1\"", "type" -> "\"INT\"")) - errorClassTest( + errorConditionTest( "non-boolean join conditions", testRelation.join(testRelation, condition = Some(Literal(1))), - errorClass = "JOIN_CONDITION_IS_NOT_BOOLEAN_TYPE", + condition = "JOIN_CONDITION_IS_NOT_BOOLEAN_TYPE", messageParameters = Map("joinCondition" -> "\"1\"", "conditionType" -> "\"INT\"")) - errorClassTest( + errorConditionTest( "missing group by", testRelation2.groupBy($"a")($"b"), "MISSING_AGGREGATION", @@ -477,27 +478,27 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { "expressionAnyValue" -> "\"any_value(b)\"") ) - errorClassTest( + errorConditionTest( "ambiguous field", nestedRelation.select($"top.duplicateField"), - errorClass = "AMBIGUOUS_REFERENCE_TO_FIELDS", + condition = "AMBIGUOUS_REFERENCE_TO_FIELDS", messageParameters = Map( "field" -> "`duplicateField`", "count" -> "2"), caseSensitive = false ) - errorClassTest( + errorConditionTest( "ambiguous field due to case insensitivity", nestedRelation.select($"top.differentCase"), - errorClass = "AMBIGUOUS_REFERENCE_TO_FIELDS", + condition = "AMBIGUOUS_REFERENCE_TO_FIELDS", messageParameters = Map( "field" -> "`differentCase`", "count" -> "2"), caseSensitive = false ) - errorClassTest( + errorConditionTest( "missing field", nestedRelation2.select($"top.c"), "FIELD_NOT_FOUND", @@ -560,14 +561,14 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { testRelation3.except(testRelation4, isAll = false), "except" :: "compatible column types" :: "map" :: "decimal" :: Nil) - errorClassTest( + errorConditionTest( "SPARK-9955: correct error message for aggregate", // When parse SQL string, we will wrap aggregate expressions with UnresolvedAlias. testRelation2.where($"bad_column" > 1).groupBy($"a")(UnresolvedAlias(max($"b"))), "UNRESOLVED_COLUMN.WITH_SUGGESTION", Map("objectName" -> "`bad_column`", "proposal" -> "`a`, `c`, `d`, `b`, `e`")) - errorClassTest( + errorConditionTest( "slide duration greater than window in time window", testRelation2.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "2 second", "0 second").as("window")), @@ -582,7 +583,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "start time greater than slide duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "1 second", "1 minute").as("window")), @@ -597,7 +598,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "start time equal to slide duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "1 second", "1 second").as("window")), @@ -612,7 +613,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "SPARK-21590: absolute value of start time greater than slide duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "1 second", "-1 minute").as("window")), @@ -627,7 +628,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "SPARK-21590: absolute value of start time equal to slide duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "1 second", "-1 second").as("window")), @@ -642,7 +643,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "negative window duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "-1 second", "1 second", "0 second").as("window")), @@ -655,7 +656,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "zero window duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "0 second", "1 second", "0 second").as("window")), @@ -668,7 +669,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "negative slide duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "-1 second", "0 second").as("window")), @@ -681,7 +682,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "zero slide duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "0 second", "0 second").as("window")), @@ -733,7 +734,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { "The generator is not supported: outside the SELECT clause, found: Sort" :: Nil ) - errorClassTest( + errorConditionTest( "an evaluated limit class must not be string", testRelation.limit(Literal(UTF8String.fromString("abc"), StringType)), "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", @@ -744,7 +745,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "an evaluated limit class must not be long", testRelation.limit(Literal(10L, LongType)), "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", @@ -755,7 +756,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "an evaluated limit class must not be null", testRelation.limit(Literal(null, IntegerType)), "INVALID_LIMIT_LIKE_EXPRESSION.IS_NULL", @@ -765,7 +766,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "num_rows in limit clause must be equal to or greater than 0", listRelation.limit(-1), "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", @@ -776,7 +777,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "an evaluated offset class must not be string", testRelation.offset(Literal(UTF8String.fromString("abc"), StringType)), "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", @@ -787,7 +788,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "an evaluated offset class must not be long", testRelation.offset(Literal(10L, LongType)), "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", @@ -798,7 +799,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "an evaluated offset class must not be null", testRelation.offset(Literal(null, IntegerType)), "INVALID_LIMIT_LIKE_EXPRESSION.IS_NULL", @@ -808,7 +809,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "num_rows in offset clause must be equal to or greater than 0", testRelation.offset(-1), "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", @@ -819,7 +820,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { ) ) - errorClassTest( + errorConditionTest( "the sum of num_rows in limit clause and num_rows in offset clause less than Int.MaxValue", testRelation.offset(Literal(2000000000, IntegerType)).limit(Literal(1000000000, IntegerType)), "SUM_OF_LIMIT_AND_OFFSET_EXCEEDS_MAX_INT", @@ -833,14 +834,14 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { """"explode(array(min(a)))", "explode(array(max(a)))"""" :: Nil ) - errorClassTest( + errorConditionTest( "EXEC IMMEDIATE - nested execute immediate not allowed", CatalystSqlParser.parsePlan("EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \\\'SELECT 42\\\''"), "NESTED_EXECUTE_IMMEDIATE", Map( "sqlString" -> "EXECUTE IMMEDIATE 'SELECT 42'")) - errorClassTest( + errorConditionTest( "EXEC IMMEDIATE - both positional and named used", CatalystSqlParser.parsePlan("EXECUTE IMMEDIATE 'SELECT 42 where ? = :first'" + " USING 1, 2 as first"), @@ -853,9 +854,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { scala.util.Right(UnresolvedAttribute("testVarA")), Seq(UnresolvedAttribute("testVarA"))) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = execImmediatePlan, - expectedErrorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + expectedErrorCondition = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", expectedMessageParameters = Map( "varType" -> "\"INT\"" )) @@ -867,9 +868,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { scala.util.Right(UnresolvedAttribute("testVarNull")), Seq(UnresolvedAttribute("testVarNull"))) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = execImmediatePlan, - expectedErrorClass = "NULL_QUERY_STRING_EXECUTE_IMMEDIATE", + expectedErrorCondition = "NULL_QUERY_STRING_EXECUTE_IMMEDIATE", expectedMessageParameters = Map("varName" -> "`testVarNull`")) } @@ -880,9 +881,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { scala.util.Left("SELECT ?"), Seq.empty) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = execImmediatePlan, - expectedErrorClass = "UNSUPPORTED_EXPR_FOR_PARAMETER", + expectedErrorCondition = "UNSUPPORTED_EXPR_FOR_PARAMETER", expectedMessageParameters = Map( "invalidExprSql" -> "\"nanvl(1, 1)\"" )) @@ -894,9 +895,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { scala.util.Left("SELECT :first"), Seq.empty) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = execImmediateSetVariablePlan, - expectedErrorClass = "ALL_PARAMETERS_MUST_BE_NAMED", + expectedErrorCondition = "ALL_PARAMETERS_MUST_BE_NAMED", expectedMessageParameters = Map( "exprs" -> "\"2\", \"3\"" )) @@ -908,9 +909,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { scala.util.Left("SET VAR testVarA = 1"), Seq(UnresolvedAttribute("testVarA"))) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = execImmediateSetVariablePlan, - expectedErrorClass = "INVALID_STATEMENT_FOR_EXECUTE_INTO", + expectedErrorCondition = "INVALID_STATEMENT_FOR_EXECUTE_INTO", expectedMessageParameters = Map( "sqlString" -> "SET VAR TESTVARA = 1" )) @@ -931,9 +932,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { assert(plan.resolved) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = plan, - expectedErrorClass = "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_APPEAR_IN_OPERATION", + expectedErrorCondition = "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_APPEAR_IN_OPERATION", expectedMessageParameters = Map( "missingAttributes" -> "\"a\", \"c\"", "input" -> "\"a\"", @@ -1059,9 +1060,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { AttributeReference("a", IntegerType)(exprId = ExprId(2)), AttributeReference("b", IntegerType)(exprId = ExprId(1)))) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = plan, - expectedErrorClass = "NESTED_AGGREGATE_FUNCTION", + expectedErrorCondition = "NESTED_AGGREGATE_FUNCTION", expectedMessageParameters = Map.empty ) } @@ -1082,9 +1083,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { joinType = Cross, condition = Some($"b" === $"d")) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = plan2, - expectedErrorClass = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", + expectedErrorCondition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", expectedMessageParameters = Map( "functionName" -> "`=`", "dataType" -> "\"MAP\"", @@ -1145,8 +1146,8 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { val a = AttributeReference("a", IntegerType)() val b = AttributeReference("b", IntegerType)() val plan = Filter($"a" === UnresolvedFunction("max", Seq(b), true), LocalRelation(a, b)) - assertAnalysisErrorClass(plan, - expectedErrorClass = "INVALID_WHERE_CONDITION", + assertAnalysisErrorCondition(plan, + expectedErrorCondition = "INVALID_WHERE_CONDITION", expectedMessageParameters = Map( "condition" -> "\"(a = max(DISTINCT b))\"", "expressionList" -> "max(DISTINCT b)")) @@ -1160,8 +1161,8 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { Project( Alias(Literal(1), "x")() :: Nil, UnresolvedRelation(TableIdentifier("t", Option("nonexist"))))))) - assertAnalysisErrorClass(plan, - expectedErrorClass = "TABLE_OR_VIEW_NOT_FOUND", + assertAnalysisErrorCondition(plan, + expectedErrorCondition = "TABLE_OR_VIEW_NOT_FOUND", Map("relationName" -> "`nonexist`.`t`")) } @@ -1170,8 +1171,8 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { (Randn("a".attr), "\"randn(a)\"")).foreach { case (r, expectedArg) => val plan = Project(Seq(r.as("r")), testRelation) - assertAnalysisErrorClass(plan, - expectedErrorClass = "SEED_EXPRESSION_IS_UNFOLDABLE", + assertAnalysisErrorCondition(plan, + expectedErrorCondition = "SEED_EXPRESSION_IS_UNFOLDABLE", expectedMessageParameters = Map( "seedExpr" -> "\"a\"", "exprWithSeed" -> expectedArg), @@ -1184,8 +1185,8 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { Randn("a") -> ("\"randn(a)\"", "\"a\"", "\"STRING\"") ).foreach { case (r, (sqlExpr, inputSql, inputType)) => val plan = Project(Seq(r.as("r")), testRelation) - assertAnalysisErrorClass(plan, - expectedErrorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + assertAnalysisErrorCondition(plan, + expectedErrorCondition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", expectedMessageParameters = Map( "sqlExpr" -> sqlExpr, "paramIndex" -> "first", @@ -1208,9 +1209,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { t.as("t2"))) ) :: Nil, sum($"c2").as("sum") :: Nil, t.as("t1")) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( plan, - expectedErrorClass = + expectedErrorCondition = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", expectedMessageParameters = Map.empty) } @@ -1226,37 +1227,37 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { Filter($"t1.c1" === $"t2.c1", t.as("t2"))) ).as("sub") :: Nil, t.as("t1")) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( plan, - expectedErrorClass = + expectedErrorCondition = "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", expectedMessageParameters = Map("sqlExpr" -> "\"scalarsubquery(c1)\"")) } - errorClassTest( + errorConditionTest( "SPARK-34920: error code to error message", testRelation2.where($"bad_column" > 1).groupBy($"a")(UnresolvedAlias(max($"b"))), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", messageParameters = Map( "objectName" -> "`bad_column`", "proposal" -> "`a`, `c`, `d`, `b`, `e`")) - errorClassTest( + errorConditionTest( "SPARK-39783: backticks in error message for candidate column with dots", // This selects a column that does not exist, // the error message suggest the existing column with correct backticks testRelation6.select($"`the`.`id`"), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", messageParameters = Map( "objectName" -> "`the`.`id`", "proposal" -> "`the.id`")) - errorClassTest( + errorConditionTest( "SPARK-39783: backticks in error message for candidate struct column", // This selects a column that does not exist, // the error message suggest the existing column with correct backticks nestedRelation2.select($"`top.aField`"), - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", messageParameters = Map( "objectName" -> "`top.aField`", "proposal" -> "`top`")) @@ -1291,9 +1292,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { "Scalar subquery must return only one column, but got 2" :: Nil) // t2.* cannot be resolved and the error should be the initial analysis exception. - assertAnalysisErrorClass( + assertAnalysisErrorCondition( Project(ScalarSubquery(t0.select(star("t2"))).as("sub") :: Nil, t1), - expectedErrorClass = "CANNOT_RESOLVE_STAR_EXPAND", + expectedErrorCondition = "CANNOT_RESOLVE_STAR_EXPAND", expectedMessageParameters = Map("targetString" -> "`t2`", "columns" -> "") ) } @@ -1306,70 +1307,70 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { val t2 = LocalRelation(b, c).as("t2") // SELECT * FROM t1 WHERE a = (SELECT sum(c) FROM t2 WHERE t1.* = t2.b) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( Filter(EqualTo(a, ScalarSubquery(t2.select(sum(c)).where(star("t1") === b))), t1), - expectedErrorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + expectedErrorCondition = "INVALID_USAGE_OF_STAR_OR_REGEX", expectedMessageParameters = Map("elem" -> "'*'", "prettyName" -> "expression `equalto`") ) // SELECT * FROM t1 JOIN t2 ON (EXISTS (SELECT 1 FROM t2 WHERE t1.* = b)) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( t1.join(t2, condition = Some(Exists(t2.select(1).where(star("t1") === b)))), - expectedErrorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + expectedErrorCondition = "INVALID_USAGE_OF_STAR_OR_REGEX", expectedMessageParameters = Map("elem" -> "'*'", "prettyName" -> "expression `equalto`") ) } test("SPARK-36488: Regular expression expansion should fail with a meaningful message") { withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( testRelation.select(Divide(UnresolvedRegex(".?", None, false), "a")), - expectedErrorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + expectedErrorCondition = "INVALID_USAGE_OF_STAR_OR_REGEX", expectedMessageParameters = Map( "elem" -> "regular expression '.?'", "prettyName" -> "expression `divide`") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( testRelation.select( Divide(UnresolvedRegex(".?", None, false), UnresolvedRegex(".*", None, false))), - expectedErrorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + expectedErrorCondition = "INVALID_USAGE_OF_STAR_OR_REGEX", expectedMessageParameters = Map( "elem" -> "regular expressions '.?', '.*'", "prettyName" -> "expression `divide`") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( testRelation.select( Divide(UnresolvedRegex(".?", None, false), UnresolvedRegex(".?", None, false))), - expectedErrorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + expectedErrorCondition = "INVALID_USAGE_OF_STAR_OR_REGEX", expectedMessageParameters = Map( "elem" -> "regular expression '.?'", "prettyName" -> "expression `divide`") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( testRelation.select(Divide(UnresolvedStar(None), "a")), - expectedErrorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + expectedErrorCondition = "INVALID_USAGE_OF_STAR_OR_REGEX", expectedMessageParameters = Map( "elem" -> "'*'", "prettyName" -> "expression `divide`") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( testRelation.select(Divide(UnresolvedStar(None), UnresolvedStar(None))), - expectedErrorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + expectedErrorCondition = "INVALID_USAGE_OF_STAR_OR_REGEX", expectedMessageParameters = Map( "elem" -> "'*'", "prettyName" -> "expression `divide`") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( testRelation.select(Divide(UnresolvedStar(None), UnresolvedRegex(".?", None, false))), - expectedErrorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + expectedErrorCondition = "INVALID_USAGE_OF_STAR_OR_REGEX", expectedMessageParameters = Map( "elem" -> "'*' and regular expression '.?'", "prettyName" -> "expression `divide`") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( testRelation.select(Least(Seq(UnresolvedStar(None), UnresolvedRegex(".*", None, false), UnresolvedRegex(".?", None, false)))), - expectedErrorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + expectedErrorCondition = "INVALID_USAGE_OF_STAR_OR_REGEX", expectedMessageParameters = Map( "elem" -> "'*' and regular expressions '.*', '.?'", "prettyName" -> "expression `least`") @@ -1377,7 +1378,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { } } - errorClassTest( + errorConditionTest( "SPARK-47572: Enforce Window partitionSpec is orderable", testRelation2.select( WindowExpression( @@ -1386,7 +1387,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { CreateMap(Literal("key") :: UnresolvedAttribute("a") :: Nil) :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, UnspecifiedFrame)).as("window")), - errorClass = "EXPRESSION_TYPE_IS_NOT_ORDERABLE", + condition = "EXPRESSION_TYPE_IS_NOT_ORDERABLE", messageParameters = Map( "expr" -> "\"_w0\"", "exprType" -> "\"MAP\"")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala index be256adbd8929..55f59f7a22574 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala @@ -48,7 +48,7 @@ class AnalysisExceptionPositionSuite extends AnalysisTest { verifyTableOrViewPosition("REFRESH TABLE unknown", "unknown") verifyTableOrViewPosition("SHOW COLUMNS FROM unknown", "unknown") // Special case where namespace is prepended to the table name. - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsePlan("SHOW COLUMNS FROM unknown IN db"), "TABLE_OR_VIEW_NOT_FOUND", Map("relationName" -> "`db`.`unknown`"), @@ -94,7 +94,7 @@ class AnalysisExceptionPositionSuite extends AnalysisTest { private def verifyPosition(sql: String, table: String): Unit = { val startPos = sql.indexOf(table) assert(startPos != -1) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsePlan(sql), "TABLE_OR_VIEW_NOT_FOUND", Map("relationName" -> s"`$table`"), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index e55a08409b642..e23a753dafe8c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -112,7 +112,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), Project(testRelation.output, testRelation)) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( Project(Seq(UnresolvedAttribute("tBl.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), "UNRESOLVED_COLUMN.WITH_SUGGESTION", @@ -359,7 +359,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { val plan = Project(Alias(In(Literal(null), Seq(Literal(true), Literal(1))), "a")() :: Nil, LocalRelation() ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( plan, "DATATYPE_MISMATCH.DATA_DIFF_TYPES", Map( @@ -555,7 +555,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { assertAnalysisSuccess(rangeWithAliases(3 :: Nil, "a" :: Nil)) assertAnalysisSuccess(rangeWithAliases(1 :: 4 :: Nil, "b" :: Nil)) assertAnalysisSuccess(rangeWithAliases(2 :: 6 :: 2 :: Nil, "c" :: Nil)) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( rangeWithAliases(3 :: Nil, "a" :: "b" :: Nil), "NUM_TABLE_VALUE_ALIASES_MISMATCH", Map("funcName" -> "`range`", "aliasesNum" -> "2", "outColsNum" -> "1")) @@ -569,12 +569,12 @@ class AnalysisSuite extends AnalysisTest with Matchers { ).select(star()) } assertAnalysisSuccess(tableColumnsWithAliases("col1" :: "col2" :: "col3" :: "col4" :: Nil)) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( tableColumnsWithAliases("col1" :: Nil), "ASSIGNMENT_ARITY_MISMATCH", Map("numExpr" -> "1", "numTarget" -> "4") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( tableColumnsWithAliases("col1" :: "col2" :: "col3" :: "col4" :: "col5" :: Nil), "ASSIGNMENT_ARITY_MISMATCH", Map("numExpr" -> "5", "numTarget" -> "4") @@ -591,12 +591,12 @@ class AnalysisSuite extends AnalysisTest with Matchers { ).select(star()) } assertAnalysisSuccess(tableColumnsWithAliases("col1" :: "col2" :: "col3" :: "col4" :: Nil)) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( tableColumnsWithAliases("col1" :: Nil), "ASSIGNMENT_ARITY_MISMATCH", Map("numExpr" -> "1", "numTarget" -> "4") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( tableColumnsWithAliases("col1" :: "col2" :: "col3" :: "col4" :: "col5" :: Nil), "ASSIGNMENT_ARITY_MISMATCH", Map("numExpr" -> "5", "numTarget" -> "4") @@ -615,12 +615,12 @@ class AnalysisSuite extends AnalysisTest with Matchers { ).select(star()) } assertAnalysisSuccess(joinRelationWithAliases("col1" :: "col2" :: "col3" :: "col4" :: Nil)) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( joinRelationWithAliases("col1" :: Nil), "ASSIGNMENT_ARITY_MISMATCH", Map("numExpr" -> "1", "numTarget" -> "4") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( joinRelationWithAliases("col1" :: "col2" :: "col3" :: "col4" :: "col5" :: Nil), "ASSIGNMENT_ARITY_MISMATCH", Map("numExpr" -> "5", "numTarget" -> "4") @@ -755,7 +755,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("SPARK-34741: Avoid ambiguous reference in MergeIntoTable") { val cond = $"a" > 1 - assertAnalysisErrorClass( + assertAnalysisErrorCondition( MergeIntoTable( testRelation, testRelation, @@ -794,7 +794,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("CTE with non-existing column alias") { - assertAnalysisErrorClass(parsePlan("WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE y = 1"), + assertAnalysisErrorCondition(parsePlan("WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE y = 1"), "UNRESOLVED_COLUMN.WITH_SUGGESTION", Map("objectName" -> "`y`", "proposal" -> "`x`"), Array(ExpectedContext("y", 46, 46)) @@ -802,7 +802,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("CTE with non-matching column alias") { - assertAnalysisErrorClass(parsePlan("WITH t(x, y) AS (SELECT 1) SELECT * FROM t WHERE x = 1"), + assertAnalysisErrorCondition( + parsePlan("WITH t(x, y) AS (SELECT 1) SELECT * FROM t WHERE x = 1"), "ASSIGNMENT_ARITY_MISMATCH", Map("numExpr" -> "2", "numTarget" -> "1"), Array(ExpectedContext("t(x, y) AS (SELECT 1)", 5, 25)) @@ -810,7 +811,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-28251: Insert into non-existing table error message is user friendly") { - assertAnalysisErrorClass(parsePlan("INSERT INTO test VALUES (1)"), + assertAnalysisErrorCondition(parsePlan("INSERT INTO test VALUES (1)"), "TABLE_OR_VIEW_NOT_FOUND", Map("relationName" -> "`test`"), Array(ExpectedContext("test", 12, 15))) } @@ -826,9 +827,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { // Bad name assert(!CollectMetrics("", sum :: Nil, testRelation, 0).resolved) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CollectMetrics("", sum :: Nil, testRelation, 0), - expectedErrorClass = "INVALID_OBSERVED_METRICS.MISSING_NAME", + expectedErrorCondition = "INVALID_OBSERVED_METRICS.MISSING_NAME", expectedMessageParameters = Map( "operator" -> "'CollectMetrics , [sum(a#x) AS sum#xL], 0\n+- LocalRelation , [a#x]\n") @@ -853,37 +854,38 @@ class AnalysisSuite extends AnalysisTest with Matchers { ) // Unwrapped attribute - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CollectMetrics("event", a :: Nil, testRelation, 0), - expectedErrorClass = "INVALID_OBSERVED_METRICS.NON_AGGREGATE_FUNC_ARG_IS_ATTRIBUTE", + expectedErrorCondition = "INVALID_OBSERVED_METRICS.NON_AGGREGATE_FUNC_ARG_IS_ATTRIBUTE", expectedMessageParameters = Map("expr" -> "\"a\"") ) // Unwrapped non-deterministic expression - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CollectMetrics("event", Rand(10).as("rnd") :: Nil, testRelation, 0), - expectedErrorClass = "INVALID_OBSERVED_METRICS.NON_AGGREGATE_FUNC_ARG_IS_NON_DETERMINISTIC", + expectedErrorCondition = + "INVALID_OBSERVED_METRICS.NON_AGGREGATE_FUNC_ARG_IS_NON_DETERMINISTIC", expectedMessageParameters = Map("expr" -> "\"rand(10) AS rnd\"") ) // Distinct aggregate - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CollectMetrics( "event", Sum(a).toAggregateExpression(isDistinct = true).as("sum") :: Nil, testRelation, 0), - expectedErrorClass = + expectedErrorCondition = "INVALID_OBSERVED_METRICS.AGGREGATE_EXPRESSION_WITH_DISTINCT_UNSUPPORTED", expectedMessageParameters = Map("expr" -> "\"sum(DISTINCT a) AS sum\"") ) // Nested aggregate - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CollectMetrics( "event", Sum(Sum(a).toAggregateExpression()).toAggregateExpression().as("sum") :: Nil, testRelation, 0), - expectedErrorClass = "INVALID_OBSERVED_METRICS.NESTED_AGGREGATES_UNSUPPORTED", + expectedErrorCondition = "INVALID_OBSERVED_METRICS.NESTED_AGGREGATES_UNSUPPORTED", expectedMessageParameters = Map("expr" -> "\"sum(sum(a)) AS sum\"") ) @@ -892,9 +894,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { RowNumber(), WindowSpecDefinition(Nil, a.asc :: Nil, SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CollectMetrics("event", windowExpr.as("rn") :: Nil, testRelation, 0), - expectedErrorClass = "INVALID_OBSERVED_METRICS.WINDOW_EXPRESSIONS_UNSUPPORTED", + expectedErrorCondition = "INVALID_OBSERVED_METRICS.WINDOW_EXPRESSIONS_UNSUPPORTED", expectedMessageParameters = Map( "expr" -> """ @@ -915,22 +917,22 @@ class AnalysisSuite extends AnalysisTest with Matchers { CollectMetrics("evt1", count :: Nil, testRelation, 0) :: Nil)) // Same children, structurally different metrics - fail - assertAnalysisErrorClass( + assertAnalysisErrorCondition( Union( CollectMetrics("evt1", count :: Nil, testRelation, 0) :: CollectMetrics("evt1", sum :: Nil, testRelation, 1) :: Nil), - expectedErrorClass = "DUPLICATED_METRICS_NAME", + expectedErrorCondition = "DUPLICATED_METRICS_NAME", expectedMessageParameters = Map("metricName" -> "evt1") ) // Different children, same metrics - fail val b = $"b".string val tblB = LocalRelation(b) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( Union( CollectMetrics("evt1", count :: Nil, testRelation, 0) :: CollectMetrics("evt1", count :: Nil, tblB, 1) :: Nil), - expectedErrorClass = "DUPLICATED_METRICS_NAME", + expectedErrorCondition = "DUPLICATED_METRICS_NAME", expectedMessageParameters = Map("metricName" -> "evt1") ) @@ -939,9 +941,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { val query = Project( b :: ScalarSubquery(subquery, Nil).as("sum") :: Nil, CollectMetrics("evt1", count :: Nil, tblB, 1)) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( query, - expectedErrorClass = "DUPLICATED_METRICS_NAME", + expectedErrorCondition = "DUPLICATED_METRICS_NAME", expectedMessageParameters = Map("metricName" -> "evt1") ) @@ -949,9 +951,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { val sumWithFilter = sum.transform { case a: AggregateExpression => a.copy(filter = Some(true)) }.asInstanceOf[NamedExpression] - assertAnalysisErrorClass( + assertAnalysisErrorCondition( CollectMetrics("evt1", sumWithFilter :: Nil, testRelation, 0), - expectedErrorClass = + expectedErrorCondition = "INVALID_OBSERVED_METRICS.AGGREGATE_EXPRESSION_WITH_FILTER_UNSUPPORTED", expectedMessageParameters = Map("expr" -> "\"sum(a) FILTER (WHERE true) AS sum\"") ) @@ -1062,9 +1064,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { AttributeReference("c", IntegerType)(), AttributeReference("d", TimestampType)()) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( Union(firstTable, secondTable), - expectedErrorClass = "INCOMPATIBLE_COLUMN_TYPE", + expectedErrorCondition = "INCOMPATIBLE_COLUMN_TYPE", expectedMessageParameters = Map( "tableOrdinalNumber" -> "second", "columnOrdinalNumber" -> "second", @@ -1074,9 +1076,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { "dataType1" -> "\"TIMESTAMP\"") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( Union(firstTable, thirdTable), - expectedErrorClass = "INCOMPATIBLE_COLUMN_TYPE", + expectedErrorCondition = "INCOMPATIBLE_COLUMN_TYPE", expectedMessageParameters = Map( "tableOrdinalNumber" -> "second", "columnOrdinalNumber" -> "third", @@ -1086,9 +1088,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { "dataType1" -> "\"TIMESTAMP\"") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( Union(firstTable, fourthTable), - expectedErrorClass = "INCOMPATIBLE_COLUMN_TYPE", + expectedErrorCondition = "INCOMPATIBLE_COLUMN_TYPE", expectedMessageParameters = Map( "tableOrdinalNumber" -> "second", "columnOrdinalNumber" -> "4th", @@ -1098,9 +1100,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { "dataType1" -> "\"TIMESTAMP\"") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( Except(firstTable, secondTable, isAll = false), - expectedErrorClass = "INCOMPATIBLE_COLUMN_TYPE", + expectedErrorCondition = "INCOMPATIBLE_COLUMN_TYPE", expectedMessageParameters = Map( "tableOrdinalNumber" -> "second", "columnOrdinalNumber" -> "second", @@ -1110,9 +1112,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { "dataType1" -> "\"TIMESTAMP\"") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( Intersect(firstTable, secondTable, isAll = false), - expectedErrorClass = "INCOMPATIBLE_COLUMN_TYPE", + expectedErrorCondition = "INCOMPATIBLE_COLUMN_TYPE", expectedMessageParameters = Map( "tableOrdinalNumber" -> "second", "columnOrdinalNumber" -> "second", @@ -1124,21 +1126,21 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-31975: Throw user facing error when use WindowFunction directly") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = testRelation2.select(RowNumber()), - expectedErrorClass = "WINDOW_FUNCTION_WITHOUT_OVER_CLAUSE", + expectedErrorCondition = "WINDOW_FUNCTION_WITHOUT_OVER_CLAUSE", expectedMessageParameters = Map("funcName" -> "\"row_number()\"") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = testRelation2.select(Sum(RowNumber())), - expectedErrorClass = "WINDOW_FUNCTION_WITHOUT_OVER_CLAUSE", + expectedErrorCondition = "WINDOW_FUNCTION_WITHOUT_OVER_CLAUSE", expectedMessageParameters = Map("funcName" -> "\"row_number()\"") ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = testRelation2.select(RowNumber() + 1), - expectedErrorClass = "WINDOW_FUNCTION_WITHOUT_OVER_CLAUSE", + expectedErrorCondition = "WINDOW_FUNCTION_WITHOUT_OVER_CLAUSE", expectedMessageParameters = Map("funcName" -> "\"row_number()\"") ) } @@ -1297,7 +1299,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { | ORDER BY grouping__id > 0 """.stripMargin), false) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsePlan( """ |SELECT grouping__id FROM ( @@ -1328,7 +1330,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { |ORDER BY c.x |""".stripMargin)) - assertAnalysisErrorClass(parsePlan( + assertAnalysisErrorCondition(parsePlan( """ |SELECT c.x |FROM VALUES NAMED_STRUCT('x', 'A', 'y', 1), NAMED_STRUCT('x', 'A', 'y', 2) AS t(c) @@ -1342,7 +1344,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-38118: Func(wrong_type) in the HAVING clause should throw data mismatch error") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = parsePlan( s""" |WITH t as (SELECT true c) @@ -1350,7 +1352,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { |FROM t |GROUP BY t.c |HAVING mean(t.c) > 0d""".stripMargin), - expectedErrorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + expectedErrorCondition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", expectedMessageParameters = Map( "sqlExpr" -> "\"mean(c)\"", "paramIndex" -> "first", @@ -1361,7 +1363,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { caseSensitive = false ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = parsePlan( s""" |WITH t as (SELECT true c, false d) @@ -1369,7 +1371,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { |FROM t |GROUP BY t.c, t.d |HAVING mean(c) > 0d""".stripMargin), - expectedErrorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + expectedErrorCondition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", expectedMessageParameters = Map( "sqlExpr" -> "\"mean(c)\"", "paramIndex" -> "first", @@ -1379,7 +1381,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { queryContext = Array(ExpectedContext("mean(c)", 91, 97)), caseSensitive = false) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = parsePlan( s""" |WITH t as (SELECT true c) @@ -1387,7 +1389,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { |FROM t |GROUP BY t.c |HAVING abs(t.c) > 0d""".stripMargin), - expectedErrorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + expectedErrorCondition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", expectedMessageParameters = Map( "sqlExpr" -> "\"abs(c)\"", "paramIndex" -> "first", @@ -1399,7 +1401,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { caseSensitive = false ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = parsePlan( s""" |WITH t as (SELECT true c, false d) @@ -1407,7 +1409,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { |FROM t |GROUP BY t.c, t.d |HAVING abs(c) > 0d""".stripMargin), - expectedErrorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + expectedErrorCondition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", expectedMessageParameters = Map( "sqlExpr" -> "\"abs(c)\"", "paramIndex" -> "first", @@ -1421,7 +1423,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-39354: should be [TABLE_OR_VIEW_NOT_FOUND]") { - assertAnalysisErrorClass(parsePlan( + assertAnalysisErrorCondition(parsePlan( s""" |WITH t1 as (SELECT 1 user_id, CAST("2022-06-02" AS DATE) dt) |SELECT * @@ -1531,13 +1533,13 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-41489: type of filter expression should be a bool") { - assertAnalysisErrorClass(parsePlan( + assertAnalysisErrorCondition(parsePlan( s""" |WITH t1 as (SELECT 1 user_id) |SELECT * |FROM t1 |WHERE 'true'""".stripMargin), - expectedErrorClass = "DATATYPE_MISMATCH.FILTER_NOT_BOOLEAN", + expectedErrorCondition = "DATATYPE_MISMATCH.FILTER_NOT_BOOLEAN", expectedMessageParameters = Map( "sqlExpr" -> "\"true\"", "filter" -> "\"true\"", "type" -> "\"STRING\"") , diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index f299c078befc0..33b9fb488c94f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -178,9 +178,9 @@ trait AnalysisTest extends PlanTest { } } - protected def assertAnalysisErrorClass( + protected def assertAnalysisErrorCondition( inputPlan: LogicalPlan, - expectedErrorClass: String, + expectedErrorCondition: String, expectedMessageParameters: Map[String, String], queryContext: Array[ExpectedContext] = Array.empty, caseSensitive: Boolean = true): Unit = { @@ -191,7 +191,7 @@ trait AnalysisTest extends PlanTest { } checkError( exception = e, - condition = expectedErrorClass, + condition = expectedErrorCondition, parameters = expectedMessageParameters, queryContext = queryContext ) @@ -199,14 +199,13 @@ trait AnalysisTest extends PlanTest { } protected def interceptParseException(parser: String => Any)( - sqlCommand: String, messages: String*)( - errorClass: Option[String] = None): Unit = { + sqlCommand: String, messages: String*)(condition: Option[String] = None): Unit = { val e = parseException(parser)(sqlCommand) messages.foreach { message => assert(e.message.contains(message)) } - if (errorClass.isDefined) { - assert(e.getErrorClass == errorClass.get) + if (condition.isDefined) { + assert(e.getErrorClass == condition.get) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala index c9e37e255ab44..6b034d3dbee09 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala @@ -41,8 +41,8 @@ class CreateTablePartitioningValidationSuite extends AnalysisTest { ignoreIfExists = false) assert(!plan.resolved) - assertAnalysisErrorClass(plan, - expectedErrorClass = "UNSUPPORTED_FEATURE.PARTITION_WITH_NESTED_COLUMN_IS_UNSUPPORTED", + assertAnalysisErrorCondition(plan, + expectedErrorCondition = "UNSUPPORTED_FEATURE.PARTITION_WITH_NESTED_COLUMN_IS_UNSUPPORTED", expectedMessageParameters = Map("cols" -> "`does_not_exist`")) } @@ -56,8 +56,8 @@ class CreateTablePartitioningValidationSuite extends AnalysisTest { ignoreIfExists = false) assert(!plan.resolved) - assertAnalysisErrorClass(plan, - expectedErrorClass = "UNSUPPORTED_FEATURE.PARTITION_WITH_NESTED_COLUMN_IS_UNSUPPORTED", + assertAnalysisErrorCondition(plan, + expectedErrorCondition = "UNSUPPORTED_FEATURE.PARTITION_WITH_NESTED_COLUMN_IS_UNSUPPORTED", expectedMessageParameters = Map("cols" -> "`does_not_exist`.`z`")) } @@ -71,8 +71,8 @@ class CreateTablePartitioningValidationSuite extends AnalysisTest { ignoreIfExists = false) assert(!plan.resolved) - assertAnalysisErrorClass(plan, - expectedErrorClass = "UNSUPPORTED_FEATURE.PARTITION_WITH_NESTED_COLUMN_IS_UNSUPPORTED", + assertAnalysisErrorCondition(plan, + expectedErrorCondition = "UNSUPPORTED_FEATURE.PARTITION_WITH_NESTED_COLUMN_IS_UNSUPPORTED", expectedMessageParameters = Map("cols" -> "`point`.`z`")) } @@ -86,8 +86,8 @@ class CreateTablePartitioningValidationSuite extends AnalysisTest { ignoreIfExists = false) assert(!plan.resolved) - assertAnalysisErrorClass(plan, - expectedErrorClass = "UNSUPPORTED_FEATURE.PARTITION_WITH_NESTED_COLUMN_IS_UNSUPPORTED", + assertAnalysisErrorCondition(plan, + expectedErrorCondition = "UNSUPPORTED_FEATURE.PARTITION_WITH_NESTED_COLUMN_IS_UNSUPPORTED", expectedMessageParameters = Map("cols" -> "`does_not_exist`, `point`.`z`")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala index 5c843d62d6d7c..b7afc803410cf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala @@ -108,14 +108,14 @@ class ResolveNaturalJoinSuite extends AnalysisTest { } test("using unresolved attribute") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( r1.join(r2, UsingJoin(Inner, Seq("d"))), - expectedErrorClass = "UNRESOLVED_USING_COLUMN_FOR_JOIN", + expectedErrorCondition = "UNRESOLVED_USING_COLUMN_FOR_JOIN", expectedMessageParameters = Map( "colName" -> "`d`", "side" -> "left", "suggestion" -> "`a`, `b`")) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( r1.join(r2, UsingJoin(Inner, Seq("b"))), - expectedErrorClass = "UNRESOLVED_USING_COLUMN_FOR_JOIN", + expectedErrorCondition = "UNRESOLVED_USING_COLUMN_FOR_JOIN", expectedMessageParameters = Map( "colName" -> "`b`", "side" -> "right", "suggestion" -> "`a`, `c`")) } @@ -126,17 +126,17 @@ class ResolveNaturalJoinSuite extends AnalysisTest { val usingPlan = r1.join(r2, UsingJoin(Inner, Seq("a")), None) checkAnalysis(usingPlan, expected, caseSensitive = true) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( r1.join(r2, UsingJoin(Inner, Seq("A"))), - expectedErrorClass = "UNRESOLVED_USING_COLUMN_FOR_JOIN", + expectedErrorCondition = "UNRESOLVED_USING_COLUMN_FOR_JOIN", expectedMessageParameters = Map( "colName" -> "`A`", "side" -> "left", "suggestion" -> "`a`, `b`")) } test("using join on nested fields") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( r5.join(r6, UsingJoin(Inner, Seq("d.f1"))), - expectedErrorClass = "UNRESOLVED_USING_COLUMN_FOR_JOIN", + expectedErrorCondition = "UNRESOLVED_USING_COLUMN_FOR_JOIN", expectedMessageParameters = Map( "colName" -> "`d`.`f1`", "side" -> "left", "suggestion" -> "`d`")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala index 4e17f4624f7e0..86718ee434311 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala @@ -71,7 +71,7 @@ class ResolveSubquerySuite extends AnalysisTest { test("lateral join with ambiguous join conditions") { val plan = lateralJoin(t1, t0.select($"b"), condition = Some($"b" === 1)) - assertAnalysisErrorClass(plan, + assertAnalysisErrorCondition(plan, "AMBIGUOUS_REFERENCE", Map("name" -> "`b`", "referenceNames" -> "[`b`, `b`]") ) } @@ -123,7 +123,7 @@ class ResolveSubquerySuite extends AnalysisTest { // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT a, b, c)) // TODO: support accessing columns from outer outer query. - assertAnalysisErrorClass( + assertAnalysisErrorCondition( lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b", $"c"))), "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", Map("objectName" -> "`a`") @@ -132,25 +132,25 @@ class ResolveSubquerySuite extends AnalysisTest { test("lateral subquery with unresolvable attributes") { // SELECT * FROM t1, LATERAL (SELECT a, c) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( lateralJoin(t1, t0.select($"a", $"c")), "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", Map("objectName" -> "`c`") ) // SELECT * FROM t1, LATERAL (SELECT a, b, c, d FROM t2) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( lateralJoin(t1, t2.select($"a", $"b", $"c", $"d")), "UNRESOLVED_COLUMN.WITH_SUGGESTION", Map("objectName" -> "`d`", "proposal" -> "`b`, `c`") ) // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.a)) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( lateralJoin(t1, lateralJoin(t2, t0.select($"t1.a"))), "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", Map("objectName" -> "`t1`.`a`") ) // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT a, b)) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b"))), "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", Map("objectName" -> "`a`") @@ -165,7 +165,7 @@ class ResolveSubquerySuite extends AnalysisTest { LateralJoin(t4, LateralSubquery(Project(Seq(xa, ya), t0), Seq(x, y)), Inner, None) ) // Analyzer will try to resolve struct first before subquery alias. - assertAnalysisErrorClass( + assertAnalysisErrorCondition( lateralJoin(t1.as("x"), t4.select($"x.a", $"x.b")), "FIELD_NOT_FOUND", Map("fieldName" -> "`b`", "fields" -> "`a`")) @@ -174,9 +174,9 @@ class ResolveSubquerySuite extends AnalysisTest { test("lateral join with unsupported expressions") { val plan = lateralJoin(t1, t0.select(($"a" + $"b").as("c")), condition = Some(sum($"a") === sum($"c"))) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( plan, - expectedErrorClass = "UNSUPPORTED_EXPR_FOR_OPERATOR", + expectedErrorCondition = "UNSUPPORTED_EXPR_FOR_OPERATOR", expectedMessageParameters = Map("invalidExprSqls" -> "\"sum(a)\", \"sum(c)\"") ) } @@ -206,17 +206,17 @@ class ResolveSubquerySuite extends AnalysisTest { LateralSubquery(Project(Seq(outerA, outerB, b, c), t2.as("t2")), Seq(a, b)), Inner, None) ) // SELECT * FROM t1, LATERAL (SELECT t2.*) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( lateralJoin(t1.as("t1"), t0.select(star("t2"))), - expectedErrorClass = "CANNOT_RESOLVE_STAR_EXPAND", + expectedErrorCondition = "CANNOT_RESOLVE_STAR_EXPAND", expectedMessageParameters = Map("targetString" -> "`t2`", "columns" -> "") ) // Check case sensitivities. // SELECT * FROM t1, LATERAL (SELECT T1.*) val plan = lateralJoin(t1.as("t1"), t0.select(star("T1"))) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( plan, - expectedErrorClass = "CANNOT_RESOLVE_STAR_EXPAND", + expectedErrorCondition = "CANNOT_RESOLVE_STAR_EXPAND", expectedMessageParameters = Map("targetString" -> "`T1`", "columns" -> "") ) assertAnalysisSuccess(plan, caseSensitive = false) @@ -232,9 +232,9 @@ class ResolveSubquerySuite extends AnalysisTest { LateralJoin(t1, LateralSubquery(t0.select(newArray.as(newArray.sql)), Seq(a, b)), Inner, None) ) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( lateralJoin(t1.as("t1"), t0.select(Count(star("t1")))), - expectedErrorClass = "INVALID_USAGE_OF_STAR_OR_REGEX", + expectedErrorCondition = "INVALID_USAGE_OF_STAR_OR_REGEX", expectedMessageParameters = Map("elem" -> "'*'", "prettyName" -> "expression `count`")) } @@ -293,9 +293,9 @@ class ResolveSubquerySuite extends AnalysisTest { :: lv(Symbol("X")) :: Nil)) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = lambdaPlanScanFromTable, - expectedErrorClass = + expectedErrorCondition = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.HIGHER_ORDER_FUNCTION", expectedMessageParameters = Map.empty[String, String]) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala index 2280463c2f244..96c9f966e5f74 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala @@ -154,16 +154,16 @@ abstract class V2ANSIWriteAnalysisSuiteBase extends V2WriteAnalysisSuiteBase { } } - override def assertAnalysisErrorClass( - inputPlan: LogicalPlan, - expectedErrorClass: String, - expectedMessageParameters: Map[String, String], - queryContext: Array[ExpectedContext] = Array.empty, - caseSensitive: Boolean = true): Unit = { + override def assertAnalysisErrorCondition( + inputPlan: LogicalPlan, + expectedErrorCondition: String, + expectedMessageParameters: Map[String, String], + queryContext: Array[ExpectedContext] = Array.empty, + caseSensitive: Boolean = true): Unit = { withSQLConf(SQLConf.STORE_ASSIGNMENT_POLICY.key -> StoreAssignmentPolicy.ANSI.toString) { - super.assertAnalysisErrorClass( + super.assertAnalysisErrorCondition( inputPlan, - expectedErrorClass, + expectedErrorCondition, expectedMessageParameters, queryContext, caseSensitive @@ -191,16 +191,16 @@ abstract class V2StrictWriteAnalysisSuiteBase extends V2WriteAnalysisSuiteBase { } } - override def assertAnalysisErrorClass( - inputPlan: LogicalPlan, - expectedErrorClass: String, - expectedMessageParameters: Map[String, String], - queryContext: Array[ExpectedContext] = Array.empty, - caseSensitive: Boolean = true): Unit = { + override def assertAnalysisErrorCondition( + inputPlan: LogicalPlan, + expectedErrorCondition: String, + expectedMessageParameters: Map[String, String], + queryContext: Array[ExpectedContext] = Array.empty, + caseSensitive: Boolean = true): Unit = { withSQLConf(SQLConf.STORE_ASSIGNMENT_POLICY.key -> StoreAssignmentPolicy.STRICT.toString) { - super.assertAnalysisErrorClass( + super.assertAnalysisErrorCondition( inputPlan, - expectedErrorClass, + expectedErrorCondition, expectedMessageParameters, queryContext, caseSensitive @@ -212,9 +212,9 @@ abstract class V2StrictWriteAnalysisSuiteBase extends V2WriteAnalysisSuiteBase { val parsedPlan = byName(table, widerTable) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`x`", @@ -235,9 +235,9 @@ abstract class V2StrictWriteAnalysisSuiteBase extends V2WriteAnalysisSuiteBase { val parsedPlan = byName(xRequiredTable, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`x`", @@ -254,9 +254,9 @@ abstract class V2StrictWriteAnalysisSuiteBase extends V2WriteAnalysisSuiteBase { val parsedPlan = byPosition(table, widerTable) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`x`", @@ -277,9 +277,9 @@ abstract class V2StrictWriteAnalysisSuiteBase extends V2WriteAnalysisSuiteBase { val parsedPlan = byPosition(xRequiredTable, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_SAFELY_CAST", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`x`", @@ -421,9 +421,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = byName(table, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", expectedMessageParameters = Map("tableName" -> "`table-name`", "colName" -> "`x`") ) } @@ -436,9 +436,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = byName(table, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", expectedMessageParameters = Map("tableName" -> "`table-name`", "colName" -> "`x`") ) } @@ -499,9 +499,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = byName(requiredTable, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", expectedMessageParameters = Map("tableName" -> "`table-name`", "colName" -> "`x`") ) } @@ -514,9 +514,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = byName(table, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", expectedMessageParameters = Map("tableName" -> "`table-name`", "colName" -> "`x`") ) } @@ -546,9 +546,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = byName(table, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = parsedPlan, - expectedErrorClass = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + expectedErrorCondition = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "tableColumns" -> "`x`, `y`", @@ -561,9 +561,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val query = TestRelation(Seq($"b".struct($"y".int, $"x".int, $"z".int), $"a".int)) val writePlan = byName(table, query) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( writePlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`b`", @@ -636,9 +636,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = byPosition(requiredTable, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = parsedPlan, - expectedErrorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + expectedErrorCondition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "tableColumns" -> "`x`, `y`", @@ -654,9 +654,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = byPosition(table, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = parsedPlan, - expectedErrorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + expectedErrorCondition = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "tableColumns" -> "`x`, `y`", @@ -693,9 +693,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = byName(table, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = parsedPlan, - expectedErrorClass = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + expectedErrorCondition = "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "tableColumns" -> "`x`, `y`", @@ -740,9 +740,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { withClue("byName") { val parsedPlan = byName(tableWithStructCol, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", expectedMessageParameters = Map("tableName" -> "`table-name`", "colName" -> "`col`.`a`") ) } @@ -792,9 +792,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = if (byNameResolution) byName(table, query) else byPosition(table, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`b`.`n2`", @@ -821,9 +821,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = if (byNameResolution) byName(table, query) else byPosition(table, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`arr`.`element`", @@ -854,9 +854,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = if (byNameResolution) byName(table, query) else byPosition(table, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`m`.`key`", @@ -887,9 +887,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan = if (byNameResolution) byName(table, query) else byPosition(table, query) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.EXTRA_STRUCT_FIELDS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`m`.`value`", @@ -921,17 +921,17 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { assertNotResolved(parsedPlan) if (byNameResolution) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`b`.`n2`.`dn3`") ) } else { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.STRUCT_MISSING_FIELDS", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.STRUCT_MISSING_FIELDS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`b`.`n2`", @@ -964,17 +964,17 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { assertNotResolved(parsedPlan) if (byNameResolution) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`arr`.`element`.`y`") ) } else { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.STRUCT_MISSING_FIELDS", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.STRUCT_MISSING_FIELDS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`arr`.`element`", @@ -1011,17 +1011,17 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { assertNotResolved(parsedPlan) if (byNameResolution) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`m`.`key`.`y`") ) } else { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.STRUCT_MISSING_FIELDS", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.STRUCT_MISSING_FIELDS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`m`.`key`", @@ -1058,17 +1058,17 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { assertNotResolved(parsedPlan) if (byNameResolution) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`m`.`value`.`y`") ) } else { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, - expectedErrorClass = "INCOMPATIBLE_DATA_FOR_TABLE.STRUCT_MISSING_FIELDS", + expectedErrorCondition = "INCOMPATIBLE_DATA_FOR_TABLE.STRUCT_MISSING_FIELDS", expectedMessageParameters = Map( "tableName" -> "`table-name`", "colName" -> "`m`.`value`", @@ -1363,7 +1363,7 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { LessThanOrEqual(UnresolvedAttribute(Seq("a")), Literal(15.0d))) assertNotResolved(parsedPlan) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan, "UNRESOLVED_COLUMN.WITH_SUGGESTION", Map("objectName" -> "`a`", "proposal" -> "`x`, `y`") @@ -1376,7 +1376,7 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan2 = OverwriteByExpression.byPosition(tableAcceptAnySchema, query, LessThanOrEqual(UnresolvedAttribute(Seq("a")), Literal(15.0d))) assertNotResolved(parsedPlan2) - assertAnalysisErrorClass( + assertAnalysisErrorCondition( parsedPlan2, "UNRESOLVED_COLUMN.WITH_SUGGESTION", Map("objectName" -> "`a`", "proposal" -> "`x`, `y`") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala index b4a768a75989a..4fe79ff53c2df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala @@ -177,9 +177,9 @@ class V2CommandsCaseSensitivitySuite None))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = alter, - expectedErrorClass = "FIELD_NOT_FOUND", + expectedErrorCondition = "FIELD_NOT_FOUND", expectedMessageParameters = Map("fieldName" -> "`f`", "fields" -> "id, data, point") ) } @@ -208,9 +208,9 @@ class V2CommandsCaseSensitivitySuite None))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = alter, - expectedErrorClass = "FIELD_NOT_FOUND", + expectedErrorCondition = "FIELD_NOT_FOUND", expectedMessageParameters = Map("fieldName" -> "`y`", "fields" -> "id, data, point, x") ) } @@ -231,9 +231,9 @@ class V2CommandsCaseSensitivitySuite None))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = alter, - expectedErrorClass = "FIELD_NOT_FOUND", + expectedErrorCondition = "FIELD_NOT_FOUND", expectedMessageParameters = Map("fieldName" -> "`z`", "fields" -> "x, y") ) } @@ -262,9 +262,9 @@ class V2CommandsCaseSensitivitySuite None))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( inputPlan = alter, - expectedErrorClass = "FIELD_NOT_FOUND", + expectedErrorCondition = "FIELD_NOT_FOUND", expectedMessageParameters = Map("fieldName" -> "`zz`", "fields" -> "x, y, z") ) } @@ -272,7 +272,7 @@ class V2CommandsCaseSensitivitySuite } test("SPARK-36372: Adding duplicate columns should not be allowed") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( AddColumns( table, Seq(QualifiedColType( @@ -401,7 +401,7 @@ class V2CommandsCaseSensitivitySuite } test("SPARK-36449: Replacing columns with duplicate name should not be allowed") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( ReplaceColumns( table, Seq(QualifiedColType(None, "f", LongType, true, None, None, None), @@ -420,7 +420,7 @@ class V2CommandsCaseSensitivitySuite withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { val expectError = if (expectErrorOnCaseSensitive) caseSensitive else !caseSensitive if (expectError) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( alter, expectedErrorClass, expectedMessageParameters, caseSensitive = caseSensitive) } else { assertAnalysisSuccess(alter, caseSensitive) @@ -438,7 +438,7 @@ class V2CommandsCaseSensitivitySuite withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { val expectError = if (expectErrorOnCaseSensitive) caseSensitive else !caseSensitive if (expectError) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( alter, errorClass, messageParameters, caseSensitive = caseSensitive) } else { assertAnalysisSuccess(alter, caseSensitive) From 632dac0f69c36d3b210e22574dfd2a496c732a8e Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 17:09:56 +0200 Subject: [PATCH 12/14] Rename errorClass to condition --- .../NamedParameterFunctionSuite.scala | 4 +-- .../analysis/UnsupportedOperationsSuite.scala | 24 ++++++++--------- .../VariantExpressionEvalUtilsSuite.scala | 4 +-- .../variant/VariantExpressionSuite.scala | 8 +++--- .../parser/TableIdentifierParserSuite.scala | 4 +-- .../catalyst/parser/UnpivotParserSuite.scala | 4 +-- .../V2CommandsCaseSensitivitySuite.scala | 26 +++++++++---------- .../sql/hive/execution/HiveDDLSuite.scala | 23 ++++++++-------- 8 files changed, 49 insertions(+), 48 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedParameterFunctionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedParameterFunctionSuite.scala index 5f1789083fa3b..02543c9fba539 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedParameterFunctionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedParameterFunctionSuite.scala @@ -98,12 +98,12 @@ class NamedParameterFunctionSuite extends AnalysisTest { } test("DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT") { - val errorClass = + val condition = "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.BOTH_POSITIONAL_AND_NAMED" checkError( exception = parseRearrangeException( signature, Seq(k1Arg, k2Arg, k3Arg, k4Arg, namedK1Arg), "foo"), - condition = errorClass, + condition = condition, parameters = Map("routineName" -> toSQLId("foo"), "parameterName" -> toSQLId("k1")) ) checkError( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index a8f479d0526c3..3e9a93dc743df 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -740,7 +740,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { testUnaryOperatorInStreamingPlan( "window", Window(Nil, Nil, Nil, _), - errorClass = "NON_TIME_WINDOW_NOT_SUPPORTED_IN_STREAMING") + condition = "NON_TIME_WINDOW_NOT_SUPPORTED_IN_STREAMING") // Output modes with aggregation and non-aggregation plans testOutputMode(Append, shouldSupportAggregation = false, shouldSupportNonAggregation = true) @@ -869,11 +869,11 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { * supports having a batch child plan, forming a batch subplan inside a streaming plan. */ def testUnaryOperatorInStreamingPlan( - operationName: String, - logicalPlanGenerator: LogicalPlan => LogicalPlan, - outputMode: OutputMode = Append, - expectedMsg: String = "", - errorClass: String = ""): Unit = { + operationName: String, + logicalPlanGenerator: LogicalPlan => LogicalPlan, + outputMode: OutputMode = Append, + expectedMsg: String = "", + condition: String = ""): Unit = { val expectedMsgs = if (expectedMsg.isEmpty) Seq(operationName) else Seq(expectedMsg) @@ -882,7 +882,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { wrapInStreaming(logicalPlanGenerator(streamRelation)), outputMode, expectedMsgs, - errorClass) + condition) assertSupportedInStreamingPlan( s"$operationName with batch relation", @@ -1030,11 +1030,11 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { plan: LogicalPlan, outputMode: OutputMode, expectedMsgs: Seq[String], - errorClass: String = ""): Unit = { + condition: String = ""): Unit = { testError( s"streaming plan - $name: not supported", expectedMsgs :+ "streaming" :+ "DataFrame" :+ "Dataset" :+ "not supported", - errorClass) { + condition) { UnsupportedOperationChecker.checkForStreaming(wrapInStreaming(plan), outputMode) } } @@ -1120,7 +1120,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { def testError( testName: String, expectedMsgs: Seq[String], - errorClass: String = "")(testBody: => Unit): Unit = { + condition: String = "")(testBody: => Unit): Unit = { test(testName) { val e = intercept[AnalysisException] { @@ -1132,8 +1132,8 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { s"actual exception message:\n\t'${e.getMessage}'") } } - if (!errorClass.isEmpty) { - assert(e.getErrorClass == errorClass) + if (!condition.isEmpty) { + assert(e.getErrorClass == condition) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionEvalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionEvalUtilsSuite.scala index 515dc98061a13..f599fead45015 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionEvalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionEvalUtilsSuite.scala @@ -116,7 +116,7 @@ class VariantExpressionEvalUtilsSuite extends SparkFunSuite { } test("parseJson negative") { - def checkException(json: String, errorClass: String, parameters: Map[String, String]): Unit = { + def checkException(json: String, condition: String, parameters: Map[String, String]): Unit = { val try_parse_json_output = VariantExpressionEvalUtils.parseJson(UTF8String.fromString(json), allowDuplicateKeys = false, failOnError = false) checkError( @@ -124,7 +124,7 @@ class VariantExpressionEvalUtilsSuite extends SparkFunSuite { VariantExpressionEvalUtils.parseJson(UTF8String.fromString(json), allowDuplicateKeys = false) }, - condition = errorClass, + condition = condition, parameters = parameters ) assert(try_parse_json_output === null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionSuite.scala index fb0bf63c01123..3fe80647466db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/variant/VariantExpressionSuite.scala @@ -45,12 +45,14 @@ class VariantExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } test("to_json malformed") { - def check(value: Array[Byte], metadata: Array[Byte], - errorClass: String = "MALFORMED_VARIANT"): Unit = { + def check( + value: Array[Byte], + metadata: Array[Byte], + condition: String = "MALFORMED_VARIANT"): Unit = { checkErrorInExpression[SparkRuntimeException]( ResolveTimeZone.resolveTimeZones( StructsToJson(Map.empty, Literal(new VariantVal(value, metadata)))), - errorClass + condition ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 6e1b5b78d9741..0f32922728814 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -296,10 +296,10 @@ class TableIdentifierParserSuite extends SQLKeywordUtils { "t:" -> ("PARSE_SYNTAX_ERROR", Map("error" -> "':'", "hint" -> ": extra input ':'")), "${some.var.x}" -> ("PARSE_SYNTAX_ERROR", Map("error" -> "'$'", "hint" -> "")), "tab:1" -> ("PARSE_SYNTAX_ERROR", Map("error" -> "':'", "hint" -> "")) - ).foreach { case (identifier, (errorClass, parameters)) => + ).foreach { case (identifier, (condition, parameters)) => checkError( exception = intercept[ParseException](parseTableIdentifier(identifier)), - condition = errorClass, + condition = condition, parameters = parameters) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/UnpivotParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/UnpivotParserSuite.scala index 3012ef6f1544d..3f59f8de95429 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/UnpivotParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/UnpivotParserSuite.scala @@ -31,8 +31,8 @@ class UnpivotParserSuite extends AnalysisTest { comparePlans(parsePlan(sqlCommand), plan, checkAnalysis = false) } - private def intercept(sqlCommand: String, errorClass: Option[String], messages: String*): Unit = - interceptParseException(parsePlan)(sqlCommand, messages: _*)(errorClass) + private def intercept(sqlCommand: String, condition: Option[String], messages: String*): Unit = + interceptParseException(parsePlan)(sqlCommand, messages: _*)(condition) test("unpivot - single value") { assertEqual( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala index 4fe79ff53c2df..5091c72ef96ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala @@ -155,7 +155,7 @@ class V2CommandsCaseSensitivitySuite Seq(QualifiedColType( Some(UnresolvedFieldName(field.init.toImmutableArraySeq)), field.last, LongType, true, None, None, None))), - expectedErrorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + expectedErrorCondition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", expectedMessageParameters = Map( "objectName" -> s"`${field.head}`", "proposal" -> "`id`, `data`, `point`") @@ -297,7 +297,7 @@ class V2CommandsCaseSensitivitySuite } test("SPARK-36381: Check column name exist case sensitive and insensitive when add column") { - alterTableErrorClass( + alterTableErrorCondition( AddColumns( table, Seq(QualifiedColType( @@ -317,7 +317,7 @@ class V2CommandsCaseSensitivitySuite } test("SPARK-36381: Check column name exist case sensitive and insensitive when rename column") { - alterTableErrorClass( + alterTableErrorCondition( RenameColumn(table, UnresolvedFieldName(Array("id").toImmutableArraySeq), "DATA"), "FIELD_ALREADY_EXISTS", Map( @@ -338,7 +338,7 @@ class V2CommandsCaseSensitivitySuite } else { alterTableTest( alter = alter, - expectedErrorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + expectedErrorCondition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", expectedMessageParameters = Map( "objectName" -> s"${toSQLId(ref.toImmutableArraySeq)}", "proposal" -> "`id`, `data`, `point`" @@ -353,7 +353,7 @@ class V2CommandsCaseSensitivitySuite Seq(Array("ID"), Array("point", "X"), Array("POINT", "X"), Array("POINT", "x")).foreach { ref => alterTableTest( alter = RenameColumn(table, UnresolvedFieldName(ref.toImmutableArraySeq), "newName"), - expectedErrorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + expectedErrorCondition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", expectedMessageParameters = Map( "objectName" -> s"${toSQLId(ref.toImmutableArraySeq)}", "proposal" -> "`id`, `data`, `point`") @@ -366,7 +366,7 @@ class V2CommandsCaseSensitivitySuite alterTableTest( AlterColumn(table, UnresolvedFieldName(ref.toImmutableArraySeq), None, Some(true), None, None, None), - expectedErrorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + expectedErrorCondition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", expectedMessageParameters = Map( "objectName" -> s"${toSQLId(ref.toImmutableArraySeq)}", "proposal" -> "`id`, `data`, `point`") @@ -379,7 +379,7 @@ class V2CommandsCaseSensitivitySuite alterTableTest( AlterColumn(table, UnresolvedFieldName(ref.toImmutableArraySeq), Some(StringType), None, None, None, None), - expectedErrorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + expectedErrorCondition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", expectedMessageParameters = Map( "objectName" -> s"${toSQLId(ref.toImmutableArraySeq)}", "proposal" -> "`id`, `data`, `point`") @@ -392,7 +392,7 @@ class V2CommandsCaseSensitivitySuite alterTableTest( AlterColumn(table, UnresolvedFieldName(ref.toImmutableArraySeq), None, None, Some("comment"), None, None), - expectedErrorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + expectedErrorCondition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", expectedMessageParameters = Map( "objectName" -> s"${toSQLId(ref.toImmutableArraySeq)}", "proposal" -> "`id`, `data`, `point`") @@ -413,7 +413,7 @@ class V2CommandsCaseSensitivitySuite private def alterTableTest( alter: => AlterTableCommand, - expectedErrorClass: String, + expectedErrorCondition: String, expectedMessageParameters: Map[String, String], expectErrorOnCaseSensitive: Boolean = true): Unit = { Seq(true, false).foreach { caseSensitive => @@ -421,7 +421,7 @@ class V2CommandsCaseSensitivitySuite val expectError = if (expectErrorOnCaseSensitive) caseSensitive else !caseSensitive if (expectError) { assertAnalysisErrorCondition( - alter, expectedErrorClass, expectedMessageParameters, caseSensitive = caseSensitive) + alter, expectedErrorCondition, expectedMessageParameters, caseSensitive = caseSensitive) } else { assertAnalysisSuccess(alter, caseSensitive) } @@ -429,9 +429,9 @@ class V2CommandsCaseSensitivitySuite } } - private def alterTableErrorClass( + private def alterTableErrorCondition( alter: => AlterTableCommand, - errorClass: String, + condition: String, messageParameters: Map[String, String], expectErrorOnCaseSensitive: Boolean = true): Unit = { Seq(true, false).foreach { caseSensitive => @@ -439,7 +439,7 @@ class V2CommandsCaseSensitivitySuite val expectError = if (expectErrorOnCaseSensitive) caseSensitive else !caseSensitive if (expectError) { assertAnalysisErrorCondition( - alter, errorClass, messageParameters, caseSensitive = caseSensitive) + alter, condition, messageParameters, caseSensitive = caseSensitive) } else { assertAnalysisSuccess(alter, caseSensitive) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index cb2799ac8bc6f..69d54a746b55d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -530,7 +530,7 @@ class HiveDDLSuite } test("create table: partition column names exist in table definition") { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( "CREATE TABLE tbl(a int) PARTITIONED BY (a string)", "COLUMN_ALREADY_EXISTS", Map("columnName" -> "`a`")) @@ -770,13 +770,12 @@ class HiveDDLSuite } } - private def assertAnalysisErrorClass( + private def assertAnalysisErrorCondition( sqlText: String, - errorClass: String, + condition: String, parameters: Map[String, String]): Unit = { val e = intercept[AnalysisException](sql(sqlText)) - checkError(e, - condition = errorClass, parameters = parameters) + checkError(e, condition = condition, parameters = parameters) } test("create table - SET TBLPROPERTIES EXTERNAL to TRUE") { @@ -1117,9 +1116,9 @@ class HiveDDLSuite test("drop table using drop view") { withTable("tab1") { sql("CREATE TABLE tab1(c1 int)") - assertAnalysisErrorClass( + assertAnalysisErrorCondition( sqlText = "DROP VIEW tab1", - errorClass = "WRONG_COMMAND_FOR_OBJECT_TYPE", + condition = "WRONG_COMMAND_FOR_OBJECT_TYPE", parameters = Map( "alternative" -> "DROP TABLE", "operation" -> "DROP VIEW", @@ -1136,9 +1135,9 @@ class HiveDDLSuite spark.range(10).write.saveAsTable("tab1") withView("view1") { sql("CREATE VIEW view1 AS SELECT * FROM tab1") - assertAnalysisErrorClass( + assertAnalysisErrorCondition( sqlText = "DROP TABLE view1", - errorClass = "WRONG_COMMAND_FOR_OBJECT_TYPE", + condition = "WRONG_COMMAND_FOR_OBJECT_TYPE", parameters = Map( "alternative" -> "DROP VIEW", "operation" -> "DROP TABLE", @@ -1322,7 +1321,7 @@ class HiveDDLSuite sql(s"USE default") val sqlDropDatabase = s"DROP DATABASE $dbName ${if (cascade) "CASCADE" else "RESTRICT"}" if (tableExists && !cascade) { - assertAnalysisErrorClass( + assertAnalysisErrorCondition( sqlDropDatabase, "SCHEMA_NOT_EMPTY", Map("schemaName" -> s"`$dbName`")) @@ -2526,13 +2525,13 @@ class HiveDDLSuite sql("CREATE TABLE tab (c1 int) PARTITIONED BY (c2 int) STORED AS PARQUET") if (!caseSensitive) { // duplicating partitioning column name - assertAnalysisErrorClass( + assertAnalysisErrorCondition( "ALTER TABLE tab ADD COLUMNS (C2 string)", "COLUMN_ALREADY_EXISTS", Map("columnName" -> "`c2`")) // duplicating data column name - assertAnalysisErrorClass( + assertAnalysisErrorCondition( "ALTER TABLE tab ADD COLUMNS (C1 string)", "COLUMN_ALREADY_EXISTS", Map("columnName" -> "`c1`")) From 6c9e2b1549f147e10e3a5aa07753481d4031d800 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 17:30:26 +0200 Subject: [PATCH 13/14] Rename errorClass in SparkThrowableSuite --- .../apache/spark/SparkThrowableSuite.scala | 34 ++++++++----------- 1 file changed, 14 insertions(+), 20 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala index d99589c171c3f..955333de345a1 100644 --- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala @@ -43,16 +43,13 @@ class SparkThrowableSuite extends SparkFunSuite { /* Used to regenerate the error class file. Run: {{{ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt \ - "core/testOnly *SparkThrowableSuite -- -t \"Error classes are correctly formatted\"" + "core/testOnly *SparkThrowableSuite -- -t \"Error conditions are correctly formatted\"" }}} */ private val regenerateCommand = "SPARK_GENERATE_GOLDEN_FILES=1 build/sbt " + "\"core/testOnly *SparkThrowableSuite -- -t \\\"Error classes match with document\\\"\"" private val errorJsonFilePath = getWorkspaceFilePath( - // Note that though we call them "error classes" here, the proper name is "error conditions", - // hence why the name of the JSON file is different. We will address this inconsistency as part - // of this ticket: https://issues.apache.org/jira/browse/SPARK-47429 "common", "utils", "src", "main", "resources", "error", "error-conditions.json") private val errorReader = new ErrorClassesJsonReader(Seq(errorJsonFilePath.toUri.toURL)) @@ -81,8 +78,8 @@ class SparkThrowableSuite extends SparkFunSuite { mapper.readValue(errorJsonFilePath.toUri.toURL, new TypeReference[Map[String, ErrorInfo]]() {}) } - test("Error classes are correctly formatted") { - val errorClassFileContents = + test("Error conditions are correctly formatted") { + val errorConditionFileContents = IOUtils.toString(errorJsonFilePath.toUri.toURL.openStream(), StandardCharsets.UTF_8) val mapper = JsonMapper.builder() .addModule(DefaultScalaModule) @@ -96,33 +93,30 @@ class SparkThrowableSuite extends SparkFunSuite { .writeValueAsString(errorReader.errorInfoMap) if (regenerateGoldenFiles) { - if (rewrittenString.trim != errorClassFileContents.trim) { - val errorClassesFile = errorJsonFilePath.toFile - logInfo(s"Regenerating error class file $errorClassesFile") - Files.delete(errorClassesFile.toPath) + if (rewrittenString.trim != errorConditionFileContents.trim) { + val errorConditionsFile = errorJsonFilePath.toFile + logInfo(s"Regenerating error conditions file $errorConditionsFile") + Files.delete(errorConditionsFile.toPath) FileUtils.writeStringToFile( - errorClassesFile, + errorConditionsFile, rewrittenString + lineSeparator, StandardCharsets.UTF_8) } } else { - assert(rewrittenString.trim == errorClassFileContents.trim) + assert(rewrittenString.trim == errorConditionFileContents.trim) } } test("SQLSTATE is mandatory") { - val errorClassesNoSqlState = errorReader.errorInfoMap.filter { + val errorConditionsNoSqlState = errorReader.errorInfoMap.filter { case (error: String, info: ErrorInfo) => !error.startsWith("_LEGACY_ERROR_TEMP") && info.sqlState.isEmpty }.keys.toSeq - assert(errorClassesNoSqlState.isEmpty, - s"Error classes without SQLSTATE: ${errorClassesNoSqlState.mkString(", ")}") + assert(errorConditionsNoSqlState.isEmpty, + s"Error classes without SQLSTATE: ${errorConditionsNoSqlState.mkString(", ")}") } test("Error class and error state / SQLSTATE invariants") { - // Unlike in the rest of the codebase, the term "error class" is used here as it is in our - // documentation as well as in the SQL standard. We can remove this comment as part of this - // ticket: https://issues.apache.org/jira/browse/SPARK-47429 val errorClassesJson = Utils.getSparkClassLoader.getResource("error/error-classes.json") val errorStatesJson = Utils.getSparkClassLoader.getResource("error/error-states.json") val mapper = JsonMapper.builder() @@ -171,9 +165,9 @@ class SparkThrowableSuite extends SparkFunSuite { .enable(SerializationFeature.INDENT_OUTPUT) .build() mapper.writeValue(tmpFile, errorReader.errorInfoMap) - val rereadErrorClassToInfoMap = mapper.readValue( + val rereadErrorConditionToInfoMap = mapper.readValue( tmpFile, new TypeReference[Map[String, ErrorInfo]]() {}) - assert(rereadErrorClassToInfoMap == errorReader.errorInfoMap) + assert(rereadErrorConditionToInfoMap == errorReader.errorInfoMap) } test("Error class names should contain only capital letters, numbers and underscores") { From 9ac26bfd055ae72543585e0a26db3ee028062906 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 9 Sep 2024 21:35:30 +0200 Subject: [PATCH 14/14] Fix indentations --- .../analysis/V2WriteAnalysisSuite.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala index 96c9f966e5f74..29c6c63ecfeab 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala @@ -155,11 +155,11 @@ abstract class V2ANSIWriteAnalysisSuiteBase extends V2WriteAnalysisSuiteBase { } override def assertAnalysisErrorCondition( - inputPlan: LogicalPlan, - expectedErrorCondition: String, - expectedMessageParameters: Map[String, String], - queryContext: Array[ExpectedContext] = Array.empty, - caseSensitive: Boolean = true): Unit = { + inputPlan: LogicalPlan, + expectedErrorCondition: String, + expectedMessageParameters: Map[String, String], + queryContext: Array[ExpectedContext] = Array.empty, + caseSensitive: Boolean = true): Unit = { withSQLConf(SQLConf.STORE_ASSIGNMENT_POLICY.key -> StoreAssignmentPolicy.ANSI.toString) { super.assertAnalysisErrorCondition( inputPlan, @@ -192,11 +192,11 @@ abstract class V2StrictWriteAnalysisSuiteBase extends V2WriteAnalysisSuiteBase { } override def assertAnalysisErrorCondition( - inputPlan: LogicalPlan, - expectedErrorCondition: String, - expectedMessageParameters: Map[String, String], - queryContext: Array[ExpectedContext] = Array.empty, - caseSensitive: Boolean = true): Unit = { + inputPlan: LogicalPlan, + expectedErrorCondition: String, + expectedMessageParameters: Map[String, String], + queryContext: Array[ExpectedContext] = Array.empty, + caseSensitive: Boolean = true): Unit = { withSQLConf(SQLConf.STORE_ASSIGNMENT_POLICY.key -> StoreAssignmentPolicy.STRICT.toString) { super.assertAnalysisErrorCondition( inputPlan,