(spark) branch master updated: [SPARK-47261][SQL] Assign better name for errors _LEGACY_ERROR_TEMP_1172, _LEGACY_ERROR_TEMP_1173, and _LEGACY_ERROR_TEMP_1174
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 731a104df307 [SPARK-47261][SQL] Assign better name for errors _LEGACY_ERROR_TEMP_1172, _LEGACY_ERROR_TEMP_1173, and _LEGACY_ERROR_TEMP_1174 731a104df307 is described below commit 731a104df30751ca0c333e621129564f34b81994 Author: junyuc25 AuthorDate: Fri Aug 9 11:54:27 2024 +0200 [SPARK-47261][SQL] Assign better name for errors _LEGACY_ERROR_TEMP_1172, _LEGACY_ERROR_TEMP_1173, and _LEGACY_ERROR_TEMP_1174 ### What changes were proposed in this pull request? Assign better name for error _LEGACY_ERROR_TEMP_1172, _LEGACY_ERROR_TEMP_1173, and _LEGACY_ERROR_TEMP_1174. ### Why are the changes needed? Proper name improves user experience with Spark SQL. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? Added new tests and ran all the tests in the suite: ``` org.apache.spark.sql.execution.datasources.parquetParquetSchemaSuite org.apache.spark.SparkThrowableSuite ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #47421 from junyuc25/SPARK-47261. Authored-by: junyuc25 Signed-off-by: Max Gekk --- .../src/main/resources/error/error-conditions.json | 33 ++- .../spark/sql/errors/QueryCompilationErrors.scala | 6 ++-- ...p-field-with-enum-as-logical-annotation.parquet | Bin 0 -> 409 bytes .../interval-using-fixed-len-byte-array.parquet| Bin 0 -> 369 bytes .../datasources/parquet/ParquetSchemaSuite.scala | 35 ++--- 5 files changed, 52 insertions(+), 22 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 26bda26fef28..4766c7790915 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3543,6 +3543,24 @@ ], "sqlState" : "42805" }, + "PARQUET_TYPE_ILLEGAL" : { +"message" : [ + "Illegal Parquet type: ." +], +"sqlState" : "42846" + }, + "PARQUET_TYPE_NOT_RECOGNIZED" : { +"message" : [ + "Unrecognized Parquet type: ." +], +"sqlState" : "42846" + }, + "PARQUET_TYPE_NOT_SUPPORTED" : { +"message" : [ + "Parquet type not yet supported: ." +], +"sqlState" : "42846" + }, "PARSE_EMPTY_STATEMENT" : { "message" : [ "Syntax error, unexpected empty statement." @@ -5881,21 +5899,6 @@ "createTableColumnTypes option column not found in schema ." ] }, - "_LEGACY_ERROR_TEMP_1172" : { -"message" : [ - "Parquet type not yet supported: ." -] - }, - "_LEGACY_ERROR_TEMP_1173" : { -"message" : [ - "Illegal Parquet type: ." -] - }, - "_LEGACY_ERROR_TEMP_1174" : { -"message" : [ - "Unrecognized Parquet type: ." -] - }, "_LEGACY_ERROR_TEMP_1181" : { "message" : [ "Stream-stream join without equality predicate is not supported." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index cf801e3caacb..09dfa6b3b603 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1995,19 +1995,19 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat def parquetTypeUnsupportedYetError(parquetType: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1172", + errorClass = "PARQUET_TYPE_NOT_SUPPORTED", messageParameters = Map("parquetType" -> parquetType)) } def illegalParquetTypeError(parquetType: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1173", + errorClass = "PARQUET_TYPE_ILLEGAL", messageParameters = Map("parquetType" -> parquetType)) } def unrecognizedParquetTypeError(field: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1174", + errorClass = "PARQUET_TYPE_NOT_RECOGNIZED", messageParameters = Map("field&quo
(spark) branch master updated: [SPARK-49111][SQL] Move withProjectAndFilter to the companion object of DataSourceV2Strategy
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c121bb557447 [SPARK-49111][SQL] Move withProjectAndFilter to the companion object of DataSourceV2Strategy c121bb557447 is described below commit c121bb557447709c19fc01b68b3a34b8838abc6d Author: Uros Stankovic AuthorDate: Fri Aug 9 09:25:28 2024 +0200 [SPARK-49111][SQL] Move withProjectAndFilter to the companion object of DataSourceV2Strategy ### What changes were proposed in this pull request? Move static method `withProjectAndFilter` to object in DataSourceV2Strategy ### Why are the changes needed? It provides better opportunities for reuse, since object of strategy is not needed anymore for function invocation, and a code is also cleaner. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Simple refactor, there is no new changes that can be tested. ### Was this patch authored or co-authored using generative AI tooling? No Closes #47606 from urosstan-db/data-source-v2-strategy-refactor. Authored-by: Uros Stankovic Signed-off-by: Max Gekk --- .../datasources/v2/DataSourceV2Strategy.scala | 51 +- 1 file changed, 31 insertions(+), 20 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index bb138c0fcd0a..b0a89173060a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -53,21 +53,6 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat import DataSourceV2Implicits._ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - private def withProjectAndFilter( - project: Seq[NamedExpression], - filters: Seq[Expression], - scan: LeafExecNode, - needsUnsafeConversion: Boolean): SparkPlan = { -val filterCondition = filters.reduceLeftOption(And) -val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan) - -if (withFilter.output != project || needsUnsafeConversion) { - ProjectExec(project, withFilter) -} else { - withFilter -} - } - private def refreshCache(r: DataSourceV2Relation)(): Unit = { session.sharedState.cacheManager.recacheByPlan(session, r) } @@ -128,12 +113,14 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat unsafeRowRDD, v1Relation, tableIdentifier) - withProjectAndFilter(project, filters, dsScan, needsUnsafeConversion = false) :: Nil + DataSourceV2Strategy.withProjectAndFilter( +project, filters, dsScan, needsUnsafeConversion = false) :: Nil case PhysicalOperation(project, filters, DataSourceV2ScanRelation(_, scan: LocalScan, output, _, _)) => val localScanExec = LocalTableScanExec(output, scan.rows().toImmutableArraySeq) - withProjectAndFilter(project, filters, localScanExec, needsUnsafeConversion = false) :: Nil + DataSourceV2Strategy.withProjectAndFilter( +project, filters, localScanExec, needsUnsafeConversion = false) :: Nil case PhysicalOperation(project, filters, relation: DataSourceV2ScanRelation) => // projection and filters were already pushed down in the optimizer. @@ -146,7 +133,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat val batchExec = BatchScanExec(relation.output, relation.scan, runtimeFilters, relation.ordering, relation.relation.table, StoragePartitionJoinParams(relation.keyGroupedPartitioning)) - withProjectAndFilter(project, postScanFilters, batchExec, !batchExec.supportsColumnar) :: Nil + DataSourceV2Strategy.withProjectAndFilter( +project, postScanFilters, batchExec, !batchExec.supportsColumnar) :: Nil case PhysicalOperation(p, f, r: StreamingDataSourceV2ScanRelation) if r.startOffset.isDefined && r.endOffset.isDefined => @@ -156,7 +144,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat r.output, r.scan, microBatchStream, r.startOffset.get, r.endOffset.get) // Add a Project here to make sure we produce unsafe rows. - withProjectAndFilter(p, f, scanExec, !scanExec.supportsColumnar) :: Nil + DataSourceV2Strategy.withProjectAndFilter(p, f, scanExec, !scanExec.supportsColumnar) :: Nil case PhysicalOperation(p, f, r: StreamingDataSourceV2ScanRelation) if r.startOffset.isDefined &
(spark) branch master updated (8d1539f7bb23 -> 1b3e5e71f7f9)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 8d1539f7bb23 [SPARK-47616][SQL] Add User Document for Mapping Spark SQL Data Types from MySQL add 1b3e5e71f7f9 [SPARK-47485][SQL][PYTHON][CONNECT] Create column with collations in dataframe API No new revisions were added by this update. Summary of changes: python/pyspark/sql/connect/types.py| 2 +- python/pyspark/sql/tests/test_dataframe.py | 6 +++ python/pyspark/sql/tests/test_types.py | 50 ++ python/pyspark/sql/types.py| 20 ++--- .../org/apache/spark/sql/types/StringType.scala| 7 ++- .../sql/execution/python/EvaluatePython.scala | 2 +- .../org/apache/spark/sql/CollationSuite.scala | 11 + 7 files changed, 72 insertions(+), 26 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (87449c3f1d65 -> d10dbaa31a44)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 87449c3f1d65 [SPARK-47563][SQL] Add map normalization on creation add d10dbaa31a44 [SPARK-47564][SQL] Always throw FAILED_READ_FILE error when fail to read files No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 50 +++--- .../apache/spark/sql/avro/AvroOutputWriter.scala | 9 +++- .../spark/sql/avro/AvroLogicalTypeSuite.scala | 2 +- .../org/apache/spark/sql/avro/AvroSuite.scala | 6 +-- ...ror-conditions-failed-read-file-error-class.md} | 27 ++-- docs/sql-error-conditions.md | 12 ++ .../spark/sql/errors/QueryExecutionErrors.scala| 41 -- .../datasources/FileFormatDataWriter.scala | 9 +--- .../sql/execution/datasources/FileScanRDD.scala| 8 +--- .../datasources/v2/FileDataSourceV2.scala | 28 ++-- .../datasources/v2/FilePartitionReader.scala | 4 -- .../spark/sql/FileBasedDataSourceSuite.scala | 12 ++ .../org/apache/spark/sql/MetadataCacheSuite.scala | 32 +- .../sql/errors/QueryExecutionErrorsSuite.scala | 4 +- .../apache/spark/sql/execution/SQLViewSuite.scala | 14 +++--- .../binaryfile/BinaryFileFormatSuite.scala | 2 +- .../sql/execution/datasources/csv/CSVSuite.scala | 27 ++-- .../sql/execution/datasources/json/JsonSuite.scala | 17 +--- .../execution/datasources/orc/OrcFilterSuite.scala | 2 +- .../execution/datasources/orc/OrcQuerySuite.scala | 31 +++--- .../execution/datasources/orc/OrcSourceSuite.scala | 14 +++--- .../parquet/ParquetFileFormatSuite.scala | 7 ++- .../datasources/parquet/ParquetFilterSuite.scala | 2 +- .../datasources/parquet/ParquetIOSuite.scala | 15 +-- .../datasources/parquet/ParquetQuerySuite.scala| 2 +- .../datasources/parquet/ParquetRowIndexSuite.scala | 2 +- .../datasources/parquet/ParquetSchemaSuite.scala | 33 ++ .../sql/execution/datasources/xml/XmlSuite.scala | 17 +--- .../spark/sql/hive/HiveMetadataCacheSuite.scala| 33 +- 29 files changed, 235 insertions(+), 227 deletions(-) copy docs/{sql-error-conditions-invalid-inline-table-error-class.md => sql-error-conditions-failed-read-file-error-class.md} (53%) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (ff38378d7e42 -> a3aa08697686)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from ff38378d7e42 [SPARK-47372][SS] Add support for range scan based key state encoder for use with state store provider add a3aa08697686 [SPARK-47358][SQL][COLLATION] Improve repeat expression support to return correct datatype No new revisions were added by this update. Summary of changes: .../sql/catalyst/expressions/stringExpressions.scala | 4 ++-- .../spark/sql/CollationStringExpressionsSuite.scala| 18 +- 2 files changed, 19 insertions(+), 3 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (b2f6474848fc -> d7db869de609)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from b2f6474848fc [SPARK-46743][SQL] Count bug after constant folding add d7db869de609 [SPARK-47256][SQL] Assign names to error classes _LEGACY_ERROR_TEMP_102[4-7] No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 48 ++ ...itions-invalid-aggregate-filter-error-class.md} | 24 ++--- docs/sql-error-conditions.md | 8 ++ .../spark/sql/catalyst/analysis/Analyzer.scala | 14 ++- .../spark/sql/errors/QueryCompilationErrors.scala | 32 --- .../sql/catalyst/analysis/AnalysisErrorSuite.scala | 21 - .../analyzer-results/group-by-filter.sql.out | 99 .../resources/sql-tests/inputs/group-by-filter.sql | 15 +++ .../sql-tests/results/group-by-filter.sql.out | 102 + 9 files changed, 294 insertions(+), 69 deletions(-) copy docs/{sql-error-conditions-invalid-time-travel-timestamp-expr-error-class.md => sql-error-conditions-invalid-aggregate-filter-error-class.md} (65%) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47539][SQL][FOLLOWUP] Fix UT about `variant`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 1b55fd3ee107 [SPARK-47539][SQL][FOLLOWUP] Fix UT about `variant` 1b55fd3ee107 is described below commit 1b55fd3ee107ecf41e1716fa8f111a7f5c80ecfc Author: panbingkun AuthorDate: Mon Mar 25 20:10:23 2024 +0500 [SPARK-47539][SQL][FOLLOWUP] Fix UT about `variant` ### What changes were proposed in this pull request? The pr aims to fix `conflict UT` by https://github.com/apache/spark/pull/45575 `Support to_json(variant)` Closes #45697 Closes #45698 ### Why are the changes needed? Fix master GA break. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Manually test. https://github.com/apache/spark/assets/15246973/aa52542a-96f4-4f3a-86a5-d4f8b7bc209d";> ./build/sbt "catalyst/testOnly org.apache.spark.sql.catalyst.expressions.ToPrettyStringSuite" https://github.com/apache/spark/assets/15246973/58a16367-f324-4673-89d8-69e5ea61e32c";> - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45702 from panbingkun/SPARK-47539_FOLLOWUP. Authored-by: panbingkun Signed-off-by: Max Gekk --- .../apache/spark/sql/catalyst/expressions/ToPrettyStringSuite.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ToPrettyStringSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ToPrettyStringSuite.scala index 4e043e72fad3..783fba3bfc0d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ToPrettyStringSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ToPrettyStringSuite.scala @@ -121,8 +121,7 @@ class ToPrettyStringSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Variant as pretty strings") { -checkEvaluation( - ToPrettyString(Literal(new VariantVal(Array[Byte](1, 2, 3), Array[Byte](4, 5, - UTF8String.fromBytes(Array[Byte](1, 2, 3)).toString) +val v = new VariantVal(Array[Byte](1, 2, 3), Array[Byte](1, 1)) +checkEvaluation(ToPrettyString(Literal(v)), UTF8String.fromString(v.toString)) } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47506][SQL] Add support to all file source formats for collated data types
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 804b2a416781 [SPARK-47506][SQL] Add support to all file source formats for collated data types 804b2a416781 is described below commit 804b2a4167813ac33f5d2e61898483a66c389059 Author: Stefan Kandic AuthorDate: Mon Mar 25 10:05:01 2024 +0500 [SPARK-47506][SQL] Add support to all file source formats for collated data types ### What changes were proposed in this pull request? Adding support and tests for collated types in all the file sources currently supported by Spark, including: - parquet - json - csv - orc - text Important to note is that collations metadata will only be preserved if these file sources are specified via the [CREATE TABLE USING DATA_SOURCE](https://spark.apache.org/docs/latest/sql-ref-syntax-ddl-create-table-datasource.html) api. Just using the dataframe api to directly write to a file will not preserve collation metadata (except in the case of parquet because it saves the schema in the file itself). ### Why are the changes needed? To have collations be compatible with all file sources users can choose from. ### Does this PR introduce _any_ user-facing change? Yes, users can now create tables with collations using all supported file sources. ### How was this patch tested? New unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45641 from stefankandic/fileSources. Authored-by: Stefan Kandic Signed-off-by: Max Gekk --- .../spark/sql/catalyst/json/JacksonGenerator.scala | 2 +- .../execution/datasources/orc/OrcSerializer.scala | 2 +- .../sql/execution/datasources/orc/OrcUtils.scala | 4 ++ .../datasources/text/TextFileFormat.scala | 2 +- .../org/apache/spark/sql/CollationSuite.scala | 49 ++ 5 files changed, 47 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index e01457ff1025..c2c6117e1e3a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -137,7 +137,7 @@ class JacksonGenerator( (row: SpecializedGetters, ordinal: Int) => gen.writeNumber(row.getDouble(ordinal)) -case StringType => +case _: StringType => (row: SpecializedGetters, ordinal: Int) => gen.writeString(row.getUTF8String(ordinal).toString) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala index 5ed73c3f78b1..75e3e13b0f7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala @@ -130,7 +130,7 @@ class OrcSerializer(dataSchema: StructType) { // Don't reuse the result object for string and binary as it would cause extra data copy. -case StringType => (getter, ordinal) => +case _: StringType => (getter, ordinal) => new Text(getter.getUTF8String(ordinal).getBytes) case BinaryType => (getter, ordinal) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index 15fa2f88e128..24943b37d059 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -305,6 +305,10 @@ object OrcUtils extends Logging { val typeDesc = new TypeDescription(TypeDescription.Category.TIMESTAMP) typeDesc.setAttribute(CATALYST_TYPE_ATTRIBUTE_NAME, t.typeName) Some(typeDesc) +case _: StringType => + val typeDesc = new TypeDescription(TypeDescription.Category.STRING) + typeDesc.setAttribute(CATALYST_TYPE_ATTRIBUTE_NAME, StringType.typeName) + Some(typeDesc) case _ => None } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index e675f70e2a0d..caa4e3ed386b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/Text
(spark) branch master updated (8cba15ed30ea -> c94090e13139)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 8cba15ed30ea [SPARK-47483][SQL] Add support for aggregation and join operations on arrays of collated strings add c94090e13139 [SPARK-47007][SQL] Add the `MapSort` expression No new revisions were added by this update. Summary of changes: .../expressions/collectionOperations.scala | 131 + .../expressions/CollectionExpressionsSuite.scala | 23 2 files changed, 154 insertions(+) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47483][SQL] Add support for aggregation and join operations on arrays of collated strings
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8cba15ed30ea [SPARK-47483][SQL] Add support for aggregation and join operations on arrays of collated strings 8cba15ed30ea is described below commit 8cba15ed30ea55185ebbc8d3601852381a4bfd97 Author: Nikola Mandic AuthorDate: Fri Mar 22 12:15:05 2024 +0500 [SPARK-47483][SQL] Add support for aggregation and join operations on arrays of collated strings ### What changes were proposed in this pull request? Example of aggregation sequence: ``` create table t(a array) using parquet; insert into t(a) values(array('a' collate utf8_binary_lcase)); insert into t(a) values(array('A' collate utf8_binary_lcase)); select distinct a from t; ``` Example of join sequence: ``` create table l(a array) using parquet; create table r(a array) using parquet; insert into l(a) values(array('a' collate utf8_binary_lcase)); insert into r(a) values(array('A' collate utf8_binary_lcase)); select * from l join r where l.a = r.a; ``` Both runs should yield one row since the arrays are considered equal. Problem is in `isBinaryStable` function which should return false if **any** of its subtypes is non-binary collated string. ### Why are the changes needed? To support aggregates and joins in arrays of collated strings properly. ### Does this PR introduce _any_ user-facing change? Yes, it fixes the described scenarios. ### How was this patch tested? Added new checks to collation suite. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45611 from nikolamand-db/SPARK-47483. Authored-by: Nikola Mandic Signed-off-by: Max Gekk --- .../spark/sql/catalyst/util/UnsafeRowUtils.scala | 6 +- .../sql/catalyst/util/UnsafeRowUtilsSuite.scala| 68 - .../org/apache/spark/sql/CollationSuite.scala | 87 +- 3 files changed, 156 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala index 0718cf110f75..0c1ce5ffa8b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala @@ -204,8 +204,8 @@ object UnsafeRowUtils { * e.g. this is not true for non-binary collations (any case/accent insensitive collation * can lead to rows being semantically equal even though their binary representations differ). */ - def isBinaryStable(dataType: DataType): Boolean = dataType.existsRecursively { -case st: StringType => CollationFactory.fetchCollation(st.collationId).isBinaryCollation -case _ => true + def isBinaryStable(dataType: DataType): Boolean = !dataType.existsRecursively { +case st: StringType => !CollationFactory.fetchCollation(st.collationId).isBinaryCollation +case _ => false } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala index c7a8bc74f4dd..b6e87c456de0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala @@ -21,7 +21,7 @@ import java.math.{BigDecimal => JavaBigDecimal} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.types.{Decimal, DecimalType, IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, Decimal, DecimalType, IntegerType, MapType, StringType, StructField, StructType} class UnsafeRowUtilsSuite extends SparkFunSuite { @@ -91,4 +91,70 @@ class UnsafeRowUtilsSuite extends SparkFunSuite { "fieldStatus:\n" + "[UnsafeRowFieldStatus] index: 0, expectedFieldType: IntegerType,")) } + + test("isBinaryStable on complex types containing collated strings") { +val nonBinaryStringType = StringType(CollationFactory.collationNameToId("UTF8_BINARY_LCASE")) + +// simple checks +assert(UnsafeRowUtils.isBinaryStable(IntegerType)) +assert(UnsafeRowUtils.isBinaryStable(StringType)) +assert(!UnsafeRowUtils.isBinaryStable(nonBinaryStringType)) + +assert(UnsafeRowUtils.isBinaryStable(ArrayTy
(spark) branch master updated: [SPARK-47479][SQL] Optimize cannot write data to relations with multiple paths error log
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5042263f8668 [SPARK-47479][SQL] Optimize cannot write data to relations with multiple paths error log 5042263f8668 is described below commit 5042263f86684090f28d9ae051a2c0aa367058b4 Author: huangxiaoping <1754789...@qq.com> AuthorDate: Fri Mar 22 10:36:25 2024 +0500 [SPARK-47479][SQL] Optimize cannot write data to relations with multiple paths error log ### What changes were proposed in this pull request? Add paths information to the error log ### Why are the changes needed? Make it easier for users to troubleshoot problems ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? No Closes #45605 from huangxiaopingRD/SPARK-47479. Authored-by: huangxiaoping <1754789...@qq.com> Signed-off-by: Max Gekk --- common/utils/src/main/resources/error/error-classes.json | 10 +- docs/sql-error-conditions-unsupported-insert-error-class.md| 4 .../org/apache/spark/sql/errors/QueryCompilationErrors.scala | 7 --- .../spark/sql/execution/datasources/DataSourceStrategy.scala | 3 ++- 4 files changed, 15 insertions(+), 9 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 2c3558232361..091f24d44f66 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -4345,6 +4345,11 @@ "Can't insert into the target." ], "subClass" : { + "MULTI_PATH" : { +"message" : [ + "Can only write data to relations with a single path but given paths are ." +] + }, "NOT_ALLOWED" : { "message" : [ "The target relation does not allow insertion." @@ -5269,11 +5274,6 @@ "The ordering of partition columns is . All partition columns having constant values need to appear before other partition columns that do not have an assigned constant value." ] }, - "_LEGACY_ERROR_TEMP_1148" : { -"message" : [ - "Can only write data to relations with a single path." -] - }, "_LEGACY_ERROR_TEMP_1149" : { "message" : [ "Fail to rebuild expression: missing key in `translatedFilterToExpr`." diff --git a/docs/sql-error-conditions-unsupported-insert-error-class.md b/docs/sql-error-conditions-unsupported-insert-error-class.md index a8f25440b00a..3f679589fd3a 100644 --- a/docs/sql-error-conditions-unsupported-insert-error-class.md +++ b/docs/sql-error-conditions-unsupported-insert-error-class.md @@ -30,6 +30,10 @@ Can't insert into the target. This error class has the following derived error classes: +## MULTI_PATH + +Can only write data to relations with a single path but given paths are ``. + ## NOT_ALLOWED The target relation `` does not allow insertion. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index c8a2bec8668c..81cd50d5f2d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1709,10 +1709,11 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "partColumns" -> targetPartitionSchema.fields.map(_.name).mkString("[", ",", "]"))) } - def cannotWriteDataToRelationsWithMultiplePathsError(): Throwable = { + def cannotWriteDataToRelationsWithMultiplePathsError(paths: Seq[Path]): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1148", - messageParameters = Map.empty) + errorClass = "UNSUPPORTED_INSERT.MULTI_PATH", + messageParameters = Map( +"paths" -> paths.mkString("[", ",", "]"))) } def failedToRebuildExpressionError(filter: Filter): Throwable = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 252ab1bcb54e..845d969df088 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/Data
(spark) branch master updated (76260807eef2 -> 3a3477b0f156)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 76260807eef2 [SPARK-46990][SQL] Fix loading empty Avro files emitted by event-hubs add 3a3477b0f156 [SPARK-47443][SQL] Window Aggregate support for collations No new revisions were added by this update. Summary of changes: .../execution/window/WindowEvaluatorFactory.scala| 13 +++-- .../scala/org/apache/spark/sql/CollationSuite.scala | 20 2 files changed, 31 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47007][SQL][PYTHON][R][CONNECT] Add the `map_sort` function
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 747846bd3ef3 [SPARK-47007][SQL][PYTHON][R][CONNECT] Add the `map_sort` function 747846bd3ef3 is described below commit 747846bd3ef38eaec204ae32e47bdcb192fd2797 Author: Stevo Mitric AuthorDate: Wed Mar 20 10:00:11 2024 +0500 [SPARK-47007][SQL][PYTHON][R][CONNECT] Add the `map_sort` function ### What changes were proposed in this pull request? Adding a new function `map_sort` to: - Scala API - Python API - R API - Spark Connect Scala Client - Spark Connect Python Client ### Why are the changes needed? In order to add the ability to do GROUP BY on map types we first have to be able to sort the maps by their key ### Does this PR introduce _any_ user-facing change? Yes, new function `map_sort` ### How was this patch tested? With new UTs ### Was this patch authored or co-authored using generative AI tooling? No Closes #45069 from stefankandic/SPARK-47007. Lead-authored-by: Stevo Mitric Co-authored-by: Stefan Kandic Co-authored-by: Stevo Mitric Signed-off-by: Max Gekk --- R/pkg/NAMESPACE| 1 + R/pkg/R/functions.R| 17 ++ R/pkg/R/generics.R | 4 + R/pkg/tests/fulltests/test_sparkSQL.R | 6 + .../scala/org/apache/spark/sql/functions.scala | 17 ++ .../apache/spark/sql/PlanGenerationTestSuite.scala | 4 + .../explain-results/function_map_sort.explain | 2 + .../query-tests/queries/function_map_sort.json | 29 .../queries/function_map_sort.proto.bin| Bin 0 -> 183 bytes .../source/reference/pyspark.sql/functions.rst | 1 + python/pyspark/sql/connect/functions/builtin.py| 7 + python/pyspark/sql/functions/builtin.py| 48 ++ python/pyspark/sql/tests/test_functions.py | 7 + .../sql/catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/collectionOperations.scala | 172 + .../expressions/CollectionExpressionsSuite.scala | 40 + .../scala/org/apache/spark/sql/functions.scala | 17 ++ .../sql-functions/sql-expression-schema.md | 1 + .../apache/spark/sql/DataFrameFunctionsSuite.scala | 82 +- 19 files changed, 455 insertions(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 3d683ba919a9..a0aa7d0f42ff 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -363,6 +363,7 @@ exportMethods("%<=>%", "map_keys", "map_values", "map_zip_with", + "map_sort", "max", "max_by", "md5", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index a7e337d3f9af..bb8085863482 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -4552,6 +4552,23 @@ setMethod("map_zip_with", ) }) +#' @details +#' \code{map_sort}: Sorts the input map in ascending or descending order according to +#' the natural ordering of the map keys. +#' +#' @rdname column_collection_functions +#' @param asc a logical flag indicating the sorting order. +#'TRUE, sorting is in ascending order. +#'FALSE, sorting is in descending order. +#' @aliases map_sort map_sort,Column-method +#' @note map_sort since 4.0.0 +setMethod("map_sort", + signature(x = "Column"), + function(x, asc = TRUE) { +jc <- callJStatic("org.apache.spark.sql.functions", "map_sort", x@jc, asc) +column(jc) + }) + #' @details #' \code{element_at}: Returns element of array at given index in \code{extraction} if #' \code{x} is array. Returns value for the given key in \code{extraction} if \code{x} is map. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 10a85c7b891a..58bdd53eae25 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1224,6 +1224,10 @@ setGeneric("map_values", function(x) { standardGeneric("map_values") }) #' @name NULL setGeneric("map_zip_with", function(x, y, f) { standardGeneric("map_zip_with") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("map_sort", function(x, asc = TRUE) { standardGeneric("map_sort") }) + #' @rdname column_aggregate_functions #' @name NULL setGeneric("max_by", function(x, y) { standardGeneric("max_by") }) diff --git a/R/p
(spark) branch master updated: [MINOR][TESTS] Collation - extending golden file coverage
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 44a88edc995e [MINOR][TESTS] Collation - extending golden file coverage 44a88edc995e is described below commit 44a88edc995e1e09adfab80e63a409f8ced3b131 Author: Aleksandar Tomic AuthorDate: Mon Mar 18 13:52:48 2024 +0500 [MINOR][TESTS] Collation - extending golden file coverage ### What changes were proposed in this pull request? This PR adds new golden file tests for collation feature: 1) DESCRIBE 3) Basic array operations 4) Removing struct test since same is already covered in golden files. ### Why are the changes needed? Extending test coverage for collation feature. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? No Closes #45515 from dbatomic/collation_golden_files_update. Authored-by: Aleksandar Tomic Signed-off-by: Max Gekk --- .../sql-tests/analyzer-results/collations.sql.out | 38 +- .../test/resources/sql-tests/inputs/collations.sql | 15 +++- .../resources/sql-tests/results/collations.sql.out | 45 +- .../org/apache/spark/sql/CollationSuite.scala | 22 --- 4 files changed, 92 insertions(+), 28 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out index 6d9bb3470be6..3a0f8eec02ba 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out @@ -37,6 +37,12 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d +- LocalRelation [col1#x, col2#x] +-- !query +describe table t1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t1`, false, [col_name#x, data_type#x, comment#x] + + -- !query select count(*) from t1 group by utf8_binary -- !query analysis @@ -207,7 +213,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false -- !query -INSERT INTO t1 VALUES (named_struct('utf8_binary', 'aaa', 'utf8_binary_lcase', 'aaa')) +insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_binary_lcase', 'aaa')) -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1] +- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_binary_lcase, cast(col1#x.utf8_binary_lcase as string collate UTF8_BINARY_LCASE)) AS c1#x] @@ -215,7 +221,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d -- !query -INSERT INTO t1 VALUES (named_struct('utf8_binary', 'AAA', 'utf8_binary_lcase', 'AAA')) +insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_binary_lcase', 'AAA')) -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1] +- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_binary_lcase, cast(col1#x.utf8_binary_lcase as string collate UTF8_BINARY_LCASE)) AS c1#x] @@ -243,3 +249,31 @@ drop table t1 -- !query analysis DropTable false, false +- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 + + +-- !query +select array_contains(ARRAY('aaa' collate utf8_binary_lcase),'AAA' collate utf8_binary_lcase) +-- !query analysis +Project [array_contains(array(collate(aaa, utf8_binary_lcase)), collate(AAA, utf8_binary_lcase)) AS array_contains(array(collate(aaa)), collate(AAA))#x] ++- OneRowRelation + + +-- !query +select array_position(ARRAY('aaa' collate utf8_binary_lcase, 'bbb' collate utf8_binary_lcase),'BBB' collate utf8_binary_lcase) +-- !query analysis +Project [array_position(array(collate(aaa, utf8_binary_lcase), collate(bbb, utf8_binary_lcase)), collate(BBB, utf8_binary_lcase)) AS array_position(array(collate(aaa), collate(bbb)), collate(BBB))#xL] ++- OneRowRelation + + +-- !query +select nullif('aaa' COL
(spark) branch master updated: [SPARK-47327][SQL] Move sort keys concurrency test to CollationFactorySuite
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6719168b6ec7 [SPARK-47327][SQL] Move sort keys concurrency test to CollationFactorySuite 6719168b6ec7 is described below commit 6719168b6ec72242e111bcb3aae75985d36fdad2 Author: Stefan Kandic AuthorDate: Sat Mar 16 09:24:22 2024 +0500 [SPARK-47327][SQL] Move sort keys concurrency test to CollationFactorySuite ### What changes were proposed in this pull request? Move concurrency test to the `CollationFactorySuite` ### Why are the changes needed? This is more appropriate location for the test as it directly uses the `CollationFactory`. Also, I just found out that `par` method is highly discouraged and that we should use `ParSeq` instead. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? With existing UTs ### Was this patch authored or co-authored using generative AI tooling? No Closes #45501 from stefankandic/moveTest. Authored-by: Stefan Kandic Signed-off-by: Max Gekk --- common/unsafe/pom.xml | 6 ++ .../apache/spark/unsafe/types/CollationFactorySuite.scala | 14 ++ .../test/scala/org/apache/spark/sql/CollationSuite.scala | 14 -- 3 files changed, 20 insertions(+), 14 deletions(-) diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index e9785ebb7ad4..13b45f55a4ad 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -47,6 +47,12 @@ ${project.version} + + org.scala-lang.modules + scala-parallel-collections_${scala.binary.version} + test + + com.ibm.icu icu4j diff --git a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala index f9927b94fd42..0a9ff7558e3a 100644 --- a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala +++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.unsafe.types +import scala.collection.parallel.immutable.ParSeq import scala.jdk.CollectionConverters.MapHasAsScala import org.apache.spark.SparkException @@ -138,4 +139,17 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig assert(result == testCase.expectedResult) }) } + + test("test concurrently generating collation keys") { +// generating ICU sort keys is not thread-safe by default so this should fail +// if we don't handle the concurrency properly on Collator level + +(0 to 10).foreach(_ => { + val collator = fetchCollation("UNICODE").collator + + ParSeq(0 to 100).foreach { _ => +collator.getCollationKey("aaa") + } +}) + } } 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 bef7417be36c..aaf3e88c9bdb 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 @@ -18,7 +18,6 @@ package org.apache.spark.sql import scala.collection.immutable.Seq -import scala.collection.parallel.CollectionConverters.ImmutableIterableIsParallelizable import scala.jdk.CollectionConverters.MapHasAsJava import org.apache.spark.SparkException @@ -413,19 +412,6 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } - test("test concurrently generating collation keys") { -// generating ICU sort keys is not thread-safe by default so this should fail -// if we don't handle the concurrency properly on Collator level - -(0 to 10).foreach(_ => { - val collator = CollationFactory.fetchCollation("UNICODE").collator - - (0 to 100).par.foreach { _ => -collator.getCollationKey("aaa") - } -}) - } - test("text writing to parquet with collation enclosed with backticks") { withTempPath{ path => sql(s"select 'a' COLLATE `UNICODE`").write.parquet(path.getAbsolutePath) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47423][SQL] Collations - Set operation support for strings with collations
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 653ac5b729e2 [SPARK-47423][SQL] Collations - Set operation support for strings with collations 653ac5b729e2 is described below commit 653ac5b729e2eba9bf097905b3fd136603b7a298 Author: Aleksandar Tomic AuthorDate: Sat Mar 16 09:21:08 2024 +0500 [SPARK-47423][SQL] Collations - Set operation support for strings with collations ### What changes were proposed in this pull request? This PR fixes support for set operations for strings with collations different from `UTF8_BINARY`. The fix is not strictly related to set operations and may resolve other problems in collation space. The fix is to add default value for `StringType` with collation. Previously the matching pattern would not catch the `StringType` with collation case and fix is simply to do pattern matching on `st: StringType` instead of relying on `StringType` match. ### Why are the changes needed? Fixing behaviour of set operations. ### Does this PR introduce _any_ user-facing change? Yes - fixing the logic that previously didn't work. ### How was this patch tested? Golden file tests are added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45536 from dbatomic/collations_and_set_ops. Authored-by: Aleksandar Tomic Signed-off-by: Max Gekk --- .../spark/sql/catalyst/expressions/literals.scala | 2 +- .../sql-tests/analyzer-results/collations.sql.out | 51 + .../test/resources/sql-tests/inputs/collations.sql | 7 +++ .../resources/sql-tests/results/collations.sql.out | 53 ++ 4 files changed, 112 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 9603647db06f..eadd4c04f4b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -195,7 +195,7 @@ object Literal { case TimestampNTZType => create(0L, TimestampNTZType) case it: DayTimeIntervalType => create(0L, it) case it: YearMonthIntervalType => create(0, it) -case StringType => Literal("") +case st: StringType => Literal(UTF8String.fromString(""), st) case BinaryType => Literal("".getBytes(StandardCharsets.UTF_8)) case CalendarIntervalType => Literal(new CalendarInterval(0, 0, 0)) case arr: ArrayType => create(Array(), arr) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out index fff2d4eab717..6d9bb3470be6 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out @@ -149,6 +149,57 @@ DropTable false, false +- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 +-- !query +select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +-- !query analysis +Except false +:- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] +: +- LocalRelation [col1#x] ++- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] + +- LocalRelation [col1#x] + + +-- !query +select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +-- !query analysis +Except All true +:- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] +: +- LocalRelation [col1#x] ++- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] + +- LocalRelation [col1#x] + + +-- !query +select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +-- !query analysis +Distinct ++- Union false, false + :- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] + : +- LocalRelation [col1#x] + +- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] + +- LocalRelation [col1#x] + + +-- !query +select col1 collate
(spark) branch master updated: [SPARK-47345][SQL][TESTS] Xml functions suite
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7c81bdf1ed17 [SPARK-47345][SQL][TESTS] Xml functions suite 7c81bdf1ed17 is described below commit 7c81bdf1ed17df31ec6d7a3ee9f18b73d8ae2bd6 Author: Yousof Hosny AuthorDate: Fri Mar 15 22:56:29 2024 +0500 [SPARK-47345][SQL][TESTS] Xml functions suite ### What changes were proposed in this pull request? Convert JsonFunctiosnSuite.scala to XML equivalent. Note that XML doesn’t implement all json functions like json_tuple, get_json_object, etc. ### Why are the changes needed? Improve unit test coverage. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45466 from yhosny/xml-functions-suite. Authored-by: Yousof Hosny Signed-off-by: Max Gekk --- .../org/apache/spark/sql/XmlFunctionsSuite.scala | 480 + 1 file changed, 480 insertions(+) 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 new file mode 100644 index ..fcfbebaa61ec --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala @@ -0,0 +1,480 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.text.SimpleDateFormat +import java.util.Locale + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ + +class XmlFunctionsSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + test("from_xml") { +val df = Seq("""1""").toDS() +val schema = new StructType().add("a", IntegerType) + +checkAnswer( + df.select(from_xml($"value", schema)), + Row(Row(1)) :: Nil) + } + + test("from_xml with option (timestampFormat)") { +val df = Seq("""26/08/2015 18:00""").toDS() +val schema = new StructType().add("time", TimestampType) +val options = Map("timestampFormat" -> "dd/MM/ HH:mm").asJava + +checkAnswer( + df.select(from_xml($"value", schema, options)), + Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0" + } + + test("from_xml with option (rowTag)") { +val df = Seq("""1""").toDS() +val schema = new StructType().add("a", IntegerType) +val options = Map("rowTag" -> "foo").asJava + +checkAnswer( + df.select(from_xml($"value", schema)), + Row(Row(1)) :: Nil) + } + + test("from_xml with option (dateFormat)") { +val df = Seq("""26/08/2015""").toDS() +val schema = new StructType().add("time", DateType) +val options = Map("dateFormat" -> "dd/MM/").asJava + +checkAnswer( + df.select(from_xml($"value", schema, options)), + Row(Row(java.sql.Date.valueOf("2015-08-26" + } + + test("from_xml missing columns") { +val df = Seq("""1""").toDS() +val schema = new StructType().add("b", IntegerType) + +checkAnswer( + df.select(from_xml($"value", schema)), + Row(Row(null)) :: Nil) + } + + test("from_xml invalid xml") { +val df = Seq("""1""").toDS() +val schema = new StructType().add("a", IntegerType) + +checkAnswer( + df.select(from_xml($"value", schema)), + Row(Row(null)) :: Nil) + } + + test("
(spark) branch master updated (6bf031796c8c -> e2c0471476ea)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 6bf031796c8c [SPARK-44740][CONNECT][TESTS][FOLLOWUP] Deduplicate `test_metadata` add e2c0471476ea [SPARK-47395] Add collate and collation to other APIs No new revisions were added by this update. Summary of changes: R/pkg/NAMESPACE| 2 + R/pkg/R/functions.R| 29 R/pkg/R/generics.R | 8 R/pkg/tests/fulltests/test_sparkSQL.R | 2 + .../scala/org/apache/spark/sql/functions.scala | 16 +++ .../apache/spark/sql/PlanGenerationTestSuite.scala | 8 ...ulls_first.explain => function_collate.explain} | 2 +- ...ls_first.explain => function_collation.explain} | 2 +- .../{column_rlike.json => function_collate.json} | 4 +- ...sition.proto.bin => function_collate.proto.bin} | Bin 189 -> 189 bytes ...{column_isNull.json => function_collation.json} | 2 +- ..._geq.proto.bin => function_collation.proto.bin} | Bin 178 -> 178 bytes .../source/reference/pyspark.sql/functions.rst | 2 + python/pyspark/sql/connect/functions/builtin.py| 14 ++ python/pyspark/sql/functions/builtin.py| 52 + python/pyspark/sql/tests/test_functions.py | 5 ++ .../scala/org/apache/spark/sql/functions.scala | 16 +++ 17 files changed, 159 insertions(+), 5 deletions(-) copy connector/connect/common/src/test/resources/query-tests/explain-results/{column_asc_nulls_first.explain => function_collate.explain} (57%) copy connector/connect/common/src/test/resources/query-tests/explain-results/{column_asc_nulls_first.explain => function_collation.explain} (61%) copy connector/connect/common/src/test/resources/query-tests/queries/{column_rlike.json => function_collate.json} (89%) copy connector/connect/common/src/test/resources/query-tests/queries/{function_bitmap_bit_position.proto.bin => function_collate.proto.bin} (85%) copy connector/connect/common/src/test/resources/query-tests/queries/{column_isNull.json => function_collation.json} (93%) copy connector/connect/common/src/test/resources/query-tests/queries/{column_geq.proto.bin => function_collation.proto.bin} (90%) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (4b1f8c3d779b -> b7aa9740249b)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 4b1f8c3d779b [SPARK-47399][SQL] Disable generated columns on expressions with collations add b7aa9740249b [SPARK-47407][SQL] Support java.sql.Types.NULL map to NullType No new revisions were added by this update. Summary of changes: .../apache/spark/sql/jdbc/PostgresIntegrationSuite.scala | 16 +++- .../spark/sql/execution/datasources/jdbc/JdbcUtils.scala | 6 +- .../org/apache/spark/sql/jdbc/PostgresDialect.scala | 1 + 3 files changed, 21 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47399][SQL] Disable generated columns on expressions with collations
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 4b1f8c3d779b [SPARK-47399][SQL] Disable generated columns on expressions with collations 4b1f8c3d779b is described below commit 4b1f8c3d779b1391b414d6d6791bed5800b600bd Author: Stefan Kandic AuthorDate: Fri Mar 15 16:12:40 2024 +0500 [SPARK-47399][SQL] Disable generated columns on expressions with collations ### What changes were proposed in this pull request? Disable the ability to use collations in expressions for generated columns. ### Why are the changes needed? Changing the collation of a column or even just changing the ICU version could lead to a differences in the resulting expression so it would be best if we simply disable it for now. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? With new unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45520 from stefankandic/disableGeneratedColumnsCollation. Authored-by: Stefan Kandic Signed-off-by: Max Gekk --- .../spark/sql/catalyst/util/GeneratedColumn.scala | 5 ++ .../org/apache/spark/sql/CollationSuite.scala | 53 ++ 2 files changed, 58 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index 28ddc16cf6b0..747a0e225a2f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, Table import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.sql.util.SchemaUtils /** * This object contains utility methods and values for Generated Columns @@ -162,6 +163,10 @@ object GeneratedColumn { s"generation expression data type ${analyzed.dataType.simpleString} " + s"is incompatible with column data type ${dataType.simpleString}") } +if (analyzed.exists(e => SchemaUtils.hasNonDefaultCollatedString(e.dataType))) { + throw unsupportedExpressionError( +"generation expression cannot contain non-default collated string type") +} } /** 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 72e72a53c4f6..bef7417be36c 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 @@ -622,4 +622,57 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { case _: SortMergeJoinExec => () }.nonEmpty) } + + test("Generated column expressions using collations - errors out") { +checkError( + exception = intercept[AnalysisException] { +sql( + s""" + |CREATE TABLE testcat.test_table( + | c1 STRING COLLATE UNICODE, + | c2 STRING COLLATE UNICODE GENERATED ALWAYS AS (SUBSTRING(c1, 0, 1)) + |) + |USING $v2Source + |""".stripMargin) + }, + errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + parameters = Map( +"fieldName" -> "c2", +"expressionStr" -> "SUBSTRING(c1, 0, 1)", +"reason" -> "generation expression cannot contain non-default collated string type")) + +checkError( + exception = intercept[AnalysisException] { +sql( + s""" + |CREATE TABLE testcat.test_table( + | c1 STRING COLLATE UNICODE, + | c2 STRING COLLATE UNICODE GENERATED ALWAYS AS (c1 || 'a' COLLATE UNICODE) + |) + |USING $v2Source + |""".stripMargin) + }, + errorClass = "UNSUPPORTED_EXPRESSION_GENERATED_COLUMN", + parameters = Map( +"fieldName" -> "c2", +"expressionStr" -> "c1 || 'a' COLLATE UNICODE", +"reason" -> "generation expression cannot contain non-default collated string type")) + +checkError( + exception = intercept[AnalysisException] { +sql( + s&
(spark) branch master updated (d41d5ecda8c1 -> 7b4ab4fa452d)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from d41d5ecda8c1 [SPARK-47390][SQL][FOLLOWUP] Fix TIME_WITH_TIMEZONE mapping in PostgresDialect add 7b4ab4fa452d [SPARK-47387][SQL] Remove some unused error classes No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 27 -- docs/sql-error-conditions.md | 12 -- 2 files changed, 39 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47388][SQL] Pass `messageParameters` by name to `require()`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e848b9665134 [SPARK-47388][SQL] Pass `messageParameters` by name to `require()` e848b9665134 is described below commit e848b9665134faa866617f9caee066e23e5b30a6 Author: Max Gekk AuthorDate: Thu Mar 14 18:25:17 2024 +0500 [SPARK-47388][SQL] Pass `messageParameters` by name to `require()` ### What changes were proposed in this pull request? In the PR, I propose to pass `messageParameters` by name to avoid eager instantiation. ### Why are the changes needed? Passing `messageParameters` by value independently from `requirement` might introduce perf regression. ### Does this PR introduce _any_ user-facing change? No, this is not a part of public API. ### How was this patch tested? By running the affected test suite: ``` $ build/sbt "test:testOnly *QueryCompilationErrorsSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45511 from MaxGekk/fix-SparkException-require. Authored-by: Max Gekk Signed-off-by: Max Gekk --- common/utils/src/main/scala/org/apache/spark/SparkException.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/utils/src/main/scala/org/apache/spark/SparkException.scala b/common/utils/src/main/scala/org/apache/spark/SparkException.scala index 1dc5ee949023..6bacdd0a4440 100644 --- a/common/utils/src/main/scala/org/apache/spark/SparkException.scala +++ b/common/utils/src/main/scala/org/apache/spark/SparkException.scala @@ -116,7 +116,7 @@ object SparkException { def require( requirement: Boolean, errorClass: String, - messageParameters: Map[String, String]): Unit = { + messageParameters: => Map[String, String]): Unit = { if (!requirement) { throw new SparkIllegalArgumentException(errorClass, messageParameters) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46654][SQL][PYTHON] Make `to_csv` explicitly indicate that it does not support some types of data
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ccdc19d587c3 [SPARK-46654][SQL][PYTHON] Make `to_csv` explicitly indicate that it does not support some types of data ccdc19d587c3 is described below commit ccdc19d587c35be227df7ad24118bddfcd77495b Author: panbingkun AuthorDate: Wed Mar 13 17:15:52 2024 +0500 [SPARK-46654][SQL][PYTHON] Make `to_csv` explicitly indicate that it does not support some types of data ### What changes were proposed in this pull request? The pr aims to make `to_csv` `explicitly` indicate that it does not support `some types of data`, includes: - Struct - Array - Map - Variant - Binary ### Why are the changes needed? - Because the `CSV specification` does not have standards for these `types`, and cannot be read back through `from_csv`. - For this case Before: https://github.com/apache/spark/assets/15246973/f298738d-8817-473c-b759-c3f83026ff33";> After: https://github.com/apache/spark/assets/15246973/f32c1282-13b4-4268-8ccb-60ae1f04c358";> ### Does this PR introduce _any_ user-facing change? Yes, `to_csv` will explicitly indicate that it does not support some types of data. ### How was this patch tested? - Add & Update UT. - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44665 from panbingkun/SPARK-46654. Authored-by: panbingkun Signed-off-by: Max Gekk --- docs/sql-migration-guide.md| 1 + python/pyspark/sql/functions/builtin.py| 18 +- .../sql/catalyst/expressions/csvExpressions.scala | 36 +++- .../org/apache/spark/sql/CsvFunctionsSuite.scala | 225 - 4 files changed, 252 insertions(+), 28 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 9f92d6fc8347..325e58bf753d 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -39,6 +39,7 @@ license: | - Since Spark 4.0, the default value of `spark.sql.orc.compression.codec` is changed from `snappy` to `zstd`. To restore the previous behavior, set `spark.sql.orc.compression.codec` to `snappy`. - Since Spark 4.0, `SELECT (*)` is equivalent to `SELECT struct(*)` instead of `SELECT *`. To restore the previous behavior, set `spark.sql.legacy.ignoreParenthesesAroundStar` to `true`. - Since Spark 4.0, the SQL config `spark.sql.legacy.allowZeroIndexInFormatString` is deprecated. Consider to change `strfmt` of the `format_string` function to use 1-based indexes. The first argument must be referenced by "1$", the second by "2$", etc. +- Since Spark 4.0, the function `to_csv` no longer supports input with the data type `STRUCT`, `ARRAY`, `MAP`, `VARIANT` and `BINARY` (because the `CSV specification` does not have standards for these data types and cannot be read back using `from_csv`), Spark will throw `DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE` exception. ## Upgrading from Spark SQL 3.4 to 3.5 diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 6320f9b922ee..d55080111592 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -15491,8 +15491,6 @@ def schema_of_csv(csv: Union[Column, str], options: Optional[Dict[str, str]] = N return _invoke_function("schema_of_csv", col, _options_to_str(options)) -# TODO(SPARK-46654) Re-enable the `Example 2` test after fixing the display -# difference between Regular Spark and Spark Connect on `df.show`. @_try_remote_functions def to_csv(col: "ColumnOrName", options: Optional[Dict[str, str]] = None) -> Column: """ @@ -15534,19 +15532,7 @@ def to_csv(col: "ColumnOrName", options: Optional[Dict[str, str]] = None) -> Col | 2,Alice| +-+ -Example 2: Converting a complex StructType to a CSV string - ->>> from pyspark.sql import Row, functions as sf ->>> data = [(1, Row(age=2, name='Alice', scores=[100, 200, 300]))] ->>> df = spark.createDataFrame(data, ("key", "value")) ->>> df.select(sf.to_csv(df.value)).show(truncate=False) # doctest: +SKIP -+---+ -|to_csv(value) | -+---+ -|2,Alice,"[100,200,300]"| -+---+ - -Example 3: Converting a StructType with null values to a CSV string +Example 2: Converting a StructType with null values to a CSV string >>> from pyspark.sql import Row, functions as sf >>> from pyspark.sql.typ
(spark) branch master updated (78531ebb8064 -> c7795bb8cc82)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 78531ebb8064 [SPARK-47340][SQL] Change "collate" in StringType typename to lowercase add c7795bb8cc82 [SPARK-47333][SQL] Use checkInputDataTypes to check the parameter types of the function `to_xml` No new revisions were added by this update. Summary of changes: .../sql/catalyst/expressions/xmlExpressions.scala | 26 +++--- .../sql/execution/datasources/xml/XmlSuite.scala | 18 +++ 2 files changed, 36 insertions(+), 8 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47340][SQL] Change "collate" in StringType typename to lowercase
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 78531ebb8064 [SPARK-47340][SQL] Change "collate" in StringType typename to lowercase 78531ebb8064 is described below commit 78531ebb8064e05f2cfd44b76af8b5d894e44f33 Author: Nikola Mandic AuthorDate: Wed Mar 13 10:29:12 2024 +0500 [SPARK-47340][SQL] Change "collate" in StringType typename to lowercase ### What changes were proposed in this pull request? Change https://github.com/apache/spark/blob/37bdf5179e01f6c8cb8c83445a155646794aabef/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala#L50 `COLLATE` to lowercase in order to follow convention. Example from DecimalType: https://github.com/apache/spark/blob/2a51242b1bdb4a1d469fc81dc0d21c4cde305753/sql/api/src/main/scala/org/apache/spark/sql/types/DecimalType.scala#L59-L63 ### Why are the changes needed? To follow convention from other data types. ### Does this PR introduce _any_ user-facing change? Yes, displayed collated string typename changes from `string COLLATE ` to `string collate `. ### How was this patch tested? Update golden files and rerun tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45475 from nikolamand-db/SPARK-47340. Authored-by: Nikola Mandic Signed-off-by: Max Gekk --- .../query-tests/queries/select_collated_string.json | 2 +- .../queries/select_collated_string.proto.bin| Bin 65 -> 65 bytes python/pyspark/sql/tests/test_types.py | 1 + python/pyspark/sql/types.py | 4 ++-- .../scala/org/apache/spark/sql/types/DataType.scala | 2 +- .../scala/org/apache/spark/sql/types/StringType.scala | 2 +- .../sql-tests/analyzer-results/collations.sql.out | 16 .../test/resources/sql-tests/results/collations.sql.out | 12 ++-- 8 files changed, 20 insertions(+), 19 deletions(-) diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json index 7bf1e0a7bb10..86595d46654c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json @@ -8,7 +8,7 @@ "planId": "0" }, "localRelation": { -"schema": "struct\u003cs:string COLLATE UTF8_BINARY_LCASE\u003e" +"schema": "struct\u003cs:string collate UTF8_BINARY_LCASE\u003e" } }, "expressions": [{ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin index bbe6d999c47c..30d816526cce 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin differ diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index f4a857ac3ee0..ac24978a59bc 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -862,6 +862,7 @@ class TypesTestsMixin: if k != "varchar" and k != "char": self.assertEqual(t(), _parse_datatype_string(k)) self.assertEqual(IntegerType(), _parse_datatype_string("int")) +self.assertEqual(StringType(), _parse_datatype_string("string collate UTF8_BINARY")) self.assertEqual(StringType(), _parse_datatype_string("string COLLATE UTF8_BINARY")) self.assertEqual(StringType(0), _parse_datatype_string("string")) self.assertEqual(StringType(0), _parse_datatype_string("string COLLATE UTF8_BINARY")) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 0b98ad346576..fda2823709ba 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -261,7 +261,7 @@ class StringType(AtomicType): def collationIdToName(self) -> str: return ( -" COLLATE %s" % StringType.collationNames[self.collationId] +" collate %s" % StringType.collationNames[self.collationId] if self.collationId != 0 else "" ) @@ -1486,7 +1486,7 @@ _all_complex_types: Dict[str, Type[Union[ArrayType, Map
(spark) branch master updated: [SPARK-46331][SQL][FOLLOWUP] Time travel should support current datetime functions
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new efc1aedce6d7 [SPARK-46331][SQL][FOLLOWUP] Time travel should support current datetime functions efc1aedce6d7 is described below commit efc1aedce6d767b40862a8bbade3e250fe7b5637 Author: Wenchen Fan AuthorDate: Tue Mar 12 21:52:32 2024 +0500 [SPARK-46331][SQL][FOLLOWUP] Time travel should support current datetime functions ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/44261 to fix one regression: we missed one exception that time travel needs to evaluate current datetime functions before the analysis phase completes. This PR fixes it by explicitly compute current datetime values in the time travel handling. ### Why are the changes needed? re-support current datetime functions as time travel timestamp ### Does this PR introduce _any_ user-facing change? no, the regression has not been released yet. ### How was this patch tested? new test ### Was this patch authored or co-authored using generative AI tooling? no Closes #45476 from cloud-fan/time-travel. Authored-by: Wenchen Fan Signed-off-by: Max Gekk --- .../apache/spark/sql/catalyst/analysis/TimeTravelSpec.scala | 13 ++--- .../apache/spark/sql/connector/DataSourceV2SQLSuite.scala | 13 + 2 files changed, 23 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TimeTravelSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TimeTravelSpec.scala index 8bfcd955497b..fecec238145e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TimeTravelSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TimeTravelSpec.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, Literal, RuntimeReplaceable, SubqueryExpression, Unevaluable} +import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Expression, Literal, SubqueryExpression, Unevaluable} +import org.apache.spark.sql.catalyst.optimizer.{ComputeCurrentTime, ReplaceExpressions} +import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.TimestampType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -42,14 +44,19 @@ object TimeTravelSpec { throw QueryCompilationErrors.invalidTimestampExprForTimeTravel( "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.INPUT", ts) } - val tsToEval = ts.transform { -case r: RuntimeReplaceable => r.replacement + val tsToEval = { +val fakeProject = Project(Seq(Alias(ts, "ts")()), OneRowRelation()) + ComputeCurrentTime(ReplaceExpressions(fakeProject)).asInstanceOf[Project] + .expressions.head.asInstanceOf[Alias].child + } + tsToEval.foreach { case _: Unevaluable => throw QueryCompilationErrors.invalidTimestampExprForTimeTravel( "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.UNEVALUABLE", ts) case e if !e.deterministic => throw QueryCompilationErrors.invalidTimestampExprForTimeTravel( "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.NON_DETERMINISTIC", ts) +case _ => } val tz = Some(sessionLocalTimeZone) // Set `ansiEnabled` to false, so that it can return null for invalid input and we can provide 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 f21c0c2b52fa..93f199dfd585 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 @@ -3003,6 +3003,19 @@ class DataSourceV2SQLSuiteV1Filter .collect() assert(res10 === Array(Row(7), Row(8))) + checkError( +exception = intercept[AnalysisException] { + // `current_date()` is a valid expression for time travel timestamp, but the test uses + // 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", +parameters = Map("relationName" -> "`t`"), +context = ExpectedContex
(spark) branch master updated (f8a49030fd68 -> 6c30525e0d71)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from f8a49030fd68 [SPARK-47295][SQL] Added ICU StringSearch for the `startsWith` and `endsWith` functions add 6c30525e0d71 [SPARK-47343][SQL] Fix NPE when `sqlString` variable value is null string in execute immediate No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 6 ++ docs/sql-error-conditions.md | 6 ++ .../sql/catalyst/analysis/executeImmediate.scala | 8 +++- .../spark/sql/errors/QueryCompilationErrors.scala | 6 ++ .../sql/catalyst/analysis/AnalysisErrorSuite.scala | 13 .../spark/sql/catalyst/analysis/AnalysisTest.scala | 4 +++- .../analyzer-results/execute-immediate.sql.out | 22 + .../sql-tests/inputs/execute-immediate.sql | 4 .../sql-tests/results/execute-immediate.sql.out| 23 ++ 9 files changed, 90 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (fc61abd89cbf -> f8a49030fd68)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from fc61abd89cbf [SPARK-47169][SQL] Disable bucketing on collated columns add f8a49030fd68 [SPARK-47295][SQL] Added ICU StringSearch for the `startsWith` and `endsWith` functions No new revisions were added by this update. Summary of changes: .../src/main/java/org/apache/spark/unsafe/types/UTF8String.java | 8 ++-- 1 file changed, 6 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (1f58f4c68e8d -> fc61abd89cbf)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 1f58f4c68e8d [SPARK-47254][SQL] Assign names to the error classes _LEGACY_ERROR_TEMP_325[1-9] add fc61abd89cbf [SPARK-47169][SQL] Disable bucketing on collated columns No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 6 + docs/sql-error-conditions.md | 6 + .../spark/sql/errors/QueryCompilationErrors.scala | 6 + .../sql/execution/datasources/BucketingUtils.scala | 6 + .../spark/sql/execution/datasources/rules.scala| 7 + .../org/apache/spark/sql/CollationSuite.scala | 30 ++ 6 files changed, 61 insertions(+) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (95aec974d8e7 -> 1f58f4c68e8d)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 95aec974d8e7 [SPARK-47347][PYTHON][CONNECT][TESTS] Factor session-related tests out of `test_connect_basic` add 1f58f4c68e8d [SPARK-47254][SQL] Assign names to the error classes _LEGACY_ERROR_TEMP_325[1-9] No new revisions were added by this update. Summary of changes: R/pkg/tests/fulltests/test_streaming.R | 3 +- .../src/main/resources/error/error-classes.json| 150 ++- ...onsistent-behavior-cross-version-error-class.md | 6 + ...itions-invalid-datetime-pattern-error-class.md} | 16 +- ...nditions-invalid-interval-format-error-class.md | 81 docs/sql-error-conditions.md | 28 +++ .../spark/ml/tuning/CrossValidatorSuite.scala | 16 +- .../catalyst/util/DateTimeFormatterHelper.scala| 14 +- .../sql/catalyst/util/SparkIntervalUtils.scala | 113 --- .../org/apache/spark/sql/types/DataType.scala | 8 +- .../org/apache/spark/sql/types/StructType.scala| 19 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 3 +- .../sql/catalyst/csv/UnivocityParserSuite.scala| 14 +- .../catalyst/parser/ExpressionParserSuite.scala| 11 -- .../util/DateTimeFormatterHelperSuite.scala| 22 ++- .../sql/catalyst/util/IntervalUtilsSuite.scala | 214 + .../org/apache/spark/sql/types/DataTypeSuite.scala | 8 +- .../apache/spark/sql/types/StructTypeSuite.scala | 12 +- .../analyzer-results/ansi/interval.sql.out | 17 +- .../sql-tests/analyzer-results/interval.sql.out| 17 +- .../sql-tests/results/ansi/interval.sql.out| 17 +- .../results/datetime-formatting-invalid.sql.out| 46 +++-- .../resources/sql-tests/results/interval.sql.out | 17 +- .../sql/execution/command/DDLParserSuite.scala | 13 -- 24 files changed, 610 insertions(+), 255 deletions(-) copy docs/{sql-error-conditions-invalid-boundary-error-class.md => sql-error-conditions-invalid-datetime-pattern-error-class.md} (71%) create mode 100644 docs/sql-error-conditions-invalid-interval-format-error-class.md - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47255][SQL] Assign names to the error classes _LEGACY_ERROR_TEMP_323[6-7] and _LEGACY_ERROR_TEMP_324[7-9]
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 10be03215775 [SPARK-47255][SQL] Assign names to the error classes _LEGACY_ERROR_TEMP_323[6-7] and _LEGACY_ERROR_TEMP_324[7-9] 10be03215775 is described below commit 10be0321577513b442cabc84269d402cffb3c73a Author: Milan Dankovic AuthorDate: Mon Mar 11 19:33:05 2024 +0500 [SPARK-47255][SQL] Assign names to the error classes _LEGACY_ERROR_TEMP_323[6-7] and _LEGACY_ERROR_TEMP_324[7-9] ### What changes were proposed in this pull request? In the PR, I propose to assign the proper names to the legacy error classes _LEGACY_ERROR_TEMP_323[6-7] and _LEGACY_ERROR_TEMP_324[7-9], and modify tests in testing suites to reflect these changes and use checkError() function. Also this PR improves the error messages. ### Why are the changes needed? Proper name improves user experience w/ Spark SQL. ### Does this PR introduce _any_ user-facing change? Yes, the PR changes an user-facing error message. ### How was this patch tested? Error _LEGACY_ERROR_TEMP_3249 is tested by running the modified test suite: `$ build/sbt "catalyst/testOnly *RowJsonSuite"` Errors _LEGACY_ERROR_TEMP_323[6-7] and _LEGACY_ERROR_TEMP_324[7-8] are tested by running the modified test suite: `$ build/sbt "catalyst/testOnly *CSVExprUtilsSuite"` Golden files are regenerated using the: `$ SPARK_GENERATE_GOLDEN_FILES=1` `$ build/sbt "core/testOnly *SparkThrowableSuite"` ### Was this patch authored or co-authored using generative AI tooling? No Closes #45423 from miland-db/miland-db/miland-legacy-error-class. Authored-by: Milan Dankovic Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 59 +- ...nditions-invalid-delimiter-value-error-class.md | 49 ++ docs/sql-error-conditions.md | 14 + .../src/main/scala/org/apache/spark/sql/Row.scala | 9 ++-- .../spark/sql/catalyst/csv/CSVExprUtils.scala | 13 +++-- .../scala/org/apache/spark/sql/RowJsonSuite.scala | 11 ++-- .../spark/sql/catalyst/csv/CSVExprUtilsSuite.scala | 10 ++-- 7 files changed, 123 insertions(+), 42 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 9717ff2ed49c..afe81b8e9bea 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1277,6 +1277,12 @@ ], "sqlState" : "58030" }, + "FAILED_ROW_TO_JSON" : { +"message" : [ + "Failed to convert the row value of the class to the target SQL type in the JSON format." +], +"sqlState" : "2203G" + }, "FIELDS_ALREADY_EXISTS" : { "message" : [ "Cannot column, because already exists in ." @@ -1862,6 +1868,34 @@ }, "sqlState" : "42623" }, + "INVALID_DELIMITER_VALUE" : { +"message" : [ + "Invalid value for delimiter." +], +"subClass" : { + "DELIMITER_LONGER_THAN_EXPECTED" : { +"message" : [ + "Delimiter cannot be more than one character: ." +] + }, + "EMPTY_STRING" : { +"message" : [ + "Delimiter cannot be empty string." +] + }, + "SINGLE_BACKSLASH" : { +"message" : [ + "Single backslash is prohibited. It has special meaning as beginning of an escape sequence. To get the backslash character, pass a string with two backslashes as the delimiter." +] + }, + "UNSUPPORTED_SPECIAL_CHARACTER" : { +"message" : [ + "Unsupported special character for delimiter: ." +] + } +}, +"sqlState" : "42602" + }, "INVALID_DRIVER_MEMORY" : { "message" : [ "System memory must be at least .", @@ -7767,16 +7801,6 @@ "The numbers of zipped arrays and field names should be the same" ] }, - "_LEGACY_ERROR_TEMP_3236" : { -"message" : [ - "Unsupported special character for delimiter: " -] - }, - "_LEGACY_ERROR_TEMP_3237" : { -"message" : [ - "Delimiter cannot be more than one character: " -] - }, "_LEGACY_ERROR_TEMP_3238" : { "message" : [ "Failed to convert value (cl
(spark) branch master updated (427f74066d57 -> 4b0d1085a782)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 427f74066d57 [SPARK-47328][SQL] Rename UCS_BASIC collation to UTF8_BINARY add 4b0d1085a782 [SPARK-47313][SQL] Added scala.MatchError handling inside QueryExecution.toInternalError No new revisions were added by this update. Summary of changes: .../apache/spark/sql/execution/QueryExecution.scala | 21 - 1 file changed, 16 insertions(+), 5 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (d5f42fc6f186 -> 427f74066d57)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from d5f42fc6f186 [SPARK-47337][SQL][DOCKER] Upgrade DB2 docker image version to 11.5.8.0 add 427f74066d57 [SPARK-47328][SQL] Rename UCS_BASIC collation to UTF8_BINARY No new revisions were added by this update. Summary of changes: .../spark/sql/catalyst/util/CollationFactory.java | 4 +- .../org/apache/spark/unsafe/types/UTF8String.java | 2 +- .../spark/unsafe/types/CollationFactorySuite.scala | 54 +++ .../queries/select_collated_string.json| 2 +- .../queries/select_collated_string.proto.bin | Bin 63 -> 65 bytes .../connect/planner/SparkConnectProtoSuite.scala | 2 +- python/pyspark/sql/tests/test_types.py | 10 +- python/pyspark/sql/types.py| 2 +- .../org/apache/spark/sql/types/StringType.scala| 4 +- .../expressions/collationExpressions.scala | 6 +- .../expressions/CollationExpressionSuite.scala | 31 ++-- .../optimizer/MergeScalarSubqueriesSuite.scala | 8 +- .../sql-functions/sql-expression-schema.md | 2 +- .../sql-tests/analyzer-results/collations.sql.out | 110 ++--- .../test/resources/sql-tests/inputs/collations.sql | 46 +++--- .../resources/sql-tests/results/collations.sql.out | 42 ++--- .../org/apache/spark/sql/CollationSuite.scala | 180 ++--- .../spark/sql/FileBasedDataSourceSuite.scala | 2 +- 18 files changed, 255 insertions(+), 252 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (72a95bcad7f1 -> 66d6a23dc3b7)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 72a95bcad7f1 [SPARK-47324][SQL] Add missing timestamp conversion for JDBC nested types add 66d6a23dc3b7 [SPARK-45827][SQL] Add variant singleton type for Java No new revisions were added by this update. Summary of changes: sql/api/src/main/java/org/apache/spark/sql/types/DataTypes.java | 5 + 1 file changed, 5 insertions(+) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [MINOR][SQL][TEST] Moving tests to related suites
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 264e00ee12bb [MINOR][SQL][TEST] Moving tests to related suites 264e00ee12bb is described below commit 264e00ee12bbbd822e52fa8ce79692c60f531495 Author: Mihailo Milosevic AuthorDate: Mon Mar 11 00:25:55 2024 +0500 [MINOR][SQL][TEST] Moving tests to related suites ### What changes were proposed in this pull request? Tests from `QueryCompilationErrorsSuite` were moved to `DDLSuite` and `JDBCTableCatalogSuite`. ### Why are the changes needed? We should move tests to related test suites in order to improve testing. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Corresponding Suites succeed. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45439 from mihailom-db/SPARK-47326. Authored-by: Mihailo Milosevic Signed-off-by: Max Gekk --- .../sql/errors/QueryCompilationErrorsSuite.scala | 74 -- .../spark/sql/execution/command/DDLSuite.scala | 17 + .../v2/jdbc/JDBCTableCatalogSuite.scala| 56 3 files changed, 73 insertions(+), 74 deletions(-) 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 c9198c86c720..4574d3328d48 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 @@ -25,13 +25,11 @@ import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test} import org.apache.spark.sql.catalyst.expressions.{Coalesce, Literal, UnsafeRow} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter -import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils case class StringLongClass(a: String, b: Long) @@ -817,78 +815,6 @@ class QueryCompilationErrorsSuite parameters = Map("extraction" -> "\"array(test)\"")) } - test("CREATE NAMESPACE with LOCATION for JDBC catalog should throw an error") { -withTempDir { tempDir => - val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" - Utils.classForName("org.h2.Driver") - withSQLConf( -"spark.sql.catalog.h2" -> classOf[JDBCTableCatalog].getName, -"spark.sql.catalog.h2.url" -> url, -"spark.sql.catalog.h2.driver" -> "org.h2.Driver") { -checkError( - exception = intercept[AnalysisException] { -sql("CREATE NAMESPACE h2.test_namespace LOCATION './samplepath'") - }, - errorClass = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND", - sqlState = "0A000", - parameters = Map("cmd" -> toSQLStmt("CREATE NAMESPACE ... LOCATION ..."))) - } -} - } - - test("ALTER NAMESPACE with property other than COMMENT " + -"for JDBC catalog should throw an exception") { -withTempDir { tempDir => - val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" - Utils.classForName("org.h2.Driver") - withSQLConf( -"spark.sql.catalog.h2" -> classOf[JDBCTableCatalog].getName, -"spark.sql.catalog.h2.url" -> url, -"spark.sql.catalog.h2.driver" -> "org.h2.Driver") { -val namespace = "h2.test_namespace" -withNamespace(namespace) { - sql(s"CREATE NAMESPACE $namespace") - checkError( -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", -sqlState = "0A000", -parameters = Map( - "cmd" -> toSQLStmt("SET NAMESPACE"), - "property" -> toSQLConf("location"))) - - checkError( -exception = intercept[AnalysisException] { -
(spark) branch master updated: [SPARK-47302][SQL] Collate keyword as identifier
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 0a94e7c7b9a8 [SPARK-47302][SQL] Collate keyword as identifier 0a94e7c7b9a8 is described below commit 0a94e7c7b9a83ecfbca59f5b93532453f462500c Author: Aleksandar Tomic AuthorDate: Fri Mar 8 13:33:21 2024 +0300 [SPARK-47302][SQL] Collate keyword as identifier ### What changes were proposed in this pull request? With this change we move away from using collation names as string literals and start treating them as identifiers, since that is the part of sql standard. Collation names are marked as multi part identifiers, since, in future, we will want to introduce user defined collations which can be part of nested namespaces in catalog. ### Why are the changes needed? Aligning with sql standard on collation syntax. ### Does this PR introduce _any_ user-facing change? Yes. Collations are still not a released feature. ### How was this patch tested? Existing tests are used. ### Was this patch authored or co-authored using generative AI tooling? Closes #45405 from dbatomic/collate_key_word_as_identifier. Lead-authored-by: Aleksandar Tomic Co-authored-by: Nikola Mandic Co-authored-by: Stefan Kandic Signed-off-by: Max Gekk --- .../queries/select_collated_string.json| 2 +- .../queries/select_collated_string.proto.bin | Bin 65 -> 63 bytes .../connect/planner/SparkConnectProtoSuite.scala | 4 +- python/pyspark/sql/tests/test_types.py | 18 - python/pyspark/sql/types.py| 4 +- .../spark/sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../sql/catalyst/parser/DataTypeAstBuilder.scala | 2 +- .../org/apache/spark/sql/types/DataType.scala | 2 +- .../org/apache/spark/sql/types/StringType.scala| 2 +- .../expressions/collationExpressions.scala | 2 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 2 +- .../sql-functions/sql-expression-schema.md | 2 +- .../sql-tests/analyzer-results/collations.sql.out | 26 ++--- .../test/resources/sql-tests/inputs/collations.sql | 10 ++--- .../resources/sql-tests/results/collations.sql.out | 22 +-- .../org/apache/spark/sql/CollationSuite.scala | 42 + .../sql/errors/QueryCompilationErrorsSuite.scala | 6 +-- 17 files changed, 79 insertions(+), 69 deletions(-) diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json index db065b36e345..00644e072190 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json @@ -8,7 +8,7 @@ "planId": "0" }, "localRelation": { -"schema": "struct\u003cs:string COLLATE \u0027UCS_BASIC_LCASE\u0027\u003e" +"schema": "struct\u003cs:string COLLATE UCS_BASIC_LCASE\u003e" } }, "expressions": [{ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin index 3a5661e54ce0..b247463daf43 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin differ diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala index bd52a16d5b22..1b50936d935a 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala @@ -1047,8 +1047,8 @@ class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest { test("SPARK-47144: Collated string") { Seq("UCS_BASIC", "UCS_BASIC_LCASE", "UNICODE", "UNICODE_CI").map(collationName => Seq( -s"select 'abc' collate '$collationName'", -s"select collation('abc' collate '$collationName')").map(query => +s"select 'abc' collate $collationName", +
(spark) branch master updated: [SPARK-47304][SQL][TESTS] Distribute tests from `DataFrameSuite` to more specific suites
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new fe2174d8caaf [SPARK-47304][SQL][TESTS] Distribute tests from `DataFrameSuite` to more specific suites fe2174d8caaf is described below commit fe2174d8caaf6f9474319a8d729a2f537038b4c1 Author: Max Gekk AuthorDate: Wed Mar 6 18:45:00 2024 +0300 [SPARK-47304][SQL][TESTS] Distribute tests from `DataFrameSuite` to more specific suites ### What changes were proposed in this pull request? In the PR, I propose to move some tests from `DataFrameSuite` out, and distribute them across more specific tests suites and new one `DataFrameShowSuite`. ### Why are the changes needed? 1. Improve maintainability of `DataFrameSuite` 2. Speed up execution of the test suite. Execution time dropped to < 1min. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the modified test suites. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45392 from MaxGekk/split-DataFrameSuite. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../apache/spark/sql/DataFrameAggregateSuite.scala | 109 ++ .../spark/sql/DataFrameComplexTypeSuite.scala | 232 +++- .../org/apache/spark/sql/DataFrameJoinSuite.scala | 19 + .../apache/spark/sql/DataFrameSelfJoinSuite.scala | 10 + .../org/apache/spark/sql/DataFrameShowSuite.scala | 487 .../org/apache/spark/sql/DataFrameStatSuite.scala | 65 ++ .../org/apache/spark/sql/DataFrameSuite.scala | 1202 +--- .../spark/sql/StatisticsCollectionSuite.scala | 125 +- .../test/scala/org/apache/spark/sql/UDFSuite.scala | 93 +- .../execution/datasources/DataSourceSuite.scala| 49 + .../sql/execution/datasources/json/JsonSuite.scala | 55 + 11 files changed, 1246 insertions(+), 1200 deletions(-) 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 ec589fa77241..21d7156a62b3 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 @@ -2200,6 +2200,115 @@ class DataFrameAggregateSuite extends QueryTest checkAnswer(df, Row(1, 2, 2) :: Row(3, 1, 1) :: Nil) } } + + private def assertDecimalSumOverflow( + df: DataFrame, ansiEnabled: Boolean, expectedAnswer: Row): Unit = { +if (!ansiEnabled) { + checkAnswer(df, expectedAnswer) +} else { + val e = intercept[ArithmeticException] { +df.collect() + } + assert(e.getMessage.contains("cannot be represented as Decimal") || +e.getMessage.contains("Overflow in sum of decimals")) +} + } + + def checkAggResultsForDecimalOverflow(aggFn: Column => Column): Unit = { +Seq("true", "false").foreach { wholeStageEnabled => + withSQLConf((SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, wholeStageEnabled)) { +Seq(true, false).foreach { ansiEnabled => + withSQLConf((SQLConf.ANSI_ENABLED.key, ansiEnabled.toString)) { +val df0 = Seq( + (BigDecimal("1000"), 1), + (BigDecimal("1000"), 1), + (BigDecimal("1000"), 2)).toDF("decNum", "intNum") +val df1 = Seq( + (BigDecimal("1000"), 2), + (BigDecimal("1000"), 2), + (BigDecimal("1000"), 2), + (BigDecimal("1000"), 2), + (BigDecimal("1000"), 2), + (BigDecimal("1000"), 2), + (BigDecimal("1000"), 2), + (BigDecimal("1000"), 2), + (BigDecimal("1000"), 2)).toDF("decNum", "intNum") +val df = df0.union(df1) +val df2 = df.withColumnRenamed("decNum", "decNum2"). + join(df, "intNum").agg(aggFn($"decNum")) + +val expectedAnswer = Row(null) +assertDecimalSumOverflow(df2, ansiEnabled, expectedAnswer) + +val decStr = "1" + "0" * 19 +val d1 = spark.range(0, 12, 1, 1) +val d2 = d1.select(expr(s"cast('$decStr' as decimal (38, 18)) as d")).agg(aggFn($"d")) +assertDecimalSumOverflow(d2, ansiEnabled, expe
(spark) branch master updated: [SPARK-47102][SQL] Add the `COLLATION_ENABLED` config flag
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6534a3398ae9 [SPARK-47102][SQL] Add the `COLLATION_ENABLED` config flag 6534a3398ae9 is described below commit 6534a3398ae9f4e14b02f4821bccde4b3671dc17 Author: Mihailo Milosevic AuthorDate: Tue Mar 5 15:11:26 2024 +0300 [SPARK-47102][SQL] Add the `COLLATION_ENABLED` config flag ### What changes were proposed in this pull request? This PR adds `COLLATION_ENABLED` config to `SQLConf` and introduces new error class `UNSUPPORTED_FEATURE.COLLATION` to appropriately report error on usage of feature under development. Closes #45218 ### Why are the changes needed? We want to make collations configurable on this flag. These changes disable usage of `collate` and `collation` functions, along with any `COLLATE` syntax when the flag is set to false. By default, the flag is set to false. ### Does this PR introduce _any_ user-facing change? Yes. It introduces new error along with an appropriate message. ### How was this patch tested? ``` ./build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.errors.QueryCompilationErrorsSuite test ./build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.catalyst.expressions.CollationExpressionSuite test ./build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.CollationSuite test ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45285 from mihailom-db/SPARK-47102. Authored-by: Mihailo Milosevic Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 5 ...r-conditions-unsupported-feature-error-class.md | 4 +++ .../expressions/collationExpressions.scala | 27 +-- .../spark/sql/catalyst/parser/AstBuilder.scala | 7 + .../spark/sql/errors/QueryCompilationErrors.scala | 6 + .../org/apache/spark/sql/internal/SQLConf.scala| 10 +++ .../spark/sql/execution/datasources/rules.scala| 21 ++- .../sql/internal/BaseSessionStateBuilder.scala | 1 + .../sql/errors/QueryCompilationErrorsSuite.scala | 31 ++ .../spark/sql/hive/HiveSessionStateBuilder.scala | 1 + 10 files changed, 110 insertions(+), 3 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 7cf3e9c533ca..c61f348e63d8 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -3921,6 +3921,11 @@ "Catalog does not support ." ] }, + "COLLATION" : { +"message" : [ + "Collation is not yet supported." +] + }, "COMBINATION_QUERY_RESULT_CLAUSES" : { "message" : [ "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY." diff --git a/docs/sql-error-conditions-unsupported-feature-error-class.md b/docs/sql-error-conditions-unsupported-feature-error-class.md index 8d42ecdce790..e580ecc63b18 100644 --- a/docs/sql-error-conditions-unsupported-feature-error-class.md +++ b/docs/sql-error-conditions-unsupported-feature-error-class.md @@ -58,6 +58,10 @@ The ANALYZE TABLE command does not support views. Catalog `` does not support ``. +## COLLATION + +Collation is not yet supported. + ## COMBINATION_QUERY_RESULT_CLAUSES Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala index a2faca95dfbc..e51d9a67b166 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala @@ -22,8 +22,10 @@ import org.apache.spark.sql.catalyst.analysis.ExpressionBuilder import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util.CollationFactory import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +// scalastyle:off line.contains.tab @ExpressionDescription( usage = "_FUNC_(expr, collationName) - Marks a given expression with the specified collation.", arguments = """ @@ -33,13 +35,24 @@ import org.apache.spark.sql.types._ """, examples = """ Examples: + > SET spark.sql.collation.enabled=true; +
(spark) branch master updated: [SPARK-46961][SS] Using ProcessorContext to store and retrieve handle
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6b5917beff30 [SPARK-46961][SS] Using ProcessorContext to store and retrieve handle 6b5917beff30 is described below commit 6b5917beff30c813a362584a135a587001df1390 Author: Eric Marnadi AuthorDate: Mon Mar 4 21:20:23 2024 +0300 [SPARK-46961][SS] Using ProcessorContext to store and retrieve handle ### What changes were proposed in this pull request? Setting the processorHandle as a part of the statefulProcessor, so that the user doesn't have to explicitly keep track of it, and can instead simply call `getStatefulProcessorHandle` ### Why are the changes needed? This enhances the usability of the State API ### Does this PR introduce _any_ user-facing change? Yes, this is an API change. This enhances usability of the StatefulProcessorHandle and the TransformWithState operator. ### How was this patch tested? Existing unit tests are sufficient ### Was this patch authored or co-authored using generative AI tooling? No Closes #45359 from ericm-db/handle-context. Authored-by: Eric Marnadi Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 7 +++ docs/sql-error-conditions.md | 7 +++ .../apache/spark/sql/errors/ExecutionErrors.scala | 6 +++ .../spark/sql/streaming/StatefulProcessor.scala| 38 --- .../streaming/TransformWithStateExec.scala | 4 +- .../streaming/TransformWithListStateSuite.scala| 14 ++ .../sql/streaming/TransformWithStateSuite.scala| 54 ++ 7 files changed, 84 insertions(+), 46 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 6ccd841ccd0f..7cf3e9c533ca 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -3337,6 +3337,13 @@ ], "sqlState" : "42802" }, + "STATE_STORE_HANDLE_NOT_INITIALIZED" : { +"message" : [ + "The handle has not been initialized for this StatefulProcessor.", + "Please only use the StatefulProcessor within the transformWithState operator." +], +"sqlState" : "42802" + }, "STATE_STORE_MULTIPLE_VALUES_PER_KEY" : { "message" : [ "Store does not support multiple values per key" diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index f026c456eb2d..7be01f8cb513 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -2091,6 +2091,13 @@ Star (*) is not allowed in a select list when GROUP BY an ordinal position is us Failed to remove default column family with reserved name=``. +### STATE_STORE_HANDLE_NOT_INITIALIZED + +[SQLSTATE: 42802](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +The handle has not been initialized for this StatefulProcessor. +Please only use the StatefulProcessor within the transformWithState operator. + ### STATE_STORE_MULTIPLE_VALUES_PER_KEY [SQLSTATE: 42802](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala index b74a67b49bda..7910c386fcf1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala @@ -53,6 +53,12 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { e) } + def stateStoreHandleNotInitialized(): SparkRuntimeException = { +new SparkRuntimeException( + errorClass = "STATE_STORE_HANDLE_NOT_INITIALIZED", + messageParameters = Map.empty) + } + def failToRecognizePatternAfterUpgradeError( pattern: String, e: Throwable): SparkUpgradeException = { new SparkUpgradeException( diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala index 76794136dd49..42a9430bf39d 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.streaming import java.io.Serializable import org.apache.spark.annotation.{Evolving, Experimental} +import org.apache.spark.sql.errors.ExecutionErrors /** * Represents the arbi
(spark) branch master updated: [SPARK-47245][SQL] Improve error code for INVALID_PARTITION_COLUMN_DATA_TYPE
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 0b2eca28c307 [SPARK-47245][SQL] Improve error code for INVALID_PARTITION_COLUMN_DATA_TYPE 0b2eca28c307 is described below commit 0b2eca28c307b4feb4edd1f53c9aecc64523b7eb Author: Stefan Kandic AuthorDate: Mon Mar 4 19:58:52 2024 +0300 [SPARK-47245][SQL] Improve error code for INVALID_PARTITION_COLUMN_DATA_TYPE ### What changes were proposed in this pull request? Improving the error code for error class `INVALID_PARTITION_COLUMN_DATA_TYPE`. ### Why are the changes needed? `0A000` means a feature is not supported, It implies that in some future it may be and the user hit a limit rather than just writing something inherently wrong. ### Does this PR introduce _any_ user-facing change? Yes, new sql error code ### How was this patch tested? UTs ### Was this patch authored or co-authored using generative AI tooling? No Closes #45355 from stefankandic/SPARK-47245-improveErrorCode. Authored-by: Stefan Kandic Signed-off-by: Max Gekk --- common/utils/src/main/resources/error/error-classes.json | 2 +- docs/sql-error-conditions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 493635d1f8d3..6ccd841ccd0f 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -2325,7 +2325,7 @@ "message" : [ "Cannot use for partition column." ], -"sqlState" : "42601" +"sqlState" : "0A000" }, "INVALID_PARTITION_OPERATION" : { "message" : [ diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 510f56f413c6..f026c456eb2d 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -1320,7 +1320,7 @@ For more details see [INVALID_PARAMETER_VALUE](sql-error-conditions-invalid-para ### INVALID_PARTITION_COLUMN_DATA_TYPE -[SQLSTATE: 42601](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) +[SQLSTATE: 0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported) Cannot use `` for partition column. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (a1e57b32e495 -> b711efd6671a)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from a1e57b32e495 [SPARK-47266][CONNECT] Make `ProtoUtils.abbreviate` return the same type as the input add b711efd6671a [SPARK-43258][SQL] Assign names to error _LEGACY_ERROR_TEMP_202[3,5] No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 22 +- docs/sql-error-conditions.md | 12 ++ .../sql/catalyst/encoders/ExpressionEncoder.scala | 2 +- .../sql/catalyst/expressions/Expression.scala | 12 +++--- .../sql/catalyst/expressions/arithmetic.scala | 4 +- .../expressions/higherOrderFunctions.scala | 2 +- .../spark/sql/errors/QueryExecutionErrors.scala| 6 +-- .../sql/errors/QueryExecutionErrorsSuite.scala | 50 -- 8 files changed, 84 insertions(+), 26 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (1135f6b4c60c -> e46d7d1c60ab)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 1135f6b4c60c [SPARK-47237][BUILD] Upgrade xmlschema-core to 2.3.1 add e46d7d1c60ab [SPARK-47158][SQL] Assign proper name and `sqlState` to `_LEGACY_ERROR_TEMP_(2134|2231)` No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 22 ++- .../scala/org/apache/spark/SparkException.scala| 3 ++- ...ror-conditions-unsupported-call-error-class.md} | 12 +-- docs/sql-error-conditions.md | 4 +++- .../src/main/scala/org/apache/spark/sql/Row.scala | 2 +- .../apache/spark/sql/errors/DataTypeErrors.scala | 10 +++-- .../apache/spark/sql/errors/ExecutionErrors.scala | 11 -- .../spark/sql/catalyst/ShuffleSpecSuite.scala | 2 +- .../catalyst/util/TimestampFormatterSuite.scala| 25 -- .../test/scala/org/apache/spark/sql/RowSuite.scala | 14 +++- .../sql/errors/QueryCompilationErrorsSuite.scala | 2 +- 11 files changed, 74 insertions(+), 33 deletions(-) copy docs/{sql-error-conditions-unsupported-default-value-error-class.md => sql-error-conditions-unsupported-call-error-class.md} (82%) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (8cda48370c1e -> 1135f6b4c60c)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 8cda48370c1e [SPARK-47227][FOLLOW][DOCS] Improve Spark Connect Documentation add 1135f6b4c60c [SPARK-47237][BUILD] Upgrade xmlschema-core to 2.3.1 No new revisions were added by this update. Summary of changes: dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (0b9387e888da -> 8cda48370c1e)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 0b9387e888da [SPARK-47216][DOCS] Refine layout of SQL performance tuning page add 8cda48370c1e [SPARK-47227][FOLLOW][DOCS] Improve Spark Connect Documentation No new revisions were added by this update. Summary of changes: docs/spark-connect-overview.md | 12 ++-- 1 file changed, 6 insertions(+), 6 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (e822775f1f07 -> 0b9387e888da)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from e822775f1f07 [SPARK-47243][SS] Correct the package name of `StateMetadataSource.scala` add 0b9387e888da [SPARK-47216][DOCS] Refine layout of SQL performance tuning page No new revisions were added by this update. Summary of changes: docs/_data/menu-sql.yaml | 14 ++--- docs/sql-performance-tuning.md | 140 +++-- 2 files changed, 86 insertions(+), 68 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47243][SS] Correct the package name of `StateMetadataSource.scala`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e822775f1f07 [SPARK-47243][SS] Correct the package name of `StateMetadataSource.scala` e822775f1f07 is described below commit e822775f1f07dcf13c8f8be5ca54ed2e9681b2a5 Author: yangjie01 AuthorDate: Fri Mar 1 22:06:47 2024 +0300 [SPARK-47243][SS] Correct the package name of `StateMetadataSource.scala` ### What changes were proposed in this pull request? The package name for `StateMetadataSource.scala` is defined as `org.apache.spark.sql.execution.datasources.v2.state`, but it is in the `org/apache/spark/sql/execution/datasources/v2/state/metadata` directory. This pr corrects its package name to `org.apache.spark.sql.execution.datasources.v2.state.metadata` to ensure consistency. ### Why are the changes needed? Correct package name of `StateMetadataSource.scala` ### Does this PR introduce _any_ user-facing change? Yes, the package name of `StateMetadataSource.scala` changed from `org.apache.spark.sql.execution.datasources.v2.state` to `org.apache.spark.sql.execution.datasources.v2.state.metadata` , but this feature was added in Apache Spark 4.0, it has not been released yet. ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #45352 from LuciferYang/correct-package-name-StateMetadataSource. Authored-by: yangjie01 Signed-off-by: Max Gekk --- .../META-INF/services/org.apache.spark.sql.sources.DataSourceRegister | 2 +- .../sql/execution/datasources/v2/state/StatePartitionReader.scala | 1 + .../execution/datasources/v2/state/metadata/StateMetadataSource.scala | 3 ++- .../apache/spark/sql/execution/streaming/IncrementalExecution.scala| 2 +- 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 9e32329beb9d..b628c753a767 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -28,5 +28,5 @@ org.apache.spark.sql.execution.streaming.sources.RateStreamProvider org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider org.apache.spark.sql.execution.datasources.binaryfile.BinaryFileFormat org.apache.spark.sql.execution.streaming.sources.RatePerMicroBatchProvider -org.apache.spark.sql.execution.datasources.v2.state.StateMetadataSource +org.apache.spark.sql.execution.datasources.v2.state.metadata.StateMetadataSource org.apache.spark.sql.execution.datasources.v2.state.StateDataSource diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala index d9fbb272ecbe..e4e28d5f8ac3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala @@ -20,6 +20,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.execution.datasources.v2.state.metadata.StateMetadataPartitionReader import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StateStoreConf, StateStoreId, StateStoreProvider, StateStoreProviderId} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala index 476a5e8e15ca..606a0ac6bc12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.v2.state +package org.apache.spark.sql.execution.datasources.v2.state.metadata import java.util @@ -28,6
(spark) branch master updated (a993f9dcd56f -> a5a282234295)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from a993f9dcd56f [SPARK-42627][SPARK-26494][SQL] Support Oracle TIMESTAMP WITH LOCAL TIME ZONE add a5a282234295 [SPARK-43255][SQL] Replace the error class _LEGACY_ERROR_TEMP_2020 by an internal error No new revisions were added by this update. Summary of changes: common/utils/src/main/resources/error/error-classes.json | 5 - .../scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala | 7 +++ 2 files changed, 3 insertions(+), 9 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (28fd3de0fea0 -> 813934c69df6)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 28fd3de0fea0 [SPARK-41392][BUILD][TESTS] Add `bouncy-castle` test dependencies to `sql/core` module for Hadoop 3.4.0 add 813934c69df6 [SPARK-47015][SQL] Disable partitioning on collated columns No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 13 +- docs/sql-error-conditions.md | 6 + .../spark/sql/errors/QueryCompilationErrors.scala | 6 ++--- .../execution/datasources/PartitioningUtils.scala | 18 ++--- .../spark/sql/execution/datasources/rules.scala| 14 +- .../org/apache/spark/sql/CollationSuite.scala | 30 ++ .../org/apache/spark/sql/SQLInsertTestSuite.scala | 2 +- .../spark/sql/connector/AlterTableTests.scala | 4 +-- 8 files changed, 71 insertions(+), 22 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (a8b6e3c906c9 -> b0a027ca4f72)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from a8b6e3c906c9 [MINOR][PYTHON][DOCS] Clarify verifySchema at createDataFrame not working with pandas DataFrame with Arrow optimization add b0a027ca4f72 [SPARK-47221][SQL] Uses signatures from CsvParser to AbstractParser No new revisions were added by this update. Summary of changes: .../org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala | 8 +--- 1 file changed, 5 insertions(+), 3 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (34026d1f54ed -> 7d2435197841)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 34026d1f54ed [SPARK-42929][CONNECT][PYTHON][TEST] test barrier mode for mapInPandas/mapInArrow add 7d2435197841 [SPARK-46077][SQL] Consider the type generated by TimestampNTZConverter in JdbcDialect.compileValue No new revisions were added by this update. Summary of changes: sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala | 4 sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala| 5 + 2 files changed, 9 insertions(+) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (6d41866e6314 -> ca7012d81b3e)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 6d41866e6314 [SPARK-47191][SQL] Avoid unnecessary relation lookup when uncaching table/view add ca7012d81b3e [SPARK-47192] Convert some _LEGACY_ERROR_TEMP_0035 errors No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 6 ++ docs/sql-error-conditions.md | 6 ++ .../spark/sql/errors/QueryParsingErrors.scala | 7 ++ .../spark/sql/catalyst/parser/AstBuilder.scala | 20 ++--- .../spark/sql/catalyst/parser/ParserUtils.scala| 4 + .../spark/sql/catalyst/parser/DDLParserSuite.scala | 56 +++--- .../spark/sql/execution/SparkSqlParser.scala | 12 +-- .../AlterTableDropPartitionParserSuite.scala | 4 +- .../sql/execution/command/DDLParserSuite.scala | 88 +++--- .../spark/sql/execution/command/DDLSuite.scala | 40 +- .../execution/command/PlanResolutionSuite.scala| 24 +++--- .../spark/sql/hive/execution/HiveDDLSuite.scala| 20 ++--- .../spark/sql/hive/execution/HiveQuerySuite.scala | 6 +- 13 files changed, 158 insertions(+), 135 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-43256][SQL] Remove error class _LEGACY_ERROR_TEMP_2021
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f0090c95ad4e [SPARK-43256][SQL] Remove error class _LEGACY_ERROR_TEMP_2021 f0090c95ad4e is described below commit f0090c95ad4eca18040104848117a7da648ffa3c Author: andrej-db AuthorDate: Tue Feb 27 22:39:41 2024 +0300 [SPARK-43256][SQL] Remove error class _LEGACY_ERROR_TEMP_2021 ### What changes were proposed in this pull request? In the PR, I propose to remove the legacy error class `_LEGACY_ERROR_TEMP_2021` as it is an internal error. ### Why are the changes needed? User experience improvement w/ Spark SQL. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Tests already exist. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45198 from andrej-db/SPARK-43256. Authored-by: andrej-db Signed-off-by: Max Gekk --- common/utils/src/main/resources/error/error-classes.json | 5 - .../src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala | 5 +++-- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 57ee72c7637f..5313de69cbcd 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -5710,11 +5710,6 @@ "Couldn't find a valid constructor on ." ] }, - "_LEGACY_ERROR_TEMP_2021" : { -"message" : [ - "Couldn't find a primary constructor on ." -] - }, "_LEGACY_ERROR_TEMP_2023" : { "message" : [ "Unresolved encoder expected, but was found." diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala index 394e56062071..4d0b47d6e93b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala @@ -202,8 +202,9 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { def primaryConstructorNotFoundError(cls: Class[_]): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2021", - messageParameters = Map("cls" -> cls.toString)) + errorClass = "INTERNAL_ERROR", + messageParameters = Map( +"message" -> s"Couldn't find a primary constructor on ${cls.toString}.")) } def cannotGetOuterPointerForInnerClassError(innerCls: Class[_]): SparkRuntimeException = { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47189][SQL] Tweak column error names and text
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c773aed87d64 [SPARK-47189][SQL] Tweak column error names and text c773aed87d64 is described below commit c773aed87d649f24ca896ffc2a1b09534588e088 Author: Nicholas Chammas AuthorDate: Tue Feb 27 18:04:00 2024 +0300 [SPARK-47189][SQL] Tweak column error names and text ### What changes were proposed in this pull request? Tweak the names and text for a few errors so they read more naturally (and correctly). ### Why are the changes needed? Just minor English improvements. ### Does this PR introduce _any_ user-facing change? Yes, these are user-facing error messages. ### How was this patch tested? No testing apart from CI. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45276 from nchammas/column-error-tweak. Authored-by: Nicholas Chammas Signed-off-by: Max Gekk --- common/utils/src/main/resources/error/error-classes.json | 6 +++--- docs/sql-error-conditions.md | 6 +++--- .../org/apache/spark/sql/errors/QueryParsingErrors.scala | 2 +- .../apache/spark/sql/catalyst/parser/DDLParserSuite.scala| 12 ++-- 4 files changed, 13 insertions(+), 13 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 17ef8e5fe469..57ee72c7637f 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -492,15 +492,15 @@ }, "sqlState" : "54000" }, - "COLUMN_ALIASES_IS_NOT_ALLOWED" : { + "COLUMN_ALIASES_NOT_ALLOWED" : { "message" : [ - "Columns aliases are not allowed in ." + "Column aliases are not allowed in ." ], "sqlState" : "42601" }, "COLUMN_ALREADY_EXISTS" : { "message" : [ - "The column already exists. Consider to choose another name or rename the existing column." + "The column already exists. Choose another name or rename the existing column." ], "sqlState" : "42711" }, diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index bb982a77fca0..f47a67b3d31f 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -400,17 +400,17 @@ Can't create array with `` elements which exceeding the array For more details see [COLLECTION_SIZE_LIMIT_EXCEEDED](sql-error-conditions-collection-size-limit-exceeded-error-class.html) -### COLUMN_ALIASES_IS_NOT_ALLOWED +### COLUMN_ALIASES_NOT_ALLOWED [SQLSTATE: 42601](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) -Columns aliases are not allowed in ``. +Column aliases are not allowed in ``. ### COLUMN_ALREADY_EXISTS [SQLSTATE: 42711](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) -The column `` already exists. Consider to choose another name or rename the existing column. +The column `` already exists. Choose another name or rename the existing column. ### COLUMN_NOT_DEFINED_IN_TABLE diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index daab055608f5..37793d97ccc3 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -42,7 +42,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { def columnAliasInOperationNotAllowedError(op: String, ctx: TableAliasContext): Throwable = { new ParseException( - errorClass = "COLUMN_ALIASES_IS_NOT_ALLOWED", + errorClass = "COLUMN_ALIASES_NOT_ALLOWED", messageParameters = Map("op" -> toSQLStmt(op)), ctx.identifierList()) } 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 7704469b2986..f80cad7dd200 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 @@ -1794,11 +1794,11 @@ class DDLParserSuite extends AnalysisTest { EqualTo(UnresolvedAttribute("t.a"), Literal(2 } - test("delete from table: columns aliases is not a
(spark) branch master updated: [SPARK-47179][SQL] Improve error message from SparkThrowableSuite for better debuggability
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9d2fd5e328fd [SPARK-47179][SQL] Improve error message from SparkThrowableSuite for better debuggability 9d2fd5e328fd is described below commit 9d2fd5e328fd298e8522a7bb9f96c7f014b8f145 Author: Haejoon Lee AuthorDate: Tue Feb 27 15:51:42 2024 +0300 [SPARK-47179][SQL] Improve error message from SparkThrowableSuite for better debuggability ### What changes were proposed in this pull request? This PR proposes to improve error message from SparkThrowableSuite for better debuggability ### Why are the changes needed? The current error message is not very actionable for developer who need regenerating the error class documentation. ### Does this PR introduce _any_ user-facing change? No API change, but the error message is changed: **Before** ``` The error class document is not up to date. Please regenerate it. ``` **After** ``` he error class document is not up to date. Please regenerate it by running `SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "core/testOnly *SparkThrowableSuite -- -t \"Error classes match with document\""` ``` ### How was this patch tested? The existing CI should pass. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45273 from itholic/improve_error_suite_debuggability. Authored-by: Haejoon Lee Signed-off-by: Max Gekk --- .../test/scala/org/apache/spark/SparkThrowableSuite.scala| 12 +--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala index 1f3b28968025..f41ad8fffd3e 100644 --- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala @@ -55,6 +55,9 @@ class SparkThrowableSuite extends SparkFunSuite { "core/testOnly *SparkThrowableSuite -- -t \"Error classes match with document\"" }}} */ + private val regenerateCommand = "SPARK_GENERATE_GOLDEN_FILES=1 build/sbt " + +"\"core/testOnly *SparkThrowableSuite -- -t \\\"Error classes match with document\\\"\"" + private val errorJsonFilePath = getWorkspaceFilePath( "common", "utils", "src", "main", "resources", "error", "error-classes.json") @@ -328,7 +331,8 @@ class SparkThrowableSuite extends SparkFunSuite { } } else { assert(subErrorDoc.trim == errorsInDoc.trim, - "The error class document is not up to date. Please regenerate it.") + "The error class document is not up to date. " + +s"Please regenerate it by running `$regenerateCommand`") } }) @@ -351,7 +355,8 @@ class SparkThrowableSuite extends SparkFunSuite { } } else { assert(sqlErrorParentDoc.trim == commonErrorsInDoc.trim, -"The error class document is not up to date. Please regenerate it.") +"The error class document is not up to date. " + + s"Please regenerate it by running `$regenerateCommand`") } val orphans = orphanedGoldenFiles() @@ -368,7 +373,8 @@ class SparkThrowableSuite extends SparkFunSuite { } } else { assert(orphans.isEmpty, -"Exist orphaned error class documents. Please regenerate it.") +"Exist orphaned error class documents. " + + s"Please regenerate it by running `$regenerateCommand`") } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (0ea318f3e23f -> e2dcb0274f13)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 0ea318f3e23f [SPARK-46947][CORE] Delay memory manager initialization until Driver plugin is loaded add e2dcb0274f13 [SPARK-47173][SS][UI] Fix a typo in streaming UI explanation No new revisions were added by this update. Summary of changes: .../apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala| 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-43259][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_2024
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6de527e9ee94 [SPARK-43259][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_2024 6de527e9ee94 is described below commit 6de527e9ee941bf17aa1d2b26c2a63d36e0bb946 Author: Mihailo Milosevic AuthorDate: Thu Feb 22 20:32:54 2024 +0300 [SPARK-43259][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_2024 ### What changes were proposed in this pull request? In the PR, I propose to assign the proper name `INVALID_EXPRESSION_ENCODER` to the legacy error class `_LEGACY_ERROR_TEMP_2024`, and add a test to the suite which uses `checkError()`. Also this PR improves the error message. ### Why are the changes needed? Proper name improves user experience w/ Spark SQL. ### Does this PR introduce _any_ user-facing change? Yes, the PR changes an user-facing error message. ### How was this patch tested? By running the modified test suite: ``` ./build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.errors.QueryExecutionErrorsSuite test ``` Closes #45095 from mihailom-db/SPARK-43259. Authored-by: Mihailo Milosevic Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 11 ++- .../src/main/resources/error/error-states.json | 6 ++ docs/sql-error-conditions-sqlstates.md | 9 + docs/sql-error-conditions.md | 6 ++ .../spark/sql/catalyst/encoders/package.scala | 2 +- .../spark/sql/errors/QueryExecutionErrors.scala| 12 +++ .../sql/errors/QueryExecutionErrorsSuite.scala | 23 -- 7 files changed, 57 insertions(+), 12 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index d4bb4920db88..17ef8e5fe469 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1877,6 +1877,12 @@ ], "sqlState" : "F" }, + "INVALID_EXPRESSION_ENCODER" : { +"message" : [ + "Found an invalid expression encoder. Expects an instance of ExpressionEncoder but got . For more information consult '/api/java/index.html?org/apache/spark/sql/Encoder.html'." +], +"sqlState" : "42001" + }, "INVALID_EXTRACT_BASE_FIELD_TYPE" : { "message" : [ "Can't extract a value from . Need a complex type [STRUCT, ARRAY, MAP] but got ." @@ -5714,11 +5720,6 @@ "Unresolved encoder expected, but was found." ] }, - "_LEGACY_ERROR_TEMP_2024" : { -"message" : [ - "Only expression encoders are supported for now." -] - }, "_LEGACY_ERROR_TEMP_2025" : { "message" : [ " must override either or ." diff --git a/common/utils/src/main/resources/error/error-states.json b/common/utils/src/main/resources/error/error-states.json index e278c75ae4fa..f696866bb15d 100644 --- a/common/utils/src/main/resources/error/error-states.json +++ b/common/utils/src/main/resources/error/error-states.json @@ -2933,6 +2933,12 @@ "standard": "Y", "usedBy": ["SQL/Foundation", "PostgreSQL", "Redshift", "Oracle", "SQL Server"] }, +"42001": { +"description": "Invalid encoder error", +"origin": "Spark", +"standard": "N", +"usedBy": ["Spark"] +}, "42501": { "description": "The authorization ID does not have the privilege to perform the specified operation on the identified object.", "origin": "DB2", diff --git a/docs/sql-error-conditions-sqlstates.md b/docs/sql-error-conditions-sqlstates.md index 85f1c5c69c33..b142c7340537 100644 --- a/docs/sql-error-conditions-sqlstates.md +++ b/docs/sql-error-conditions-sqlstates.md @@ -238,6 +238,15 @@ Spark SQL uses the following `SQLSTATE` classes: AMBIGUOUS_REFERENCE_TO_FIELDS, INVALID_COLUMN_OR_FIELD_DATA_TYPE, INVALID_EXTRACT_BASE_FIELD_TYPE, INVALID_EXTRACT_FIELD_TYPE, INVALID_FIELD_NAME + + + 42001 + Invalid encoder error + + + + INVALID_EXPRESSION_ENCODER + 42601 diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index e458cd5a337b..0745de995799 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -1144,6 +1144,12 @@ SQLSTATE: F Executor memor
(spark) branch master updated: [SPARK-42328][SQL] Remove _LEGACY_ERROR_TEMP_1175 from error classes
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 78f7c30e140f [SPARK-42328][SQL] Remove _LEGACY_ERROR_TEMP_1175 from error classes 78f7c30e140f is described below commit 78f7c30e140fd8cf4a80b783dd7e9ee4d1b4d7e2 Author: Nikola Mandic AuthorDate: Thu Feb 22 12:09:02 2024 +0300 [SPARK-42328][SQL] Remove _LEGACY_ERROR_TEMP_1175 from error classes ### What changes were proposed in this pull request? Only occurrence of `_LEGACY_ERROR_TEMP_1175` appears under conversion from Spark data types to Parquet. All supported documented [Spark data types](https://spark.apache.org/docs/latest/sql-ref-datatypes.html) are covered in the [conversion function](https://github.com/apache/spark/blob/3e0808c33f185c13808ce2d547ce9ba0057d31a6/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala#L517-L745) (`VarcharType` and `CharType` are not present [...] Convert the error class to `INTERNAL_ERROR`. ### Why are the changes needed? Remove legacy error classes as part of activity in [SPARK-37935](https://issues.apache.org/jira/browse/SPARK-37935). ### Does this PR introduce _any_ user-facing change? If the Spark works correctly, user shouldn't be able to run into `INTERNAL_ERROR` by using the public API. ### How was this patch tested? Added test to `QueryCompilationErrorsSuite` and tested with sbt: ``` project sql testOnly *QueryCompilationErrorsSuite ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45183 from nikolamand-db/nikolamand-db/SPARK-42328. Authored-by: Nikola Mandic Signed-off-by: Max Gekk --- .../utils/src/main/resources/error/error-classes.json | 5 - .../spark/sql/errors/QueryCompilationErrors.scala | 5 +++-- .../sql/errors/QueryCompilationErrorsSuite.scala | 19 +++ 3 files changed, 22 insertions(+), 7 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index c6149ce35a43..d4bb4920db88 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -5118,11 +5118,6 @@ "Unrecognized Parquet type: ." ] }, - "_LEGACY_ERROR_TEMP_1175" : { -"message" : [ - "Unsupported data type ." -] - }, "_LEGACY_ERROR_TEMP_1181" : { "message" : [ "Stream-stream join without equality predicate is not supported." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 53338f38ed6d..e96474862b1d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1908,8 +1908,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat def cannotConvertDataTypeToParquetTypeError(field: StructField): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1175", - messageParameters = Map("dataType" -> field.dataType.catalogString)) + errorClass = "INTERNAL_ERROR", + messageParameters = Map("message" -> +s"Cannot convert Spark data type ${toSQLType(field.dataType)} to any Parquet type.")) } def incompatibleViewSchemaChangeError( 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 e22399c326f6..d4e4a41155ea 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 @@ -24,6 +24,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test} import org.apache.spark.sql.catalyst.expressions.{Coalesce, Literal, UnsafeRow} import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.functions._ @@ -962,6 +963,24 @@ class QueryCompilationErrorsSuite "methodName" -> "update", "classNa
(spark) branch master updated: [SPARK-47087][SQL] Raise Spark's exception with an error class in config value check
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9ec46a691a88 [SPARK-47087][SQL] Raise Spark's exception with an error class in config value check 9ec46a691a88 is described below commit 9ec46a691a880bac9f79b30c870dc99e9626101d Author: Max Gekk AuthorDate: Mon Feb 19 17:31:22 2024 +0300 [SPARK-47087][SQL] Raise Spark's exception with an error class in config value check ### What changes were proposed in this pull request? In the PR, I propose to extend the `TypedConfigBuilder` API by new method: ```scala def checkValue( validator: T => Boolean, errorClass: String, parameters: Map[String, String]): TypedConfigBuilder[T] = { ``` which raises `SparkIllegalArgumentException` with an error class when `checkValue` fails. As an example, I ported the check of the SQL config `spark.sql.session.timeZone` on the new method. ### Why are the changes needed? To improve user's experience with Spark SQL by migration on unified error framework. ### Does this PR introduce _any_ user-facing change? It can if user's code depends on particular format of error messages. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "test:testOnly *SQLConfSuite" $ build/sbt "core/testOnly *SparkThrowableSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45156 from MaxGekk/fix-set-invalid-tz. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 13 .../spark/internal/config/ConfigBuilder.scala | 19 +++ ...or-conditions-invalid-conf-value-error-class.md | 37 ++ docs/sql-error-conditions.md | 8 + .../org/apache/spark/sql/internal/SQLConf.scala| 3 +- .../sql-tests/analyzer-results/timezone.sql.out| 11 +-- .../resources/sql-tests/results/timezone.sql.out | 11 +-- .../apache/spark/sql/internal/SQLConfSuite.scala | 32 --- 8 files changed, 117 insertions(+), 17 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 38161ff87720..6c953174865f 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1758,6 +1758,19 @@ ], "sqlState" : "42000" }, + "INVALID_CONF_VALUE" : { +"message" : [ + "The value '' in the config \"\" is invalid." +], +"subClass" : { + "TIME_ZONE" : { +"message" : [ + "Cannot resolve the given timezone." +] + } +}, +"sqlState" : "22022" + }, "INVALID_CURSOR" : { "message" : [ "The cursor is invalid." diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala index 954980dcb943..303d856ca2c5 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala @@ -22,6 +22,7 @@ import java.util.regex.PatternSyntaxException import scala.util.matching.Regex +import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.network.util.{ByteUnit, JavaUtils} import org.apache.spark.util.Utils @@ -111,6 +112,24 @@ private[spark] class TypedConfigBuilder[T]( } } + /** Checks if the user-provided value for the config matches the validator. + * If it doesn't match, raise Spark's exception with the given error class. */ + def checkValue( + validator: T => Boolean, + errorClass: String, + parameters: Map[String, String]): TypedConfigBuilder[T] = { +transform { v => + if (!validator(v)) { +throw new SparkIllegalArgumentException( + errorClass = "INVALID_CONF_VALUE." + errorClass, + messageParameters = parameters ++ Map( +"confValue" -> v.toString, +"confName" -> parent.key)) + } + v +} + } + /** Check that user-provided values for the config match a pre-defined set. */ def checkValues(validValues: Set[T]): TypedConfigBuilder[T] = { transform { v => diff --git a/docs/sql-error-conditions-invalid-conf-value-error-class.md b/docs/sql-error-conditions-invalid-conf-value-error-class.md new fil
(spark) branch branch-3.5 updated: [SPARK-47072][SQL][3.5] Fix supported interval formats in error messages
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 93a09ea279e6 [SPARK-47072][SQL][3.5] Fix supported interval formats in error messages 93a09ea279e6 is described below commit 93a09ea279e6bd2515ced66d8f38053e4a5514ce Author: Max Gekk AuthorDate: Mon Feb 19 10:29:08 2024 +0300 [SPARK-47072][SQL][3.5] Fix supported interval formats in error messages ### What changes were proposed in this pull request? In the PR, I propose to add one more field to keys of `supportedFormat` in `IntervalUtils` because current implementation has duplicate keys that overwrites each other. For instance, the following keys are the same: ``` (YM.YEAR, YM.MONTH) ... (DT.DAY, DT.HOUR) ``` because `YM.YEAR = DT.DAY = 0` and `YM.MONTH = DT.HOUR = 1`. This is a backport of https://github.com/apache/spark/pull/45127. ### Why are the changes needed? To fix the incorrect error message when Spark cannot parse ANSI interval string. For example, the expected format should be some year-month format but Spark outputs day-time one: ```sql spark-sql (default)> select interval '-\t2-2\t' year to month; Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: - 2-2 . (line 1, pos 16) == SQL == select interval '-\t2-2\t' year to month ^^^ ``` ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? By running the existing test suite: ``` $ build/sbt "test:testOnly *IntervalUtilsSuite" ``` and regenerating the golden files: ``` $ SPARK_GENERATE_GOLDEN_FILES=1 PYSPARK_PYTHON=python3 build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Authored-by: Max Gekk (cherry picked from commit 074fcf2807000d342831379de0fafc1e49a6bf19) Closes #45139 from MaxGekk/fix-supportedFormat-3.5. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../spark/sql/catalyst/util/IntervalUtils.scala| 33 +- .../sql/catalyst/expressions/CastSuiteBase.scala | 8 +++--- .../analyzer-results/ansi/interval.sql.out | 2 +- .../sql-tests/analyzer-results/interval.sql.out| 2 +- .../sql-tests/results/ansi/interval.sql.out| 2 +- .../resources/sql-tests/results/interval.sql.out | 2 +- 6 files changed, 27 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index e051cfc37f12..4d90007400ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -107,25 +107,30 @@ object IntervalUtils extends SparkIntervalUtils { fallBackNotice: Option[String] = None) = { throw new IllegalArgumentException( s"Interval string does not match $intervalStr format of " + -s"${supportedFormat((startFiled, endField)).map(format => s"`$format`").mkString(", ")} " + +s"${supportedFormat((intervalStr, startFiled, endField)) + .map(format => s"`$format`").mkString(", ")} " + s"when cast to $typeName: ${input.toString}" + s"${fallBackNotice.map(s => s", $s").getOrElse("")}") } val supportedFormat = Map( -(YM.YEAR, YM.MONTH) -> Seq("[+|-]y-m", "INTERVAL [+|-]'[+|-]y-m' YEAR TO MONTH"), -(YM.YEAR, YM.YEAR) -> Seq("[+|-]y", "INTERVAL [+|-]'[+|-]y' YEAR"), -(YM.MONTH, YM.MONTH) -> Seq("[+|-]m", "INTERVAL [+|-]'[+|-]m' MONTH"), -(DT.DAY, DT.DAY) -> Seq("[+|-]d", "INTERVAL [+|-]'[+|-]d' DAY"), -(DT.DAY, DT.HOUR) -> Seq("[+|-]d h", "INTERVAL [+|-]'[+|-]d h' DAY TO HOUR"), -(DT.DAY, DT.MINUTE) -> Seq("[+|-]d h:m", "INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE"), -(DT.DAY, DT.SECOND) -> Seq("[+|-]d h:m:s.n", "INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND"), -(DT.HOUR, DT.HOUR) -> Seq("[+|-]h", "INTERVAL [+|-]'[+|-]h' HOUR"), -(DT.HOUR, DT.MINUTE) -> Seq("[+|-]h:m", "INTERVAL [+|-]&
(spark) branch branch-3.4 updated: [SPARK-47072][SQL][3.4] Fix supported interval formats in error messages
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch branch-3.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.4 by this push: new 081c7a7947a4 [SPARK-47072][SQL][3.4] Fix supported interval formats in error messages 081c7a7947a4 is described below commit 081c7a7947a47bf0b2bfd478abdd4b78a1db3ddb Author: Max Gekk AuthorDate: Mon Feb 19 10:26:47 2024 +0300 [SPARK-47072][SQL][3.4] Fix supported interval formats in error messages ### What changes were proposed in this pull request? In the PR, I propose to add one more field to keys of `supportedFormat` in `IntervalUtils` because current implementation has duplicate keys that overwrites each other. For instance, the following keys are the same: ``` (YM.YEAR, YM.MONTH) ... (DT.DAY, DT.HOUR) ``` because `YM.YEAR = DT.DAY = 0` and `YM.MONTH = DT.HOUR = 1`. This is a backport of https://github.com/apache/spark/pull/45127. ### Why are the changes needed? To fix the incorrect error message when Spark cannot parse ANSI interval string. For example, the expected format should be some year-month format but Spark outputs day-time one: ```sql spark-sql (default)> select interval '-\t2-2\t' year to month; Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: - 2-2 . (line 1, pos 16) == SQL == select interval '-\t2-2\t' year to month ^^^ ``` ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? By running the existing test suite: ``` $ build/sbt "test:testOnly *IntervalUtilsSuite" ``` and regenerating the golden files: ``` $ SPARK_GENERATE_GOLDEN_FILES=1 PYSPARK_PYTHON=python3 build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Authored-by: Max Gekk (cherry picked from commit 074fcf2807000d342831379de0fafc1e49a6bf19) Closes #45140 from MaxGekk/fix-supportedFormat-3.4. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../spark/sql/catalyst/util/IntervalUtils.scala| 33 +- .../sql/catalyst/expressions/CastSuiteBase.scala | 8 +++--- .../sql-tests/results/ansi/interval.sql.out| 2 +- .../resources/sql-tests/results/interval.sql.out | 2 +- 4 files changed, 25 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 7b574e987d91..93b8f2f512ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -124,25 +124,30 @@ object IntervalUtils { fallBackNotice: Option[String] = None) = { throw new IllegalArgumentException( s"Interval string does not match $intervalStr format of " + -s"${supportedFormat((startFiled, endField)).map(format => s"`$format`").mkString(", ")} " + +s"${supportedFormat((intervalStr, startFiled, endField)) + .map(format => s"`$format`").mkString(", ")} " + s"when cast to $typeName: ${input.toString}" + s"${fallBackNotice.map(s => s", $s").getOrElse("")}") } val supportedFormat = Map( -(YM.YEAR, YM.MONTH) -> Seq("[+|-]y-m", "INTERVAL [+|-]'[+|-]y-m' YEAR TO MONTH"), -(YM.YEAR, YM.YEAR) -> Seq("[+|-]y", "INTERVAL [+|-]'[+|-]y' YEAR"), -(YM.MONTH, YM.MONTH) -> Seq("[+|-]m", "INTERVAL [+|-]'[+|-]m' MONTH"), -(DT.DAY, DT.DAY) -> Seq("[+|-]d", "INTERVAL [+|-]'[+|-]d' DAY"), -(DT.DAY, DT.HOUR) -> Seq("[+|-]d h", "INTERVAL [+|-]'[+|-]d h' DAY TO HOUR"), -(DT.DAY, DT.MINUTE) -> Seq("[+|-]d h:m", "INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE"), -(DT.DAY, DT.SECOND) -> Seq("[+|-]d h:m:s.n", "INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND"), -(DT.HOUR, DT.HOUR) -> Seq("[+|-]h", "INTERVAL [+|-]'[+|-]h' HOUR"), -(DT.HOUR, DT.MINUTE) -> Seq("[+|-]h:m", "INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE"), -(DT.HOUR, DT.SECOND) -> Seq("[+|-]h:m:s.n", "INTERVAL [+|-]'[+|-]h:m:s.n'
(spark) branch master updated: [MINOR][SQL] Remove `unsupportedOperationMsg` from `CaseInsensitiveStringMap`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 081809667611 [MINOR][SQL] Remove `unsupportedOperationMsg` from `CaseInsensitiveStringMap` 081809667611 is described below commit 081809667611ceca926e9ef66f0bc3a786039314 Author: yangjie01 AuthorDate: Mon Feb 19 10:25:01 2024 +0300 [MINOR][SQL] Remove `unsupportedOperationMsg` from `CaseInsensitiveStringMap` ### What changes were proposed in this pull request? This pr remove unused private field `unsupportedOperationMsg` from `CaseInsensitiveStringMap`, it is replaced by `_LEGACY_ERROR_TEMP_3206` after https://github.com/apache/spark/pull/44549 is merged, ### Why are the changes needed? Remove unused private field. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #45161 from LuciferYang/minior-CaseInsensitiveStringMap. Authored-by: yangjie01 Signed-off-by: Max Gekk --- .../main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java b/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java index 1c5c38ba705f..00a3de692fbf 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java @@ -45,8 +45,6 @@ import java.util.Set; public class CaseInsensitiveStringMap implements Map { private static final Logger logger = LoggerFactory.getLogger(CaseInsensitiveStringMap.class); - private String unsupportedOperationMsg = "CaseInsensitiveStringMap is read-only."; - public static CaseInsensitiveStringMap empty() { return new CaseInsensitiveStringMap(new HashMap<>(0)); } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47072][SQL] Fix supported interval formats in error messages
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 074fcf280700 [SPARK-47072][SQL] Fix supported interval formats in error messages 074fcf280700 is described below commit 074fcf2807000d342831379de0fafc1e49a6bf19 Author: Max Gekk AuthorDate: Fri Feb 16 14:20:37 2024 +0300 [SPARK-47072][SQL] Fix supported interval formats in error messages ### What changes were proposed in this pull request? In the PR, I propose to add one more field to keys of `supportedFormat` in `IntervalUtils` because current implementation has duplicate keys that overwrites each other. For instance, the following keys are the same: ``` (YM.YEAR, YM.MONTH) ... (DT.DAY, DT.HOUR) ``` because `YM.YEAR = DT.DAY = 0` and `YM.MONTH = DT.HOUR = 1`. ### Why are the changes needed? To fix the incorrect error message when Spark cannot parse ANSI interval string. For example, the expected format should be some year-month format but Spark outputs day-time one: ```sql spark-sql (default)> select interval '-\t2-2\t' year to month; Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: - 2-2 . (line 1, pos 16) == SQL == select interval '-\t2-2\t' year to month ^^^ ``` ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? By running the existing test suite: ``` $ build/sbt "test:testOnly *IntervalUtilsSuite" ``` and regenerating the golden files: ``` $ SPARK_GENERATE_GOLDEN_FILES=1 PYSPARK_PYTHON=python3 build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45127 from MaxGekk/fix-supportedFormat. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../spark/sql/catalyst/util/IntervalUtils.scala| 32 -- .../sql/catalyst/expressions/CastSuiteBase.scala | 8 +++--- .../analyzer-results/ansi/interval.sql.out | 2 +- .../sql-tests/analyzer-results/interval.sql.out| 2 +- .../sql-tests/results/ansi/interval.sql.out| 2 +- .../resources/sql-tests/results/interval.sql.out | 2 +- 6 files changed, 26 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 9b4b0302ee77..3a7c7b0904dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -110,7 +110,7 @@ object IntervalUtils extends SparkIntervalUtils { errorClass = "_LEGACY_ERROR_TEMP_3214", messageParameters = Map( "intervalStr" -> intervalStr, -"supportedFormat" -> supportedFormat((startFiled, endField)) +"supportedFormat" -> supportedFormat((intervalStr, startFiled, endField)) .map(format => s"`$format`").mkString(", "), "typeName" -> typeName, "input" -> input.toString, @@ -118,19 +118,23 @@ object IntervalUtils extends SparkIntervalUtils { } val supportedFormat = Map( -(YM.YEAR, YM.MONTH) -> Seq("[+|-]y-m", "INTERVAL [+|-]'[+|-]y-m' YEAR TO MONTH"), -(YM.YEAR, YM.YEAR) -> Seq("[+|-]y", "INTERVAL [+|-]'[+|-]y' YEAR"), -(YM.MONTH, YM.MONTH) -> Seq("[+|-]m", "INTERVAL [+|-]'[+|-]m' MONTH"), -(DT.DAY, DT.DAY) -> Seq("[+|-]d", "INTERVAL [+|-]'[+|-]d' DAY"), -(DT.DAY, DT.HOUR) -> Seq("[+|-]d h", "INTERVAL [+|-]'[+|-]d h' DAY TO HOUR"), -(DT.DAY, DT.MINUTE) -> Seq("[+|-]d h:m", "INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE"), -(DT.DAY, DT.SECOND) -> Seq("[+|-]d h:m:s.n", "INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND"), -(DT.HOUR, DT.HOUR) -> Seq("[+|-]h", "INTERVAL [+|-]'[+|-]h' HOUR"), -(DT.HOUR, DT.MINUTE) -> Seq("[+|-]h:m", "INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE"), -(DT.HOUR, DT.SECOND) -> Seq("[+|-]h:m:s.n", "INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND"), -(DT.MINUTE, DT.MINUTE) -> Seq("[+|-]m", "INTERVAL [+|-]'[+|-]m
(spark) branch master updated: [SPARK-47060][SQL][TESTS] Check `SparkIllegalArgumentException` instead of `IllegalArgumentException` in `catalyst`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 64fa13b5744f [SPARK-47060][SQL][TESTS] Check `SparkIllegalArgumentException` instead of `IllegalArgumentException` in `catalyst` 64fa13b5744f is described below commit 64fa13b5744fac11d62c35a9b1e1f8e7917bcc18 Author: Max Gekk AuthorDate: Fri Feb 16 11:44:39 2024 +0300 [SPARK-47060][SQL][TESTS] Check `SparkIllegalArgumentException` instead of `IllegalArgumentException` in `catalyst` ### What changes were proposed in this pull request? In the PR, I propose to use `checkError()` in tests of `catalyst` to check `SparkIllegalArgumentException`, and its fields. ### Why are the changes needed? By checking `SparkIllegalArgumentException` and its fields like error class and message parameters prevents replacing `SparkIllegalArgumentException` back to `IllegalArgumentException`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the modified test suites. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45118 from MaxGekk/migrate-IllegalArgumentException-catalyst-tests. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../spark/sql/catalyst/util/IntervalUtils.scala| 3 +- .../scala/org/apache/spark/sql/RowJsonSuite.scala | 21 +++-- .../test/scala/org/apache/spark/sql/RowTest.scala | 4 +- .../spark/sql/catalyst/csv/CSVExprUtilsSuite.scala | 11 ++- .../sql/catalyst/csv/UnivocityParserSuite.scala| 42 +- .../sql/catalyst/expressions/CastSuiteBase.scala | 73 ++--- .../expressions/DateExpressionsSuite.scala | 11 ++- .../expressions/ExpressionEvalHelper.scala | 7 ++ .../sql/catalyst/expressions/TimeWindowSuite.scala | 2 +- .../sql/catalyst/expressions/TryCastSuite.scala| 10 ++- .../streaming/InternalOutputModesSuite.scala | 4 +- .../util/DateTimeFormatterHelperSuite.scala| 49 ++- .../sql/catalyst/util/DatetimeFormatterSuite.scala | 6 +- .../spark/sql/connector/catalog/CatalogSuite.scala | 94 +++--- .../catalog/SupportsPartitionManagementSuite.scala | 12 +-- .../org/apache/spark/sql/types/DataTypeSuite.scala | 43 +- .../apache/spark/sql/types/StructTypeSuite.scala | 20 +++-- 17 files changed, 238 insertions(+), 174 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index a016fd7d5881..9b4b0302ee77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import scala.util.control.NonFatal -import org.apache.spark.SparkIllegalArgumentException +import org.apache.spark.{SparkIllegalArgumentException, SparkThrowable} import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.DateTimeConstants._ @@ -202,6 +202,7 @@ object IntervalUtils extends SparkIntervalUtils { try { f } catch { + case e: SparkThrowable => throw e case NonFatal(e) => throw new SparkIllegalArgumentException( errorClass = "_LEGACY_ERROR_TEMP_3213", 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 1962fca66c05..e5914d4a88d2 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 @@ -21,7 +21,7 @@ import java.time.LocalDate import org.json4s.JsonAST.{JArray, JBool, JDecimal, JDouble, JLong, JNull, JObject, JString, JValue} -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.catalyst.encoders.{ExamplePoint, ExamplePointUDT} import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.types._ @@ -128,12 +128,17 @@ class RowJsonSuite extends SparkFunSuite { } test("unsupported type") { -val e = intercept[IllegalArgumentException] { - val row = new GenericRowWithSchema( -Array((1, 2)), -new StructType().add("a", ObjectType(classOf[(Int, Int)]))) - row.jsonValue -} -assert(e.getMessage.contains("Failed to convert value")) +checkError( + exception = intercept[SparkIllegalArgumentException]
(spark) branch master updated: [SPARK-47059][SQL] Attach error context for ALTER COLUMN v1 command
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e2cd71a4cd54 [SPARK-47059][SQL] Attach error context for ALTER COLUMN v1 command e2cd71a4cd54 is described below commit e2cd71a4cd54bbdf5af76d3edfbb2fc8c1b067b6 Author: Wenchen Fan AuthorDate: Thu Feb 15 18:36:11 2024 +0300 [SPARK-47059][SQL] Attach error context for ALTER COLUMN v1 command ### What changes were proposed in this pull request? This is a small fix to improve the error message for ALTER COLUMN. We attach the error context for v1 command as well, making it consistent with v2 command. ### Why are the changes needed? better error message ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? updated tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #45121 from cloud-fan/context. Authored-by: Wenchen Fan Signed-off-by: Max Gekk --- .../spark/sql/errors/QueryCompilationErrors.scala | 7 +-- .../apache/spark/sql/execution/command/ddl.scala | 2 +- .../analyzer-results/change-column.sql.out | 9 +++- .../sql-tests/analyzer-results/charvarchar.sql.out | 9 +++- .../sql-tests/results/change-column.sql.out| 9 +++- .../sql-tests/results/charvarchar.sql.out | 9 +++- .../execution/command/CharVarcharDDLTestBase.scala | 24 ++ 7 files changed, 44 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 46028817e8eb..53338f38ed6d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2637,7 +2637,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat def alterTableChangeColumnNotSupportedForColumnTypeError( tableName: String, originColumn: StructField, - newColumn: StructField): Throwable = { + newColumn: StructField, + origin: Origin): Throwable = { new AnalysisException( errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", messageParameters = Map( @@ -2645,7 +2646,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "originName" -> toSQLId(originColumn.name), "originType" -> toSQLType(originColumn.dataType), "newName" -> toSQLId(newColumn.name), -"newType"-> toSQLType(newColumn.dataType))) +"newType"-> toSQLType(newColumn.dataType)), + origin = origin +) } def cannotAlterPartitionColumn( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index dc1c5b3fd580..a5e48784ada1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -390,7 +390,7 @@ case class AlterTableChangeColumnCommand( // Throw an AnalysisException if the column name/dataType is changed. if (!columnEqual(originColumn, newColumn, resolver)) { throw QueryCompilationErrors.alterTableChangeColumnNotSupportedForColumnTypeError( -toSQLId(table.identifier.nameParts), originColumn, newColumn) +toSQLId(table.identifier.nameParts), originColumn, newColumn, this.origin) } val newDataSchema = table.dataSchema.fields.map { field => diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out index a3d4388ab84f..07edfa5e95e1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out @@ -69,7 +69,14 @@ org.apache.spark.sql.AnalysisException "originName" : "`a`", "originType" : "\"INT\"", "table" : "`spark_catalog`.`default`.`test_change`" - } + }, + "queryContext" : [ { +"objectType" : "", +"objectName" : "", +"startIndex" : 1, +"stopIndex" : 44, +"fragment" : "ALTER TABLE test_change CHANGE a TYPE STRING" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/s
(spark) branch master updated: [SPARK-47045][SQL] Replace `IllegalArgumentException` by `SparkIllegalArgumentException` in `sql/api`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9ef552691e1d [SPARK-47045][SQL] Replace `IllegalArgumentException` by `SparkIllegalArgumentException` in `sql/api` 9ef552691e1d is described below commit 9ef552691e1d4725d7a64b45e6cdee9e5e75f992 Author: Max Gekk AuthorDate: Thu Feb 15 10:28:21 2024 +0300 [SPARK-47045][SQL] Replace `IllegalArgumentException` by `SparkIllegalArgumentException` in `sql/api` ### What changes were proposed in this pull request? In the PR, I propose to replace all `IllegalArgumentException` by `SparkIllegalArgumentException` in `sql/api` code base, and introduce new legacy error classes with the `_LEGACY_ERROR_TEMP_` prefix. ### Why are the changes needed? To unify Spark SQL exception, and port Java exceptions on Spark exceptions with error classes. ### Does this PR introduce _any_ user-facing change? Yes, it can if user's code assumes some particular format of `IllegalArgumentException` messages. ### How was this patch tested? By running existing test suites like: ``` $ build/sbt "core/testOnly *SparkThrowableSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45098 from MaxGekk/migrate-IllegalArgumentException-sql. Authored-by: Max Gekk Signed-off-by: Max Gekk --- R/pkg/tests/fulltests/test_streaming.R | 3 +- .../src/main/resources/error/error-classes.json| 70 +++ .../src/main/scala/org/apache/spark/sql/Row.scala | 11 ++- .../catalyst/streaming/InternalOutputModes.scala | 7 +- .../catalyst/util/DateTimeFormatterHelper.scala| 18 +++-- .../sql/catalyst/util/SparkIntervalUtils.scala | 8 ++- .../sql/catalyst/util/TimestampFormatter.scala | 6 +- .../spark/sql/execution/streaming/Triggers.scala | 5 +- .../org/apache/spark/sql/types/DataType.scala | 19 ++--- .../org/apache/spark/sql/types/StructType.scala| 25 --- .../results/datetime-formatting-invalid.sql.out| 81 +- .../org/apache/spark/sql/JsonFunctionsSuite.scala | 13 ++-- 12 files changed, 206 insertions(+), 60 deletions(-) diff --git a/R/pkg/tests/fulltests/test_streaming.R b/R/pkg/tests/fulltests/test_streaming.R index 8804471e640c..67479726b57c 100644 --- a/R/pkg/tests/fulltests/test_streaming.R +++ b/R/pkg/tests/fulltests/test_streaming.R @@ -257,7 +257,8 @@ test_that("Trigger", { "Value for trigger.processingTime must be a non-empty string.") expect_error(write.stream(df, "memory", queryName = "times", outputMode = "append", - trigger.processingTime = "invalid"), "illegal argument") + trigger.processingTime = "invalid"), + "Error parsing 'invalid' to interval, unrecognized number 'invalid'") expect_error(write.stream(df, "memory", queryName = "times", outputMode = "append", trigger.once = ""), "Value for trigger.once must be TRUE.") diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 5884c9267119..38161ff87720 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -7767,6 +7767,76 @@ "Single backslash is prohibited. It has special meaning as beginning of an escape sequence. To get the backslash character, pass a string with two backslashes as the delimiter." ] }, + "_LEGACY_ERROR_TEMP_3249" : { +"message" : [ + "Failed to convert value (class of }) with the type of to JSON." +] + }, + "_LEGACY_ERROR_TEMP_3250" : { +"message" : [ + "Failed to convert the JSON string '' to a field." +] + }, + "_LEGACY_ERROR_TEMP_3251" : { +"message" : [ + "Failed to convert the JSON string '' to a data type." +] + }, + "_LEGACY_ERROR_TEMP_3252" : { +"message" : [ + " does not exist. Available: " +] + }, + "_LEGACY_ERROR_TEMP_3253" : { +"message" : [ + " do(es) not exist. Available: " +] + }, + "_LEGACY_ERROR_TEMP_3254" : { +"message" : [ + " does not exist. Available: " +] + }, + "_LEGACY_ERROR_TEMP_3255" : { +"message" : [ + "Error parsing '' to interval, " +] + }, + "_LEGACY_ERROR_TEMP_3256&q
(spark) branch master updated: [SPARK-46832][SQL] Introducing Collate and Collation expressions
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 861cca3da4c4 [SPARK-46832][SQL] Introducing Collate and Collation expressions 861cca3da4c4 is described below commit 861cca3da4c446761ccff007c89b214a691b0a72 Author: Aleksandar Tomic AuthorDate: Wed Feb 14 19:14:50 2024 +0300 [SPARK-46832][SQL] Introducing Collate and Collation expressions ### What changes were proposed in this pull request? This PR adds E2E support for `collate` and `collation` expressions. Following changes were made to get us there: 1) Set the right ordering for `PhysicalStringType` based on `collationId`. 2) UTF8String is now just a data holder class - it no longer implements `Comparable` interface. All comparisons must be done through `CollationFactory`. 3) `collate` and `collation` expressions are added. Special syntax for `collate` is enabled - `'hello world' COLLATE 'target_collation' 4) First set of tests is added that covers both core expression and E2E collation tests. ### Why are the changes needed? This PR is part of larger collation track. For more details please refer to design doc attached in parent JIRA ticket. ### Does this PR introduce _any_ user-facing change? This test adds two new expressions and opens up new syntax. ### How was this patch tested? Basic tests are added. In follow up PRs we will add support for more advanced operators and keep adding tests alongside new feature support. ### Was this patch authored or co-authored using generative AI tooling? Yes. Closes #45064 from dbatomic/stringtype_compare. Lead-authored-by: Aleksandar Tomic Co-authored-by: Stefan Kandic Signed-off-by: Max Gekk --- .../spark/sql/catalyst/util/CollationFactory.java | 5 +- .../org/apache/spark/unsafe/types/UTF8String.java | 59 ++- .../apache/spark/unsafe/types/UTF8StringSuite.java | 24 +-- .../types/UTF8StringPropertyCheckSuite.scala | 2 +- .../spark/sql/catalyst/parser/SqlBaseParser.g4 | 1 + .../spark/sql/catalyst/encoders/RowEncoder.scala | 2 +- .../org/apache/spark/sql/types/StringType.scala| 23 ++- .../sql/catalyst/CatalystTypeConverters.scala | 2 +- .../sql/catalyst/analysis/FunctionRegistry.scala | 2 + .../spark/sql/catalyst/encoders/EncoderUtils.scala | 2 +- .../sql/catalyst/expressions/ToStringBase.scala| 4 +- .../aggregate/BloomFilterAggregate.scala | 4 +- .../expressions/codegen/CodeGenerator.scala| 13 +- .../expressions/collationExpressions.scala | 100 .../spark/sql/catalyst/parser/AstBuilder.scala | 8 + .../sql/catalyst/types/PhysicalDataType.scala | 4 +- .../catalyst/expressions/CodeGenerationSuite.scala | 9 +- .../expressions/CollationExpressionSuite.scala | 77 + .../apache/spark/sql/execution/HiveResult.scala| 2 +- .../spark/sql/execution/columnar/ColumnStats.scala | 4 +- .../sql-functions/sql-expression-schema.md | 2 + .../org/apache/spark/sql/CollationSuite.scala | 177 + .../sql/expressions/ExpressionInfoSuite.scala | 5 +- 23 files changed, 484 insertions(+), 47 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java index 018fb6cbeb9f..83cac849e848 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java @@ -112,7 +112,7 @@ public final class CollationFactory { collationTable[0] = new Collation( "UCS_BASIC", null, - UTF8String::compareTo, + UTF8String::binaryCompare, "1.0", s -> (long)s.hashCode(), true); @@ -122,7 +122,7 @@ public final class CollationFactory { collationTable[1] = new Collation( "UCS_BASIC_LCASE", null, - Comparator.comparing(UTF8String::toLowerCase), + (s1, s2) -> s1.toLowerCase().binaryCompare(s2.toLowerCase()), "1.0", (s) -> (long)s.toLowerCase().hashCode(), false); @@ -132,7 +132,6 @@ public final class CollationFactory { "UNICODE", Collator.getInstance(ULocale.ROOT), "153.120.0.0", true); collationTable[2].collator.setStrength(Collator.TERTIARY); - // UNICODE case-insensitive comparison (ROOT locale, in ICU + Secondary strength). collationTable[3] = new Collation( "UNICODE_CI", Collator.getInstance(ULocale.ROOT), "153.120.0.0", false); diff --gi
(spark) branch master updated: [SPARK-47028][SQL][TESTS] Check `SparkUnsupportedOperationException` instead of `UnsupportedOperationException`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 49bcde612c59 [SPARK-47028][SQL][TESTS] Check `SparkUnsupportedOperationException` instead of `UnsupportedOperationException` 49bcde612c59 is described below commit 49bcde612c598fcf3c76cbd91a3dbf11d1b7f1b2 Author: Max Gekk AuthorDate: Tue Feb 13 13:50:18 2024 +0300 [SPARK-47028][SQL][TESTS] Check `SparkUnsupportedOperationException` instead of `UnsupportedOperationException` ### What changes were proposed in this pull request? In the PR, I propose to use `checkError()` in tests of `sql` to check `SparkUnsupportedOperationException`, and its fields. ### Why are the changes needed? By checking `SparkUnsupportedOperationException` and its fields like error class and message parameters prevents replacing `SparkUnsupportedOperationException` back to `UnsupportedOperationException`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the modified test suites. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45082 from MaxGekk/intercept-UnsupportedOperationException-tests. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../test/scala/org/apache/spark/sql/RowTest.scala | 6 +- .../spark/sql/catalyst/ScalaReflectionSuite.scala | 23 +--- .../encoders/EncoderErrorMessageSuite.scala| 12 ++-- .../catalyst/encoders/ExpressionEncoderSuite.scala | 10 ++-- .../sql/catalyst/json/JacksonGeneratorSuite.scala | 6 +- .../spark/sql/connector/catalog/CatalogSuite.scala | 4 +- .../sql/util/CaseInsensitiveStringMapSuite.scala | 6 +- .../scala/org/apache/spark/sql/DatasetSuite.scala | 33 ++- .../spark/sql/ScalaReflectionRelationSuite.scala | 29 + .../sql/connector/DataSourceV2FunctionSuite.scala | 4 +- .../binaryfile/BinaryFileFormatSuite.scala | 16 ++--- .../datasources/v2/V2SessionCatalogSuite.scala | 11 ++-- .../streaming/CompactibleFileStreamLogSuite.scala | 11 ++-- .../sources/RatePerMicroBatchProviderSuite.scala | 20 --- .../sources/RateStreamProviderSuite.scala | 19 +++--- .../streaming/sources/TextSocketStreamSuite.scala | 12 ++-- .../spark/sql/streaming/GroupStateSuite.scala | 68 +++--- 17 files changed, 160 insertions(+), 130 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala index 840d80ffed13..985443773943 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala @@ -24,7 +24,7 @@ import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.must.Matchers import org.scalatest.matchers.should.Matchers._ -import org.apache.spark.SparkException +import org.apache.spark.{SparkException, SparkUnsupportedOperationException} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericRow, GenericRowWithSchema} import org.apache.spark.sql.types._ @@ -45,10 +45,10 @@ class RowTest extends AnyFunSpec with Matchers { describe("Row (without schema)") { it("throws an exception when accessing by fieldName") { - intercept[UnsupportedOperationException] { + intercept[SparkUnsupportedOperationException] { noSchemaRow.fieldIndex("col1") } - intercept[UnsupportedOperationException] { + intercept[SparkUnsupportedOperationException] { noSchemaRow.getAs("col1") } } 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 bbb62acd0250..daa8d12613f2 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 @@ -22,7 +22,7 @@ import java.sql.{Date, Timestamp} import scala.reflect.ClassTag import scala.reflect.runtime.universe.{typeTag, TypeTag} -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkUnsupportedOperationException} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.FooEnum.FooEnum import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue @@ -490,17 +490,22 @@ class ScalaReflectionSuite extends SparkFunSuite { } test("SPARK-29026: schemaFor for trait without companion object throws exception ") { -val e = intercept[UnsupportedOperationException] { - schemaFor[TraitProductWithoutCompanion
(spark) branch master updated: [SPARK-46950][CORE][SQL] Align `not available codec` error-class
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 88d4681dd6ab [SPARK-46950][CORE][SQL] Align `not available codec` error-class 88d4681dd6ab is described below commit 88d4681dd6ab37d65d704b0119ba2a54801fffde Author: panbingkun AuthorDate: Sat Feb 3 10:21:58 2024 +0300 [SPARK-46950][CORE][SQL] Align `not available codec` error-class ### What changes were proposed in this pull request? The pr aims to align `not available codec` error-class, includes: - In `core`, convert `CODEC_NOT_AVAILABLE` to `CODEC_NOT_AVAILABLE.WITH_CONF_SUGGESTION`. - In `sql`, use `CODEC_NOT_AVAILABLE.WITH_AVAILABLE_CODECS_SUGGESTION` to cover `IllegalArgumentException`. ### Why are the changes needed? When I was solving a case where `the compression option` is `null`, as follows: https://github.com/apache/spark/assets/15246973/0e2abdad-0c1c-4ade-9f48-014bc4372fc6";> I found that we could first complete the extraction logic for `the codec error`. At the same time, I found that there was already an `CODEC_NOT_AVAILABLE` error in the `error-classes.json`, but it had some differences from the error prompt in `SQL` for compress. Therefore, I proposed using `CODEC_NOT_AVAILABLE` as `a parent class error` and adding `two subclass errors`: `WITH_AVAILABLE_CODECS_SUGGESTION` and `WITH_CONF_SUGGESTION`. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? - Update existed UT. - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44992 from panbingkun/codec_improve. Lead-authored-by: panbingkun Co-authored-by: Maxim Gekk Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 14 +++- .../org/apache/spark/errors/SparkCoreErrors.scala | 13 ++- .../org/apache/spark/io/CompressionCodec.scala | 9 ++--- .../apache/spark/io/CompressionCodecSuite.scala| 2 +- ...r-conditions-codec-not-available-error-class.md | 41 ++ docs/sql-error-conditions.md | 6 ++-- .../sql/catalyst/util/CompressionCodecs.scala | 7 ++-- .../spark/sql/errors/QueryExecutionErrors.scala| 8 + .../sql/execution/datasources/orc/OrcOptions.scala | 4 +-- .../datasources/parquet/ParquetOptions.scala | 5 +-- .../ParquetCompressionCodecPrecedenceSuite.scala | 14 +--- .../sql/execution/datasources/text/TextSuite.scala | 18 ++ 12 files changed, 112 insertions(+), 29 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 6d88f5ee511c..8399311cbfc4 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -441,8 +441,20 @@ }, "CODEC_NOT_AVAILABLE" : { "message" : [ - "The codec is not available. Consider to set the config to ." + "The codec is not available." ], +"subClass" : { + "WITH_AVAILABLE_CODECS_SUGGESTION" : { +"message" : [ + "Available codecs are ." +] + }, + "WITH_CONF_SUGGESTION" : { +"message" : [ + "Consider to set the config to ." +] + } +}, "sqlState" : "56038" }, "CODEC_SHORT_NAME_NOT_FOUND" : { diff --git a/core/src/main/scala/org/apache/spark/errors/SparkCoreErrors.scala b/core/src/main/scala/org/apache/spark/errors/SparkCoreErrors.scala index 641310f2a0c5..a131f8233b0d 100644 --- a/core/src/main/scala/org/apache/spark/errors/SparkCoreErrors.scala +++ b/core/src/main/scala/org/apache/spark/errors/SparkCoreErrors.scala @@ -24,7 +24,9 @@ import scala.jdk.CollectionConverters._ import org.apache.hadoop.fs.Path -import org.apache.spark.{SparkException, SparkRuntimeException, SparkUnsupportedOperationException, TaskNotSerializableException} +import org.apache.spark.{SparkException, SparkIllegalArgumentException, SparkRuntimeException, SparkUnsupportedOperationException, TaskNotSerializableException} +import org.apache.spark.internal.config.IO_COMPRESSION_CODEC +import org.apache.spark.io.CompressionCodec.FALLBACK_COMPRESSION_CODEC import org.apache.spark.memory.SparkOutOfMemoryError import org.apache.spark.scheduler.{BarrierJobRunWithDynamicAllocationException, BarrierJobSlotsNumberCheckFailed, BarrierJobUnsupportedRDDChainException} import org.apache.spark.shuffle.{FetchFailedException, ShuffleManager} @@ -490,6 +492,15 @@ private[spark] object SparkCoreErrors {
(spark) branch master updated: [SPARK-46890][SQL] Fix CSV parsing bug with existence default values and column pruning
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 16ac82092bb7 [SPARK-46890][SQL] Fix CSV parsing bug with existence default values and column pruning 16ac82092bb7 is described below commit 16ac82092bb775aafd010e2fb02b7ddc1eceea73 Author: Daniel Tenedorio AuthorDate: Sat Feb 3 08:50:44 2024 +0300 [SPARK-46890][SQL] Fix CSV parsing bug with existence default values and column pruning ### What changes were proposed in this pull request? This PR fixes a CSV parsing bug with existence default values and column pruning (https://issues.apache.org/jira/browse/SPARK-46890). The bug fix includes disabling column pruning specifically when checking the CSV header schema against the required schema expected by Catalyst. This makes the expected schema match what the CSV parser provides, since later we also happen instruct the CSV parser to disable column pruning and instead read each entire row in order to correctly assign the default value(s) during execution. ### Why are the changes needed? Before this change, queries from a subset of the columns in a CSV table whose `CREATE TABLE` statement contained default values would return an internal exception. For example: ``` CREATE TABLE IF NOT EXISTS products ( product_id INT, name STRING, price FLOAT default 0.0, quantity INT default 0 ) USING CSV OPTIONS ( header 'true', inferSchema 'false', enforceSchema 'false', path '/Users/maximgekk/tmp/products.csv' ); ``` The CSV file products.csv: ``` product_id,name,price,quantity 1,Apple,0.50,100 2,Banana,0.25,200 3,Orange,0.75,50 ``` The query fails: ``` spark-sql (default)> SELECT price FROM products; 24/01/28 11:43:09 ERROR Executor: Exception in task 0.0 in stage 8.0 (TID 6) java.lang.IllegalArgumentException: Number of column in CSV header is not equal to number of fields in the schema: Header length: 4, schema size: 1 CSV file: file:///Users/Daniel.Tenedorio/tmp/products.csv ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This PR adds test coverage. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44939 from dtenedor/debug-csv-default. Authored-by: Daniel Tenedorio Signed-off-by: Max Gekk --- .../apache/spark/sql/catalyst/csv/CSVOptions.scala | 15 - .../spark/sql/catalyst/csv/UnivocityParser.scala | 4 +-- .../execution/datasources/csv/CSVFileFormat.scala | 5 ++- .../v2/csv/CSVPartitionReaderFactory.scala | 6 +++- .../sql/execution/datasources/csv/CSVSuite.scala | 38 ++ 5 files changed, 62 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index c5a6bf5076de..f4ade722791c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -26,8 +26,10 @@ import com.univocity.parsers.csv.{CsvParserSettings, CsvWriterSettings, Unescape import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.{DataSourceOptions, FileSourceOptions} import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumnsUtils.EXISTS_DEFAULT_COLUMN_METADATA_KEY import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} +import org.apache.spark.sql.types.StructType class CSVOptions( @transient val parameters: CaseInsensitiveMap[String], @@ -278,13 +280,24 @@ class CSVOptions( .getOrElse(UNESCAPED_QUOTE_HANDLING, "STOP_AT_DELIMITER").toUpperCase(Locale.ROOT)) /** + * Returns true if column pruning is enabled and there are no existence column default values in + * the [[schema]]. + * * The column pruning feature can be enabled either via the CSV option `columnPruning` or * in non-multiline mode via initialization of CSV options by the SQL config: * `spark.sql.csv.parser.columnPruning.enabled`. * The feature is disabled in the `multiLine` mode because of the issue: * https://github.com/uniVocity/univocity-parsers/issues/529 + * + * We disable column pruning when there are any column defaults, instead preferring to reach in + * each row and then post-process it to substitute the default values after. */ - val isColumnPruning
(spark) branch master updated: [SPARK-46915][SQL] Simplify `UnaryMinus` `Abs` and align error class
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e35e29a0517d [SPARK-46915][SQL] Simplify `UnaryMinus` `Abs` and align error class e35e29a0517d is described below commit e35e29a0517db930e12fe801f0f0ab1a31c3b23e Author: panbingkun AuthorDate: Fri Feb 2 20:33:31 2024 +0300 [SPARK-46915][SQL] Simplify `UnaryMinus` `Abs` and align error class ### What changes were proposed in this pull request? The pr aims to: - simplify `UnaryMinus` & `Abs` - convert error-class `_LEGACY_ERROR_TEMP_2043` to `ARITHMETIC_OVERFLOW`, and remove it. ### Why are the changes needed? 1.When the data type in `UnaryMinus` and `Abs` is `ByteType` or `ShortType`, if `an overflow exception` occurs, the corresponding error class is: `_LEGACY_ERROR_TEMP_2043` But when the data type is `IntegerType` or `LongType`, if `an overflow exception` occurs, its corresponding error class is: ARITHMETIC_OVERFLOW, We should unify it. 2.In the `codegen` logic of `UnaryMinus` and `Abs`, there is a difference between the logic of generating code when the data type is `ByteType` or `ShortType` and when the data type is `IntegerType` or `LongType`. We can unify it and simplify the code. ### Does this PR introduce _any_ user-facing change? Yes, ### How was this patch tested? - Update existed UT. - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44942 from panbingkun/UnaryMinus_improve. Authored-by: panbingkun Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 5 --- .../sql/catalyst/expressions/arithmetic.scala | 45 -- .../spark/sql/errors/QueryExecutionErrors.scala| 8 .../org/apache/spark/sql/types/numerics.scala | 6 +-- .../expressions/ArithmeticExpressionSuite.scala| 27 - 5 files changed, 36 insertions(+), 55 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 136825ab374d..6d88f5ee511c 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -5747,11 +5747,6 @@ ". If necessary set to false to bypass this error." ] }, - "_LEGACY_ERROR_TEMP_2043" : { -"message" : [ - "- caused overflow." -] - }, "_LEGACY_ERROR_TEMP_2045" : { "message" : [ "Unsupported table change: " diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 9f1b42ad84d3..0f95ae821ab0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -60,23 +60,15 @@ case class UnaryMinus( override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = dataType match { case _: DecimalType => defineCodeGen(ctx, ev, c => s"$c.unary_$$minus()") -case ByteType | ShortType if failOnError => +case ByteType | ShortType | IntegerType | LongType if failOnError => + val typeUtils = TypeUtils.getClass.getCanonicalName.stripSuffix("$") + val refDataType = ctx.addReferenceObj("refDataType", dataType, dataType.getClass.getName) nullSafeCodeGen(ctx, ev, eval => { val javaBoxedType = CodeGenerator.boxedType(dataType) -val javaType = CodeGenerator.javaType(dataType) -val originValue = ctx.freshName("origin") s""" - |$javaType $originValue = ($javaType)($eval); - |if ($originValue == $javaBoxedType.MIN_VALUE) { - | throw QueryExecutionErrors.unaryMinusCauseOverflowError($originValue); - |} - |${ev.value} = ($javaType)(-($originValue)); - """.stripMargin - }) -case IntegerType | LongType if failOnError => - val mathUtils = MathUtils.getClass.getCanonicalName.stripSuffix("$") - nullSafeCodeGen(ctx, ev, eval => { -s"${ev.value} = $mathUtils.negateExact($eval);" + |${ev.value} = ($javaBoxedType)$typeUtils.getNumeric( + | $refDataType, $failOnError).negate($eval); + """.stripMargin }) case dt: NumericType => nullSafeCodeGen(ctx, ev, eval => { val originValue = ctx.freshName("origin") @@ -181,23 +173,16 @@ case class Abs(ch
(spark) branch master updated: [SPARK-46831][SQL] Collations - Extending StringType and PhysicalStringType with collationId field
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e211dbdee42c [SPARK-46831][SQL] Collations - Extending StringType and PhysicalStringType with collationId field e211dbdee42c is described below commit e211dbdee42c887c99635623a0312857a240ebaa Author: Aleksandar Tomic AuthorDate: Mon Jan 29 17:15:29 2024 +0300 [SPARK-46831][SQL] Collations - Extending StringType and PhysicalStringType with collationId field ### What changes were proposed in this pull request? This PR represents initial change for introducing collation concept into Spark engine. For higher level overview please take a look at the umbrella [JIRA](https://issues.apache.org/jira/browse/SPARK-46830). This PR extends both `StringType` and `PhysicalStringType` with collationId field. At this point this is just a noop field. In the following PRs this field will be used for fetching right UTF8String comparison rules from global collation table. Goal is to make sure that we keep backwards compatibility - this is ensured by keeping singleton `object StringType` that inherits `StringType(DEFAULT_COLLATION_ID)`. DEFAULT_COLLATION_ID represents UTF8 Binary collation rules (i.e. byte for byte comparison, that is already used in Spark). Hence, any code that relies on `StringType` will stay binary compatible with this version. It may be hard to see end state from just this initial PR. For reviewers who want to see how this will fit in the end state, please take a look at this draft [PR](https://github.com/apache/spark/pull/44537). ### Why are the changes needed? Please refer to umbrella JIRA ticket for collation effort. ### Does this PR introduce _any_ user-facing change? At this point No. ### How was this patch tested? This initial PR doesn't introduce any surface level changes. ### Was this patch authored or co-authored using generative AI tooling? No Closes #44901 from dbatomic/string_with_collation_type. Authored-by: Aleksandar Tomic Signed-off-by: Max Gekk --- project/MimaExcludes.scala | 2 ++ .../main/scala/org/apache/spark/sql/types/StringType.scala | 9 ++--- .../scala/org/apache/spark/sql/catalyst/InternalRow.scala| 2 +- .../catalyst/expressions/InterpretedUnsafeProjection.scala | 2 +- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 4 ++-- .../org/apache/spark/sql/catalyst/expressions/literals.scala | 2 +- .../spark/sql/catalyst/expressions/namedExpressions.scala| 2 +- .../apache/spark/sql/catalyst/types/PhysicalDataType.scala | 12 +++- .../org/apache/spark/sql/execution/columnar/ColumnType.scala | 3 ++- .../spark/sql/execution/columnar/ColumnarDataTypeUtils.scala | 2 +- 10 files changed, 24 insertions(+), 16 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index eb4c130cc6a9..43723742be97 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -107,6 +107,8 @@ object MimaExcludes { // SPARK-46410: Assign error classes/subclasses to JdbcUtils.classifyException ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.jdbc.JdbcDialect.classifyException"), +// [SPARK-464878][CORE][SQL] (false alert). Invalid rule for StringType extension. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.types.StringType.this"), (problem: Problem) => problem match { case MissingClassProblem(cls) => !cls.fullName.startsWith("org.sparkproject.jpmml") && diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala index 5985238a863e..bd2ff8475741 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -23,9 +23,10 @@ import org.apache.spark.annotation.Stable * The data type representing `String` values. Please use the singleton `DataTypes.StringType`. * * @since 1.3.0 + * @param collationId The id of collation for this StringType. */ @Stable -class StringType private() extends AtomicType { +class StringType private(val collationId: Int) extends AtomicType { /** * The default size of a value of the StringType is 20 bytes. */ @@ -38,5 +39,7 @@ class StringType private() extends AtomicType { * @since 1.3.0 */ @Stable -case object StringType extends StringType - +case object StringType extends StringType(0) { + val DEFAULT_COLLATION_ID = 0 + def apply(collationId: Int): StringType = new StringType(collationId) +} diff --git a/sql/cata
(spark) branch branch-3.4 updated: [SPARK-46862][SQL][FOLLOWUP] Fix column pruning without schema enforcing in V1 CSV datasource
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch branch-3.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.4 by this push: new 5254840547b4 [SPARK-46862][SQL][FOLLOWUP] Fix column pruning without schema enforcing in V1 CSV datasource 5254840547b4 is described below commit 5254840547b48390ad98dd61dc1fc505a82ce8fc Author: Max Gekk AuthorDate: Sat Jan 27 19:22:52 2024 +0300 [SPARK-46862][SQL][FOLLOWUP] Fix column pruning without schema enforcing in V1 CSV datasource ### What changes were proposed in this pull request? In the PR, I propose to invoke `CSVOptons.isColumnPruningEnabled` introduced by https://github.com/apache/spark/pull/44872 while matching of CSV header to a schema in the V1 CSV datasource. ### Why are the changes needed? To fix the failure when column pruning happens and a schema is not enforced: ```scala scala> spark.read. | option("multiLine", true). | option("header", true). | option("escape", "\""). | option("enforceSchema", false). | csv("/Users/maximgekk/tmp/es-939111-data.csv"). | count() 24/01/27 12:43:14 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 3) java.lang.IllegalArgumentException: Number of column in CSV header is not equal to number of fields in the schema: Header length: 4, schema size: 0 CSV file: file:///Users/maximgekk/tmp/es-939111-data.csv ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "test:testOnly *CSVv1Suite" $ build/sbt "test:testOnly *CSVv2Suite" $ build/sbt "test:testOnly *CSVLegacyTimeParserSuite" $ build/sbt "testOnly *.CsvFunctionsSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44910 from MaxGekk/check-header-column-pruning. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit bc51c9fea3645c6ae1d9e1e83b0f94f8b849be20) Signed-off-by: Max Gekk --- .../sql/execution/datasources/csv/CSVFileFormat.scala | 6 +++--- .../spark/sql/execution/datasources/csv/CSVSuite.scala| 15 +-- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 069ad9562a7d..0ff96f073f03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -100,12 +100,12 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) -val columnPruning = sparkSession.sessionState.conf.csvColumnPruning val parsedOptions = new CSVOptions( options, - columnPruning, + sparkSession.sessionState.conf.csvColumnPruning, sparkSession.sessionState.conf.sessionLocalTimeZone, sparkSession.sessionState.conf.columnNameOfCorruptRecord) +val isColumnPruningEnabled = parsedOptions.isColumnPruningEnabled // Check a field requirement for corrupt records here to throw an exception in a driver side ExprUtils.verifyColumnNameOfCorruptRecord(dataSchema, parsedOptions.columnNameOfCorruptRecord) @@ -125,7 +125,7 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { actualRequiredSchema, parsedOptions, actualFilters) - val schema = if (columnPruning) actualRequiredSchema else actualDataSchema + val schema = if (isColumnPruningEnabled) actualRequiredSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( schema, parsedOptions, source = s"CSV file: ${file.urlEncodedPath}", isStartOfFile) 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 b3fb9effebca..23dd218e1479 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 @@ -3166,12 +3166,15 @@ abstract class CSVSuite withTempPath { path => Files.write(path.toPath, data.getByte
(spark) branch branch-3.5 updated: [SPARK-46862][SQL][FOLLOWUP] Fix column pruning without schema enforcing in V1 CSV datasource
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new a2854ba5d852 [SPARK-46862][SQL][FOLLOWUP] Fix column pruning without schema enforcing in V1 CSV datasource a2854ba5d852 is described below commit a2854ba5d852e2001b96636a8964494c45fc27d3 Author: Max Gekk AuthorDate: Sat Jan 27 19:22:52 2024 +0300 [SPARK-46862][SQL][FOLLOWUP] Fix column pruning without schema enforcing in V1 CSV datasource ### What changes were proposed in this pull request? In the PR, I propose to invoke `CSVOptons.isColumnPruningEnabled` introduced by https://github.com/apache/spark/pull/44872 while matching of CSV header to a schema in the V1 CSV datasource. ### Why are the changes needed? To fix the failure when column pruning happens and a schema is not enforced: ```scala scala> spark.read. | option("multiLine", true). | option("header", true). | option("escape", "\""). | option("enforceSchema", false). | csv("/Users/maximgekk/tmp/es-939111-data.csv"). | count() 24/01/27 12:43:14 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 3) java.lang.IllegalArgumentException: Number of column in CSV header is not equal to number of fields in the schema: Header length: 4, schema size: 0 CSV file: file:///Users/maximgekk/tmp/es-939111-data.csv ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "test:testOnly *CSVv1Suite" $ build/sbt "test:testOnly *CSVv2Suite" $ build/sbt "test:testOnly *CSVLegacyTimeParserSuite" $ build/sbt "testOnly *.CsvFunctionsSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44910 from MaxGekk/check-header-column-pruning. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit bc51c9fea3645c6ae1d9e1e83b0f94f8b849be20) Signed-off-by: Max Gekk --- .../sql/execution/datasources/csv/CSVFileFormat.scala | 6 +++--- .../spark/sql/execution/datasources/csv/CSVSuite.scala| 15 +-- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 069ad9562a7d..0ff96f073f03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -100,12 +100,12 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) -val columnPruning = sparkSession.sessionState.conf.csvColumnPruning val parsedOptions = new CSVOptions( options, - columnPruning, + sparkSession.sessionState.conf.csvColumnPruning, sparkSession.sessionState.conf.sessionLocalTimeZone, sparkSession.sessionState.conf.columnNameOfCorruptRecord) +val isColumnPruningEnabled = parsedOptions.isColumnPruningEnabled // Check a field requirement for corrupt records here to throw an exception in a driver side ExprUtils.verifyColumnNameOfCorruptRecord(dataSchema, parsedOptions.columnNameOfCorruptRecord) @@ -125,7 +125,7 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { actualRequiredSchema, parsedOptions, actualFilters) - val schema = if (columnPruning) actualRequiredSchema else actualDataSchema + val schema = if (isColumnPruningEnabled) actualRequiredSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( schema, parsedOptions, source = s"CSV file: ${file.urlEncodedPath}", isStartOfFile) 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 6690bf101fa7..a91adb787838 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 @@ -3215,12 +3215,15 @@ abstract class CSVSuite withTempPath { path => Files.write(path.toPath, data.getByte
(spark) branch master updated: [SPARK-46862][SQL][FOLLOWUP] Fix column pruning without schema enforcing in V1 CSV datasource
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new bc51c9fea364 [SPARK-46862][SQL][FOLLOWUP] Fix column pruning without schema enforcing in V1 CSV datasource bc51c9fea364 is described below commit bc51c9fea3645c6ae1d9e1e83b0f94f8b849be20 Author: Max Gekk AuthorDate: Sat Jan 27 19:22:52 2024 +0300 [SPARK-46862][SQL][FOLLOWUP] Fix column pruning without schema enforcing in V1 CSV datasource ### What changes were proposed in this pull request? In the PR, I propose to invoke `CSVOptons.isColumnPruningEnabled` introduced by https://github.com/apache/spark/pull/44872 while matching of CSV header to a schema in the V1 CSV datasource. ### Why are the changes needed? To fix the failure when column pruning happens and a schema is not enforced: ```scala scala> spark.read. | option("multiLine", true). | option("header", true). | option("escape", "\""). | option("enforceSchema", false). | csv("/Users/maximgekk/tmp/es-939111-data.csv"). | count() 24/01/27 12:43:14 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 3) java.lang.IllegalArgumentException: Number of column in CSV header is not equal to number of fields in the schema: Header length: 4, schema size: 0 CSV file: file:///Users/maximgekk/tmp/es-939111-data.csv ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "test:testOnly *CSVv1Suite" $ build/sbt "test:testOnly *CSVv2Suite" $ build/sbt "test:testOnly *CSVLegacyTimeParserSuite" $ build/sbt "testOnly *.CsvFunctionsSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44910 from MaxGekk/check-header-column-pruning. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../sql/execution/datasources/csv/CSVFileFormat.scala | 6 +++--- .../spark/sql/execution/datasources/csv/CSVSuite.scala| 15 +-- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 32370562003f..9516a7729481 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -100,12 +100,12 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) -val columnPruning = sparkSession.sessionState.conf.csvColumnPruning val parsedOptions = new CSVOptions( options, - columnPruning, + sparkSession.sessionState.conf.csvColumnPruning, sparkSession.sessionState.conf.sessionLocalTimeZone, sparkSession.sessionState.conf.columnNameOfCorruptRecord) +val isColumnPruningEnabled = parsedOptions.isColumnPruningEnabled // Check a field requirement for corrupt records here to throw an exception in a driver side ExprUtils.verifyColumnNameOfCorruptRecord(dataSchema, parsedOptions.columnNameOfCorruptRecord) @@ -125,7 +125,7 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { actualRequiredSchema, parsedOptions, actualFilters) - val schema = if (columnPruning) actualRequiredSchema else actualDataSchema + val schema = if (isColumnPruningEnabled) actualRequiredSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( schema, parsedOptions, source = s"CSV file: ${file.urlEncodedPath}", isStartOfFile) 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 2ec9e1086b92..c7f25c633e0b 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 @@ -3237,12 +3237,15 @@ abstract class CSVSuite withTempPath { path => Files.write(path.toPath, data.getBytes(StandardCharsets.UTF_8)) - val df = spark.read -.option("multiline", "true") -
(spark) branch master updated: [SPARK-46875][SQL] When the `mode` is null, a `NullPointException` should `not` be thrown
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6d29c72f6c2b [SPARK-46875][SQL] When the `mode` is null, a `NullPointException` should `not` be thrown 6d29c72f6c2b is described below commit 6d29c72f6c2bdf534c2b079f9a3563bf8cfc8dab Author: panbingkun AuthorDate: Sat Jan 27 10:53:39 2024 +0300 [SPARK-46875][SQL] When the `mode` is null, a `NullPointException` should `not` be thrown ### What changes were proposed in this pull request? The pr aims to provide better prompts when option's `mode` is null. ### Why are the changes needed? In the original logic, if the mode is null, Spark will throw a `NullPointerException`, which is obviously unfriendly to the user. ``` val cars = spark.read .format("csv") .options(Map("header" -> "true", "mode" -> null)) .load(testFile(carsFile)) cars.show(false) ``` Before: ``` Cannot invoke "String.toUpperCase(java.util.Locale)" because "mode" is null java.lang.NullPointerException: Cannot invoke "String.toUpperCase(java.util.Locale)" because "mode" is null at org.apache.spark.sql.catalyst.util.ParseMode$.fromString(ParseMode.scala:50) at org.apache.spark.sql.catalyst.csv.CSVOptions.$anonfun$parseMode$1(CSVOptions.scala:105) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.csv.CSVOptions.(CSVOptions.scala:105) at org.apache.spark.sql.catalyst.csv.CSVOptions.(CSVOptions.scala:49) at org.apache.spark.sql.execution.datasources.csv.CSVFileFormat.inferSchema(CSVFileFormat.scala:60) ``` After: It will fall back to `PermissiveMode` mode and then display the data normally, as shown below: ``` 18:54:06.727 WARN org.apache.spark.sql.catalyst.util.ParseMode: mode is null and not a valid parse mode. Using PERMISSIVE. ++-+-+--+-+ |year|make |model|comment |blank| ++-+-+--+-+ |2012|Tesla|S|No comment|NULL | |1997|Ford |E350 |Go get one now they are going fast|NULL | |2015|Chevy|Volt |NULL |NULL | ++-+-+--+-+ ``` ### Does this PR introduce _any_ user-facing change? Yes, When `mode` is null, it fallback to `PermissiveMode ` instead of throwing a `NullPointerException`. ### How was this patch tested? - Add new UT. - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44900 from panbingkun/SPARK-46875. Authored-by: panbingkun Signed-off-by: Max Gekk --- .../apache/spark/sql/catalyst/util/ParseMode.scala| 19 --- .../sql/execution/datasources/csv/CSVSuite.scala | 10 ++ 2 files changed, 22 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala index 2beb875d1751..b35da8e2c80f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala @@ -47,12 +47,17 @@ object ParseMode extends Logging { /** * Returns the parse mode from the given string. */ - def fromString(mode: String): ParseMode = mode.toUpperCase(Locale.ROOT) match { -case PermissiveMode.name => PermissiveMode -case DropMalformedMode.name => DropMalformedMode -case FailFastMode.name => FailFastMode -case _ => - logWarning(s"$mode is not a valid parse mode. Using ${PermissiveMode.name}.") - PermissiveMode + def fromString(mode: String): ParseMode = Option(mode).map { +v => v.toUpperCase(Locale.ROOT) match { + case PermissiveMode.name => PermissiveMode + case DropMalformedMode.name => DropMalformedMode + case FailFastMode.name => FailFastMode + case _ => +logWarning(s"$v is not a valid parse mode. Using ${PermissiveMode.name}.") +PermissiveMode +} + }.getOrElse { +logWarning(s"mode is null and not a valid parse mode. Using ${PermissiveMode.name}.") +PermissiveMode } } 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 8e6282bd5a42..2ec9e1086b92 100644 --- a
(spark) branch branch-3.4 updated: [SPARK-46862][SQL] Disable CSV column pruning in the multi-line mode
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch branch-3.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.4 by this push: new 113ca5132f3c [SPARK-46862][SQL] Disable CSV column pruning in the multi-line mode 113ca5132f3c is described below commit 113ca5132f3c252105b2a344193fbdad612d93e7 Author: Max Gekk AuthorDate: Fri Jan 26 11:02:15 2024 +0300 [SPARK-46862][SQL] Disable CSV column pruning in the multi-line mode ### What changes were proposed in this pull request? In the PR, I propose to disable the column pruning feature in the CSV datasource for the `multiLine` mode. ### Why are the changes needed? To workaround the issue in the `uniVocity` parser used by the CSV datasource: https://github.com/uniVocity/univocity-parsers/issues/529 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "test:testOnly *CSVv1Suite" $ build/sbt "test:testOnly *CSVv2Suite" $ build/sbt "test:testOnly *CSVLegacyTimeParserSuite" $ build/sbt "testOnly *.CsvFunctionsSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44872 from MaxGekk/csv-disable-column-pruning. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit 829e742df8251c6f5e965cb08ad454ac3ee1a389) Signed-off-by: Max Gekk --- .../apache/spark/sql/catalyst/csv/CSVOptions.scala | 10 + .../spark/sql/catalyst/csv/UnivocityParser.scala | 2 +- .../v2/csv/CSVPartitionReaderFactory.scala | 2 +- .../sql/execution/datasources/csv/CSVSuite.scala | 25 +- 4 files changed, 36 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 1a9de5bc35ed..72382daf2760 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -274,6 +274,15 @@ class CSVOptions( val unescapedQuoteHandling: UnescapedQuoteHandling = UnescapedQuoteHandling.valueOf(parameters .getOrElse(UNESCAPED_QUOTE_HANDLING, "STOP_AT_DELIMITER").toUpperCase(Locale.ROOT)) + /** + * The column pruning feature can be enabled either via the CSV option `columnPruning` or + * in non-multiline mode via initialization of CSV options by the SQL config: + * `spark.sql.csv.parser.columnPruning.enabled`. + * The feature is disabled in the `multiLine` mode because of the issue: + * https://github.com/uniVocity/univocity-parsers/issues/529 + */ + val isColumnPruningEnabled: Boolean = getBool(COLUMN_PRUNING, !multiLine && columnPruning) + def asWriterSettings: CsvWriterSettings = { val writerSettings = new CsvWriterSettings() val format = writerSettings.getFormat @@ -373,4 +382,5 @@ object CSVOptions extends DataSourceOptions { val SEP = "sep" val DELIMITER = "delimiter" newOption(SEP, DELIMITER) + val COLUMN_PRUNING = newOption("columnPruning") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index b58649da61c0..59b2857f6b60 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -72,7 +72,7 @@ class UnivocityParser( // positions. Generally assigned by input configuration options, except when input column(s) have // default values, in which case we omit the explicit indexes in order to know how many tokens // were present in each line instead. - private def columnPruning: Boolean = options.columnPruning && + private def columnPruning: Boolean = options.isColumnPruningEnabled && !requiredSchema.exists(_.metadata.contains(EXISTS_DEFAULT_COLUMN_METADATA_KEY)) // When column pruning is enabled, the parser only parses the required columns based on diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index 37f6ae4aaa9f..cef5a71ca9c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -58,7 +58,7 @@ case class CSVP
(spark) branch branch-3.5 updated: [SPARK-46862][SQL] Disable CSV column pruning in the multi-line mode
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new cf4e8672dd1a [SPARK-46862][SQL] Disable CSV column pruning in the multi-line mode cf4e8672dd1a is described below commit cf4e8672dd1a4e6309bb530191be7d92a793cb28 Author: Max Gekk AuthorDate: Fri Jan 26 11:02:15 2024 +0300 [SPARK-46862][SQL] Disable CSV column pruning in the multi-line mode ### What changes were proposed in this pull request? In the PR, I propose to disable the column pruning feature in the CSV datasource for the `multiLine` mode. ### Why are the changes needed? To workaround the issue in the `uniVocity` parser used by the CSV datasource: https://github.com/uniVocity/univocity-parsers/issues/529 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "test:testOnly *CSVv1Suite" $ build/sbt "test:testOnly *CSVv2Suite" $ build/sbt "test:testOnly *CSVLegacyTimeParserSuite" $ build/sbt "testOnly *.CsvFunctionsSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44872 from MaxGekk/csv-disable-column-pruning. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit 829e742df8251c6f5e965cb08ad454ac3ee1a389) Signed-off-by: Max Gekk --- .../apache/spark/sql/catalyst/csv/CSVOptions.scala | 10 + .../spark/sql/catalyst/csv/UnivocityParser.scala | 2 +- .../v2/csv/CSVPartitionReaderFactory.scala | 2 +- .../sql/execution/datasources/csv/CSVSuite.scala | 25 +- 4 files changed, 36 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 845c815c5648..c5a6bf5076de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -277,6 +277,15 @@ class CSVOptions( val unescapedQuoteHandling: UnescapedQuoteHandling = UnescapedQuoteHandling.valueOf(parameters .getOrElse(UNESCAPED_QUOTE_HANDLING, "STOP_AT_DELIMITER").toUpperCase(Locale.ROOT)) + /** + * The column pruning feature can be enabled either via the CSV option `columnPruning` or + * in non-multiline mode via initialization of CSV options by the SQL config: + * `spark.sql.csv.parser.columnPruning.enabled`. + * The feature is disabled in the `multiLine` mode because of the issue: + * https://github.com/uniVocity/univocity-parsers/issues/529 + */ + val isColumnPruningEnabled: Boolean = getBool(COLUMN_PRUNING, !multiLine && columnPruning) + def asWriterSettings: CsvWriterSettings = { val writerSettings = new CsvWriterSettings() val format = writerSettings.getFormat @@ -376,4 +385,5 @@ object CSVOptions extends DataSourceOptions { val SEP = "sep" val DELIMITER = "delimiter" newOption(SEP, DELIMITER) + val COLUMN_PRUNING = newOption("columnPruning") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index b99ee630d4b2..804c5d358ad6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -72,7 +72,7 @@ class UnivocityParser( // positions. Generally assigned by input configuration options, except when input column(s) have // default values, in which case we omit the explicit indexes in order to know how many tokens // were present in each line instead. - private def columnPruning: Boolean = options.columnPruning && + private def columnPruning: Boolean = options.isColumnPruningEnabled && !requiredSchema.exists(_.metadata.contains(EXISTS_DEFAULT_COLUMN_METADATA_KEY)) // When column pruning is enabled, the parser only parses the required columns based on diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index 37f6ae4aaa9f..cef5a71ca9c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -58,7 +58,7 @@ case class CSVP
(spark) branch master updated: [SPARK-46862][SQL] Disable CSV column pruning in the multi-line mode
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 829e742df825 [SPARK-46862][SQL] Disable CSV column pruning in the multi-line mode 829e742df825 is described below commit 829e742df8251c6f5e965cb08ad454ac3ee1a389 Author: Max Gekk AuthorDate: Fri Jan 26 11:02:15 2024 +0300 [SPARK-46862][SQL] Disable CSV column pruning in the multi-line mode ### What changes were proposed in this pull request? In the PR, I propose to disable the column pruning feature in the CSV datasource for the `multiLine` mode. ### Why are the changes needed? To workaround the issue in the `uniVocity` parser used by the CSV datasource: https://github.com/uniVocity/univocity-parsers/issues/529 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "test:testOnly *CSVv1Suite" $ build/sbt "test:testOnly *CSVv2Suite" $ build/sbt "test:testOnly *CSVLegacyTimeParserSuite" $ build/sbt "testOnly *.CsvFunctionsSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44872 from MaxGekk/csv-disable-column-pruning. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../apache/spark/sql/catalyst/csv/CSVOptions.scala | 10 + .../spark/sql/catalyst/csv/UnivocityParser.scala | 2 +- .../v2/csv/CSVPartitionReaderFactory.scala | 2 +- .../sql/execution/datasources/csv/CSVSuite.scala | 25 +- 4 files changed, 36 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 845c815c5648..c5a6bf5076de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -277,6 +277,15 @@ class CSVOptions( val unescapedQuoteHandling: UnescapedQuoteHandling = UnescapedQuoteHandling.valueOf(parameters .getOrElse(UNESCAPED_QUOTE_HANDLING, "STOP_AT_DELIMITER").toUpperCase(Locale.ROOT)) + /** + * The column pruning feature can be enabled either via the CSV option `columnPruning` or + * in non-multiline mode via initialization of CSV options by the SQL config: + * `spark.sql.csv.parser.columnPruning.enabled`. + * The feature is disabled in the `multiLine` mode because of the issue: + * https://github.com/uniVocity/univocity-parsers/issues/529 + */ + val isColumnPruningEnabled: Boolean = getBool(COLUMN_PRUNING, !multiLine && columnPruning) + def asWriterSettings: CsvWriterSettings = { val writerSettings = new CsvWriterSettings() val format = writerSettings.getFormat @@ -376,4 +385,5 @@ object CSVOptions extends DataSourceOptions { val SEP = "sep" val DELIMITER = "delimiter" newOption(SEP, DELIMITER) + val COLUMN_PRUNING = newOption("columnPruning") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index eb7e120277bb..34a8b3d09047 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -71,7 +71,7 @@ class UnivocityParser( // positions. Generally assigned by input configuration options, except when input column(s) have // default values, in which case we omit the explicit indexes in order to know how many tokens // were present in each line instead. - private def columnPruning: Boolean = options.columnPruning && + private def columnPruning: Boolean = options.isColumnPruningEnabled && !requiredSchema.exists(_.metadata.contains(EXISTS_DEFAULT_COLUMN_METADATA_KEY)) // When column pruning is enabled, the parser only parses the required columns based on diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index 37f6ae4aaa9f..cef5a71ca9c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -58,7 +58,7 @@ case class CSVPartitionReaderFactory( actualReadDataSchema, options, filters) -
(spark) branch master updated: [SPARK-46850][SQL] Convert `_LEGACY_ERROR_TEMP_2102 ` to `UNSUPPORTED_DATATYPE`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 43adfa070a40 [SPARK-46850][SQL] Convert `_LEGACY_ERROR_TEMP_2102 ` to `UNSUPPORTED_DATATYPE` 43adfa070a40 is described below commit 43adfa070a40832d8316be8db164e3aca8a4f593 Author: panbingkun AuthorDate: Thu Jan 25 18:04:17 2024 +0300 [SPARK-46850][SQL] Convert `_LEGACY_ERROR_TEMP_2102 ` to `UNSUPPORTED_DATATYPE` ### What changes were proposed in this pull request? The pr aims to - convert `_LEGACY_ERROR_TEMP_2102` to `UNSUPPORTED_DATATYPE`. - remove some outdated comments. ### Why are the changes needed? The changes improve the error framework. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Add new UT - Pass GA ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44871 from panbingkun/LEGACY_ERROR_TEMP_2102. Lead-authored-by: panbingkun Co-authored-by: Maxim Gekk Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 5 - .../spark/sql/catalyst/csv/UnivocityParser.scala | 6 ++--- .../spark/sql/catalyst/json/JacksonParser.scala| 4 ++-- .../spark/sql/errors/QueryExecutionErrors.scala| 7 -- .../spark/sql/execution/columnar/ColumnType.scala | 4 ++-- .../org/apache/spark/sql/CsvFunctionsSuite.scala | 26 +- .../sql/execution/columnar/ColumnTypeSuite.scala | 14 +++- 7 files changed, 39 insertions(+), 27 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 1f3122a502c5..64d65fd4beed 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -5966,11 +5966,6 @@ "Not support non-primitive type now." ] }, - "_LEGACY_ERROR_TEMP_2102" : { -"message" : [ - "Unsupported type: ." -] - }, "_LEGACY_ERROR_TEMP_2103" : { "message" : [ "Dictionary encoding should not be used because of dictionary overflow." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index b99ee630d4b2..eb7e120277bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -30,13 +30,12 @@ import org.apache.spark.sql.catalyst.expressions.{Cast, EmptyRow, ExprUtils, Gen import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ -import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.errors.{ExecutionErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String - /** * Constructs a parser for a given schema that translates CSV data to an [[InternalRow]]. * @@ -273,8 +272,7 @@ class UnivocityParser( case udt: UserDefinedType[_] => makeConverter(name, udt.sqlType, nullable) -// We don't actually hit this exception though, we keep it for understandability -case _ => throw QueryExecutionErrors.unsupportedTypeError(dataType) +case _ => throw ExecutionErrors.unsupportedDataTypeError(dataType) } private def nullSafeDatum( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index eace96ac8729..36f37888b084 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ -import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.errors.{ExecutionErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ @@ -381,7 +381,7 @@ class JacksonParser( } // We d
(spark) branch master updated: [SPARK-46818][SQL] Improve error messages for Range with non-foldable inputs
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c3872cd4f1b7 [SPARK-46818][SQL] Improve error messages for Range with non-foldable inputs c3872cd4f1b7 is described below commit c3872cd4f1b7c27956c067da11a26ae5e2829ed3 Author: allisonwang-db AuthorDate: Thu Jan 25 10:30:03 2024 +0300 [SPARK-46818][SQL] Improve error messages for Range with non-foldable inputs ### What changes were proposed in this pull request? This PR improves the error messages for the `Range` function when the input arguments are not foldable. ### Why are the changes needed? To make error messages more user-friendly. For example, before this PR, Spark throws an internal error: ``` SELECT * FROM range(0, (select 1)); [INTERNAL_ERROR] Cannot evaluate expression: scalar-subquery#1306 [] SQLSTATE: XX000 ``` After this PR, the error message will be: ``` [NON_FOLDABLE_ARGUMENT] The function `range` requires the parameter `end` to be a foldable expression of the type "BIGINT", but the actual argument is a non-foldable. SQLSTATE: 42K08; line 1 pos 14 ``` ### Does this PR introduce _any_ user-facing change? Yes. Error message improvement. ### How was this patch tested? New SQL tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #44857 from allisonwang-db/spark-46818-fix-range. Authored-by: allisonwang-db Signed-off-by: Max Gekk --- .../plans/logical/basicLogicalOperators.scala | 28 + .../table-valued-functions.sql.out | 44 .../sql-tests/inputs/table-valued-functions.sql| 4 ++ .../results/table-valued-functions.sql.out | 48 ++ 4 files changed, 116 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index fbbae16130c0..7c2dfd31f4e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -965,7 +965,12 @@ object Range { if (SQLConf.get.ansiEnabled) AnsiTypeCoercion else TypeCoercion } - private def castAndEval[T](expression: Expression, dataType: DataType, paramIndex: Int): T = { + private def castAndEval[T]( + expression: Expression, dataType: DataType, paramIndex: Int, paramName: String): T = { +if (!expression.foldable) { + throw QueryCompilationErrors.nonFoldableArgumentError( +"range", paramName, dataType) +} typeCoercion.implicitCast(expression, dataType) .map(_.eval()) .filter(_ != null) @@ -975,11 +980,11 @@ object Range { }.asInstanceOf[T] } - def toLong(expression: Expression, paramIndex: Int): Long = -castAndEval[Long](expression, LongType, paramIndex) + def toLong(expression: Expression, paramIndex: Int, paramName: String): Long = +castAndEval[Long](expression, LongType, paramIndex, paramName) - def toInt(expression: Expression, paramIndex: Int): Int = -castAndEval[Int](expression, IntegerType, paramIndex) + def toInt(expression: Expression, paramIndex: Int, paramName: String): Int = +castAndEval[Int](expression, IntegerType, paramIndex, paramName) } @ExpressionDescription( @@ -1025,12 +1030,19 @@ case class Range( require(step != 0, s"step ($step) cannot be 0") def this(start: Expression, end: Expression, step: Expression, numSlices: Expression) = { -this(Range.toLong(start, 1), Range.toLong(end, 2), Range.toLong(step, 3), - Some(Range.toInt(numSlices, 4))) +this( + Range.toLong(start, 1, "start"), + Range.toLong(end, 2, "end"), + Range.toLong(step, 3, "step"), + Some(Range.toInt(numSlices, 4, "numSlices"))) } def this(start: Expression, end: Expression, step: Expression) = -this(Range.toLong(start, 1), Range.toLong(end, 2), Range.toLong(step, 3), None) +this( + Range.toLong(start, 1, "start"), + Range.toLong(end, 2, "end"), + Range.toLong(step, 3, "step"), + None) def this(start: Expression, end: Expression) = this(start, end, Literal.create(1L, LongType)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/table-valued-functions.sql.out index fe4c6ec26056..9ee8d741aed3 100644 --- a/sql/core/s
(spark) branch master updated: [SPARK-46795][SQL] Replace `UnsupportedOperationException` by `SparkUnsupportedOperationException` in `sql/core`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new fa60a7e216e6 [SPARK-46795][SQL] Replace `UnsupportedOperationException` by `SparkUnsupportedOperationException` in `sql/core` fa60a7e216e6 is described below commit fa60a7e216e63b1edb199b1610b26197815c656b Author: Max Gekk AuthorDate: Tue Jan 23 20:42:16 2024 +0300 [SPARK-46795][SQL] Replace `UnsupportedOperationException` by `SparkUnsupportedOperationException` in `sql/core` ### What changes were proposed in this pull request? In the PR, I propose to replace all `UnsupportedOperationException` by `SparkUnsupportedOperationException` in `sql/core` code base, and introduce new legacy error classes with the `_LEGACY_ERROR_TEMP_` prefix. ### Why are the changes needed? To unify Spark SQL exception, and port Java exceptions on Spark exceptions with error classes. ### Does this PR introduce _any_ user-facing change? Yes, it can if user's code assumes some particular format of `UnsupportedOperationException` messages. ### How was this patch tested? By running the modified test suites: ``` $ build/sbt "core/testOnly *SparkThrowableSuite" $ build/sbt "test:testOnly *FileBasedDataSourceSuite" $ build/sbt "test:testOnly *ColumnarRulesSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44772 from MaxGekk/migrate-UnsupportedOperationException-sql. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 150 + .../sql/execution/columnar/ColumnDictionary.java | 3 +- .../datasources/orc/OrcArrayColumnVector.java | 25 ++-- .../datasources/orc/OrcAtomicColumnVector.java | 9 +- .../datasources/orc/OrcMapColumnVector.java| 25 ++-- .../datasources/orc/OrcStructColumnVector.java | 25 ++-- .../parquet/ParquetVectorUpdaterFactory.java | 3 +- .../parquet/VectorizedColumnReader.java| 16 ++- .../parquet/VectorizedParquetRecordReader.java | 3 +- .../parquet/VectorizedPlainValuesReader.java | 3 +- .../datasources/parquet/VectorizedReaderBase.java | 51 +++ .../parquet/VectorizedRleValuesReader.java | 41 +++--- .../execution/vectorized/ColumnVectorUtils.java| 4 +- .../execution/vectorized/MutableColumnarRow.java | 10 +- .../spark/sql/artifact/ArtifactManager.scala | 5 +- .../spark/sql/execution/SparkStrategies.scala | 7 +- .../spark/sql/execution/UnsafeRowSerializer.scala | 15 ++- .../sql/execution/WholeStageCodegenExec.scala | 12 +- .../execution/adaptive/OptimizeSkewedJoin.scala| 3 +- .../execution/aggregate/SortAggregateExec.scala| 5 +- .../columnar/GenerateColumnAccessor.scala | 11 +- .../sql/execution/datasources/jdbc/JdbcUtils.scala | 12 +- .../datasources/parquet/ParquetUtils.scala | 17 ++- .../spark/sql/execution/joins/HashedRelation.scala | 28 ++-- .../FlatMapGroupsInPandasWithStateExec.scala | 6 +- .../streaming/AcceptsLatestSeenOffsetHandler.scala | 7 +- .../AvailableNowMicroBatchStreamWrapper.scala | 5 +- .../streaming/AvailableNowSourceWrapper.scala | 5 +- .../sql/execution/streaming/GroupStateImpl.scala | 4 +- .../sources/RatePerMicroBatchProvider.scala| 3 +- .../sources/RatePerMicroBatchStream.scala | 4 +- .../streaming/state/HDFSBackedStateStoreMap.scala | 3 +- .../sql/execution/window/WindowFunctionFrame.scala | 5 +- .../org/apache/spark/sql/jdbc/DB2Dialect.scala | 8 +- .../org/apache/spark/sql/jdbc/H2Dialect.scala | 15 ++- .../org/apache/spark/sql/jdbc/JdbcDialects.scala | 47 --- .../org/apache/spark/sql/jdbc/MySQLDialect.scala | 8 +- .../org/apache/spark/sql/jdbc/OracleDialect.scala | 8 +- .../apache/spark/sql/util/MapperRowCounter.scala | 8 +- .../scala/org/apache/spark/sql/DatasetSuite.scala | 6 +- .../spark/sql/FileBasedDataSourceSuite.scala | 10 +- .../sql/connector/DataSourceV2FunctionSuite.scala | 3 +- .../spark/sql/connector/DataSourceV2Suite.scala| 2 +- .../sql/connector/TableCapabilityCheckSuite.scala | 5 +- .../spark/sql/connector/V1ReadFallbackSuite.scala | 5 +- .../spark/sql/execution/ColumnarRulesSuite.scala | 5 +- .../apache/spark/sql/execution/PlannerSuite.scala | 3 +- .../spark/sql/execution/SparkPlanSuite.scala | 4 +- .../execution/columnar/ColumnarDataTypeUtils.scala | 5 +- 49 files changed, 451 insertions(+), 216 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 8f4e04ba545
(spark) branch master updated: [SPARK-46673][PYTHON][DOCS] Refine docstring `aes_encrypt/aes_decrypt/try_aes_decrypt`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9556da8834b0 [SPARK-46673][PYTHON][DOCS] Refine docstring `aes_encrypt/aes_decrypt/try_aes_decrypt` 9556da8834b0 is described below commit 9556da8834b0b6ef6d4237a46a62cadd839c88e7 Author: panbingkun AuthorDate: Mon Jan 22 11:18:40 2024 +0300 [SPARK-46673][PYTHON][DOCS] Refine docstring `aes_encrypt/aes_decrypt/try_aes_decrypt` ### What changes were proposed in this pull request? The pr aims to refine docstring of `aes_encrypt/aes_decrypt/try_aes_decrypt`. ### Why are the changes needed? To improve PySpark documentation. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Pass GA. - Manually test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44750 from panbingkun/SPARK-46673. Authored-by: panbingkun Signed-off-by: Max Gekk --- python/pyspark/sql/functions/builtin.py | 246 ++-- 1 file changed, 201 insertions(+), 45 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index ca2efde0b3c2..d3a94fe4b9e9 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -18836,6 +18836,8 @@ def nvl2(col1: "ColumnOrName", col2: "ColumnOrName", col3: "ColumnOrName") -> Co return _invoke_function_over_columns("nvl2", col1, col2, col3) +# TODO(SPARK-46738) Re-enable testing that includes the 'Cast' operation after +# fixing the display difference between Regular Spark and Spark Connect on `Cast`. @_try_remote_functions def aes_encrypt( input: "ColumnOrName", @@ -18877,50 +18879,96 @@ def aes_encrypt( Optional additional authenticated data. Only supported for GCM mode. This can be any free-form input and must be provided for both encryption and decryption. +Returns +--- +:class:`~pyspark.sql.Column` +A new column that contains an encrypted value. + Examples + +Example 1: Encrypt data with key, mode, padding, iv and aad. + +>>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([( ... "Spark", "abcdefghijklmnop12345678ABCDEFGH", "GCM", "DEFAULT", ... "", "This is an AAD mixed into the input",)], ... ["input", "key", "mode", "padding", "iv", "aad"] ... ) ->>> df.select(base64(aes_encrypt( -... df.input, df.key, df.mode, df.padding, to_binary(df.iv, lit("hex")), df.aad) -... ).alias('r')).collect() -[Row(r='QiYi+sTLm7KD9UcZ2nlRdYDe/PX4')] +>>> df.select(sf.base64(sf.aes_encrypt( +... df.input, df.key, df.mode, df.padding, sf.to_binary(df.iv, sf.lit("hex")), df.aad) +... )).show(truncate=False) ++---+ +|base64(aes_encrypt(input, key, mode, padding, to_binary(iv, hex), aad))| ++---+ +|QiYi+sTLm7KD9UcZ2nlRdYDe/PX4 | ++---+ ->>> df.select(base64(aes_encrypt( -... df.input, df.key, df.mode, df.padding, to_binary(df.iv, lit("hex"))) -... ).alias('r')).collect() -[Row(r='QiYi+sRNYDAOTjdSEcYBFsAWPL1f')] +Example 2: Encrypt data with key, mode, padding and iv. +>>> import pyspark.sql.functions as sf +>>> df = spark.createDataFrame([( +... "Spark", "abcdefghijklmnop12345678ABCDEFGH", "GCM", "DEFAULT", +... "", "This is an AAD mixed into the input",)], +... ["input", "key", "mode", "padding", "iv", "aad"] +... ) +>>> df.select(sf.base64(sf.aes_encrypt( +... df.input, df.key, df.mode, df.padding, sf.to_binary(df.iv, sf.lit("hex"))) +... )).show(truncate=False) +++ +|base64(aes_encrypt(input, key, mode, padding, to_binary(iv, hex), ))| +++ +|QiYi+sRNYDAOTjdSEcYBFsAW
(spark) branch master updated: [MINOR][DOCS] Adjust formatting of "Building Spark" docs
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7eec8be37a87 [MINOR][DOCS] Adjust formatting of "Building Spark" docs 7eec8be37a87 is described below commit 7eec8be37a871cf62b2cb0b22be0734bb1a31041 Author: Nicholas Chammas AuthorDate: Sun Jan 21 23:44:36 2024 +0300 [MINOR][DOCS] Adjust formatting of "Building Spark" docs ### What changes were proposed in this pull request? Update the "Building Spark" docs as follows: - Add mention of Scaleway to the Jenkins section so developers understand its relevance (since we retired the project's own Jenkins infra some time back). - Add syntax highlighting to some of the code blocks. - Adjust the headings of some sections that were set too deep. ### Why are the changes needed? This makes the guide a bit more up-to-date and easier to read. ### Does this PR introduce _any_ user-facing change? Yes, it changes the public documentation on building Spark. ### How was this patch tested? I manually built the docs and reviewed the results. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44804 from nchammas/building-spark-docs. Authored-by: Nicholas Chammas Signed-off-by: Max Gekk --- docs/building-spark.md | 38 +- 1 file changed, 25 insertions(+), 13 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 9fea22426978..3d12b521c024 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -34,7 +34,9 @@ Spark requires Scala 2.13; support for Scala 2.12 was removed in Spark 4.0.0. You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`: -export MAVEN_OPTS="-Xss64m -Xmx2g -XX:ReservedCodeCacheSize=1g" +```sh +export MAVEN_OPTS="-Xss64m -Xmx2g -XX:ReservedCodeCacheSize=1g" +``` (The `ReservedCodeCacheSize` setting is optional but recommended.) If you don't add these parameters to `MAVEN_OPTS`, you may see errors and warnings like the following: @@ -171,12 +173,16 @@ assembly JAR (for developers who build with SBT). For more information about ho When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: --Xmax-classfile-name -128 +```xml +-Xmax-classfile-name +128 +``` and in `project/SparkBuild.scala` add: -scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), +```scala +scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), +``` to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. @@ -214,7 +220,7 @@ If you are building Spark for use in a Python environment and you wish to pip in **Note:** Due to packaging requirements you can not directly pip install from the Python directory, rather you must first build the sdist package as described above. -Alternatively, you can also run make-distribution with the --pip option. +Alternatively, you can also run `make-distribution.sh` with the `--pip` option. ## PySpark Tests with Maven or SBT @@ -276,11 +282,15 @@ Enable the profile (e.g. 2.13): ## Running Jenkins tests with GitHub Enterprise +While the Spark project does not maintain its own Jenkins infrastructure, [community members like Scaleway][scaleway] do. + +[scaleway]: https://spark.apache.org/developer-tools.html#scaleway + To run tests with Jenkins: ./dev/run-tests-jenkins -If use an individual repository or a repository on GitHub Enterprise, export below environment variables before running above command. +If you use an individual repository or a repository on GitHub Enterprise, export the environment variables below before running the above command. ### Related environment variables @@ -302,7 +312,7 @@ If use an individual repository or a repository on GitHub Enterprise, export bel -### Building and testing on IPv6-only environment +# Building and testing on an IPv6-only environment Use Apache Spark GitBox URL because GitHub doesn't support IPv6 yet. @@ -310,13 +320,15 @@ Use Apache Spark GitBox URL because GitHub doesn't support IPv6 yet. To build and run tests on IPv6-only environment, the following configurations are required. -export SPARK_LOCAL_HOSTNAME="your-IPv6-address" # e.g. '[2600:1700:232e:3de0:...]' -export D
(spark) branch master updated: [SPARK-46009][SQL][DOCS][FOLLOWUP] Change the error code for invalid inverse distribution function
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 26d3eca0a8d3 [SPARK-46009][SQL][DOCS][FOLLOWUP] Change the error code for invalid inverse distribution function 26d3eca0a8d3 is described below commit 26d3eca0a8d3303d0bb9450feb6575ed145bbd7e Author: beliefer AuthorDate: Sun Jan 21 23:32:28 2024 +0300 [SPARK-46009][SQL][DOCS][FOLLOWUP] Change the error code for invalid inverse distribution function ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/43910 and propose to change the error code for invalid inverse distribution function. ### Why are the changes needed? Based on the discussion at https://github.com/apache/spark/pull/43910#discussion_r1460131124 ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? GA tests. ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes #44811 from beliefer/SPARK-46009_followup. Authored-by: beliefer Signed-off-by: Max Gekk --- common/utils/src/main/resources/error/README.md| 2 +- common/utils/src/main/resources/error/error-classes.json | 2 +- ...itions-invalid-inverse-distribution-function-error-class.md | 2 +- docs/sql-error-conditions.md | 2 +- .../src/test/resources/sql-tests/analyzer-results/mode.sql.out | 8 .../resources/sql-tests/analyzer-results/percentiles.sql.out | 10 +- sql/core/src/test/resources/sql-tests/results/mode.sql.out | 8 .../src/test/resources/sql-tests/results/percentiles.sql.out | 10 +- 8 files changed, 22 insertions(+), 22 deletions(-) diff --git a/common/utils/src/main/resources/error/README.md b/common/utils/src/main/resources/error/README.md index b062c7739072..17e5a88cd884 100644 --- a/common/utils/src/main/resources/error/README.md +++ b/common/utils/src/main/resources/error/README.md @@ -881,6 +881,7 @@ The following SQLSTATEs are collated from: |42K0H|42 |Syntax error or Access Rule violation |K0H |A cyclic invocation has been detected. |Spark |N |Spark | |42K0I|42 |Syntax error or Access Rule violation |K0I |SQL Config not found. |Spark |N |Spark | |42K0J|42 |Syntax error or Access Rule violation |K0J |Property not found. |Spark |N |Spark | +|42K0K|IM |Syntax error or Access Rule violation |K0K |Invalid inverse distribution function |Spark |N |Spark | |42KD0|42 |Syntax error or Access Rule violation |KD0 |Ambiguous name reference. |Databricks |N |Databricks | |42KD1|42 |Syntax error or Access Rule violation |KD1 |Operation not supported in READ ONLY session mode. |Databricks |N |Databricks | |42KD2|42 |Syntax error or Access Rule violation |KD2 |The source and target table names of a SYNC operaton must be the same.|Databricks |N |Databricks | @@ -1310,7 +1311,6 @@ The following SQLSTATEs are collated from: |HZ320|HZ |RDA-specific condition|320 |version not supported |RDA/SQL|Y |RDA/SQL | |HZ321|HZ |RDA-specific condition|321 |TCP/IP error|RDA/SQL|Y |RDA/SQL | |HZ322|HZ |RDA-specific condition|322 |TLS alert |RDA/SQL|Y |RDA/SQL | -|ID001|IM |Invalid inverse distribution function |001 |Invalid inverse distribution function |SQL/Foundation |N |SQL/F
(spark) branch master updated: [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 15c9ec7c [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression 15c9ec7c is described below commit 15c9ec7ca3b66ec413b7964a374cb9508a80 Author: hannahkamundson AuthorDate: Sun Jan 21 23:28:17 2024 +0300 [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression ### What changes were proposed in this pull request? - I created `SparkException.require` which is the same as the Scala `require` precondition except it uses `SparkIllegalArgumentException`. - I changed 2 `require`s to use this `SparkException.require` in `ComplexTypeMergingExpression` - The exception message was updated for these `require`s - A SQLState error was added for the `require`s in `ComplexTypeMergingExpression` ### Why are the changes needed? All user facing exceptions should be `SparkException`s. The `require`s need to be changed to match this convention. ### Does this PR introduce _any_ user-facing change? Yes. - `ComplexTypeMergingExpression throws a `SparkIllegalArgumentException` instead of `IllegalArgumentException`. - It also updates the SqlState for these - It also updates the messages associated with the exceptions thrown - It also updates the docs with the new error classes. ### How was this patch tested? Unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #44336 from hannahkamundson/SPARK-42332. Lead-authored-by: hannahkamundson Co-authored-by: Hannah <48397717+hannahkamund...@users.noreply.github.com> Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 18 +++ .../scala/org/apache/spark/SparkException.scala| 15 + ...lex-expression-unsupported-input-error-class.md | 36 ++ docs/sql-error-conditions.md | 8 + .../sql/catalyst/expressions/Expression.scala | 19 +++- .../sql/errors/QueryCompilationErrorsSuite.scala | 35 ++--- 6 files changed, 118 insertions(+), 13 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 8794a8632c52..072081f48448 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -506,6 +506,24 @@ ], "sqlState" : "22004" }, + "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT" : { +"message" : [ + "Cannot process input data types for the expression: ." +], +"subClass" : { + "MISMATCHED_TYPES" : { +"message" : [ + "All input types must be the same except nullable, containsNull, valueContainsNull flags, but found the input types ." +] + }, + "NO_INPUTS" : { +"message" : [ + "The collection of input data types must not be empty." +] + } +}, +"sqlState" : "42K09" + }, "CONCURRENT_QUERY" : { "message" : [ "Another instance of this query was just started by a concurrent session." diff --git a/common/utils/src/main/scala/org/apache/spark/SparkException.scala b/common/utils/src/main/scala/org/apache/spark/SparkException.scala index 67bdc23b5f08..ebb6e772249b 100644 --- a/common/utils/src/main/scala/org/apache/spark/SparkException.scala +++ b/common/utils/src/main/scala/org/apache/spark/SparkException.scala @@ -106,6 +106,21 @@ object SparkException { messageParameters = Map("message" -> msg), cause = cause) } + + /** + * This is like the Scala require precondition, except it uses SparkIllegalArgumentException. + * @param requirement The requirement you want to check + * @param errorClass The error class to type if the requirement isn't passed + * @param messageParameters Message parameters to append to the message + */ + def require( + requirement: Boolean, + errorClass: String, + messageParameters: Map[String, String]): Unit = { +if (!requirement) { + throw new SparkIllegalArgumentException(errorClass, messageParameters) +} + } } /** diff --git a/docs/sql-error-conditions-complex-expression-unsupported-input-error-class.md b/docs/sql-error-conditions-complex-expression-unsupported-input-error-class.md new file mode 100644 index ..e8b8630c19aa --- /dev/null +++ b/docs/sql-error-conditions-complex-expression-unsupported-input-err
(spark) branch master updated: [SPARK-46725][SQL] Add DAYNAME function
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f7f424d19c59 [SPARK-46725][SQL] Add DAYNAME function f7f424d19c59 is described below commit f7f424d19c59c6fef8d08a68a49861458a06abcf Author: Petar Vasiljevic AuthorDate: Sun Jan 21 22:27:19 2024 +0300 [SPARK-46725][SQL] Add DAYNAME function ### What changes were proposed in this pull request? Added DAYNAME function that returns three letter abbreviation day name for the specified date to: - Scala API - Python API - R API - Spark Connect Scala Client - Spark Connect Python Client ### Why are the changes needed? For parity with Snowflake ### Does this PR introduce _any_ user-facing change? Yes, since new function DAYNAME is added ### How was this patch tested? Tested on new unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #44758 from PetarVasiljevic-DB/function_daytime. Lead-authored-by: Petar Vasiljevic Co-authored-by: PetarVasiljevic-DB <156098003+petarvasiljevic...@users.noreply.github.com> Signed-off-by: Max Gekk --- R/pkg/R/functions.R| 14 + R/pkg/R/generics.R | 4 +++ R/pkg/tests/fulltests/test_sparkSQL.R | 1 + .../scala/org/apache/spark/sql/functions.scala | 9 ++ .../apache/spark/sql/PlanGenerationTestSuite.scala | 4 +++ .../explain-results/function_dayname.explain | 2 ++ .../query-tests/queries/function_dayname.json | 25 .../query-tests/queries/function_dayname.proto.bin | Bin 0 -> 121 bytes .../source/reference/pyspark.sql/functions.rst | 1 + python/pyspark/sql/connect/functions/builtin.py| 7 + python/pyspark/sql/functions/builtin.py| 33 + python/pyspark/sql/tests/test_functions.py | 6 .../sql/catalyst/analysis/FunctionRegistry.scala | 1 + .../catalyst/expressions/datetimeExpressions.scala | 19 .../spark/sql/catalyst/util/DateTimeUtils.scala| 11 +++ .../expressions/DateExpressionsSuite.scala | 11 +++ .../scala/org/apache/spark/sql/functions.scala | 9 ++ .../sql-functions/sql-expression-schema.md | 1 + .../org/apache/spark/sql/DateFunctionsSuite.scala | 12 19 files changed, 170 insertions(+) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 0db25ad8739f..5106a83bd0ec 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -1105,6 +1105,20 @@ setMethod("monthname", column(jc) }) +#' @details +#' \code{dayname}: Extracts the three-letter abbreviated day name from a +#' given date/timestamp/string. +#' +#' @rdname column_datetime_functions +#' @aliases dayname dayname,Column-method +#' @note dayname since 4.0.0 +setMethod("dayname", + signature(x = "Column"), + function(x) { +jc <- callJStatic("org.apache.spark.sql.functions", "dayname", x@jc) +column(jc) + }) + #' @details #' \code{decode}: Computes the first argument into a string from a binary using the provided #' character set. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 92febc02710d..26e81733055a 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1024,6 +1024,10 @@ setGeneric("dayofyear", function(x) { standardGeneric("dayofyear") }) #' @name NULL setGeneric("monthname", function(x) { standardGeneric("monthname") }) +#' @rdname column_datetime_functions +#' @name NULL +setGeneric("dayname", function(x) { standardGeneric("dayname") }) + #' @rdname column_string_functions #' @name NULL setGeneric("decode", function(x, charset) { standardGeneric("decode") }) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 118c853a00df..630781a57e44 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -2063,6 +2063,7 @@ test_that("date functions on a DataFrame", { expect_equal(collect(select(df, year(df$b)))[, 1], c(2012, 2013, 2014)) expect_equal(collect(select(df, month(df$b)))[, 1], c(12, 12, 12)) expect_equal(collect(select(df, monthname(df$b)))[, 1], c("Dec", "Dec", "Dec")) + expect_equal(collect(select(df, dayname(df$b)))[, 1], c("Thu", "Sat", "Mon")) expect_equal(collect(select(df, last_day(df$b)))[, 1], c(as.Da
(spark) branch master updated: [SPARK-46629] Fix for STRUCT type DDL not picking up nullability and comment
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new b01b88ad68a2 [SPARK-46629] Fix for STRUCT type DDL not picking up nullability and comment b01b88ad68a2 is described below commit b01b88ad68a264396001d99c63bf58fb56d2be68 Author: Vitalii Li AuthorDate: Thu Jan 18 17:28:45 2024 +0300 [SPARK-46629] Fix for STRUCT type DDL not picking up nullability and comment ### What changes were proposed in this pull request? This change adds logic to generate correct DDL for nested fields in STRUCT. In particular instead of generating list of fields with data type names it will add `NOT NULL` qualifier when necessary and field comment when present. For a table: ``` CREATE TABLE t(field STRUCT); SHOW CREATE TABLE t; ``` Before: ``` CREATE TABLE t(field STRUCT) ``` After ``` CREATE TABLE t(field STRUCT) ``` Closes #41016 ### Why are the changes needed? Generate correct DDL. ### Does this PR introduce _any_ user-facing change? No, we do not document behavior of this command for struct case. ### How was this patch tested? New unit test. ### Was this patch authored or co-authored using generative AI tooling? No Closes #44644 from vitaliili-db/SPARK-46629. Authored-by: Vitalii Li Signed-off-by: Max Gekk --- .../spark/sql/protobuf/ProtobufSerdeSuite.scala | 4 +++- .../org/apache/spark/sql/types/StructField.scala| 9 ++--- .../expressions/ConditionalExpressionSuite.scala| 5 +++-- .../apache/spark/sql/types/StructTypeSuite.scala| 16 .../analyzer-results/ansi/literals.sql.out | 2 +- .../sql-tests/analyzer-results/literals.sql.out | 2 +- .../typeCoercion/native/mapconcat.sql.out | 4 ++-- .../analyzer-results/xml-functions.sql.out | 2 +- .../sql-tests/results/ansi/literals.sql.out | 2 +- .../resources/sql-tests/results/literals.sql.out| 2 +- .../results/typeCoercion/native/mapconcat.sql.out | 4 ++-- .../sql-tests/results/xml-functions.sql.out | 2 +- .../apache/spark/sql/DataFrameFunctionsSuite.scala | 2 +- .../spark/sql/DataFrameSetOperationsSuite.scala | 3 ++- .../apache/spark/sql/FileBasedDataSourceSuite.scala | 4 ++-- .../apache/spark/sql/GeneratorFunctionSuite.scala | 4 ++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 4 ++-- .../sql/errors/QueryCompilationErrorsSuite.scala| 6 +++--- .../execution/command/v2/ShowCreateTableSuite.scala | 21 + .../org/apache/spark/sql/sources/InsertSuite.scala | 2 +- 20 files changed, 72 insertions(+), 28 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 56a980d05fbd..03285c73f1ff 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 @@ -127,9 +127,11 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { "protobufType" -> "FieldMissingInProto", "toType" -> toSQLType(CATALYST_STRUCT))) -assertFailedConversionMessage(protoFile, +assertFailedConversionMessage( + protoFile, Serializer, BY_NAME, + nonnullCatalyst, errorClass = "UNABLE_TO_CONVERT_TO_PROTOBUF_MESSAGE_TYPE", params = Map( "protobufType" -> "FieldMissingInProto", diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala index ca15d23b601e..66f9557db213 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -148,11 +148,14 @@ case class StructField( .map(" COMMENT '" + _ + "'") .getOrElse("") + private lazy val nullDDL = if (nullable) "" else " NOT NULL" + /** * Returns a string containing a schema in SQL format. For example the following value: * `StructField("eventId", IntegerType)` will be converted to `eventId`: INT. */ - private[sql] def sql = s"${QuotingUtils.quoteIfNeeded(name)}: ${dataType.sql}$getDDLComment" + private[sql] def sql = +s"${QuotingUtils.quoteIfNeeded(name)}: ${dataType.sql}$nullDDL$getDDLComment" /** * Returns a string containing a schema in DDL format. For e
(spark) branch master updated: [SPARK-46618][SQL] Improve error messages for DATA_SOURCE_NOT_FOUND error
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 72e3f06c7d94 [SPARK-46618][SQL] Improve error messages for DATA_SOURCE_NOT_FOUND error 72e3f06c7d94 is described below commit 72e3f06c7d94a69f9ff218ccb560ffa5c0f14884 Author: allisonwang-db AuthorDate: Thu Jan 18 10:47:50 2024 +0300 [SPARK-46618][SQL] Improve error messages for DATA_SOURCE_NOT_FOUND error ### What changes were proposed in this pull request? This PR improves the error messages for the `DATA_SOURCE_NOT_FOUND` error. ### Why are the changes needed? To make the error messages more user-friendly and update to date. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing unit tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #44620 from allisonwang-db/spark-46618-not-found-err. Authored-by: allisonwang-db Signed-off-by: Max Gekk --- common/utils/src/main/resources/error/error-classes.json | 2 +- docs/sql-error-conditions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 2fa86de3daa3..601110309ddb 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -895,7 +895,7 @@ }, "DATA_SOURCE_NOT_FOUND" : { "message" : [ - "Failed to find the data source: . Please find packages at `https://spark.apache.org/third-party-projects.html`."; + "Failed to find the data source: . Make sure the provider name is correct and the package is properly registered and compatible with your Spark version." ], "sqlState" : "42K02" }, diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 35b50d6c6e4f..008346a018e3 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -496,7 +496,7 @@ Data source '``' not found. Please make sure the data source is regist [SQLSTATE: 42K02](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) -Failed to find the data source: ``. Please find packages at `https://spark.apache.org/third-party-projects.html`. +Failed to find the data source: ``. Make sure the provider name is correct and the package is properly registered and compatible with your Spark version. ### DATA_SOURCE_TABLE_SCHEMA_MISMATCH - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46739][SQL] Add the error class `UNSUPPORTED_CALL`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new bccbdb72b004 [SPARK-46739][SQL] Add the error class `UNSUPPORTED_CALL` bccbdb72b004 is described below commit bccbdb72b004f90ed8605d5bcaaf8b4605d8d099 Author: Max Gekk AuthorDate: Wed Jan 17 15:41:01 2024 +0300 [SPARK-46739][SQL] Add the error class `UNSUPPORTED_CALL` ### What changes were proposed in this pull request? In the PR, I propose to add new error class for unsupported method calls, and remove similar legacy error classes. New `apply()` method of `SparkUnsupportedOperationException` extracts method and class name from stack traces automatically, and places them to error class parameters. ### Why are the changes needed? To improve code maintenance by avoid boilerplate code (extract class and method names automatically), and to clean up `error-classes.json`. ### Does this PR introduce _any_ user-facing change? Yes, it can if user's code depends on the error class or message format of `SparkUnsupportedOperationException`. ### How was this patch tested? By running new test: ``` $ build/sbt "test:testOnly *QueryCompilationErrorsSuite" ``` and the affected test suites: ``` $ build/sbt "core/testOnly *SparkThrowableSuite" $ build/sbt "test:testOnly *ShuffleSpecSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44757 from MaxGekk/unsupported_call-error-class. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 181 + .../scala/org/apache/spark/SparkException.scala| 13 ++ docs/sql-error-conditions.md | 6 + .../sql/catalyst/expressions/UnsafeArrayData.java | 4 +- .../spark/sql/catalyst/expressions/UnsafeRow.java | 2 +- .../spark/sql/connector/write/WriteBuilder.java| 2 +- .../spark/sql/vectorized/ArrowColumnVector.java| 26 +-- .../apache/spark/sql/vectorized/ColumnarArray.java | 4 +- .../spark/sql/vectorized/ColumnarBatchRow.java | 6 +- .../apache/spark/sql/vectorized/ColumnarRow.java | 6 +- .../spark/sql/catalyst/ProjectingInternalRow.scala | 8 +- .../spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/analysis/FunctionRegistry.scala | 14 +- .../catalog/FunctionExpressionBuilder.scala| 2 +- .../sql/catalyst/catalog/functionResources.scala | 2 +- .../spark/sql/catalyst/expressions/grouping.scala | 6 +- .../catalyst/expressions/namedExpressions.scala| 18 +- .../spark/sql/catalyst/expressions/ordering.scala | 2 +- .../spark/sql/catalyst/plans/joinTypes.scala | 2 +- .../sql/catalyst/plans/physical/partitioning.scala | 4 +- .../catalyst/util/ResolveDefaultColumnsUtil.scala | 2 +- .../spark/sql/catalyst/ShuffleSpecSuite.scala | 13 +- .../sql/errors/QueryCompilationErrorsSuite.scala | 14 +- 23 files changed, 99 insertions(+), 240 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 19817ced3356..2fa86de3daa3 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -3592,6 +3592,12 @@ ], "sqlState" : "0A000" }, + "UNSUPPORTED_CALL" : { +"message" : [ + "Cannot call the method \"\" of the class \"\"." +], +"sqlState" : "0A000" + }, "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING" : { "message" : [ "The char/varchar type can't be used in the table schema.", @@ -7133,16 +7139,6 @@ "Cannot bind a V1 function." ] }, - "_LEGACY_ERROR_TEMP_3111" : { -"message" : [ - "" -] - }, - "_LEGACY_ERROR_TEMP_3112" : { -"message" : [ - "Operation unsupported for " -] - }, "_LEGACY_ERROR_TEMP_3113" : { "message" : [ "UnresolvedTableSpec doesn't have a data type" @@ -7153,76 +7149,11 @@ "UnresolvedTableSpec doesn't have a data type" ] }, - "_LEGACY_ERROR_TEMP_3115" : { -"message" : [ - "" -] - }, - "_LEGACY_ERROR_TEMP_3116" : { -"message" : [ - "" -] - }, - "_LEGACY_ERROR_TEMP_3117" : { -"message" : [ - "Cannot modify " -] - }, - "_LEGACY_ERROR_TEMP_3118&
(spark) branch master updated: [SPARK-46727][SQL] Port `classifyException()` in JDBC dialects on error classes
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ebd2b78f87fa [SPARK-46727][SQL] Port `classifyException()` in JDBC dialects on error classes ebd2b78f87fa is described below commit ebd2b78f87fa6086c41d5e6bcade5efeefac75d0 Author: Max Gekk AuthorDate: Tue Jan 16 17:47:09 2024 +0300 [SPARK-46727][SQL] Port `classifyException()` in JDBC dialects on error classes ### What changes were proposed in this pull request? In the PR, I propose to port the existing `classifyException()` method which accepts a description to new one w/ an error class added by https://github.com/apache/spark/pull/44358. The modified JDBC dialects are: DB2, H2, Oracle, MS SQL Server, MySQL and PostgreSQL. ### Why are the changes needed? The old method `classifyException()` which accepts a `description` only has been deprecated already by ... ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By existing integration tests, and the modified test suite: ``` $ build/sbt "test:testOnly *JDBCV2Suite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44739 from MaxGekk/port-jdbc-classifyException. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala | 4 +- .../org/apache/spark/sql/jdbc/DB2Dialect.scala | 17 +-- .../org/apache/spark/sql/jdbc/H2Dialect.scala | 34 ++--- .../apache/spark/sql/jdbc/MsSqlServerDialect.scala | 17 +-- .../org/apache/spark/sql/jdbc/MySQLDialect.scala | 29 ++- .../apache/spark/sql/jdbc/PostgresDialect.scala| 56 -- .../org/apache/spark/sql/jdbc/JDBCV2Suite.scala| 6 +-- 7 files changed, 91 insertions(+), 72 deletions(-) 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 d1d247967b4b..bae274788212 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 @@ -279,7 +279,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu sql(s"CREATE index i1 ON $catalogName.new_table (col1)") }, errorClass = "INDEX_ALREADY_EXISTS", - parameters = Map("indexName" -> "i1", "tableName" -> "new_table") + parameters = Map("indexName" -> "`i1`", "tableName" -> "`new_table`") ) sql(s"DROP index i1 ON $catalogName.new_table") @@ -304,7 +304,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu sql(s"DROP index i1 ON $catalogName.new_table") }, errorClass = "INDEX_NOT_FOUND", - parameters = Map("indexName" -> "i1", "tableName" -> "new_table") + parameters = Map("indexName" -> "`i1`", "tableName" -> "`new_table`") ) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala index d5a132c7dd48..f745e466ed9e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala @@ -144,16 +144,23 @@ private object DB2Dialect extends JdbcDialect { s"COMMENT ON SCHEMA ${quoteIdentifier(schema)} IS ''" } - override def classifyException(message: String, e: Throwable): AnalysisException = { + override def classifyException( + e: Throwable, + errorClass: String, + messageParameters: Map[String, String], + description: String): AnalysisException = { e match { case sqlException: SQLException => sqlException.getSQLState match { // https://www.ibm.com/docs/en/db2/11.5?topic=messages-sqlstate - case "42893" => throw NonEmptyNamespaceException( -namespace = Array.empty, details = message, cause = Some(e)) - case _ => super.classifyException(message, e) + case "42893" => +throw NonEmptyNamespaceException( + namespace = messageParameters.get("namespace").toArray, + details = sqlException.getMessage, + cause = Som
(spark) branch master updated: [SPARK-46655][SQL] Skip query context catching in `DataFrame` methods
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d30796021174 [SPARK-46655][SQL] Skip query context catching in `DataFrame` methods d30796021174 is described below commit d30796021174d8dc5595054d00c83ccdf0eb5b38 Author: Max Gekk AuthorDate: Thu Jan 11 17:08:29 2024 +0300 [SPARK-46655][SQL] Skip query context catching in `DataFrame` methods ### What changes were proposed in this pull request? In the PR, I propose to do not catch DataFrame query context in DataFrame methods but leave that close to `Column` functions. ### Why are the changes needed? To improve user experience with Spark DataFrame/Dataset APIs, and provide more precise context of errors. ### Does this PR introduce _any_ user-facing change? No, since the feature hasn't been released yet. ### How was this patch tested? By running the modified test suites. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44501 from MaxGekk/exclude-funcs-withOrigin. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../main/scala/org/apache/spark/sql/Dataset.scala | 359 + .../main/scala/org/apache/spark/sql/package.scala | 7 +- .../spark/sql/DataFrameSetOperationsSuite.scala| 6 +- .../apache/spark/sql/streaming/StreamSuite.scala | 4 +- 4 files changed, 160 insertions(+), 216 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index ff1bd8c73e6f..d792cdbcf865 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -512,11 +512,9 @@ class Dataset[T] private[sql]( * @group basic * @since 3.4.0 */ - def to(schema: StructType): DataFrame = withOrigin { -withPlan { - val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] - Project.matchSchema(logicalPlan, replaced, sparkSession.sessionState.conf) -} + def to(schema: StructType): DataFrame = withPlan { +val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] +Project.matchSchema(logicalPlan, replaced, sparkSession.sessionState.conf) } /** @@ -776,13 +774,12 @@ class Dataset[T] private[sql]( */ // We only accept an existing column name, not a derived column here as a watermark that is // defined on a derived column cannot referenced elsewhere in the plan. - def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = withOrigin { -withTypedPlan { - val parsedDelay = IntervalUtils.fromIntervalString(delayThreshold) - require(!IntervalUtils.isNegative(parsedDelay), -s"delay threshold ($delayThreshold) should not be negative.") - EventTimeWatermark(UnresolvedAttribute(eventTime), parsedDelay, logicalPlan) -} + def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = withTypedPlan { +val parsedDelay = IntervalUtils.fromIntervalString(delayThreshold) +require(!IntervalUtils.isNegative(parsedDelay), + s"delay threshold ($delayThreshold) should not be negative.") +EliminateEventTimeWatermark( + EventTimeWatermark(UnresolvedAttribute(eventTime), parsedDelay, logicalPlan)) } /** @@ -954,10 +951,8 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: Dataset[_]): DataFrame = withOrigin { -withPlan { - Join(logicalPlan, right.logicalPlan, joinType = Inner, None, JoinHint.NONE) -} + def join(right: Dataset[_]): DataFrame = withPlan { +Join(logicalPlan, right.logicalPlan, joinType = Inner, None, JoinHint.NONE) } /** @@ -1090,23 +1085,22 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: Dataset[_], usingColumns: Seq[String], joinType: String): DataFrame = -withOrigin { - // Analyze the self join. The assumption is that the analyzer will disambiguate left vs right - // by creating a new instance for one of the branch. - val joined = sparkSession.sessionState.executePlan( -Join(logicalPlan, right.logicalPlan, joinType = JoinType(joinType), None, JoinHint.NONE)) -.analyzed.asInstanceOf[Join] - - withPlan { -Join( - joined.left, - joined.right, - UsingJoin(JoinType(joinType), usingColumns.toIndexedSeq), - None, - JoinHint.NONE) - } + def join(right: Dataset[_], usingColumns: Seq[String], joinType: String): DataFrame = { +// Analyze the self join. The assumption is that the analyzer will disambiguate left vs right
(spark) branch master updated: [SPARK-46608][SQL] Restore backward compatibility of `JdbcDialect.classifyException`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2957bc9fa1f0 [SPARK-46608][SQL] Restore backward compatibility of `JdbcDialect.classifyException` 2957bc9fa1f0 is described below commit 2957bc9fa1f0d49f6fcfbf5ebcd43df1a76504b6 Author: Max Gekk AuthorDate: Mon Jan 8 17:01:50 2024 +0300 [SPARK-46608][SQL] Restore backward compatibility of `JdbcDialect.classifyException` ### What changes were proposed in this pull request? In the PR, I propose to restore `classifyException()` of `JdbcDialect` before the commit https://github.com/apache/spark/commit/14a933bbe2eb1c71988f475036735f07b2e1fa6a, and extends `classifyException()` with the error class parameter by `description`: ```scala def classifyException( e: Throwable, errorClass: String, messageParameters: Map[String, String], description: String): AnalysisException ``` The `description` parameter has the same meaning as `message` in the old version of `classifyException()` which is deprecated. Also old implementation of `classifyException()` has been restored in JDBC dialects: MySQL, PostgreSQL and so on. ### Why are the changes needed? To restore compatibility with existing JDBC dialects. ### Does this PR introduce _any_ user-facing change? No, this PR restores the behaviour prior https://github.com/apache/spark/pull/44358. ### How was this patch tested? By running the affected test suite: ``` $ build/sbt "core/testOnly *SparkThrowableSuite" ``` and modified test suite: ``` $ build/sbt "test:testOnly *JDBCV2Suite" $ build/sbt "test:testOnly *JDBCTableCatalogSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #9 from MaxGekk/restore-jdbc-classifyException. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json| 5 ++ .../org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala | 6 +-- ...sql-error-conditions-failed-jdbc-error-class.md | 4 ++ .../sql/execution/datasources/jdbc/JdbcUtils.scala | 6 ++- .../execution/datasources/v2/jdbc/JDBCTable.scala | 6 ++- .../datasources/v2/jdbc/JDBCTableCatalog.scala | 33 - .../org/apache/spark/sql/jdbc/DB2Dialect.scala | 16 ++- .../org/apache/spark/sql/jdbc/H2Dialect.scala | 33 ++--- .../org/apache/spark/sql/jdbc/JdbcDialects.scala | 22 - .../apache/spark/sql/jdbc/MsSqlServerDialect.scala | 16 ++- .../org/apache/spark/sql/jdbc/MySQLDialect.scala | 28 ++- .../apache/spark/sql/jdbc/PostgresDialect.scala| 55 ++ .../v2/jdbc/JDBCTableCatalogSuite.scala| 29 +--- .../org/apache/spark/sql/jdbc/JDBCV2Suite.scala| 6 +-- 14 files changed, 143 insertions(+), 122 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 1812e9d76450..c7f8f59a7679 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1185,6 +1185,11 @@ "message" : [ "Check that the table exists." ] + }, + "UNCLASSIFIED" : { +"message" : [ + "" +] } }, "sqlState" : "HV000" 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 b93106b0ce78..d1d247967b4b 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 @@ -224,7 +224,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu val e = intercept[AnalysisException] { sql(s"CREATE TABLE $catalogName.new_table (i INT) TBLPROPERTIES('a'='1')") } - assert(e.getErrorClass == "FAILED_JDBC.CREATE_TABLE") + assert(e.getErrorClass == "FAILED_JDBC.UNCLASSIFIED") testCreateTableWithProperty(s"$catalogName.new_table") } } @@ -279,7 +279,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu sql(s"CREATE index i1 ON $catalogName.new_table (col1)") }, errorClass = "INDEX_ALREADY_EXISTS", - parameters = Map("indexName&q
(spark) branch master updated: [SPARK-46515] Add MONTHNAME function
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e8dfcd3081ab [SPARK-46515] Add MONTHNAME function e8dfcd3081ab is described below commit e8dfcd3081abe16b2115bb2944a2b1cb547eca8e Author: Stefan Kandic AuthorDate: Fri Jan 5 23:20:51 2024 +0300 [SPARK-46515] Add MONTHNAME function ### What changes were proposed in this pull request? Added MONTHNAME function which returns three-letter abbreviated month name for a given date to: - Scala API - Python API - R API - Spark Connect Scala Client - Spark Connect Python Client ### Why are the changes needed? for parity with Snowflake ### Does this PR introduce _any_ user-facing change? Yes, new MONTHNAME function ### How was this patch tested? With newly added unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #44483 from stefankandic/monthname-function. Authored-by: Stefan Kandic Signed-off-by: Max Gekk --- R/pkg/R/functions.R| 14 ++ R/pkg/R/generics.R | 4 +++ R/pkg/tests/fulltests/test_sparkSQL.R | 1 + .../scala/org/apache/spark/sql/functions.scala | 9 +++ .../apache/spark/sql/PlanGenerationTestSuite.scala | 4 +++ .../explain-results/function_monthname.explain | 2 ++ .../query-tests/queries/function_monthname.json| 25 + .../queries/function_monthname.proto.bin | Bin 0 -> 123 bytes .../source/reference/pyspark.sql/functions.rst | 1 + python/pyspark/sql/connect/functions/builtin.py| 7 + python/pyspark/sql/functions/builtin.py| 30 + python/pyspark/sql/tests/test_functions.py | 6 + .../sql/catalyst/analysis/FunctionRegistry.scala | 1 + .../catalyst/expressions/datetimeExpressions.scala | 17 .../spark/sql/catalyst/util/DateTimeUtils.scala| 12 + .../expressions/DateExpressionsSuite.scala | 11 .../scala/org/apache/spark/sql/functions.scala | 9 +++ .../sql-functions/sql-expression-schema.md | 1 + .../org/apache/spark/sql/DateFunctionsSuite.scala | 12 + 19 files changed, 166 insertions(+) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 14b4cb8cbdaa..0db25ad8739f 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -1091,6 +1091,20 @@ setMethod("dayofyear", column(jc) }) +#' @details +#' \code{monthname}: Extracts the three-letter abbreviated month name from a +#' given date/timestamp/string. +#' +#' @rdname column_datetime_functions +#' @aliases monthname monthname,Column-method +#' @note monthname since 4.0.0 +setMethod("monthname", + signature(x = "Column"), + function(x) { +jc <- callJStatic("org.apache.spark.sql.functions", "monthname", x@jc) +column(jc) + }) + #' @details #' \code{decode}: Computes the first argument into a string from a binary using the provided #' character set. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 2f46f63ba38a..92febc02710d 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1020,6 +1020,10 @@ setGeneric("dayofweek", function(x) { standardGeneric("dayofweek") }) #' @name NULL setGeneric("dayofyear", function(x) { standardGeneric("dayofyear") }) +#' @rdname column_datetime_functions +#' @name NULL +setGeneric("monthname", function(x) { standardGeneric("monthname") }) + #' @rdname column_string_functions #' @name NULL setGeneric("decode", function(x, charset) { standardGeneric("decode") }) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index c1a5292195af..118c853a00df 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -2062,6 +2062,7 @@ test_that("date functions on a DataFrame", { expect_equal(collect(select(df, weekofyear(df$b)))[, 1], c(50, 50, 51)) expect_equal(collect(select(df, year(df$b)))[, 1], c(2012, 2013, 2014)) expect_equal(collect(select(df, month(df$b)))[, 1], c(12, 12, 12)) + expect_equal(collect(select(df, monthname(df$b)))[, 1], c("Dec", "Dec", "Dec")) expect_equal(collect(select(df, last_day(df$b)))[, 1], c(as.Date("2012-12-31"), as.Date("2013-12-31"), as.Date("2014-12-31"))) expect_equal(collect(select(df, next_day(df
(spark) branch master updated: [SPARK-46592][DOCKER][TESTS][FOLLOWUP] Add newline to ORACLE Docker preparation sql
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 70b90c8b5778 [SPARK-46592][DOCKER][TESTS][FOLLOWUP] Add newline to ORACLE Docker preparation sql 70b90c8b5778 is described below commit 70b90c8b5778dc11998cbb5d6dbbb4a3bb7c8145 Author: Kent Yao AuthorDate: Fri Jan 5 11:54:15 2024 +0300 [SPARK-46592][DOCKER][TESTS][FOLLOWUP] Add newline to ORACLE Docker preparation sql ### What changes were proposed in this pull request? `semicolon` is not able to break SQLs in a file into separated statements ### Why are the changes needed? a followup for SPARK-46592 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Check the Oracle docker log Before ``` STDOUT: CONTAINER: running /docker-entrypoint-initdb.d/install.sql ... STDOUT: ALTER SESSION SET DDL_LOCK_TIMEOUT = 30;ALTER USER SYSTEM IDENTIFIED BY "Th1s1sThe0racle#Pass" STDOUT: * STDOUT: ERROR at line 1: STDOUT: ORA-00922: missing or invalid option STDOUT: Help: https://docs.oracle.com/error-help/db/ora-00922/ ``` After ``` STDOUT: CONTAINER: DONE: running /docker-entrypoint-initdb.d/install.sql STDOUT: Session altered. STDOUT: User altered. ``` ### Was this patch authored or co-authored using generative AI tooling? no Closes #44609 from yaooqinn/SPARK-46592-F. Authored-by: Kent Yao Signed-off-by: Max Gekk --- .../test/scala/org/apache/spark/sql/jdbc/OracleDatabaseOnDocker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleDatabaseOnDocker.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleDatabaseOnDocker.scala index 9b77469b2f8a..98d57d6a7475 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleDatabaseOnDocker.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleDatabaseOnDocker.scala @@ -51,7 +51,7 @@ class OracleDatabaseOnDocker extends DatabaseOnDocker with Logging { // means that the lock will no wait. We set the timeout to 30 seconds to try again. // TODO: This workaround should be removed once the issue is fixed in the image. // https://github.com/gvenzl/oci-oracle-free/issues/35 - writer.write("ALTER SESSION SET DDL_LOCK_TIMEOUT = 30;") + writer.write("ALTER SESSION SET DDL_LOCK_TIMEOUT = 30;\n") writer.write(s"""ALTER USER SYSTEM IDENTIFIED BY "$oracle_password";""") writer.close() val newBind = new Bind( - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46596][CORE][TESTS] Correct package name of `SslTestUtils`
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ddcfe6b10980 [SPARK-46596][CORE][TESTS] Correct package name of `SslTestUtils` ddcfe6b10980 is described below commit ddcfe6b10980e25017190d4d9f41b39c56b29d02 Author: yangjie01 AuthorDate: Thu Jan 4 21:50:19 2024 +0300 [SPARK-46596][CORE][TESTS] Correct package name of `SslTestUtils` ### What changes were proposed in this pull request? The package name for `SslTestUtils` is defined as `org.apache.spark`, but it is in the `org/apache/spark/util/` directory. This pr corrects its package name to ensure consistency. ### Why are the changes needed? Correct package name of `SslTestUtils` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44596 from LuciferYang/SslTestUtils-package-name. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: Max Gekk --- .../test/scala/org/apache/spark/SslExternalShuffleServiceSuite.scala | 1 + core/src/test/scala/org/apache/spark/SslShuffleNettySuite.scala | 2 ++ .../org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala | 2 +- .../apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala | 4 ++-- core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala | 2 +- .../test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala | 2 +- .../apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala| 4 ++-- .../org/apache/spark/storage/SslBlockManagerReplicationSuite.scala| 3 ++- core/src/test/scala/org/apache/spark/util/SslTestUtils.scala | 3 ++- 9 files changed, 14 insertions(+), 9 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SslExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/SslExternalShuffleServiceSuite.scala index 3ce1f11a7acd..119ad6b0530a 100644 --- a/core/src/test/scala/org/apache/spark/SslExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/SslExternalShuffleServiceSuite.scala @@ -22,6 +22,7 @@ import org.apache.spark.internal.config import org.apache.spark.network.TransportContext import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExternalBlockHandler +import org.apache.spark.util.SslTestUtils /** * This suite creates an external shuffle server and routes all shuffle fetches through it. diff --git a/core/src/test/scala/org/apache/spark/SslShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/SslShuffleNettySuite.scala index 7eaff7d37a81..ca4f0e8f7527 100644 --- a/core/src/test/scala/org/apache/spark/SslShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/SslShuffleNettySuite.scala @@ -17,6 +17,8 @@ package org.apache.spark +import org.apache.spark.util.SslTestUtils + class SslShuffleNettySuite extends ShuffleNettySuite { override def beforeAll(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index a8c9550c6b76..7bd84c810c42 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -45,7 +45,7 @@ import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded, SparkListenerExecutorRemoved, TaskDescription} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{KillTask, LaunchTask} import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.util.{SerializableBuffer, ThreadUtils, Utils} +import org.apache.spark.util.{SerializableBuffer, SslTestUtils, ThreadUtils, Utils} class CoarseGrainedExecutorBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 5c234ef95500..124c089b4710 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -32,7 +32,7 @@ import org.scalatest.matchers.must.Matchers import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar -import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite, SslTestUtils} +import org.apache.spark
(spark) branch master updated: [SPARK-46597][SQL][TESTS] Optimize the run tests command in the doc of XXIntegrationSuite in `docker-integration-tests` module
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 760de230eb6e [SPARK-46597][SQL][TESTS] Optimize the run tests command in the doc of XXIntegrationSuite in `docker-integration-tests` module 760de230eb6e is described below commit 760de230eb6e48db93aad7df1f8cf98cf5e62494 Author: yangjie01 AuthorDate: Thu Jan 4 21:45:50 2024 +0300 [SPARK-46597][SQL][TESTS] Optimize the run tests command in the doc of XXIntegrationSuite in `docker-integration-tests` module ### What changes were proposed in this pull request? The class descriptions of the test cases in the `docker-integration-tests` module provide suggested commands for executing the test cases, but these commands will search for corresponding test cases in all modules. This pr adds the module name `docker-integration-tests` to these commands, so that the test commands only search for corresponding test cases in the `docker-integration-tests` module. ### Why are the changes needed? Refine docs. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? I have manually checked the new command locally. ### Was this patch authored or co-authored using generative AI tooling? No Closes #44597 from LuciferYang/docker-integration-tests-cmd. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: Max Gekk --- .../src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala | 2 +- .../test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala | 3 ++- .../scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala | 2 +- .../scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala | 2 +- .../test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala | 2 +- .../scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala | 2 +- .../scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala | 3 ++- 7 files changed, 9 insertions(+), 7 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala index e4ee7abf7dd5..df2932684880 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.tags.DockerTest * {{{ * ENABLE_DOCKER_INTEGRATION_TESTS=1 DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.6.0a * ./build/sbt -Pdocker-integration-tests - * "testOnly org.apache.spark.sql.jdbc.DB2IntegrationSuite" + * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.DB2IntegrationSuite" * }}} */ @DockerTest diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala index 66e2afbb6eff..d221fbfab622 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala @@ -34,7 +34,8 @@ import org.apache.spark.tags.DockerTest * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.6.0a): * {{{ * ENABLE_DOCKER_INTEGRATION_TESTS=1 DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.6.0a - * ./build/sbt -Pdocker-integration-tests "testOnly *DB2KrbIntegrationSuite" + * ./build/sbt -Pdocker-integration-tests + * "docker-integration-tests/testOnly *DB2KrbIntegrationSuite" * }}} */ @DockerTest diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala index 49c9e3dba0d7..6825c001f767 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.tags.DockerTest * {{{ * ENABLE_DOCKER_INTEGRATION_TESTS=1 MARIADB_DOCKER_IMAGE_NAME=mariadb:10.5.12 * ./build/sbt -Pdocker-integration-tests - * "testOnly org.apache.spark.sql.jdbc.MariaDBKrbIntegrationSuite" + * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.MariaDBKrbIntegrationSuite" * }}} */ @DockerTest diff --
(spark) branch master updated: [SPARK-46592][DOCKER][TEST] OracleIntegrationSuite is flaky because of ORA-04021
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9ec426dd81c5 [SPARK-46592][DOCKER][TEST] OracleIntegrationSuite is flaky because of ORA-04021 9ec426dd81c5 is described below commit 9ec426dd81c579946e869bfeed935743137deda1 Author: Kent Yao AuthorDate: Thu Jan 4 21:44:38 2024 +0300 [SPARK-46592][DOCKER][TEST] OracleIntegrationSuite is flaky because of ORA-04021 ### What changes were proposed in this pull request? This PR mounts `docker-entrypoint-initdb.d` to OracleDocker to set DDL_LOCK_TIMEOUT and retry `alter user system` to ensure the password of SYSTEM user is correctly set. ### Why are the changes needed? https://github.com/apache/spark/actions/runs/7405337059/job/20148144542 https://github.com/apache/spark/actions/runs/7403889993/job/20144539155 https://github.com/apache/spark/actions/runs/7401841987/job/20138483574 Fix test flakiness caused by ORA-04021 ``` 22:09.284 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: CONTAINER: Resetting SYS and SYSTEM passwords. 24/01/04 04:22:09.284 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: 24/01/04 04:22:09.284 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: User altered. 24/01/04 04:22:09.284 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: 24/01/04 04:22:09.284 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: ALTER USER SYSTEM IDENTIFIED BY "Th1s1sThe0racle#Pass" 24/01/04 04:22:09.284 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: * 24/01/04 04:22:09.284 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: ERROR at line 1: 24/01/04 04:22:09.285 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: ORA-65048: error encountered when processing the current DDL statement in 24/01/04 04:22:09.285 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: pluggable database FREEPDB1 24/01/04 04:22:09.285 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: ORA-04021: timeout occurred while waiting to lock object 24/01/04 04:22:09.285 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: Help: https://docs.oracle.com/error-help/db/ora-65048/ 24/01/04 04:22:09.285 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: 24/01/04 04:22:09.285 docker-java-stream-568158377 INFO OracleIntegrationSuite: STDOUT: ``` ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? OracleIntegrationSuite ### Was this patch authored or co-authored using generative AI tooling? no Closes #44594 from yaooqinn/SPARK-46592. Authored-by: Kent Yao Signed-off-by: Max Gekk --- .../spark/sql/jdbc/OracleDatabaseOnDocker.scala| 67 ++ .../spark/sql/jdbc/OracleIntegrationSuite.scala| 14 + .../spark/sql/jdbc/v2/OracleIntegrationSuite.scala | 16 +- .../spark/sql/jdbc/v2/OracleNamespaceSuite.scala | 16 +- 4 files changed, 72 insertions(+), 41 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleDatabaseOnDocker.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleDatabaseOnDocker.scala new file mode 100644 index ..9b77469b2f8a --- /dev/null +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleDatabaseOnDocker.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.io.{File, PrintWriter} + +import com.github.dockerjava.api.model._ + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +class OracleDatabaseOnDocker extends DatabaseOnDocker with Logging { + lazy override val imageName = +sys.env.getOrElse("
(spark) branch master updated: [SPARK-46576][SQL] Improve error messages for unsupported data source save mode
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 69c46876b5a7 [SPARK-46576][SQL] Improve error messages for unsupported data source save mode 69c46876b5a7 is described below commit 69c46876b5a76c2de6a149ea7663fad18027e387 Author: allisonwang-db AuthorDate: Thu Jan 4 09:40:40 2024 +0300 [SPARK-46576][SQL] Improve error messages for unsupported data source save mode ### What changes were proposed in this pull request? This PR renames the error class `_LEGACY_ERROR_TEMP_1308` to `UNSUPPORTED_DATA_SOURCE_SAVE_MODE` and improves its error messages. ### Why are the changes needed? To make the error more user-friendly. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #44576 from allisonwang-db/spark-46576-unsupported-save-mode. Authored-by: allisonwang-db Signed-off-by: Max Gekk --- .../src/main/resources/error/error-classes.json | 11 ++- .../apache/spark/sql/kafka010/KafkaSinkSuite.scala | 2 +- docs/sql-error-conditions.md| 6 ++ .../spark/sql/errors/QueryCompilationErrors.scala | 4 ++-- .../spark/sql/connector/DataSourceV2Suite.scala | 8 .../execution/python/PythonDataSourceSuite.scala| 21 +++-- 6 files changed, 34 insertions(+), 18 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index bcaf8a74c08d..9cade1197dca 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -3588,6 +3588,12 @@ ], "sqlState" : "0A000" }, + "UNSUPPORTED_DATA_SOURCE_SAVE_MODE" : { +"message" : [ + "The data source '' cannot be written in the mode. Please use either the \"Append\" or \"Overwrite\" mode instead." +], +"sqlState" : "0A000" + }, "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE" : { "message" : [ "The datasource doesn't support the column of the type ." @@ -5403,11 +5409,6 @@ "There is a 'path' option set and save() is called with a path parameter. Either remove the path option, or call save() without the parameter. To ignore this check, set '' to 'true'." ] }, - "_LEGACY_ERROR_TEMP_1308" : { -"message" : [ - "TableProvider implementation cannot be written with mode, please use Append or Overwrite modes instead." -] - }, "_LEGACY_ERROR_TEMP_1309" : { "message" : [ "insertInto() can't be used together with partitionBy(). Partition columns have already been defined for the table. It is not necessary to use partitionBy()." diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 6753f8be54bf..5566785c4d56 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -557,7 +557,7 @@ class KafkaSinkBatchSuiteV2 extends KafkaSinkBatchSuiteBase { test("batch - unsupported save modes") { testUnsupportedSaveModes((mode) => - Seq(s"cannot be written with ${mode.name} mode", "does not support truncate")) + Seq(s"cannot be written in the \"${mode.name}\" mode", "does not support truncate")) } test("generic - write big data with small producer buffer") { diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index c6108e97b4c5..89de607b0f22 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -2332,6 +2332,12 @@ Unsupported data source type for direct query on files: `` Unsupported data type ``. +### UNSUPPORTED_DATA_SOURCE_SAVE_MODE + +[SQLSTATE: 0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported) + +The data source '``' cannot be written in the `` mode. Please use either the "Append" or "Overwrite" mode instead. + ### UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE [SQLSTATE: 0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported) diff --git a/sql/catalyst/src/main/sca
(spark) branch master updated: [SPARK-46539][SQL] SELECT * EXCEPT(all fields from a struct) results in an assertion failure
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9c46d9dcd195 [SPARK-46539][SQL] SELECT * EXCEPT(all fields from a struct) results in an assertion failure 9c46d9dcd195 is described below commit 9c46d9dcd19551dbdef546adec73d5799364ab0b Author: Stefan Kandic AuthorDate: Wed Jan 3 21:52:37 2024 +0300 [SPARK-46539][SQL] SELECT * EXCEPT(all fields from a struct) results in an assertion failure ### What changes were proposed in this pull request? Fixing the assertion error which occurs when we do SELECT .. EXCEPT(every field from a struct) by adding a check for an empty struct ### Why are the changes needed? Because this is a valid query that should just return an empty struct rather than fail during serialization. ### Does this PR introduce _any_ user-facing change? Yes, users should no longer see this error and instead get an empty struct '{ }' ### How was this patch tested? By adding new UT to existing selectExcept tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #44527 from stefankandic/select-except-err. Authored-by: Stefan Kandic Signed-off-by: Max Gekk --- .../spark/sql/catalyst/encoders/ExpressionEncoder.scala| 12 ++-- .../sql-tests/analyzer-results/selectExcept.sql.out| 12 .../src/test/resources/sql-tests/inputs/selectExcept.sql | 1 + .../test/resources/sql-tests/results/selectExcept.sql.out | 14 ++ 4 files changed, 37 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 74d7a5e7a675..654f39393636 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -325,11 +325,19 @@ case class ExpressionEncoder[T]( assert(serializer.forall(_.references.isEmpty), "serializer cannot reference any attributes.") assert(serializer.flatMap { ser => val boundRefs = ser.collect { case b: BoundReference => b } -assert(boundRefs.nonEmpty, - "each serializer expression should contain at least one `BoundReference`") +assert(boundRefs.nonEmpty || isEmptyStruct(ser), + "each serializer expression should contain at least one `BoundReference` or it " + + "should be an empty struct. This is required to ensure that there is a reference point " + + "for the serialized object or that the serialized object is intentionally left empty." +) boundRefs }.distinct.length <= 1, "all serializer expressions must use the same BoundReference.") + private def isEmptyStruct(expr: NamedExpression): Boolean = expr.dataType match { +case struct: StructType => struct.isEmpty +case _ => false + } + /** * Returns a new copy of this encoder, where the `deserializer` is resolved and bound to the * given schema. diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/selectExcept.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/selectExcept.sql.out index 3b8594d832c6..49ea7ed4edcf 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/selectExcept.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/selectExcept.sql.out @@ -121,6 +121,18 @@ Project [id#x, name#x, named_struct(f1, data#x.f1, s2, named_struct(f3, data#x.s +- LocalRelation [id#x, name#x, data#x] +-- !query +SELECT * EXCEPT (data.f1, data.s2) FROM tbl_view +-- !query analysis +Project [id#x, name#x, named_struct() AS data#x] ++- SubqueryAlias tbl_view + +- View (`tbl_view`, [id#x,name#x,data#x]) + +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] + +- Project [id#x, name#x, data#x] ++- SubqueryAlias tbl_view + +- LocalRelation [id#x, name#x, data#x] + + -- !query SELECT * EXCEPT (id, name, data) FROM tbl_view -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/selectExcept.sql b/sql/core/src/test/resources/sql-tests/inputs/selectExcept.sql index e07e4f1117c2..08d56aeda0a8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/selectExcept.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/selectExcept.sql @@ -20,6 +20,7 @@ SELECT * EXCEPT (data) FROM tbl_view; SELECT * EXCEPT (data.f1) FROM tbl_view; SELECT * EXCEPT (data.s2) FROM tbl_view; SELECT * EXCEPT (data
(spark) branch master updated: [SPARK-46524][SQL] Improve error messages for invalid save mode
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a3d999292f8e [SPARK-46524][SQL] Improve error messages for invalid save mode a3d999292f8e is described below commit a3d999292f8e99269dfd0289e2f5aca7e5ea4fae Author: allisonwang-db AuthorDate: Wed Jan 3 15:43:53 2024 +0300 [SPARK-46524][SQL] Improve error messages for invalid save mode ### What changes were proposed in this pull request? This PR improves the error messages when writing a data frame with an invalid save mode. ### Why are the changes needed? To improve the error messages. Before this PR, Spark throws an java.lang.IllegalArgumentException: `java.lang.IllegalArgumentException: Unknown save mode: foo. Accepted save modes are 'overwrite', 'append', 'ignore', 'error', 'errorifexists', 'default'.` After this PR, the error will have a proper error class: `[INVALID_SAVE_MODE] The specified save mode "foo" is invalid. Valid save modes include "append", "overwrite", "ignore", "error", "errorifexists", and "default"." ` ### Does this PR introduce _any_ user-facing change? Yes. The error messages will be changed. ### How was this patch tested? New unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #44508 from allisonwang-db/spark-46524-invalid-save-mode. Authored-by: allisonwang-db Signed-off-by: Max Gekk --- R/pkg/tests/fulltests/test_sparkSQL.R| 2 +- common/utils/src/main/resources/error/error-classes.json | 6 ++ docs/sql-error-conditions.md | 6 ++ .../org/apache/spark/sql/errors/QueryCompilationErrors.scala | 7 +++ .../src/main/scala/org/apache/spark/sql/DataFrameWriter.scala| 3 +-- .../spark/sql/execution/python/PythonDataSourceSuite.scala | 9 + 6 files changed, 30 insertions(+), 3 deletions(-) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 0d96f708a544..c1a5292195af 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1414,7 +1414,7 @@ test_that("test HiveContext", { # Invalid mode expect_error(saveAsTable(df, "parquetest", "parquet", mode = "abc", path = parquetDataPath), - "illegal argument - Unknown save mode: abc") + "Error in mode : analysis error - \\[INVALID_SAVE_MODE\\].*") unsetHiveContext() } }) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 87e43fe0e38c..bcaf8a74c08d 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -2239,6 +2239,12 @@ ], "sqlState" : "42613" }, + "INVALID_SAVE_MODE" : { +"message" : [ + "The specified save mode is invalid. Valid save modes include \"append\", \"overwrite\", \"ignore\", \"error\", \"errorifexists\", and \"default\"." +], +"sqlState" : "42000" + }, "INVALID_SCHEMA" : { "message" : [ "The input schema is not a valid schema string." diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 3f4074af9b78..c6108e97b4c5 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -1271,6 +1271,12 @@ For more details see [INVALID_PARTITION_OPERATION](sql-error-conditions-invalid- Parameterized query must either use positional, or named parameters, but not both. +### INVALID_SAVE_MODE + +[SQLSTATE: 42000](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +The specified save mode `` is invalid. Valid save modes include "append", "overwrite", "ignore", "error", "errorifexists", and "default". + ### [INVALID_SCHEMA](sql-error-conditions-invalid-schema-error-class.html) [SQLSTATE: 42K07](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index bc847d1c0069..b844ee2bdc45 100644 --- a/sql/catalyst/src/main/scala/org/apache/spar