From 84cbea762d3cd73c83109069c141762b2f2f9121 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 8 Feb 2023 10:24:00 +0800 Subject: [PATCH 01/13] refine default column value framework --- .../queries/function_udf_2.12.json | 8 +- .../queries/function_udf_2.12.proto.bin | Bin 11257 -> 11257 bytes .../sql/jdbc/v2/V2JDBCNamespaceTest.scala | 3 +- .../main/resources/error/error-classes.json | 29 ++- .../spark/sql/connector/catalog/Column.java | 77 +++++++ .../connector/catalog/ColumnDefaultValue.java | 61 +++++ .../catalog/StagingTableCatalog.java | 67 +++++- .../spark/sql/connector/catalog/Table.java | 11 + .../sql/connector/catalog/TableCatalog.java | 23 +- .../sql/connector/catalog/TableChange.java | 20 +- .../sql/catalyst/analysis/Analyzer.scala | 32 +-- .../sql/catalyst/analysis/CheckAnalysis.scala | 10 +- .../catalyst/analysis/v2ResolutionPlans.scala | 8 +- .../sql/catalyst/parser/AstBuilder.scala | 135 ++++++----- .../catalyst/plans/logical/statements.scala | 18 -- .../plans/logical/v2AlterTableCommands.scala | 81 +++++-- .../catalyst/plans/logical/v2Commands.scala | 95 +++++--- .../util/ResolveDefaultColumnsUtil.scala | 212 +++++++++--------- .../catalog/CatalogV2Implicits.scala | 7 + .../sql/connector/catalog/CatalogV2Util.scala | 73 ++++-- .../write/RowLevelOperationTable.scala | 3 +- .../sql/errors/QueryCompilationErrors.scala | 38 ++-- .../datasources/v2/DataSourceV2Relation.scala | 3 +- .../apache/spark/sql/internal/SQLConf.scala | 13 +- .../sql/internal/connector/ColumnImpl.scala | 30 +++ .../connector/SimpleTableProvider.scala | 3 +- .../apache/spark/sql/types/StructField.scala | 8 +- .../apache/spark/sql/types/StructType.scala | 28 ++- .../catalog/ExternalCatalogSuite.scala | 37 +-- .../catalog/SessionCatalogSuite.scala | 67 ------ .../sql/catalyst/parser/DDLParserSuite.scala | 187 ++++++++------- .../sql/connector/catalog/CatalogSuite.scala | 103 ++++----- .../catalog/CatalogV2UtilSuite.scala | 4 +- .../catalog/InMemoryTableCatalog.scala | 12 +- ...pportsAtomicPartitionManagementSuite.scala | 4 +- .../SupportsPartitionManagementSuite.scala | 7 +- .../spark/sql/types/StructTypeSuite.scala | 22 +- .../analysis/ReplaceCharWithVarchar.scala | 16 +- .../analysis/ResolveSessionCatalog.scala | 58 +++-- .../spark/sql/execution/command/ddl.scala | 35 ++- .../spark/sql/execution/command/tables.scala | 26 +-- .../datasources/DataSourceStrategy.scala | 10 +- .../sql/execution/datasources/rules.scala | 1 - .../datasources/v2/CreateTableExec.scala | 7 +- .../datasources/v2/DataSourceV2Strategy.scala | 21 +- .../datasources/v2/DescribeTableExec.scala | 29 ++- .../datasources/v2/FileDataSourceV2.scala | 3 +- .../datasources/v2/ReplaceTableExec.scala | 13 +- .../datasources/v2/ShowCreateTableExec.scala | 3 +- .../datasources/v2/V2SessionCatalog.scala | 14 +- .../v2/WriteToDataSourceV2Exec.scala | 22 +- .../sources/TextSocketSourceProvider.scala | 5 +- .../internal/BaseSessionStateBuilder.scala | 2 +- .../spark/sql/internal/CatalogImpl.scala | 6 +- .../sql/streaming/DataStreamReader.scala | 3 +- .../sql/streaming/DataStreamWriter.scala | 2 +- ...SourceV2DataFrameSessionCatalogSuite.scala | 2 +- .../DataSourceV2DataFrameSuite.scala | 3 +- .../sql/connector/DataSourceV2SQLSuite.scala | 20 +- .../connector/DeleteFromTableSuiteBase.scala | 4 +- .../connector/TestV2SessionCatalogBase.scala | 11 +- .../V2CommandsCaseSensitivitySuite.scala | 125 ++++------- .../WriteDistributionAndOrderingSuite.scala | 2 +- .../command/PlanResolutionSuite.scala | 56 +++-- .../InMemoryTableMetricSuite.scala | 3 +- .../v2/V2SessionCatalogSuite.scala | 96 ++++---- .../sql/sources/DataSourceAnalysisSuite.scala | 3 +- .../spark/sql/sources/InsertSuite.scala | 86 ++----- .../sql/hive/HiveSessionStateBuilder.scala | 2 +- .../apache/spark/sql/hive/InsertSuite.scala | 6 +- 70 files changed, 1231 insertions(+), 1003 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.json b/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.json index 76738354e15b7..010dfa84e8596 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.json @@ -21,7 +21,7 @@ } }], "scalarScalaUdf": { - "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQyWQlE7Ce2cPkCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAABdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAAXNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwc3IARm9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJvZHVjdEVuY29kZXIcqKluUDodYQIAA0wABmNsc1RhZ3EAfgAPTAAGZmllbGRzcQB+AAJMAAZzY2hlbWF0ACdMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0VHlwZTt4cHNyACZzY2FsYS5yZWZsZWN0LkNsYXNzVGFnJEdlbmVyaWNDbGFzc1RhZwAAAAAAAAABAgABTAAMcnVudGltZUNsYXNzdAARTGphdmEvbGFuZy9DbGFzczt4cHZyAAxzY2FsYS5UdXBsZTHPDUf18JuzPAIAAUwAAl8xcQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcgAsc2NhbGEuY29sbGVjdGlvbi5pbW11dGFibGUuTGlzdFNlcmlhbGl6ZUVuZCSKXGNb91MLbQIAAHhweHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RUeXBl9I34EN1tjkUCAAlJAAlfaGFzaENvZGVCAAhiaXRtYXAkMFoAGWhhc0V4aXN0ZW5jZURlZmF1bHRWYWx1ZXNbABZleGlzdGVuY2VEZWZhdWx0VmFsdWVzcQB+AAhbABhleGlzdGVuY2VEZWZhdWx0c0JpdG1hc2t0AAJbWkwADWZpZWxkTmFtZXNTZXR0ACBMc2NhbGEvY29sbGVjdGlvbi9pbW11dGFibGUvU2V0O1sABmZpZWxkc3QAKVtMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0RmllbGQ7TAALbmFtZVRvRmllbGR0ABZMc2NhbGEvY29sbGVjdGlvbi9NYXA7TAALbmFtZVRvSW5kZXhxAH4ANnhwAAAAAAAAcHBwdXIAKVtMb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQ7tWPEaGAaDUcCAAB4cAAAAAFzcgAmb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQrgSSJZ9l3nwIABFoACG51bGxhYmxlTAAIZGF0YVR5cGVxAH4AEEwACG1ldGFkYXRhcQB+ACRMAARuYW1lcQB+ABR4cABxAH4AGHEAfgApcQB+ACxwcA==", + "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQyWQlE7Ce2cPkCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAABdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAAXNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwc3IARm9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJvZHVjdEVuY29kZXIcqKluUDodYQIAA0wABmNsc1RhZ3EAfgAPTAAGZmllbGRzcQB+AAJMAAZzY2hlbWF0ACdMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0VHlwZTt4cHNyACZzY2FsYS5yZWZsZWN0LkNsYXNzVGFnJEdlbmVyaWNDbGFzc1RhZwAAAAAAAAABAgABTAAMcnVudGltZUNsYXNzdAARTGphdmEvbGFuZy9DbGFzczt4cHZyAAxzY2FsYS5UdXBsZTHPDUf18JuzPAIAAUwAAl8xcQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcgAsc2NhbGEuY29sbGVjdGlvbi5pbW11dGFibGUuTGlzdFNlcmlhbGl6ZUVuZCSKXGNb91MLbQIAAHhweHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RUeXBllANIqWlTBxgCAAlJAAlfaGFzaENvZGVCAAhiaXRtYXAkMFoAGWhhc0V4aXN0ZW5jZURlZmF1bHRWYWx1ZXNbABZleGlzdGVuY2VEZWZhdWx0VmFsdWVzcQB+AAhbABhleGlzdGVuY2VEZWZhdWx0c0JpdG1hc2t0AAJbWkwADWZpZWxkTmFtZXNTZXR0ACBMc2NhbGEvY29sbGVjdGlvbi9pbW11dGFibGUvU2V0O1sABmZpZWxkc3QAKVtMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0RmllbGQ7TAALbmFtZVRvRmllbGR0ABZMc2NhbGEvY29sbGVjdGlvbi9NYXA7TAALbmFtZVRvSW5kZXhxAH4ANnhwAAAAAAAAcHBwdXIAKVtMb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQ7tWPEaGAaDUcCAAB4cAAAAAFzcgAmb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQrgSSJZ9l3nwIABFoACG51bGxhYmxlTAAIZGF0YVR5cGVxAH4AEEwACG1ldGFkYXRhcQB+ACRMAARuYW1lcQB+ABR4cABxAH4AGHEAfgApcQB+ACxwcA==", "nullable": true } } @@ -39,7 +39,7 @@ } }], "scalarScalaUdf": { - "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQzyD4NN4Grh74CAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAACdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAAnNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFzcgBGb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRQcm9kdWN0RW5jb2RlchyoqW5QOh1hAgADTAAGY2xzVGFncQB+AA9MAAZmaWVsZHNxAH4AAkwABnNjaGVtYXQAJ0xvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9TdHJ1Y3RUeXBlO3hwc3IAJnNjYWxhLnJlZmxlY3QuQ2xhc3NUYWckR2VuZXJpY0NsYXNzVGFnAAAAAAAAAAECAAFMAAxydW50aW1lQ2xhc3N0ABFMamF2YS9sYW5nL0NsYXNzO3hwdnIADHNjYWxhLlR1cGxlMi6UZn1bkvn1AgACTAACXzFxAH4AAUwAAl8ycQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NyACxzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0U2VyaWFsaXplRW5kJIpcY1v3UwttAgAAeHB4c3IAJW9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdFR5cGX0jfgQ3W2ORQIACUkACV9oYXNoQ29kZUIACGJpdG1hcCQwWgAZaGFzRXhpc3RlbmNlRGVmYXVsdFZhbHVlc1sAFmV4aXN0ZW5jZURlZmF1bHRWYWx1ZXNxAH4ACFsAGGV4aXN0ZW5jZURlZmF1bHRzQml0bWFza3QAAltaTAANZmllbGROYW1lc1NldHQAIExzY2FsYS9jb2xsZWN0aW9uL2ltbXV0YWJsZS9TZXQ7WwAGZmllbGRzdAApW0xvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9TdHJ1Y3RGaWVsZDtMAAtuYW1lVG9GaWVsZHQAFkxzY2FsYS9jb2xsZWN0aW9uL01hcDtMAAtuYW1lVG9JbmRleHEAfgA4eHAAAAAAAABwcHB1cgApW0xvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RGaWVsZDu1Y8RoYBoNRwIAAHhwAAAAAnNyACZvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RGaWVsZCuBJIln2XefAgAEWgAIbnVsbGFibGVMAAhkYXRhVHlwZXEAfgAQTAAIbWV0YWRhdGFxAH4AJEwABG5hbWVxAH4AFHhwAHEAfgAYcQB+AClxAH4ALHNxAH4APABxAH4AGHEAfgApcQB+ADFwcA==", + "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQzyD4NN4Grh74CAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAACdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAAnNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFzcgBGb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRQcm9kdWN0RW5jb2RlchyoqW5QOh1hAgADTAAGY2xzVGFncQB+AA9MAAZmaWVsZHNxAH4AAkwABnNjaGVtYXQAJ0xvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9TdHJ1Y3RUeXBlO3hwc3IAJnNjYWxhLnJlZmxlY3QuQ2xhc3NUYWckR2VuZXJpY0NsYXNzVGFnAAAAAAAAAAECAAFMAAxydW50aW1lQ2xhc3N0ABFMamF2YS9sYW5nL0NsYXNzO3hwdnIADHNjYWxhLlR1cGxlMi6UZn1bkvn1AgACTAACXzFxAH4AAUwAAl8ycQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NyACxzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0U2VyaWFsaXplRW5kJIpcY1v3UwttAgAAeHB4c3IAJW9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdFR5cGWUA0ipaVMHGAIACUkACV9oYXNoQ29kZUIACGJpdG1hcCQwWgAZaGFzRXhpc3RlbmNlRGVmYXVsdFZhbHVlc1sAFmV4aXN0ZW5jZURlZmF1bHRWYWx1ZXNxAH4ACFsAGGV4aXN0ZW5jZURlZmF1bHRzQml0bWFza3QAAltaTAANZmllbGROYW1lc1NldHQAIExzY2FsYS9jb2xsZWN0aW9uL2ltbXV0YWJsZS9TZXQ7WwAGZmllbGRzdAApW0xvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9TdHJ1Y3RGaWVsZDtMAAtuYW1lVG9GaWVsZHQAFkxzY2FsYS9jb2xsZWN0aW9uL01hcDtMAAtuYW1lVG9JbmRleHEAfgA4eHAAAAAAAABwcHB1cgApW0xvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RGaWVsZDu1Y8RoYBoNRwIAAHhwAAAAAnNyACZvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RGaWVsZCuBJIln2XefAgAEWgAIbnVsbGFibGVMAAhkYXRhVHlwZXEAfgAQTAAIbWV0YWRhdGFxAH4AJEwABG5hbWVxAH4AFHhwAHEAfgAYcQB+AClxAH4ALHNxAH4APABxAH4AGHEAfgApcQB+ADFwcA==", "nullable": true } } @@ -61,7 +61,7 @@ } }], "scalarScalaUdf": { - "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQ009cpyjjQtFMCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAADdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAA3NyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFxAH4AEXNyAEZvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByb2R1Y3RFbmNvZGVyHKipblA6HWECAANMAAZjbHNUYWdxAH4AD0wABmZpZWxkc3EAfgACTAAGc2NoZW1hdAAnTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL1N0cnVjdFR5cGU7eHBzcgAmc2NhbGEucmVmbGVjdC5DbGFzc1RhZyRHZW5lcmljQ2xhc3NUYWcAAAAAAAAAAQIAAUwADHJ1bnRpbWVDbGFzc3QAEUxqYXZhL2xhbmcvQ2xhc3M7eHB2cgAMc2NhbGEuVHVwbGUzibsxap5k00kCAANMAAJfMXEAfgABTAACXzJxAH4AAUwAAl8zcQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NxAH4AIwBxAH4AEXEAfgApdAACXzNxAH4AL3EAfgAvc3IALHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLkxpc3RTZXJpYWxpemVFbmQkilxjW/dTC20CAAB4cHhzcgAlb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0VHlwZfSN+BDdbY5FAgAJSQAJX2hhc2hDb2RlQgAIYml0bWFwJDBaABloYXNFeGlzdGVuY2VEZWZhdWx0VmFsdWVzWwAWZXhpc3RlbmNlRGVmYXVsdFZhbHVlc3EAfgAIWwAYZXhpc3RlbmNlRGVmYXVsdHNCaXRtYXNrdAACW1pMAA1maWVsZE5hbWVzU2V0dAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL1NldDtbAAZmaWVsZHN0AClbTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL1N0cnVjdEZpZWxkO0wAC25hbWVUb0ZpZWxkdAAWTHNjYWxhL2NvbGxlY3Rpb24vTWFwO0wAC25hbWVUb0luZGV4cQB+ADp4cAAAAAAAAHBwcHVyAClbTG9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdEZpZWxkO7VjxGhgGg1HAgAAeHAAAAADc3IAJm9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdEZpZWxkK4EkiWfZd58CAARaAAhudWxsYWJsZUwACGRhdGFUeXBlcQB+ABBMAAhtZXRhZGF0YXEAfgAkTAAEbmFtZXEAfgAUeHAAcQB+ABhxAH4AKXEAfgAsc3EAfgA+AHEAfgAYcQB+AClxAH4AMXNxAH4APgBxAH4AGHEAfgApcQB+ADNwcA==", + "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQ009cpyjjQtFMCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAADdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAA3NyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFxAH4AEXNyAEZvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByb2R1Y3RFbmNvZGVyHKipblA6HWECAANMAAZjbHNUYWdxAH4AD0wABmZpZWxkc3EAfgACTAAGc2NoZW1hdAAnTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL1N0cnVjdFR5cGU7eHBzcgAmc2NhbGEucmVmbGVjdC5DbGFzc1RhZyRHZW5lcmljQ2xhc3NUYWcAAAAAAAAAAQIAAUwADHJ1bnRpbWVDbGFzc3QAEUxqYXZhL2xhbmcvQ2xhc3M7eHB2cgAMc2NhbGEuVHVwbGUzibsxap5k00kCAANMAAJfMXEAfgABTAACXzJxAH4AAUwAAl8zcQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NxAH4AIwBxAH4AEXEAfgApdAACXzNxAH4AL3EAfgAvc3IALHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLkxpc3RTZXJpYWxpemVFbmQkilxjW/dTC20CAAB4cHhzcgAlb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0VHlwZZQDSKlpUwcYAgAJSQAJX2hhc2hDb2RlQgAIYml0bWFwJDBaABloYXNFeGlzdGVuY2VEZWZhdWx0VmFsdWVzWwAWZXhpc3RlbmNlRGVmYXVsdFZhbHVlc3EAfgAIWwAYZXhpc3RlbmNlRGVmYXVsdHNCaXRtYXNrdAACW1pMAA1maWVsZE5hbWVzU2V0dAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL1NldDtbAAZmaWVsZHN0AClbTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL1N0cnVjdEZpZWxkO0wAC25hbWVUb0ZpZWxkdAAWTHNjYWxhL2NvbGxlY3Rpb24vTWFwO0wAC25hbWVUb0luZGV4cQB+ADp4cAAAAAAAAHBwcHVyAClbTG9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdEZpZWxkO7VjxGhgGg1HAgAAeHAAAAADc3IAJm9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdEZpZWxkK4EkiWfZd58CAARaAAhudWxsYWJsZUwACGRhdGFUeXBlcQB+ABBMAAhtZXRhZGF0YXEAfgAkTAAEbmFtZXEAfgAUeHAAcQB+ABhxAH4AKXEAfgAsc3EAfgA+AHEAfgAYcQB+AClxAH4AMXNxAH4APgBxAH4AGHEAfgApcQB+ADNwcA==", "nullable": true } } @@ -87,7 +87,7 @@ } }], "scalarScalaUdf": { - "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQ1hQsS9jxAO/gCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAAEdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAABHNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFxAH4AEXEAfgARc3IARm9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJvZHVjdEVuY29kZXIcqKluUDodYQIAA0wABmNsc1RhZ3EAfgAPTAAGZmllbGRzcQB+AAJMAAZzY2hlbWF0ACdMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0VHlwZTt4cHNyACZzY2FsYS5yZWZsZWN0LkNsYXNzVGFnJEdlbmVyaWNDbGFzc1RhZwAAAAAAAAABAgABTAAMcnVudGltZUNsYXNzdAARTGphdmEvbGFuZy9DbGFzczt4cHZyAAxzY2FsYS5UdXBsZTTIfhC+GxSuCgIABEwAAl8xcQB+AAFMAAJfMnEAfgABTAACXzNxAH4AAUwAAl80cQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NxAH4AIwBxAH4AEXEAfgApdAACXzNxAH4AL3EAfgAvc3EAfgAjAHEAfgARcQB+ACl0AAJfNHEAfgAvcQB+AC9zcgAsc2NhbGEuY29sbGVjdGlvbi5pbW11dGFibGUuTGlzdFNlcmlhbGl6ZUVuZCSKXGNb91MLbQIAAHhweHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RUeXBl9I34EN1tjkUCAAlJAAlfaGFzaENvZGVCAAhiaXRtYXAkMFoAGWhhc0V4aXN0ZW5jZURlZmF1bHRWYWx1ZXNbABZleGlzdGVuY2VEZWZhdWx0VmFsdWVzcQB+AAhbABhleGlzdGVuY2VEZWZhdWx0c0JpdG1hc2t0AAJbWkwADWZpZWxkTmFtZXNTZXR0ACBMc2NhbGEvY29sbGVjdGlvbi9pbW11dGFibGUvU2V0O1sABmZpZWxkc3QAKVtMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0RmllbGQ7TAALbmFtZVRvRmllbGR0ABZMc2NhbGEvY29sbGVjdGlvbi9NYXA7TAALbmFtZVRvSW5kZXhxAH4APHhwAAAAAAAAcHBwdXIAKVtMb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQ7tWPEaGAaDUcCAAB4cAAAAARzcgAmb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQrgSSJZ9l3nwIABFoACG51bGxhYmxlTAAIZGF0YVR5cGVxAH4AEEwACG1ldGFkYXRhcQB+ACRMAARuYW1lcQB+ABR4cABxAH4AGHEAfgApcQB+ACxzcQB+AEAAcQB+ABhxAH4AKXEAfgAxc3EAfgBAAHEAfgAYcQB+AClxAH4AM3NxAH4AQABxAH4AGHEAfgApcQB+ADVwcA==", + "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQ1hQsS9jxAO/gCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAAEdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAABHNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFxAH4AEXEAfgARc3IARm9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJvZHVjdEVuY29kZXIcqKluUDodYQIAA0wABmNsc1RhZ3EAfgAPTAAGZmllbGRzcQB+AAJMAAZzY2hlbWF0ACdMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0VHlwZTt4cHNyACZzY2FsYS5yZWZsZWN0LkNsYXNzVGFnJEdlbmVyaWNDbGFzc1RhZwAAAAAAAAABAgABTAAMcnVudGltZUNsYXNzdAARTGphdmEvbGFuZy9DbGFzczt4cHZyAAxzY2FsYS5UdXBsZTTIfhC+GxSuCgIABEwAAl8xcQB+AAFMAAJfMnEAfgABTAACXzNxAH4AAUwAAl80cQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NxAH4AIwBxAH4AEXEAfgApdAACXzNxAH4AL3EAfgAvc3EAfgAjAHEAfgARcQB+ACl0AAJfNHEAfgAvcQB+AC9zcgAsc2NhbGEuY29sbGVjdGlvbi5pbW11dGFibGUuTGlzdFNlcmlhbGl6ZUVuZCSKXGNb91MLbQIAAHhweHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RUeXBllANIqWlTBxgCAAlJAAlfaGFzaENvZGVCAAhiaXRtYXAkMFoAGWhhc0V4aXN0ZW5jZURlZmF1bHRWYWx1ZXNbABZleGlzdGVuY2VEZWZhdWx0VmFsdWVzcQB+AAhbABhleGlzdGVuY2VEZWZhdWx0c0JpdG1hc2t0AAJbWkwADWZpZWxkTmFtZXNTZXR0ACBMc2NhbGEvY29sbGVjdGlvbi9pbW11dGFibGUvU2V0O1sABmZpZWxkc3QAKVtMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0RmllbGQ7TAALbmFtZVRvRmllbGR0ABZMc2NhbGEvY29sbGVjdGlvbi9NYXA7TAALbmFtZVRvSW5kZXhxAH4APHhwAAAAAAAAcHBwdXIAKVtMb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQ7tWPEaGAaDUcCAAB4cAAAAARzcgAmb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQrgSSJZ9l3nwIABFoACG51bGxhYmxlTAAIZGF0YVR5cGVxAH4AEEwACG1ldGFkYXRhcQB+ACRMAARuYW1lcQB+ABR4cABxAH4AGHEAfgApcQB+ACxzcQB+AEAAcQB+ABhxAH4AKXEAfgAxc3EAfgBAAHEAfgAYcQB+AClxAH4AM3NxAH4AQABxAH4AGHEAfgApcQB+ADVwcA==", "nullable": true } } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.proto.bin index aa1971d3112fcbdacea1120789b10ca3a9ff1026..fe3efd5577b102c6e877216128a839d78c5433b1 100644 GIT binary patch delta 78 xcmewv{xf{TUT%&l%pNN ()." ] }, + "INVALID_COLUMN_DEFAULT_VALUE" : { + "message" : [ + "Cannot CREATE/REPLACE/ALTER TABLE because column has an invalid DEFAULT value: ." + ], + "subClass" : { + "INCOMPATIBLE_DATA_TYPE" : { + "message" : [ + "The default value has an incompatible data type with the column:", + "Default value: ", + "Column: " + ] + }, + "NOT_CONSTANT" : { + "message" : [ + "The default value must be a constant, or can be folded to a constant." + ] + } + } + }, "INVALID_COLUMN_NAME_AS_PATH" : { "message" : [ "The datasource cannot save the column because its name contains some characters that are not allowed in file paths. Please, use an alias to rename it." @@ -3617,21 +3636,11 @@ "Invalid DEFAULT value for column : fails to parse as a valid literal value." ] }, - "_LEGACY_ERROR_TEMP_1345" : { - "message" : [ - "Failed to execute command because DEFAULT values are not supported for target data source with table provider: \"\"." - ] - }, "_LEGACY_ERROR_TEMP_1346" : { "message" : [ "Failed to execute command because DEFAULT values are not supported when adding new columns to previously existing target data source with table provider: \"\"." ] }, - "_LEGACY_ERROR_TEMP_1347" : { - "message" : [ - "Failed to execute command because subquery expressions are not allowed in DEFAULT values." - ] - }, "_LEGACY_ERROR_TEMP_2000" : { "message" : [ ". If necessary set to false to bypass this error." diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java new file mode 100644 index 0000000000000..08ce6629d7f1f --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java @@ -0,0 +1,77 @@ +/* + * 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.connector.catalog; + +import javax.annotation.Nullable; + +import org.apache.spark.sql.internal.connector.ColumnImpl; +import org.apache.spark.sql.types.DataType; + +public interface Column { + + static Column create(String name, DataType dataType) { + return create(name, dataType, true); + } + + static Column create(String name, DataType dataType, boolean nullable) { + return create(name, dataType, nullable, null, null, null); + } + + static Column create( + String name, + DataType dataType, + boolean nullable, + String comment, + ColumnDefaultValue defaultValue, + String metadataInJSON) { + return new ColumnImpl(name, dataType, nullable, comment, defaultValue, metadataInJSON); + } + + /** + * Returns the name of this table column. + */ + String name(); + + /** + * Returns the data type of this table column. + */ + DataType dataType(); + + /** + * Returns true if this column may produce null values. + */ + boolean nullable(); + + /** + * Returns the comment of this table column. Null means no comment. + */ + @Nullable + String comment(); + + /** + * Returns the default value of this table column. Null means no default value. + */ + @Nullable + ColumnDefaultValue defaultValue(); + + /** + * Returns the column metadata in JSON format. + */ + @Nullable + String metadataInJSON(); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java new file mode 100644 index 0000000000000..9cf45f678a228 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java @@ -0,0 +1,61 @@ +/* + * 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.connector.catalog; + +import java.util.Objects; +import javax.annotation.Nonnull; + +import org.apache.spark.sql.connector.expressions.Literal; + +public class ColumnDefaultValue { + private String sql; + private Literal value; + + public ColumnDefaultValue(String sql, Literal value) { + this.sql = sql; + this.value = value; + } + + @Nonnull + public String getSql() { + return sql; + } + + @Nonnull + public Literal getValue() { + return value; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof ColumnDefaultValue)) return false; + ColumnDefaultValue that = (ColumnDefaultValue) o; + return sql.equals(that.sql) && value.equals(that.value); + } + + @Override + public int hashCode() { + return Objects.hash(sql, value); + } + + @Override + public String toString() { + return "ColumnDefaultValue{sql='" + sql + "\', value=" + value + '}'; + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java index 35455a0ed9975..4337a7c615208 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java @@ -54,6 +54,19 @@ @Evolving public interface StagingTableCatalog extends TableCatalog { + /** + * Stage the creation of a table, preparing it to be committed into the metastore. + *

+ * This is deprecated. Please override + * {@link #stageCreate(Identifier, Column[], Transform[], Map)} instead. + */ + @Deprecated + StagedTable stageCreate( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException; + /** * Stage the creation of a table, preparing it to be committed into the metastore. *

@@ -64,7 +77,7 @@ public interface StagingTableCatalog extends TableCatalog { * committed, an exception should be thrown by {@link StagedTable#commitStagedChanges()}. * * @param ident a table identifier - * @param schema the schema of the new table, as a struct type + * @param columns the column of the new table * @param partitions transforms to use for partitioning data in the table * @param properties a string map of table properties * @return metadata for the new table @@ -72,11 +85,26 @@ public interface StagingTableCatalog extends TableCatalog { * @throws UnsupportedOperationException If a requested partition transform is not supported * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) */ - StagedTable stageCreate( + default StagedTable stageCreate( + Identifier ident, + Column[] columns, + Transform[] partitions, + Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException { + return stageCreate(ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties); + } + + /** + * Stage the replacement of a table, preparing it to be committed into the metastore when the + * returned table's {@link StagedTable#commitStagedChanges()} is called. + *

+ * This is deprecated, please override + * {@link #stageReplace(Identifier, StructType, Transform[], Map)} instead. + */ + StagedTable stageReplace( Identifier ident, StructType schema, Transform[] partitions, - Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException; + Map properties) throws NoSuchNamespaceException, NoSuchTableException; /** * Stage the replacement of a table, preparing it to be committed into the metastore when the @@ -97,7 +125,7 @@ StagedTable stageCreate( * operation. * * @param ident a table identifier - * @param schema the schema of the new table, as a struct type + * @param columns the columns of the new table * @param partitions transforms to use for partitioning data in the table * @param properties a string map of table properties * @return metadata for the new table @@ -105,11 +133,27 @@ StagedTable stageCreate( * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) * @throws NoSuchTableException If the table does not exist */ - StagedTable stageReplace( + default StagedTable stageReplace( + Identifier ident, + Column[] columns, + Transform[] partitions, + Map properties) throws NoSuchNamespaceException, NoSuchTableException { + return stageReplace( + ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties); + } + + /** + * Stage the creation or replacement of a table, preparing it to be committed into the metastore + * when the returned table's {@link StagedTable#commitStagedChanges()} is called. + *

+ * This is deprecated, please override + * {@link #stageCreateOrReplace(Identifier, Column[], Transform[], Map)} instead. + */ + StagedTable stageCreateOrReplace( Identifier ident, StructType schema, Transform[] partitions, - Map properties) throws NoSuchNamespaceException, NoSuchTableException; + Map properties) throws NoSuchNamespaceException; /** * Stage the creation or replacement of a table, preparing it to be committed into the metastore @@ -129,16 +173,19 @@ StagedTable stageReplace( * the staged changes are committed but the table doesn't exist at commit time. * * @param ident a table identifier - * @param schema the schema of the new table, as a struct type + * @param columns the columns of the new table * @param partitions transforms to use for partitioning data in the table * @param properties a string map of table properties * @return metadata for the new table * @throws UnsupportedOperationException If a requested partition transform is not supported * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) */ - StagedTable stageCreateOrReplace( + default StagedTable stageCreateOrReplace( Identifier ident, - StructType schema, + Column[] columns, Transform[] partitions, - Map properties) throws NoSuchNamespaceException; + Map properties) throws NoSuchNamespaceException { + return stageCreateOrReplace( + ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties); + } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java index 8f7a87404837c..b9753a08aba96 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java @@ -51,9 +51,20 @@ public interface Table { /** * Returns the schema of this table. If the table is not readable and doesn't have a schema, an * empty schema can be returned here. + *

+ * This is deprecated. Please override {@link #columns} instead. */ + @Deprecated StructType schema(); + /** + * Returns the columns of this table. If the table is not readable and doesn't have a schema, an + * empty array can be returned here. + */ + default Column[] columns() { + return CatalogV2Util.structTypeToV2Columns(schema()); + } + /** * Returns the physical partitioning of this table. */ diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index b04c7e55138e1..82622d65205ec 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -159,11 +159,24 @@ default boolean tableExists(Identifier ident) { } } + /** + * Create a table in the catalog. + *

+ * This is deprecated. Please override + * {@link #createTable(Identifier, Column[], Transform[], Map)} instead. + */ + @Deprecated + Table createTable( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException; + /** * Create a table in the catalog. * * @param ident a table identifier - * @param schema the schema of the new table, as a struct type + * @param columns the columns of the new table. * @param partitions transforms to use for partitioning data in the table * @param properties a string map of table properties * @return metadata for the new table @@ -171,11 +184,13 @@ default boolean tableExists(Identifier ident) { * @throws UnsupportedOperationException If a requested partition transform is not supported * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) */ - Table createTable( + default Table createTable( Identifier ident, - StructType schema, + Column[] columns, Transform[] partitions, - Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException; + Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException { + return createTable(ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties); + } /** * Apply a set of {@link TableChange changes} to a table in the catalog. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java index cf735ed94521c..29dc7fb825789 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java @@ -22,6 +22,7 @@ import javax.annotation.Nullable; import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.connector.expressions.Literal; import org.apache.spark.sql.types.DataType; /** @@ -140,7 +141,7 @@ static TableChange addColumn( boolean isNullable, String comment, ColumnPosition position, - String defaultValue) { + ColumnDefaultValue defaultValue) { return new AddColumn(fieldNames, dataType, isNullable, comment, position, defaultValue); } @@ -228,7 +229,7 @@ static TableChange updateColumnPosition(String[] fieldNames, ColumnPosition newP * If the field does not exist, the change will result in an {@link IllegalArgumentException}. * * @param fieldNames field names of the column to update - * @param newDefaultValue the new default value + * @param newDefaultValue the new default value SQL string (Spark SQL dialect). * @return a TableChange for the update */ static TableChange updateColumnDefaultValue(String[] fieldNames, String newDefaultValue) { @@ -383,7 +384,9 @@ interface ColumnChange extends TableChange { } /** - * A TableChange to add a field. + * A TableChange to add a field. The implementation may need to back-fill all the existing data + * to add this new column, or remember the column default value specified here and let the reader + * to fill the column value when reading these existing data that do not have this column. *

* If the field already exists, the change must result in an {@link IllegalArgumentException}. * If the new field is nested and its parent does not exist or is not a struct, the change must @@ -395,7 +398,7 @@ final class AddColumn implements ColumnChange { private final boolean isNullable; private final String comment; private final ColumnPosition position; - private final String defaultValue; + private final ColumnDefaultValue defaultValue; private AddColumn( String[] fieldNames, @@ -403,7 +406,7 @@ private AddColumn( boolean isNullable, String comment, ColumnPosition position, - String defaultValue) { + ColumnDefaultValue defaultValue) { this.fieldNames = fieldNames; this.dataType = dataType; this.isNullable = isNullable; @@ -436,7 +439,7 @@ public ColumnPosition position() { } @Nullable - public String defaultValue() { return defaultValue; } + public ColumnDefaultValue defaultValue() { return defaultValue; } @Override public boolean equals(Object o) { @@ -691,6 +694,11 @@ public String[] fieldNames() { return fieldNames; } + /** + * Returns the column default value SQL string (Spark SQL dialect). The default value literal + * is not provided as updating column default values does not need to back-fill existing data. + */ + @Nullable public String newDefaultValue() { return newDefaultValue; } @Override diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9a2648a79a576..46deaf309e99f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1249,7 +1249,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } SubqueryAlias( catalog.name +: ident.asMultipartIdentifier, - StreamingRelationV2(None, table.name, table, options, table.schema.toAttributes, + StreamingRelationV2(None, table.name, table, options, table.columns.toAttributes, Some(catalog), Some(ident), v1Fallback)) } else { SubqueryAlias( @@ -3683,7 +3683,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case u: UnresolvedFieldName => resolveFieldNames(table, u.name, u) } - case a @ AddColumns(r: ResolvedTable, cols) if !a.resolved => + case a @ AddColumns(r: ResolvedTable, cols) => // 'colsToAdd' keeps track of new columns being added. It stores a mapping from a // normalized parent name of fields to field names that belong to the parent. // For example, if we add columns "a.b.c", "a.b.d", and "a.c", 'colsToAdd' will become @@ -3703,40 +3703,40 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor ResolvedFieldPosition(ColumnPosition.after(colName)) case None => throw QueryCompilationErrors.referenceColNotFoundForAlterTableChangesError( - col.colName, allFields) + col.column.name, allFields) } case _ => ResolvedFieldPosition(u.position) } case resolved => resolved } - colsToAdd(resolvedParentName) = fieldsAdded :+ col.colName + colsToAdd(resolvedParentName) = fieldsAdded :+ col.column.name resolvedPosition } - val schema = r.table.schema + val schema = r.table.columns.asSchema val resolvedCols = cols.map { col => col.path match { - case Some(parent: UnresolvedFieldName) => - // Adding a nested field, need to resolve the parent column and position. - val resolvedParent = resolveFieldNames(r, parent.name, parent) - val parentSchema = resolvedParent.field.dataType match { + case RootTableSchema => + // Adding to the root. Just need to resolve position. + val resolvedPosition = resolvePosition(col, schema, Nil) + col.copy(position = resolvedPosition) + case parent: ResolvedFieldName => + val parentSchema = parent.field.dataType match { case s: StructType => s case _ => throw QueryCompilationErrors.invalidFieldName( col.name, parent.name, parent.origin) } - val resolvedPosition = resolvePosition(col, parentSchema, resolvedParent.name) - col.copy(path = Some(resolvedParent), position = resolvedPosition) - case _ => - // Adding to the root. Just need to resolve position. - val resolvedPosition = resolvePosition(col, schema, Nil) + val resolvedPosition = resolvePosition(col, parentSchema, parent.name) col.copy(position = resolvedPosition) + // This should not happen. All `UnresolvedFieldName` should have been resolved before. + case _ => col } } val resolved = a.copy(columnsToAdd = resolvedCols) resolved.copyTagsFrom(a) resolved - case a @ AlterColumn( - table: ResolvedTable, ResolvedFieldName(path, field), dataType, _, _, position, _) => + case a @ AlterColumn(table: ResolvedTable, + ResolvedFieldName(path, field), dataType, _, _, position, _) => val newDataType = dataType.flatMap { dt => // Hive style syntax provides the column type, even if it may not have changed. val existing = CharVarcharUtils.getRawType(field.metadata).getOrElse(field.dataType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index e95c21ad985e6..98a52bde25927 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.trees.TreePattern.{LATERAL_COLUMN_ALIAS_REFERENCE, UNRESOLVED_WINDOW_EXPRESSION} -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils, TypeUtils} +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, ResolveDefaultColumns => DefaultColumnUtil, StringUtils, TypeUtils} import org.apache.spark.sql.connector.catalog.{LookupCatalog, SupportsPartitionManagement} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} import org.apache.spark.sql.internal.SQLConf @@ -240,6 +240,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB // general unresolved check below to throw a more tailored error message. ResolveReferencesInAggregate.checkUnresolvedGroupByAll(operator) + // Early checks for column default values, to produce better error messages + DefaultColumnUtil.checkDefaultValuesInPlan(operator) + getAllExpressions(operator).foreach(_.foreachUp { case a: Attribute if !a.resolved => failUnresolvedAttribute(operator, a, "UNRESOLVED_COLUMN") @@ -591,8 +594,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case create: V2CreateTablePlan => val references = create.partitioning.flatMap(_.references).toSet + val tableSchema = create.tableSchema val badReferences = references.map(_.fieldNames).flatMap { column => - create.tableSchema.findNestedField(column) match { + tableSchema.findNestedField(column) match { case Some(_) => None case _ => @@ -606,7 +610,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB messageParameters = Map("cols" -> badReferences.mkString(", "))) } - create.tableSchema.foreach(f => TypeUtils.failWithIntervalType(f.dataType)) + tableSchema.foreach(f => TypeUtils.failWithIntervalType(f.dataType)) case write: V2WriteCommand if write.resolved => write.query.schema.foreach(f => TypeUtils.failWithIntervalType(f.dataType)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index e6be5c2395551..412cc9cb11212 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -102,6 +102,7 @@ sealed trait FieldName extends LeafExpression with Unevaluable { } case class UnresolvedFieldName(name: Seq[String]) extends FieldName { + assert(name.length > 0) override lazy val resolved = false } @@ -181,7 +182,7 @@ object ResolvedTable { catalog: TableCatalog, identifier: Identifier, table: Table): ResolvedTable = { - val schema = CharVarcharUtils.replaceCharVarcharWithStringInSchema(table.schema) + val schema = CharVarcharUtils.replaceCharVarcharWithStringInSchema(table.columns.asSchema) ResolvedTable(catalog, identifier, table, schema.toAttributes) } } @@ -195,6 +196,10 @@ case class ResolvedFieldName(path: Seq[String], field: StructField) extends Fiel def name: Seq[String] = path :+ field.name } +case object RootTableSchema extends FieldName { + def name: Seq[String] = Nil +} + case class ResolvedFieldPosition(position: ColumnPosition) extends FieldPosition @@ -246,6 +251,7 @@ case class ResolvedIdentifier( catalog: CatalogPlugin, identifier: Identifier) extends LeafNodeWithoutStats { override def output: Seq[Attribute] = Nil + def name: String = (catalog.name +: identifier.namespace() :+ identifier.name()).quoted } // A fake v2 catalog to hold temp views. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index aea496b872bc2..b95fe5c7ba6f6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils, ResolveDefaultColumns} +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, getZoneId, stringToDate, stringToTimestamp, stringToTimestampWithoutTimeZone} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog} import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition @@ -2955,7 +2955,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Create top level table schema. */ protected def createSchema(ctx: CreateOrReplaceTableColTypeListContext): StructType = { - StructType(Option(ctx).toArray.flatMap(visitCreateOrReplaceTableColTypeList)) + val columns = Option(ctx).toArray.flatMap(visitCreateOrReplaceTableColTypeList) + StructType(columns.map { c => + if (c.defaultValue.isDefined) { + throw QueryParsingErrors.defaultColumnNotImplementedYetError(ctx) + } + val field = StructField(c.name, c.dataType, c.nullable) + c.comment.map(field.withComment).getOrElse(field) + }) } /** @@ -2988,17 +2995,18 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Create a [[StructType]] from a number of CREATE TABLE column definitions. */ override def visitCreateOrReplaceTableColTypeList( - ctx: CreateOrReplaceTableColTypeListContext): Seq[StructField] = withOrigin(ctx) { + ctx: CreateOrReplaceTableColTypeListContext): Seq[Column] = withOrigin(ctx) { ctx.createOrReplaceTableColType().asScala.map(visitCreateOrReplaceTableColType).toSeq } /** - * Create a top level [[StructField]] from a CREATE TABLE column definition. + * Create a top level column from a CREATE TABLE column definition. */ override def visitCreateOrReplaceTableColType( - ctx: CreateOrReplaceTableColTypeContext): StructField = withOrigin(ctx) { + ctx: CreateOrReplaceTableColTypeContext): Column = withOrigin(ctx) { import ctx._ + val name: String = colName.getText // Check that no duplicates exist among any CREATE TABLE column options specified. var nullable = true var defaultExpression: Option[DefaultExpressionContext] = None @@ -3007,49 +3015,35 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit if (option.NULL != null) { if (!nullable) { throw QueryParsingErrors.duplicateCreateTableColumnOption( - option, colName.getText, "NOT NULL") + option, name, "NOT NULL") } nullable = false } Option(option.defaultExpression()).foreach { expr => + if (!conf.getConf(SQLConf.ENABLE_DEFAULT_COLUMNS)) { + throw QueryParsingErrors.defaultColumnNotEnabledError(ctx) + } if (defaultExpression.isDefined) { throw QueryParsingErrors.duplicateCreateTableColumnOption( - option, colName.getText, "DEFAULT") + option, name, "DEFAULT") } defaultExpression = Some(expr) } Option(option.commentSpec()).foreach { spec => if (commentSpec.isDefined) { throw QueryParsingErrors.duplicateCreateTableColumnOption( - option, colName.getText, "COMMENT") + option, name, "COMMENT") } commentSpec = Some(spec) } } - val builder = new MetadataBuilder - // Add comment to metadata - commentSpec.map(visitCommentSpec).foreach { - builder.putString("comment", _) - } - // Add the 'DEFAULT expression' clause in the column definition, if any, to the column metadata. - defaultExpression.map(visitDefaultExpression).foreach { field => - if (conf.getConf(SQLConf.ENABLE_DEFAULT_COLUMNS)) { - // Add default to metadata - builder.putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, field) - builder.putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, field) - } else { - throw QueryParsingErrors.defaultColumnNotEnabledError(ctx) - } - } - - val name: String = colName.getText - - StructField( + Column( name = name, dataType = typedVisit[DataType](ctx.dataType), nullable = nullable, - metadata = builder.build()) + comment = commentSpec.map(visitCommentSpec), + defaultValue = defaultExpression.map(visitDefaultExpression)) } /** @@ -3100,20 +3094,17 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit string(visitStringLit(ctx.stringLit)) } - /** - * Create a default string. - */ - override def visitDefaultExpression(ctx: DefaultExpressionContext): String = withOrigin(ctx) { - val exprCtx = ctx.expression() - // Make sure it can be converted to Catalyst expressions. - expression(exprCtx) - // Extract the raw expression text so that we can save the user provided text. We don't - // use `Expression.sql` to avoid storing incorrect text caused by bugs in any expression's - // `sql` method. Note: `exprCtx.getText` returns a string without spaces, so we need to - // get the text from the underlying char stream instead. - val start = exprCtx.getStart.getStartIndex - val end = exprCtx.getStop.getStopIndex - exprCtx.getStart.getInputStream.getText(new Interval(start, end)) + override def visitDefaultExpression(ctx: DefaultExpressionContext): DefaultValueExpression = { + withOrigin(ctx) { + val exprCtx = ctx.expression() + // Extract the raw expression text so that we can save the user provided text. This will be + // used in error reporting. Note: `exprCtx.getText` returns a string without spaces, so we + // need to get the text from the underlying char stream instead. + val start = exprCtx.getStart.getStartIndex + val end = exprCtx.getStop.getStopIndex + val originalSQL = exprCtx.getStart.getInputStream.getText(new Interval(start, end)) + DefaultValueExpression(expression(exprCtx), originalSQL) + } } /** @@ -3248,7 +3239,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * types like `i INT`, which should be appended to the existing table schema. */ type TableClauses = ( - Seq[Transform], Seq[StructField], Option[BucketSpec], Map[String, String], + Seq[Transform], Seq[Column], Option[BucketSpec], Map[String, String], Map[String, String], Option[String], Option[String], Option[SerdeInfo]) /** @@ -3276,12 +3267,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Parse a list of transforms or columns. */ override def visitPartitionFieldList( - ctx: PartitionFieldListContext): (Seq[Transform], Seq[StructField]) = withOrigin(ctx) { + ctx: PartitionFieldListContext): (Seq[Transform], Seq[Column]) = withOrigin(ctx) { val (transforms, columns) = ctx.fields.asScala.map { case transform: PartitionTransformContext => (Some(visitPartitionTransform(transform)), None) case field: PartitionColumnContext => - (None, Some(visitColType(field.colType))) + val f = visitColType(field.colType) + val col = Column(f.name, f.dataType, f.nullable, f.getComment(), defaultValue = None) + (None, Some(col)) }.unzip (transforms.flatten.toSeq, columns.flatten.toSeq) @@ -3736,13 +3729,13 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit private def partitionExpressions( partTransforms: Seq[Transform], - partCols: Seq[StructField], + partCols: Seq[Column], ctx: ParserRuleContext): Seq[Transform] = { if (partTransforms.nonEmpty) { if (partCols.nonEmpty) { val references = partTransforms.map(_.describe()).mkString(", ") val columns = partCols - .map(field => s"${field.name} ${field.dataType.simpleString}") + .map(column => s"${column.name} ${column.dataType.simpleString}") .mkString(", ") operationNotAllowed( s"""PARTITION BY: Cannot mix partition expressions and partition columns: @@ -3830,9 +3823,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit case _ => // Note: table schema includes both the table columns list and the partition columns // with data type. - val schema = StructType(columns ++ partCols) CreateTable(UnresolvedIdentifier(table), - schema, partitioning, tableSpec, ignoreIfExists = ifNotExists) + columns ++ partCols, partitioning, tableSpec, ignoreIfExists = ifNotExists) } } @@ -3900,9 +3892,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit case _ => // Note: table schema includes both the table columns list and the partition columns // with data type. - val schema = StructType(columns ++ partCols) ReplaceTable(UnresolvedIdentifier(table), - schema, partitioning, tableSpec, orCreate = orCreate) + columns ++ partCols, partitioning, tableSpec, orCreate = orCreate) } } @@ -3982,20 +3973,21 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit override def visitQualifiedColTypeWithPosition( ctx: QualifiedColTypeWithPositionContext): QualifiedColType = withOrigin(ctx) { val name = typedVisit[Seq[String]](ctx.name) - // Add the 'DEFAULT expression' clause in the column definition, if any, to the column metadata. val defaultExpr = Option(ctx.defaultExpression()).map(visitDefaultExpression) if (defaultExpr.isDefined && !conf.getConf(SQLConf.ENABLE_DEFAULT_COLUMNS)) { throw QueryParsingErrors.defaultColumnNotEnabledError(ctx) } QualifiedColType( - path = if (name.length > 1) Some(UnresolvedFieldName(name.init)) else None, - colName = name.last, - dataType = typedVisit[DataType](ctx.dataType), - nullable = ctx.NULL == null, - comment = Option(ctx.commentSpec()).map(visitCommentSpec), + path = if (name.length > 1) UnresolvedFieldName(name.init) else RootTableSchema, position = Option(ctx.colPosition).map( pos => UnresolvedFieldPosition(typedVisit[ColumnPosition](pos))), - default = defaultExpr) + column = Column( + name = name.last, + dataType = typedVisit[DataType](ctx.dataType), + nullable = ctx.NULL == null, + comment = Option(ctx.commentSpec()).map(visitCommentSpec), + defaultValue = defaultExpr + )) } /** @@ -4040,6 +4032,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * ALTER TABLE table1 ALTER COLUMN a.b.c SET NOT NULL * ALTER TABLE table1 ALTER COLUMN a.b.c DROP NOT NULL * ALTER TABLE table1 ALTER COLUMN a.b.c COMMENT 'new comment' + * ALTER TABLE table1 ALTER COLUMN a.b.c SET DEFAULT value + * ALTER TABLE table1 ALTER COLUMN a.b.c DROP DEFAULT * ALTER TABLE table1 ALTER COLUMN a.b.c FIRST * ALTER TABLE table1 ALTER COLUMN a.b.c AFTER x * }}} @@ -4076,19 +4070,18 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } else { None } - val setDefaultExpression: Option[String] = - if (action.defaultExpression != null) { - Option(action.defaultExpression()).map(visitDefaultExpression) - } else if (action.dropDefault != null) { - Some("") - } else { - None - } - if (setDefaultExpression.isDefined && !conf.getConf(SQLConf.ENABLE_DEFAULT_COLUMNS)) { + val defaultExpression: Option[Expression] = if (action.defaultExpression != null) { + Some(visitDefaultExpression(action.defaultExpression)) + } else if (action.dropDefault != null) { + Some(DropDefaultColumnValue) + } else { + None + } + if (defaultExpression.isDefined && !conf.getConf(SQLConf.ENABLE_DEFAULT_COLUMNS)) { throw QueryParsingErrors.defaultColumnNotEnabledError(ctx) } - assert(Seq(dataType, nullable, comment, position, setDefaultExpression) + assert(Seq(dataType, nullable, comment, position, defaultExpression) .count(_.nonEmpty) == 1) AlterColumn( @@ -4098,7 +4091,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit nullable = nullable, comment = comment, position = position, - setDefaultExpression = setDefaultExpression) + defaultExpression = defaultExpression) } /** @@ -4134,7 +4127,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit comment = Option(ctx.colType().commentSpec()).map(visitCommentSpec), position = Option(ctx.colPosition).map( pos => UnresolvedFieldPosition(typedVisit[ColumnPosition](pos))), - setDefaultExpression = None) + defaultExpression = None) } override def visitHiveReplaceColumns( @@ -4154,7 +4147,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit "Column position", "REPLACE COLUMNS", ctx) } val col = typedVisit[QualifiedColType](colType) - if (col.path.isDefined) { + if (col.path != RootTableSchema) { throw QueryParsingErrors.operationInHiveStyleCommandUnsupportedError( "Replacing with a nested column", "REPLACE COLUMNS", ctx) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index af70f07bc8792..9fbb5892e405f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -17,11 +17,9 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.analysis.{FieldName, FieldPosition} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.trees.{LeafLike, UnaryLike} import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.types.DataType /** * A logical plan node that contains exactly what was parsed from SQL. @@ -121,22 +119,6 @@ object SerdeInfo { } } -/** - * Column data as parsed by ALTER TABLE ... (ADD|REPLACE) COLUMNS. - */ -case class QualifiedColType( - path: Option[FieldName], - colName: String, - dataType: DataType, - nullable: Boolean, - comment: Option[String], - position: Option[FieldPosition], - default: Option[String]) { - def name: Seq[String] = path.map(_.name).getOrElse(Nil) :+ colName - - def resolved: Boolean = path.forall(_.resolved) && position.forall(_.resolved) -} - /** * An INSERT INTO statement, as parsed from SQL. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala index 94f2a57066356..39b3211bb819d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.analysis.{FieldName, FieldPosition} +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.analysis.{FieldName, FieldPosition, ResolvedTable, RootTableSchema} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression, Unevaluable} import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.connector.catalog.{TableCatalog, TableChange} import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.{DataType, NullType} /** * The base trait for commands that need to alter a v2 table with [[TableChange]]s. @@ -31,6 +33,10 @@ trait AlterTableCommand extends UnaryCommand { def changes: Seq[TableChange] def table: LogicalPlan final override def child: LogicalPlan = table + def resolvedTable: ResolvedTable = { + assert(childrenResolved) + table.asInstanceOf[ResolvedTable] + } } /** @@ -96,6 +102,30 @@ case class UnsetTableProperties( copy(table = newChild) } +/** + * Column data as parsed by ALTER TABLE ... (ADD|REPLACE) COLUMNS. + */ +case class QualifiedColType( + path: FieldName, + column: Column, + position: Option[FieldPosition] = None) extends Expression with Unevaluable { + def name: Seq[String] = path.name :+ column.name + override def children: Seq[Expression] = path +: column +: position.toSeq + + override def dataType: DataType = throw SparkException.internalError( + "QualifiedColType.dataType should not be called.") + override def nullable: Boolean = throw SparkException.internalError( + "QualifiedColType.nullable should not be called.") + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): Expression = { + copy( + newChildren(0).asInstanceOf[FieldName], + newChildren(1).asInstanceOf[Column], + newChildren.drop(2).headOption.map(_.asInstanceOf[FieldPosition])) + } +} + /** * The logical plan of the ALTER TABLE ... ADD COLUMNS command. */ @@ -103,24 +133,22 @@ case class AddColumns( table: LogicalPlan, columnsToAdd: Seq[QualifiedColType]) extends AlterTableCommand { columnsToAdd.foreach { c => - TypeUtils.failWithIntervalType(c.dataType) + TypeUtils.failWithIntervalType(c.column.dataType) } - override lazy val resolved: Boolean = table.resolved && columnsToAdd.forall(_.resolved) - override def changes: Seq[TableChange] = { columnsToAdd.map { col => - require(col.path.forall(_.resolved), + require(col.path.resolved, "FieldName should be resolved before it's converted to TableChange.") require(col.position.forall(_.resolved), "FieldPosition should be resolved before it's converted to TableChange.") TableChange.addColumn( col.name.toArray, - col.dataType, - col.nullable, - col.comment.orNull, + col.column.dataType, + col.column.nullable, + col.column.comment.orNull, col.position.map(_.position).orNull, - col.default.orNull) + col.column.defaultValue.map(_.toV2).orNull) } } @@ -135,11 +163,9 @@ case class ReplaceColumns( table: LogicalPlan, columnsToAdd: Seq[QualifiedColType]) extends AlterTableCommand { columnsToAdd.foreach { c => - TypeUtils.failWithIntervalType(c.dataType) + TypeUtils.failWithIntervalType(c.column.dataType) } - override lazy val resolved: Boolean = table.resolved && columnsToAdd.forall(_.resolved) - override def changes: Seq[TableChange] = { // REPLACE COLUMNS deletes all the existing columns and adds new columns specified. require(table.resolved) @@ -148,15 +174,15 @@ case class ReplaceColumns( TableChange.deleteColumn(Array(name), false /* ifExists */) } val addChanges = columnsToAdd.map { col => - assert(col.path.isEmpty) + assert(col.path == RootTableSchema) assert(col.position.isEmpty) TableChange.addColumn( col.name.toArray, - col.dataType, - col.nullable, - col.comment.orNull, + col.column.dataType, + col.column.nullable, + col.column.comment.orNull, null, - col.default.orNull) + col.column.defaultValue.map(_.toV2).orNull) } deleteChanges ++ addChanges } @@ -199,6 +225,12 @@ case class RenameColumn( copy(table = newChild) } +// A fake expression to indicate a drop column default value action in `AlterColumn`. +case object DropDefaultColumnValue extends LeafExpression with Unevaluable { + override def nullable: Boolean = true + override def dataType: DataType = NullType +} + /** * The logical plan of the ALTER TABLE ... ALTER COLUMN command. */ @@ -209,7 +241,9 @@ case class AlterColumn( nullable: Option[Boolean], comment: Option[String], position: Option[FieldPosition], - setDefaultExpression: Option[String]) extends AlterTableCommand { + defaultExpression: Option[Expression]) extends AlterTableCommand { + assert(column != RootTableSchema, "AlterTable.column must be a real (nested) column.") + override def changes: Seq[TableChange] = { require(column.resolved, "FieldName should be resolved before it's converted to TableChange.") val colName = column.name.toArray @@ -227,8 +261,13 @@ case class AlterColumn( "FieldPosition should be resolved before it's converted to TableChange.") TableChange.updateColumnPosition(colName, newPosition.position) } - val defaultValueChange = setDefaultExpression.map { newDefaultExpression => - TableChange.updateColumnDefaultValue(colName, newDefaultExpression) + val defaultValueChange = defaultExpression.map { + case DropDefaultColumnValue => + TableChange.updateColumnDefaultValue(colName, null) + case d: DefaultValueExpression => + TableChange.updateColumnDefaultValue(colName, d.originalSQL) + case other => throw SparkException.internalError( + "Unexpected expression in AlterColumn.defaultExpression: " + other) } typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange ++ defaultValueChange } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 9508b2fb99336..247af2c058e16 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -17,19 +17,21 @@ package org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, EliminateSubqueryAliases, FieldName, NamedRelation, PartitionSpec, ResolvedIdentifier, UnresolvedException} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.FunctionResource -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, MetadataAttribute, NamedExpression, Unevaluable, V2ExpressionUtils} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, Literal, MetadataAttribute, NamedExpression, UnaryExpression, Unevaluable, V2ExpressionUtils} import org.apache.spark.sql.catalyst.plans.DescribeCommandSchema import org.apache.spark.sql.catalyst.trees.BinaryLike import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, RowDeltaUtils, WriteDeltaProjections} -import org.apache.spark.sql.connector.catalog._ -import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.catalog.{Column => V2Column, ColumnDefaultValue} +import org.apache.spark.sql.connector.expressions.{LiteralValue, Transform} import org.apache.spark.sql.connector.expressions.filter.Predicate import org.apache.spark.sql.connector.write.{DeltaWrite, RowLevelOperation, RowLevelOperationTable, SupportsDelta, Write} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.internal.connector.ColumnImpl import org.apache.spark.sql.types.{BooleanType, DataType, IntegerType, MetadataBuilder, StringType, StructField, StructType} // For v2 DML commands, it may end up with the v1 fallback code path and need to build a DataFrame @@ -391,9 +393,18 @@ case class WriteDelta( /** A trait used for logical plan nodes that create or replace V2 table definitions. */ trait V2CreateTablePlan extends LogicalPlan { - def tableName: Identifier + def name: LogicalPlan + def tableSpec: TableSpec def partitioning: Seq[Transform] - def tableSchema: StructType + def columns: Seq[Column] + def tableSchema: StructType = StructType(columns.map { col => + // Schema only cares about the tree structure with name and data type. + StructField(col.name, col.dataType, col.nullable) + }) + def resolvedName: ResolvedIdentifier = { + assert(childrenResolved) + name.asInstanceOf[ResolvedIdentifier] + } /** * Creates a copy of this node with the new partitioning transforms. This method is used to @@ -402,23 +413,66 @@ trait V2CreateTablePlan extends LogicalPlan { def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan } +/** + * A fake expression to hold the column default value expression and its original SQL text. + */ +case class DefaultValueExpression(child: Expression, originalSQL: String) + extends UnaryExpression with Unevaluable { + override def dataType: DataType = child.dataType + override protected def withNewChildInternal(newChild: Expression): Expression = + copy(child = newChild) + + // Convert the default expression to ColumnDefaultValue, which is required by DS v2 APIs. + def toV2: ColumnDefaultValue = child match { + case Literal(value, dataType) => + new ColumnDefaultValue(originalSQL, LiteralValue(value, dataType)) + // Analyzer makes sure the column default value is a constant. + case other => throw SparkException.internalError( + "Default value must be a literal, but got " + other) + } +} + +/** + * Column definition for tables. This is an expression so that analyzer can resolve the default + * value expression in DDL commands automatically. + */ +case class Column( + name: String, + dataType: DataType, + nullable: Boolean = true, + comment: Option[String] = None, + defaultValue: Option[DefaultValueExpression] = None, + metadataInJSON: Option[String] = None) extends Expression with Unevaluable { + override def children: Seq[Expression] = defaultValue.toSeq + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): Expression = { + copy(defaultValue = newChildren.headOption.map(_.asInstanceOf[DefaultValueExpression])) + } + + def toV2Column: V2Column = { + ColumnImpl( + name, + dataType, + nullable, + comment.orNull, + defaultValue.map(_.toV2).orNull, + metadataInJSON.orNull) + } +} + /** * Create a new table with a v2 catalog. */ case class CreateTable( name: LogicalPlan, - tableSchema: StructType, + columns: Seq[Column], partitioning: Seq[Transform], tableSpec: TableSpec, ignoreIfExists: Boolean) extends UnaryCommand with V2CreateTablePlan { override def child: LogicalPlan = name - override def tableName: Identifier = { - assert(child.resolved) - child.asInstanceOf[ResolvedIdentifier].identifier - } - override protected def withNewChildInternal(newChild: LogicalPlan): V2CreateTablePlan = copy(name = newChild) @@ -440,15 +494,11 @@ case class CreateTableAsSelect( analyzedQuery: Option[LogicalPlan] = None) extends BinaryCommand with V2CreateTablePlan with KeepAnalyzedQuery { + override def columns: Seq[Column] = query.schema.toColumns override def tableSchema: StructType = query.schema override def left: LogicalPlan = name override def right: LogicalPlan = query - override def tableName: Identifier = { - assert(left.resolved) - left.asInstanceOf[ResolvedIdentifier].identifier - } - override lazy val resolved: Boolean = childrenResolved && { // the table schema is created from the query schema, so the only resolution needed is to check // that the columns referenced by the table's partitioning exist in the query schema @@ -479,18 +529,13 @@ case class CreateTableAsSelect( */ case class ReplaceTable( name: LogicalPlan, - tableSchema: StructType, + columns: Seq[Column], partitioning: Seq[Transform], tableSpec: TableSpec, orCreate: Boolean) extends UnaryCommand with V2CreateTablePlan { override def child: LogicalPlan = name - override def tableName: Identifier = { - assert(child.resolved) - child.asInstanceOf[ResolvedIdentifier].identifier - } - override protected def withNewChildInternal(newChild: LogicalPlan): V2CreateTablePlan = copy(name = newChild) @@ -515,6 +560,7 @@ case class ReplaceTableAsSelect( analyzedQuery: Option[LogicalPlan] = None) extends BinaryCommand with V2CreateTablePlan with KeepAnalyzedQuery { + override def columns: Seq[Column] = query.schema.toColumns override def tableSchema: StructType = query.schema override def left: LogicalPlan = name override def right: LogicalPlan = query @@ -526,11 +572,6 @@ case class ReplaceTableAsSelect( references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) } - override def tableName: Identifier = { - assert(name.resolved) - name.asInstanceOf[ResolvedIdentifier].identifier - } - override def storeAnalyzedQuery(): Command = copy(analyzedQuery = Some(query)) override protected def withNewChildrenInternal( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala index 667c0988d0cc3..780cc4e08565c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.util +import java.util.{Map => JMap} + import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException @@ -25,11 +27,10 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.{Literal => ExprLiteral} -import org.apache.spark.sql.catalyst.optimizer.ConstantFolding -import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION -import org.apache.spark.sql.connector.catalog.{CatalogManager, FunctionCatalog, Identifier} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, FunctionCatalog, Identifier, TableCatalog} import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf @@ -68,77 +69,9 @@ object ResolveDefaultColumns { val CURRENT_DEFAULT_COLUMN_NAME = "DEFAULT" /** - * Finds "current default" expressions in CREATE/REPLACE TABLE columns and constant-folds them. - * - * The results are stored in the "exists default" metadata of the same columns. For example, in - * the event of this statement: - * - * CREATE TABLE T(a INT, b INT DEFAULT 5 + 5) - * - * This method constant-folds the "current default" value, stored in the CURRENT_DEFAULT metadata - * of the "b" column, to "10", storing the result in the "exists default" value within the - * EXISTS_DEFAULT metadata of that same column. Meanwhile the "current default" metadata of this - * "b" column retains its original value of "5 + 5". - * - * The reason for constant-folding the EXISTS_DEFAULT is to make the end-user visible behavior the - * same, after executing an ALTER TABLE ADD COLUMNS command with DEFAULT value, as if the system - * had performed an exhaustive backfill of the provided value to all previously existing rows in - * the table instead. We choose to avoid doing such a backfill because it would be a - * time-consuming and costly operation. Instead, we elect to store the EXISTS_DEFAULT in the - * column metadata for future reference when querying data out of the data source. In turn, each - * data source then takes responsibility to provide the constant-folded value in the - * EXISTS_DEFAULT metadata for such columns where the value is not present in storage. - * - * @param tableSchema represents the names and types of the columns of the statement to process. - * @param tableProvider provider of the target table to store default values for, if any. - * @param statementType name of the statement being processed, such as INSERT; useful for errors. - * @param addNewColumnToExistingTable true if the statement being processed adds a new column to - * a table that already exists. - * @return a copy of `tableSchema` with field metadata updated with the constant-folded values. - */ - def constantFoldCurrentDefaultsToExistDefaults( - tableSchema: StructType, - tableProvider: Option[String], - statementType: String, - addNewColumnToExistingTable: Boolean): StructType = { - if (SQLConf.get.enableDefaultColumns) { - val keywords: Array[String] = - SQLConf.get.getConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS) - .toLowerCase().split(",").map(_.trim) - val allowedTableProviders: Array[String] = - keywords.map(_.stripSuffix("*")) - val addColumnExistingTableBannedProviders: Array[String] = - keywords.filter(_.endsWith("*")).map(_.stripSuffix("*")) - val givenTableProvider: String = tableProvider.getOrElse("").toLowerCase() - val newFields: Seq[StructField] = tableSchema.fields.map { field => - if (field.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY)) { - // Make sure that the target table has a provider that supports default column values. - if (!allowedTableProviders.contains(givenTableProvider)) { - throw QueryCompilationErrors - .defaultReferencesNotAllowedInDataSource(statementType, givenTableProvider) - } - if (addNewColumnToExistingTable && - givenTableProvider.nonEmpty && - addColumnExistingTableBannedProviders.contains(givenTableProvider)) { - throw QueryCompilationErrors - .addNewDefaultColumnToExistingTableNotAllowed(statementType, givenTableProvider) - } - val analyzed: Expression = analyze(field, statementType) - val newMetadata: Metadata = new MetadataBuilder().withMetadata(field.metadata) - .putString(EXISTS_DEFAULT_COLUMN_METADATA_KEY, analyzed.sql).build() - field.copy(metadata = newMetadata) - } else { - field - } - } - StructType(newFields) - } else { - tableSchema - } - } - - /** - * Parses and analyzes the DEFAULT column text in `field`, returning an error upon failure. + * Parses and analyzes the DEFAULT column text in `field`. The default value has already been + * validated in CREATE/REPLACE/ALTER TABLE commands. We don't need to validate it again when + * reading it out. * * @param field represents the DEFAULT column value whose "default" metadata to parse * and analyze. @@ -153,46 +86,19 @@ object ResolveDefaultColumns { metadataKey: String = CURRENT_DEFAULT_COLUMN_METADATA_KEY): Expression = { // Parse the expression. val colText: String = field.metadata.getString(metadataKey) - lazy val parser = new CatalystSqlParser() - val parsed: Expression = try { - parser.parseExpression(colText) - } catch { - case ex: ParseException => - throw new AnalysisException( - s"Failed to execute $statementType command because the destination table column " + - s"${field.name} has a DEFAULT value of $colText which fails to parse as a valid " + - s"expression: ${ex.getMessage}") - } - // Check invariants before moving on to analysis. - if (parsed.containsPattern(PLAN_EXPRESSION)) { - throw QueryCompilationErrors.defaultValuesMayNotContainSubQueryExpressions() - } + val parser = new CatalystSqlParser() + val parsed: Expression = parser.parseExpression(colText) // Analyze the parse result. - val plan = try { - val analyzer: Analyzer = DefaultColumnAnalyzer - val analyzed = analyzer.execute(Project(Seq(Alias(parsed, field.name)()), OneRowRelation())) - analyzer.checkAnalysis(analyzed) - ConstantFolding(analyzed) - } catch { - case ex: AnalysisException => - throw new AnalysisException( - s"Failed to execute $statementType command because the destination table column " + - s"${field.name} has a DEFAULT value of $colText which fails to resolve as a valid " + - s"expression: ${ex.getMessage}") - } + val analyzer: Analyzer = DefaultColumnAnalyzer + val plan = analyzer.execute(Project(Seq(Alias(parsed, field.name)()), OneRowRelation())) val analyzed: Expression = plan.collectFirst { case Project(Seq(a: Alias), OneRowRelation()) => a.child }.get // Perform implicit coercion from the provided expression type to the required column type. if (field.dataType == analyzed.dataType) { analyzed - } else if (Cast.canUpCast(analyzed.dataType, field.dataType)) { - Cast(analyzed, field.dataType) } else { - throw new AnalysisException( - s"Failed to execute $statementType command because the destination table column " + - s"${field.name} has a DEFAULT value with type ${field.dataType}, but the " + - s"statement provided a value of incompatible type ${analyzed.dataType}") + Cast(analyzed, field.dataType) } } /** @@ -284,6 +190,100 @@ object ResolveDefaultColumns { rows.toSeq } + def checkDefaultValuesInPlan(plan: LogicalPlan, isForV1: Boolean = false): Unit = { + plan match { + // Do not check anything if the children are not resolved yet. + case _ if !plan.childrenResolved => + case AlterColumn(t: ResolvedTable, col: ResolvedFieldName, _, _, _, _, + Some(default: DefaultValueExpression)) => + checkTableProvider(t.catalog, t.name, getTableProviderFromProp(t.table.properties())) + checkDefaultValue(default, t.name, col.name, col.field.dataType, isForV1) + + case cmd: V2CreateTablePlan if cmd.columns.exists(_.defaultValue.isDefined) => + val ident = cmd.resolvedName + checkTableProvider(ident.catalog, ident.name, cmd.tableSpec.provider) + cmd.columns.filter(_.defaultValue.isDefined).foreach { col => + val Column(name, dataType, _, _, Some(default), _) = col + // CREATE/REPLACE TABLE only has top-level columns + val colName = Seq(name) + checkDefaultValue(default, ident.name, colName, dataType, isForV1) + } + + case cmd: AlterTableCommand => + val table = cmd.resolvedTable + cmd.transformExpressionsDown { + case q @ QualifiedColType(path, Column(name, dataType, _, _, Some(default), _), _) + if path.resolved => + checkTableProvider( + table.catalog, table.name, getTableProviderFromProp(table.table.properties())) + checkDefaultValue( + default, + table.name, + path.name :+ name, + dataType, + isForV1) + q + } + + case _ => + } + } + + private def getTableProviderFromProp(props: JMap[String, String]): Option[String] = { + Option(props.get(TableCatalog.PROP_PROVIDER)) + } + + private def checkTableProvider( + catalog: CatalogPlugin, + tableName: String, + provider: Option[String]): Unit = { + // We only need to check table provider for the session catalog. Other custom v2 catalogs + // can check table providers in their implementations of createTable, alterTable, etc. + if (CatalogV2Util.isSessionCatalog(catalog)) { + val conf = SQLConf.get + val allowedProviders: Array[String] = conf.getConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS) + .toLowerCase().split(",").map(_.trim) + val providerName = provider.getOrElse(conf.defaultDataSourceName).toLowerCase() + if (!allowedProviders.contains(providerName)) { + throw QueryCompilationErrors.defaultReferencesNotAllowedInDataSource(tableName) + } + } + } + + private def checkDefaultValue( + default: DefaultValueExpression, + tblName: String, + colName: Seq[String], + targetType: DataType, + isForV1: Boolean): Unit = { + if (default.resolved) { + if (!default.child.foldable) { + throw QueryCompilationErrors.notConstantDefaultValueError( + tblName, colName, default.originalSQL) + } + if (!Cast.canUpCast(default.child.dataType, targetType)) { + throw QueryCompilationErrors.incompatibleTypeDefaultValueError( + tblName, colName, targetType, default.child, default.originalSQL) + } + } else { + // Ideally we should let the rest of `CheckAnalysis` to report errors about why the default + // expression is unresolved. But we should report a better error here if the default + // expression references columns or contains subquery expressions, which means it's not a + // constant for sure. + if (default.references.nonEmpty || default.containsPattern(PLAN_EXPRESSION)) { + throw QueryCompilationErrors.notConstantDefaultValueError( + tblName, colName, default.originalSQL) + } + // When converting to v1 commands, the plan is not fully resolved and we can't do a complete + // analysis check. There is no "rest of CheckAnalysis" to report better errors and we must + // fail here. This is temporary and we can remove it when using v2 commands by default. + if (isForV1) { + throw QueryCompilationErrors.notConstantDefaultValueError( + tblName, colName, default.originalSQL) + } + } + } + /** * This is an Analyzer for processing default column values using built-in functions only. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 0c9282f9675fa..12858887bb5b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -21,10 +21,12 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIfNeeded import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, LogicalExpressions, Transform} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.types.StructType /** * Conversion helpers for working with v2 [[CatalogPlugin]]. @@ -183,6 +185,11 @@ private[sql] object CatalogV2Implicits { } } + implicit class ColumnsHelper(columns: Array[Column]) { + def asSchema: StructType = CatalogV2Util.v2ColumnsToStructType(columns) + def toAttributes: Seq[AttributeReference] = asSchema.toAttributes + } + def parseColumnPath(name: String): Seq[String] = { CatalystSqlParser.parseMultipartIdentifier(name) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 72c557c8d7726..b2b805ce16bb2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -23,12 +23,12 @@ import java.util.Collections import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.analysis.{AsOfTimestamp, AsOfVersion, NamedRelation, NoSuchDatabaseException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchTableException, TimeTravelSpec} +import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{SerdeInfo, TableSpec} -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, MapType, Metadata, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils @@ -133,32 +133,27 @@ private[sql] object CatalogV2Util { */ def applySchemaChanges( schema: StructType, - changes: Seq[TableChange], - tableProvider: Option[String], - statementType: String): StructType = { + changes: Seq[TableChange]): StructType = { changes.foldLeft(schema) { (schema, change) => change match { case add: AddColumn => add.fieldNames match { case Array(name) => val field = StructField(name, add.dataType, nullable = add.isNullable) - val fieldWithDefault: StructField = - Option(add.defaultValue).map(field.withCurrentDefaultValue).getOrElse(field) + val fieldWithDefault: StructField = encodeDefaultValue(add.defaultValue(), field) val fieldWithComment: StructField = Option(add.comment).map(fieldWithDefault.withComment).getOrElse(fieldWithDefault) - addField(schema, fieldWithComment, add.position(), tableProvider, statementType, true) + addField(schema, fieldWithComment, add.position()) case names => replace(schema, names.init, parent => parent.dataType match { case parentType: StructType => val field = StructField(names.last, add.dataType, nullable = add.isNullable) - val fieldWithDefault: StructField = - Option(add.defaultValue).map(field.withCurrentDefaultValue).getOrElse(field) + val fieldWithDefault: StructField = encodeDefaultValue(add.defaultValue(), field) val fieldWithComment: StructField = Option(add.comment).map(fieldWithDefault.withComment) .getOrElse(fieldWithDefault) Some(parent.copy(dataType = - addField(parentType, fieldWithComment, add.position(), tableProvider, - statementType, true))) + addField(parentType, fieldWithComment, add.position()))) case _ => throw new IllegalArgumentException(s"Not a struct: ${names.init.last}") }) @@ -188,8 +183,7 @@ private[sql] object CatalogV2Util { throw new IllegalArgumentException("Field not found: " + name) } val withFieldRemoved = StructType(struct.fields.filter(_ != oldField)) - addField(withFieldRemoved, oldField, update.position(), tableProvider, statementType, - false) + addField(withFieldRemoved, oldField, update.position()) } update.fieldNames() match { @@ -209,8 +203,10 @@ private[sql] object CatalogV2Util { // The new DEFAULT value string will be non-empty for any DDL commands that set the // default value, such as "ALTER TABLE t ALTER COLUMN c SET DEFAULT ..." (this is // enforced by the parser). On the other hand, commands that drop the default value such - // as "ALTER TABLE t ALTER COLUMN c DROP DEFAULT" will set this string to empty. - if (update.newDefaultValue().nonEmpty) { + // as "ALTER TABLE t ALTER COLUMN c DROP DEFAULT" will set this string to null. + // Note: we should only update the "current default", as the previous "exist default" + // should still be applied when reading existing data files without the column. + if (update.newDefaultValue() != null) { Some(field.withCurrentDefaultValue(update.newDefaultValue())) } else { Some(field.clearCurrentDefaultValue) @@ -229,11 +225,8 @@ private[sql] object CatalogV2Util { private def addField( schema: StructType, field: StructField, - position: ColumnPosition, - tableProvider: Option[String], - statementType: String, - addNewColumnToExistingTable: Boolean): StructType = { - val newSchema: StructType = if (position == null) { + position: ColumnPosition): StructType = { + if (position == null) { schema.add(field) } else if (position.isInstanceOf[First]) { StructType(field +: schema.fields) @@ -246,8 +239,6 @@ private[sql] object CatalogV2Util { val (before, after) = schema.fields.splitAt(fieldIndex + 1) StructType(before ++ (field +: after)) } - constantFoldCurrentDefaultsToExistDefaults( - newSchema, tableProvider, statementType, addNewColumnToExistingTable) } private def replace( @@ -431,4 +422,40 @@ private[sql] object CatalogV2Util { .getOrElse(catalogManager.v2SessionCatalog) .asTableCatalog } + + def v2ColumnsToStructType(columns: Array[Column]): StructType = { + StructType(columns.map(v2ColumnToStructField)) + } + + def v2ColumnToStructField(col: Column): StructField = { + val metadata = Option(col.metadataInJSON()).map(Metadata.fromJson).getOrElse(Metadata.empty) + var f = StructField(col.name(), col.dataType(), col.nullable(), metadata) + Option(col.comment()).foreach { comment => + f = f.withComment(comment) + } + Option(col.defaultValue()).foreach { default => + f = encodeDefaultValue(default, f) + } + f + } + + // For built-in file sources, we encode the default value in StructField metadata. An analyzer + // rule will check the special metadata and change the DML input plan to fill the default value. + private def encodeDefaultValue(defaultValue: ColumnDefaultValue, f: StructField): StructField = { + Option(defaultValue).map { default => + // The "exist default" is used to back-fill the existing data when new columns are added, and + // should be a fixed value which was evaluated at the definition time. For example, if the + // default value is `current_date()`, the "exist default" should be the value of + // `current_date()` when the column was defined/altered, instead of when back-fall happens. + // Note: the back-fill here is a logical concept. The data source can keep the existing + // data unchanged and let the data reader to return "exist default" for missing + // columns. + val existingDefault = Literal(default.getValue.value(), default.getValue.dataType()).sql + f.withExistenceDefaultValue(existingDefault).withCurrentDefaultValue(default.getSql) + }.getOrElse(f) + } + + def structTypeToV2Columns(schema: StructType): Array[Column] = { + schema.toColumns.map(_.toV2Column) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/RowLevelOperationTable.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/RowLevelOperationTable.scala index d1f7ba000c62a..07acacd9a35d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/RowLevelOperationTable.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/RowLevelOperationTable.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.connector.write import java.util -import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsRowLevelOperations, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.catalog.{Column, SupportsRead, SupportsRowLevelOperations, SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -38,6 +38,7 @@ private[sql] case class RowLevelOperationTable( override def name: String = table.name override def schema: StructType = table.schema + override def columns: Array[Column] = table.columns() override def capabilities: util.Set[TableCapability] = table.capabilities override def toString: String = table.toString 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 74dd5879061fe..c927cad9d8211 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 @@ -3148,28 +3148,40 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { "defaultValue" -> defaultValue)) } - def defaultReferencesNotAllowedInDataSource( - statementType: String, dataSource: String): Throwable = { + def defaultReferencesNotAllowedInDataSource(tableName: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1345", + errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", messageParameters = Map( - "statementType" -> statementType, - "dataSource" -> dataSource)) + "tableName" -> tableName, + "operation" -> "column default value")) } - def addNewDefaultColumnToExistingTableNotAllowed( - statementType: String, dataSource: String): Throwable = { + def notConstantDefaultValueError( + tableName: String, + colName: Seq[String], + defaultValue: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1346", + errorClass = "INVALID_COLUMN_DEFAULT_VALUE.NOT_CONSTANT", messageParameters = Map( - "statementType" -> statementType, - "dataSource" -> dataSource)) + "tableName" -> tableName, + "colName" -> toSQLId(colName), + "defaultValue" -> defaultValue)) } - def defaultValuesMayNotContainSubQueryExpressions(): Throwable = { + def incompatibleTypeDefaultValueError( + tableName: String, + colName: Seq[String], + colType: DataType, + defaultExpression: Expression, + defaultValue: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1347", - messageParameters = Map.empty) + errorClass = "INVALID_COLUMN_DEFAULT_VALUE.INCOMPATIBLE_DATA_TYPE", + messageParameters = Map( + "tableName" -> tableName, + "colName" -> toSQLId(colName), + "defaultValue" -> defaultValue, + "defaultValueType" -> toSQLType(defaultExpression.dataType), + "colType" -> toSQLType(colType))) } def nullableColumnOrFieldError(name: Seq[String]): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 3db2ec6b8d9c3..94f6e3e31eef9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -196,9 +196,10 @@ object DataSourceV2Relation { catalog: Option[CatalogPlugin], identifier: Option[Identifier], options: CaseInsensitiveStringMap): DataSourceV2Relation = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ // The v2 source may return schema containing char/varchar type. We replace char/varchar // with "annotated" string type here as the query engine doesn't support char/varchar yet. - val schema = CharVarcharUtils.replaceCharVarcharWithStringInSchema(table.schema) + val schema = CharVarcharUtils.replaceCharVarcharWithStringInSchema(table.columns.asSchema) DataSourceV2Relation(table, schema.toAttributes, catalog, identifier, options) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8d8aacbc9cbcc..f2297a2bdaad7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3100,15 +3100,14 @@ object SQLConf { buildConf("spark.sql.defaultColumn.allowedProviders") .internal() .doc("List of table providers wherein SQL commands are permitted to assign DEFAULT column " + - "values. Comma-separated list, whitespace ignored, case-insensitive. If an asterisk " + - "appears after any table provider in this list, any command may assign DEFAULT column " + - "except `ALTER TABLE ... ADD COLUMN`. Otherwise, if no asterisk appears, all commands " + - "are permitted. This is useful because in order for such `ALTER TABLE ... ADD COLUMN` " + - "commands to work, the target data source must include support for substituting in the " + - "provided values when the corresponding fields are not present in storage.") + "values. Comma-separated list, whitespace ignored, case-insensitive. The data sources " + + "must support substituting in the provided values when the corresponding fields are not " + + "present in storage. This can happen when users create an external table and specify " + + "columns with default value, or add columns with default value to an existing non-empty " + + "table.") .version("3.4.0") .stringConf - .createWithDefault("csv,json,orc,parquet") + .createWithDefault("csv,json,orc,parquet,hive") val JSON_GENERATOR_WRITE_NULL_IF_WITH_DEFAULT_VALUE = buildConf("spark.sql.jsonGenerator.writeNullIfWithDefaultValue") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala new file mode 100644 index 0000000000000..642246144ff2e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala @@ -0,0 +1,30 @@ +/* + * 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.internal.connector + +import org.apache.spark.sql.connector.catalog.{Column, ColumnDefaultValue} +import org.apache.spark.sql.types.DataType + +// The default implementation of v2 column. +case class ColumnImpl( + name: String, + dataType: DataType, + nullable: Boolean, + comment: String, + defaultValue: ColumnDefaultValue, + metadataInJSON: String) extends Column diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SimpleTableProvider.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SimpleTableProvider.scala index 7bfe1df1117ac..f8b237195fa88 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SimpleTableProvider.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SimpleTableProvider.scala @@ -37,7 +37,8 @@ trait SimpleTableProvider extends TableProvider { } override def inferSchema(options: CaseInsensitiveStringMap): StructType = { - getOrLoadTable(options).schema() + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + getOrLoadTable(options).columns.asSchema } override def getTable( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala index 432e06a28a2a4..42c372546b559 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -89,7 +89,7 @@ case class StructField( /** * Updates the StructField with a new current default value. */ - def withCurrentDefaultValue(value: String): StructField = { + private[sql] def withCurrentDefaultValue(value: String): StructField = { val newMetadata = new MetadataBuilder() .withMetadata(metadata) .putString(CURRENT_DEFAULT_COLUMN_METADATA_KEY, value) @@ -100,7 +100,7 @@ case class StructField( /** * Clears the StructField of its current default value, if any. */ - def clearCurrentDefaultValue(): StructField = { + private[sql] def clearCurrentDefaultValue(): StructField = { val newMetadata = new MetadataBuilder() .withMetadata(metadata) .remove(CURRENT_DEFAULT_COLUMN_METADATA_KEY) @@ -111,7 +111,7 @@ case class StructField( /** * Return the current default value of this StructField. */ - def getCurrentDefaultValue(): Option[String] = { + private[sql] def getCurrentDefaultValue(): Option[String] = { if (metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY)) { Option(metadata.getString(CURRENT_DEFAULT_COLUMN_METADATA_KEY)) } else { @@ -122,7 +122,7 @@ case class StructField( /** * Updates the StructField with a new existence default value. */ - def withExistenceDefaultValue(value: String): StructField = { + private[sql] def withExistenceDefaultValue(value: String): StructField = { val newMetadata = new MetadataBuilder() .withMetadata(metadata) .putString(EXISTS_DEFAULT_COLUMN_METADATA_KEY, value) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index a9c3829a7211d..6786ffd745375 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -27,9 +27,10 @@ import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} +import org.apache.spark.sql.catalyst.plans.logical.{Column, DefaultValueExpression} import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.catalyst.types.{PhysicalDataType, PhysicalStructType} -import org.apache.spark.sql.catalyst.util.{truncatedString, StringUtils} +import org.apache.spark.sql.catalyst.util.{truncatedString, ResolveDefaultColumns, StringUtils} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} @@ -394,6 +395,31 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru protected[sql] def toAttributes: Seq[AttributeReference] = map(field => field.toAttribute) + private[sql] def toColumns: Array[Column] = fields.map { f => + val defaultValue = f.getCurrentDefaultValue().map { sql => + val existDefaultOpt = f.getExistenceDefaultValue() + assert(existDefaultOpt.isDefined, "current and exist default must be both set or neither") + val e = CatalystSqlParser.parseExpression(f.getExistenceDefaultValue().get) + assert(e.resolved && e.foldable, + "exist default must be simple SQL string that is resolved and foldable after parsing, " + + "but got: " + existDefaultOpt.get) + DefaultValueExpression(e, sql) + } + + val cleanedMetadata = new MetadataBuilder() + .withMetadata(f.metadata) + .remove("comment") + .remove(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY) + .remove(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY) + .build() + val metadataInJSON = if (cleanedMetadata == Metadata.empty) { + None + } else { + Some(cleanedMetadata.json) + } + Column(f.name, f.dataType, f.nullable, f.getComment(), defaultValue, metadataInJSON) + } + def treeString: String = treeString(Int.MaxValue) def treeString(maxDepth: Int): String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 32eb884942763..fe5cc543e8495 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, NoSuchDatabaseException, NoSuchFunctionException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -1048,37 +1047,11 @@ abstract class CatalogTestUtils { identifier = TableIdentifier(name, database), tableType = CatalogTableType.EXTERNAL, storage = storageFormat.copy(locationUri = Some(Utils.createTempDir().toURI)), - schema = if (defaultColumns) { - new StructType() - .add("col1", "int") - .add("col2", "string") - .add("a", IntegerType, nullable = true, - new MetadataBuilder().putString( - ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "42") - .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "41").build()) - .add("b", StringType, nullable = false, - new MetadataBuilder().putString( - ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "\"abc\"").build()) - // The default value fails to parse. - .add("c", LongType, nullable = false, - new MetadataBuilder().putString( - ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "_@#$%").build()) - // The default value fails to resolve. - .add("d", LongType, nullable = false, - new MetadataBuilder().putString( - ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, - "(select min(x) from badtable)").build()) - // The default value fails to coerce to the required type. - .add("e", BooleanType, nullable = false, - new MetadataBuilder().putString( - ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "41 + 1").build()) - } else { - new StructType() - .add("col1", "int") - .add("col2", "string") - .add("a", "int") - .add("b", "string") - }, + schema = new StructType() + .add("col1", "int") + .add("col2", "string") + .add("a", "int") + .add("b", "string"), provider = Some(defaultProvider), partitionColumnNames = Seq("a", "b"), bucketSpec = Some(BucketSpec(4, Seq("col1"), Nil))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index a7254865c1ed1..2593fdcee5a25 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{LeafCommand, LogicalPlan, Project, Range, SubqueryAlias, View} -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER @@ -122,72 +121,6 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { assert(e.contains(s"`$name` is not a valid name for tables/databases.")) } - test("create table with default columns") { - def test: Unit = withBasicCatalog { catalog => - assert(catalog.externalCatalog.listTables("db1").isEmpty) - assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) - catalog.createTable(newTable( - "tbl3", Some("db1"), defaultColumns = true), ignoreIfExists = false) - catalog.createTable(newTable( - "tbl3", Some("db2"), defaultColumns = true), ignoreIfExists = false) - assert(catalog.externalCatalog.listTables("db1").toSet == Set("tbl3")) - assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2", "tbl3")) - // Inspect the default column values. - val db1tbl3 = catalog.externalCatalog.getTable("db1", "tbl3") - val currentDefault = ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY - - def findField(name: String, schema: StructType): StructField = - schema.fields.filter(_.name == name).head - val columnA: StructField = findField("a", db1tbl3.schema) - val columnB: StructField = findField("b", db1tbl3.schema) - val columnC: StructField = findField("c", db1tbl3.schema) - val columnD: StructField = findField("d", db1tbl3.schema) - val columnE: StructField = findField("e", db1tbl3.schema) - - val defaultValueColumnA: String = columnA.metadata.getString(currentDefault) - val defaultValueColumnB: String = columnB.metadata.getString(currentDefault) - val defaultValueColumnC: String = columnC.metadata.getString(currentDefault) - val defaultValueColumnD: String = columnD.metadata.getString(currentDefault) - val defaultValueColumnE: String = columnE.metadata.getString(currentDefault) - - assert(defaultValueColumnA == "42") - assert(defaultValueColumnB == "\"abc\"") - assert(defaultValueColumnC == "_@#$%") - assert(defaultValueColumnD == "(select min(x) from badtable)") - assert(defaultValueColumnE == "41 + 1") - - // Analyze the default column values. - val statementType = "CREATE TABLE" - assert(ResolveDefaultColumns.analyze(columnA, statementType).sql == "42") - assert(ResolveDefaultColumns - .analyze(columnA, statementType, ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY) - .sql == "41") - assert(ResolveDefaultColumns.analyze(columnB, statementType).sql == "'abc'") - assert(intercept[AnalysisException] { - ResolveDefaultColumns.analyze(columnC, statementType) - }.getMessage.contains("fails to parse as a valid expression")) - assert(intercept[AnalysisException] { - ResolveDefaultColumns.analyze(columnD, statementType) - }.getMessage.contains("subquery expressions are not allowed in DEFAULT values")) - assert(intercept[AnalysisException] { - ResolveDefaultColumns.analyze(columnE, statementType) - }.getMessage.contains("statement provided a value of incompatible type")) - - // Make sure that constant-folding default values does not take place when the feature is - // disabled. - withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") { - val result: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( - db1tbl3.schema, db1tbl3.provider, "CREATE TABLE", false) - val columnEWithFeatureDisabled: StructField = findField("e", result) - // No constant-folding has taken place to the EXISTS_DEFAULT metadata. - assert(!columnEWithFeatureDisabled.metadata.contains("EXISTS_DEFAULT")) - } - } - withSQLConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS.key -> "csv,hive,json,orc,parquet") { - test - } - } - test("create databases using invalid names") { withEmptyCatalog { catalog => testInvalidName( 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 0efbd75ad93d0..7bf86390d8b47 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 @@ -23,12 +23,11 @@ import org.apache.spark.SparkThrowable import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions.{EqualTo, Hex, Literal} import org.apache.spark.sql.catalyst.plans.logical.{TableSpec => LogicalTableSpec, _} -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition.{after, first} import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.connector.expressions.LogicalExpressions.bucket import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{IntegerType, LongType, MetadataBuilder, StringType, StructType, TimestampType} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, TimestampType} import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class DDLParserSuite extends AnalysisTest { @@ -47,9 +46,10 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab(a INT COMMENT 'test', b STRING NOT NULL) USING parquet" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType() - .add("a", IntegerType, nullable = true, "test") - .add("b", StringType, nullable = false)), + Some(Seq( + Column("a", IntegerType, comment = Some("test")), + Column("b", StringType, nullable = false) + )), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -75,7 +75,7 @@ class DDLParserSuite extends AnalysisTest { sql, TableSpec( Seq("my_tab"), - Some(new StructType().add("a", IntegerType).add("b", StringType)), + Some(Seq(Column("a", IntegerType), Column("b", StringType))), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -93,9 +93,7 @@ class DDLParserSuite extends AnalysisTest { "USING parquet PARTITIONED BY (a)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType() - .add("a", IntegerType, nullable = true, "test") - .add("b", StringType)), + Some(Seq(Column("a", IntegerType, comment = Some("test")), Column("b", StringType))), Seq(IdentityTransform(FieldReference("a"))), Map.empty[String, String], Some("parquet"), @@ -136,10 +134,11 @@ class DDLParserSuite extends AnalysisTest { """.stripMargin val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType() - .add("a", IntegerType) - .add("b", StringType) - .add("ts", TimestampType)), + Some(Seq( + Column("a", IntegerType), + Column("b", StringType), + Column("ts", TimestampType) + )), Seq( IdentityTransform(FieldReference("a")), BucketTransform(LiteralValue(16, IntegerType), Seq(FieldReference("b"))), @@ -171,7 +170,7 @@ class DDLParserSuite extends AnalysisTest { val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("a", IntegerType).add("b", StringType)), + Some(Seq(Column("a", IntegerType), Column("b", StringType))), List(bucket(5, Array(FieldReference.column("a")), Array(FieldReference.column("b")))), Map.empty[String, String], Some("parquet"), @@ -190,7 +189,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("a", IntegerType).add("b", StringType)), + Some(Seq(Column("a", IntegerType), Column("b", StringType))), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -210,7 +209,7 @@ class DDLParserSuite extends AnalysisTest { " TBLPROPERTIES('test' = 'test')" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("a", IntegerType).add("b", StringType)), + Some(Seq(Column("a", IntegerType), Column("b", StringType))), Seq.empty[Transform], Map("test" -> "test"), Some("parquet"), @@ -227,15 +226,15 @@ class DDLParserSuite extends AnalysisTest { val createSql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" val expectedTableSpec = TableSpec( - Seq("my_tab"), - Some(new StructType().add("a", IntegerType).add("b", StringType)), - Seq.empty[Transform], - Map.empty[String, String], - Some("parquet"), - Map.empty[String, String], - Some("/tmp/file"), - None, - None) + Seq("my_tab"), + Some(Seq(Column("a", IntegerType), Column("b", StringType))), + Seq.empty[Transform], + Map.empty[String, String], + Some("parquet"), + Map.empty[String, String], + Some("/tmp/file"), + None, + None) Seq(createSql, replaceSql).foreach { sql => testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } @@ -246,7 +245,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE 1m.2g(a INT) USING parquet" val expectedTableSpec = TableSpec( Seq("1m", "2g"), - Some(new StructType().add("a", IntegerType)), + Some(Seq(Column("a", IntegerType))), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -264,7 +263,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab (id bigint) PARTITIONED BY (part string)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("id", LongType).add("part", StringType)), + Some(Seq(Column("id", LongType), Column("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -282,7 +281,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab PARTITIONED BY (part string)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("part", StringType)), + Some(Seq(Column("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -300,7 +299,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab (id bigint) USING parquet PARTITIONED BY (part string)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("id", LongType).add("part", StringType)), + Some(Seq(Column("id", LongType), Column("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], Some("parquet"), @@ -373,7 +372,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("id", LongType).add("part", StringType)), + Some(Seq(Column("id", LongType), Column("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -398,7 +397,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("id", LongType).add("part", StringType)), + Some(Seq(Column("id", LongType), Column("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -455,7 +454,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("id", LongType).add("part", StringType)), + Some(Seq(Column("id", LongType), Column("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -506,7 +505,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("id", LongType).add("part", StringType)), + Some(Seq(Column("id", LongType), Column("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -529,7 +528,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(new StructType().add("id", LongType).add("part", StringType)), + Some(Seq(Column("id", LongType), Column("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -872,7 +871,7 @@ class DDLParserSuite extends AnalysisTest { sql, TableSpec( Seq("table_name"), - Some(new StructType), + Some(Seq()), Seq.empty[Transform], Map.empty[String, String], Some("json"), @@ -1035,7 +1034,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMN x int"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None) + Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType)) ))) } @@ -1044,8 +1043,8 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS x int, y string"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None), - QualifiedColType(None, "y", StringType, true, None, None, None) + Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType)), + QualifiedColType(RootTableSchema, Column("y", StringType)) ))) } @@ -1054,7 +1053,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS x int"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None) + Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType)) ))) } @@ -1063,7 +1062,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS (x int)"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None) + Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType)) ))) } @@ -1072,7 +1071,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS (x int COMMENT 'doc')"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(None, "x", IntegerType, true, Some("doc"), None, None) + Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType, comment = Some("doc"))) ))) } @@ -1081,7 +1080,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMN x int NOT NULL"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType(None, "x", IntegerType, false, None, None, None) + Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType, nullable = false)) ))) } @@ -1090,7 +1089,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMN x int COMMENT 'doc'"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType(None, "x", IntegerType, true, Some("doc"), None, None) + Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType, comment = Some("doc"))) ))) } @@ -1100,13 +1099,9 @@ class DDLParserSuite extends AnalysisTest { AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq(QualifiedColType( - None, - "x", - IntegerType, - true, - None, - Some(UnresolvedFieldPosition(first())), - None) + RootTableSchema, + Column("x", IntegerType), + Some(UnresolvedFieldPosition(first()))) ))) comparePlans( @@ -1114,13 +1109,9 @@ class DDLParserSuite extends AnalysisTest { AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq(QualifiedColType( - None, - "x", - IntegerType, - true, - None, - Some(UnresolvedFieldPosition(after("y"))), - None) + RootTableSchema, + Column("x", IntegerType), + Some(UnresolvedFieldPosition(after("y")))) ))) } @@ -1130,7 +1121,8 @@ class DDLParserSuite extends AnalysisTest { AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq(QualifiedColType( - Some(UnresolvedFieldName(Seq("x", "y"))), "z", IntegerType, true, Some("doc"), None, None) + UnresolvedFieldName(Seq("x", "y")), + Column("z", IntegerType, comment = Some("doc"))) ))) } @@ -1141,21 +1133,12 @@ class DDLParserSuite extends AnalysisTest { UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq( QualifiedColType( - Some(UnresolvedFieldName(Seq("x", "y"))), - "z", - IntegerType, - true, - Some("doc"), - None, - None), + UnresolvedFieldName(Seq("x", "y")), + Column("z", IntegerType, comment = Some("doc"))), QualifiedColType( - Some(UnresolvedFieldName(Seq("a"))), - "b", - StringType, - true, - None, - Some(UnresolvedFieldPosition(first())), - None) + UnresolvedFieldName(Seq("a")), + Column("b", StringType), + Some(UnresolvedFieldPosition(first()))) ))) } @@ -1393,21 +1376,21 @@ class DDLParserSuite extends AnalysisTest { parsePlan(sql1), ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), - Seq(QualifiedColType(None, "x", StringType, true, None, None, None)))) + Seq(QualifiedColType(RootTableSchema, Column("x", StringType))))) comparePlans( parsePlan(sql2), ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), - Seq(QualifiedColType(None, "x", StringType, true, Some("x1"), None, None)))) + Seq(QualifiedColType(RootTableSchema, Column("x", StringType, comment = Some("x1")))))) comparePlans( parsePlan(sql3), ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), Seq( - QualifiedColType(None, "x", StringType, true, Some("x1"), None, None), - QualifiedColType(None, "y", IntegerType, true, None, None, None) + QualifiedColType(RootTableSchema, Column("x", StringType, comment = Some("x1"))), + QualifiedColType(RootTableSchema, Column("y", IntegerType)) ))) comparePlans( @@ -1415,8 +1398,8 @@ class DDLParserSuite extends AnalysisTest { ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), Seq( - QualifiedColType(None, "x", StringType, true, Some("x1"), None, None), - QualifiedColType(None, "y", IntegerType, true, Some("y1"), None, None) + QualifiedColType(RootTableSchema, Column("x", StringType, comment = Some("x1"))), + QualifiedColType(RootTableSchema, Column("y", IntegerType, comment = Some("y1"))) ))) val sql5 = "ALTER TABLE table_name PARTITION (a='1') REPLACE COLUMNS (x string)" @@ -2417,7 +2400,7 @@ class DDLParserSuite extends AnalysisTest { private case class TableSpec( name: Seq[String], - schema: Option[StructType], + columns: Option[Seq[Column]], partitioning: Seq[Transform], properties: Map[String, String], provider: Option[String], @@ -2433,7 +2416,7 @@ class DDLParserSuite extends AnalysisTest { case create: CreateTable => TableSpec( create.name.asInstanceOf[UnresolvedIdentifier].nameParts, - Some(create.tableSchema), + Some(create.columns), create.partitioning, create.tableSpec.properties, create.tableSpec.provider, @@ -2445,7 +2428,7 @@ class DDLParserSuite extends AnalysisTest { case replace: ReplaceTable => TableSpec( replace.name.asInstanceOf[UnresolvedIdentifier].nameParts, - Some(replace.tableSchema), + Some(replace.columns), replace.partitioning, replace.tableSpec.properties, replace.tableSpec.provider, @@ -2456,7 +2439,7 @@ class DDLParserSuite extends AnalysisTest { case ctas: CreateTableAsSelect => TableSpec( ctas.name.asInstanceOf[UnresolvedIdentifier].nameParts, - Some(ctas.query).filter(_.resolved).map(_.schema), + if (ctas.query.resolved) Some(ctas.columns) else None, ctas.partitioning, ctas.tableSpec.properties, ctas.tableSpec.provider, @@ -2468,7 +2451,7 @@ class DDLParserSuite extends AnalysisTest { case rtas: ReplaceTableAsSelect => TableSpec( rtas.name.asInstanceOf[UnresolvedIdentifier].nameParts, - Some(rtas.query).filter(_.resolved).map(_.schema), + if (rtas.query.resolved) Some(rtas.columns) else None, rtas.partitioning, rtas.tableSpec.properties, rtas.tableSpec.provider, @@ -2505,7 +2488,7 @@ class DDLParserSuite extends AnalysisTest { val sql = "CREATE TABLE 1m.2g(a INT)" val expectedTableSpec = TableSpec( Seq("1m", "2g"), - Some(new StructType().add("a", IntegerType)), + Some(Seq(Column("a", IntegerType))), Seq.empty[Transform], Map.empty[String, String], None, @@ -2551,34 +2534,40 @@ class DDLParserSuite extends AnalysisTest { test("SPARK-38335: Implement parser support for DEFAULT values for columns in tables") { // These CREATE/REPLACE TABLE statements should parse successfully. - val schemaWithDefaultColumn = new StructType() - .add("a", IntegerType, true) - .add("b", StringType, false, - new MetadataBuilder() - .putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "\"abc\"") - .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "\"abc\"").build()) + val columnsWithDefaultValue = Seq( + Column("a", IntegerType), + Column( + "b", + StringType, + nullable = false, + defaultValue = Some(DefaultValueExpression(Literal("abc"), "'abc'"))) + ) val createTableResult = - CreateTable(UnresolvedIdentifier(Seq("my_tab")), schemaWithDefaultColumn, + CreateTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithDefaultValue, Seq.empty[Transform], LogicalTableSpec(Map.empty[String, String], Some("parquet"), Map.empty[String, String], None, None, None, false), false) // Parse the CREATE TABLE statement twice, swapping the order of the NOT NULL and DEFAULT // options, to make sure that the parser accepts any ordering of these options. comparePlans(parsePlan( - "CREATE TABLE my_tab(a INT, b STRING NOT NULL DEFAULT \"abc\") USING parquet"), + "CREATE TABLE my_tab(a INT, b STRING NOT NULL DEFAULT 'abc') USING parquet"), createTableResult) comparePlans(parsePlan( - "CREATE TABLE my_tab(a INT, b STRING DEFAULT \"abc\" NOT NULL) USING parquet"), + "CREATE TABLE my_tab(a INT, b STRING DEFAULT 'abc' NOT NULL) USING parquet"), createTableResult) comparePlans(parsePlan("REPLACE TABLE my_tab(a INT, " + - "b STRING NOT NULL DEFAULT \"abc\") USING parquet"), - ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), schemaWithDefaultColumn, + "b STRING NOT NULL DEFAULT 'abc') USING parquet"), + ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithDefaultValue, Seq.empty[Transform], LogicalTableSpec(Map.empty[String, String], Some("parquet"), Map.empty[String, String], None, None, None, false), false)) // These ALTER TABLE statements should parse successfully. comparePlans( parsePlan("ALTER TABLE t1 ADD COLUMN x int NOT NULL DEFAULT 42"), AddColumns(UnresolvedTable(Seq("t1"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType(None, "x", IntegerType, false, None, None, Some("42"))))) + Seq(QualifiedColType(RootTableSchema, Column( + "x", + IntegerType, + nullable = false, + defaultValue = Some(DefaultValueExpression(Literal(42), "42"))))))) comparePlans( parsePlan("ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT 42"), AlterColumn( @@ -2588,7 +2577,7 @@ class DDLParserSuite extends AnalysisTest { None, None, None, - Some("42"))) + Some(DefaultValueExpression(Literal(42), "42")))) // It is possible to pass an empty string default value using quotes. comparePlans( parsePlan("ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT ''"), @@ -2599,7 +2588,7 @@ class DDLParserSuite extends AnalysisTest { None, None, None, - Some("''"))) + Some(DefaultValueExpression(Literal(""), "''")))) // It is not possible to pass an empty string default value without using quotes. // This results in a parsing error. val sql1 = "ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT " @@ -2624,7 +2613,7 @@ class DDLParserSuite extends AnalysisTest { None, None, None, - Some(""))) + Some(DropDefaultColumnValue))) // Make sure that the parser returns an exception when the feature is disabled. withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") { val sql = "CREATE TABLE my_tab(a INT, b STRING NOT NULL DEFAULT \"abc\") USING parquet" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala index 032b04bb887de..6be50f36c848a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, ScalarFunction, UnboundFunction} -import org.apache.spark.sql.connector.expressions.LogicalExpressions +import org.apache.spark.sql.connector.expressions.{LogicalExpressions, Transform} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -37,6 +37,7 @@ class CatalogSuite extends SparkFunSuite { import CatalogV2Implicits._ private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] + private val emptyTrans: Array[Transform] = Array.empty private val schema: StructType = new StructType() .add("id", IntegerType) .add("data", StringType) @@ -74,13 +75,13 @@ class CatalogSuite extends SparkFunSuite { intercept[NoSuchNamespaceException](catalog.listTables(Array("ns"))) - catalog.createTable(ident1, schema, Array.empty, emptyProps) + catalog.createTable(ident1, schema, emptyTrans, emptyProps) assert(catalog.listTables(Array("ns")).toSet == Set(ident1)) intercept[NoSuchNamespaceException](catalog.listTables(Array("ns2"))) - catalog.createTable(ident3, schema, Array.empty, emptyProps) - catalog.createTable(ident2, schema, Array.empty, emptyProps) + catalog.createTable(ident3, schema, emptyTrans, emptyProps) + catalog.createTable(ident2, schema, emptyTrans, emptyProps) assert(catalog.listTables(Array("ns")).toSet == Set(ident1, ident2)) assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) @@ -100,7 +101,7 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) assert(parsed == Seq("test", "`", ".", "test_table")) @@ -118,7 +119,7 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) - val table = catalog.createTable(testIdent, schema, Array.empty, properties) + val table = catalog.createTable(testIdent, schema, emptyTrans, properties) val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) assert(parsed == Seq("test", "`", ".", "test_table")) @@ -133,10 +134,10 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) val exc = intercept[TableAlreadyExistsException] { - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) } checkErrorTableAlreadyExists(exc, testIdentQuoted) @@ -149,7 +150,7 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) @@ -161,7 +162,7 @@ class CatalogSuite extends SparkFunSuite { test("loadTable") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) val loaded = catalog.loadTable(testIdent) assert(table.name == loaded.name) @@ -182,7 +183,7 @@ class CatalogSuite extends SparkFunSuite { test("invalidateTable") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) catalog.invalidateTable(testIdent) val loaded = catalog.loadTable(testIdent) @@ -203,7 +204,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add property") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.properties.asScala == Map()) @@ -222,7 +223,7 @@ class CatalogSuite extends SparkFunSuite { val properties = new util.HashMap[String, String]() properties.put("prop-1", "1") - val table = catalog.createTable(testIdent, schema, Array.empty, properties) + val table = catalog.createTable(testIdent, schema, emptyTrans, properties) assert(table.properties.asScala == Map("prop-1" -> "1")) @@ -241,7 +242,7 @@ class CatalogSuite extends SparkFunSuite { val properties = new util.HashMap[String, String]() properties.put("prop-1", "1") - val table = catalog.createTable(testIdent, schema, Array.empty, properties) + val table = catalog.createTable(testIdent, schema, emptyTrans, properties) assert(table.properties.asScala == Map("prop-1" -> "1")) @@ -257,7 +258,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: remove missing property") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.properties.asScala == Map()) @@ -273,7 +274,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add top-level column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -285,7 +286,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add required column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -298,7 +299,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add column with comment") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -315,7 +316,7 @@ class CatalogSuite extends SparkFunSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -330,7 +331,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add column to primitive field fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -348,7 +349,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add field to missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -364,7 +365,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: update column data type") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -380,7 +381,7 @@ class CatalogSuite extends SparkFunSuite { val originalSchema = new StructType() .add("id", IntegerType, nullable = false) .add("data", StringType) - val table = catalog.createTable(testIdent, originalSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, originalSchema, emptyTrans, emptyProps) assert(table.schema == originalSchema) @@ -394,7 +395,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: update missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -410,7 +411,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add comment") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -426,7 +427,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: replace comment") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -445,7 +446,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add comment to missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -461,7 +462,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: rename top-level column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -478,7 +479,7 @@ class CatalogSuite extends SparkFunSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -497,7 +498,7 @@ class CatalogSuite extends SparkFunSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -513,7 +514,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: rename missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -532,7 +533,7 @@ class CatalogSuite extends SparkFunSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -549,7 +550,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: delete top-level column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -566,7 +567,7 @@ class CatalogSuite extends SparkFunSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -582,7 +583,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: delete missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -604,7 +605,7 @@ class CatalogSuite extends SparkFunSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -635,7 +636,7 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) @@ -667,7 +668,7 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) assert(!catalog.tableExists(testIdentNew)) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) catalog.renameTable(testIdent, testIdentNew) @@ -692,8 +693,8 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) assert(!catalog.tableExists(testIdentNew)) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) - catalog.createTable(testIdentNew, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdentNew, schema, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) assert(catalog.tableExists(testIdentNew)) @@ -719,8 +720,8 @@ class CatalogSuite extends SparkFunSuite { val ident1 = Identifier.of(Array("ns1", "ns2"), "test_table_1") val ident2 = Identifier.of(Array("ns1", "ns2"), "test_table_2") - catalog.createTable(ident1, schema, Array.empty, emptyProps) - catalog.createTable(ident2, schema, Array.empty, emptyProps) + catalog.createTable(ident1, schema, emptyTrans, emptyProps) + catalog.createTable(ident2, schema, emptyTrans, emptyProps) assert(catalog.listNamespaces === Array(Array("ns1"))) assert(catalog.listNamespaces(Array()) === Array(Array("ns1"))) @@ -734,8 +735,8 @@ class CatalogSuite extends SparkFunSuite { val ident2 = Identifier.of(Array("ns1", "ns2"), "test_table_2") catalog.createNamespace(Array("ns1"), Map("property" -> "value").asJava) - catalog.createTable(ident1, schema, Array.empty, emptyProps) - catalog.createTable(ident2, schema, Array.empty, emptyProps) + catalog.createTable(ident1, schema, emptyTrans, emptyProps) + catalog.createTable(ident2, schema, emptyTrans, emptyProps) assert(catalog.listNamespaces === Array(Array("ns1"))) assert(catalog.listNamespaces(Array()) === Array(Array("ns1"))) @@ -756,7 +757,7 @@ class CatalogSuite extends SparkFunSuite { test("loadNamespaceMetadata: no metadata, table exists") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) val metadata = catalog.loadNamespaceMetadata(testNs) @@ -777,7 +778,7 @@ class CatalogSuite extends SparkFunSuite { val catalog = newCatalog() catalog.createNamespace(testNs, Map("property" -> "value").asJava) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) val metadata = catalog.loadNamespaceMetadata(testNs) @@ -810,7 +811,7 @@ class CatalogSuite extends SparkFunSuite { test("createNamespace: fail if namespace already exists from table") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(catalog.namespaceExists(testNs) === true) assert(catalog.loadNamespaceMetadata(testNs).asScala === Map.empty) @@ -852,7 +853,7 @@ class CatalogSuite extends SparkFunSuite { val catalog = newCatalog() catalog.createNamespace(testNs, Map("property" -> "value").asJava) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(catalog.dropNamespace(testNs, cascade = true)) @@ -882,7 +883,7 @@ class CatalogSuite extends SparkFunSuite { test("alterNamespace: create metadata if missing and table exists") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) catalog.alterNamespace(testNs, NamespaceChange.setProperty("property", "value")) @@ -902,7 +903,7 @@ class CatalogSuite extends SparkFunSuite { test("truncate non-partitioned table") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) .asInstanceOf[InMemoryTable] table.withData(Array( new BufferedRows("3").withRow(InternalRow(0, "abc", "3")), @@ -920,7 +921,7 @@ class CatalogSuite extends SparkFunSuite { new StructType() .add("col0", IntegerType) .add("part0", IntegerType), - Array(LogicalExpressions.identity(LogicalExpressions.parseReference("part0"))), + Array[Transform](LogicalExpressions.identity(LogicalExpressions.parseReference("part0"))), util.Collections.emptyMap[String, String]) val partTable = table.asInstanceOf[InMemoryPartitionTable] val partIdent = InternalRow.apply(0) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala index da5cfab8be3c7..eda401ceb6bdf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala @@ -21,14 +21,14 @@ import org.mockito.Mockito.{mock, when} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.IntegerType class CatalogV2UtilSuite extends SparkFunSuite { test("Load relation should encode the identifiers for V2Relations") { val testCatalog = mock(classOf[TableCatalog]) val ident = mock(classOf[Identifier]) val table = mock(classOf[Table]) - when(table.schema()).thenReturn(new StructType().add("i", "int")) + when(table.columns()).thenReturn(Array(Column.create("i", IntegerType))) when(testCatalog.loadTable(ident)).thenReturn(table) val r = CatalogV2Util.loadRelation(testCatalog, ident) assert(r.isDefined) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala index 06ee588329cee..302df6dd11edc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala @@ -84,6 +84,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { invalidatedTables.add(ident) } + // TODO: remove it when no tests calling this deprecated method. override def createTable( ident: Identifier, schema: StructType, @@ -93,6 +94,15 @@ class BasicInMemoryTableCatalog extends TableCatalog { Array.empty, None) } + override def createTable( + ident: Identifier, + columns: Array[Column], + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + val schema = CatalogV2Util.v2ColumnsToStructType(columns) + createTable(ident, schema, partitions, properties) + } + def createTable( ident: Identifier, schema: StructType, @@ -120,7 +130,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { override def alterTable(ident: Identifier, changes: TableChange*): Table = { val table = loadTable(ident).asInstanceOf[InMemoryTable] val properties = CatalogV2Util.applyPropertiesChanges(table.properties, changes) - val schema = CatalogV2Util.applySchemaChanges(table.schema, changes, None, "ALTER TABLE") + val schema = CatalogV2Util.applySchemaChanges(table.schema, changes) // fail if the last column in the schema was dropped if (schema.fields.isEmpty) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala index 0590ca721cc8d..90ed106d8ed1b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala @@ -22,7 +22,7 @@ import java.util import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, PartitionsAlreadyExistException} -import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference} +import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference, Transform} import org.apache.spark.sql.types.{IntegerType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -41,7 +41,7 @@ class SupportsAtomicPartitionManagementSuite extends SparkFunSuite { .add("id", IntegerType) .add("data", StringType) .add("dt", StringType), - Array(LogicalExpressions.identity(ref("dt"))), + Array[Transform](LogicalExpressions.identity(ref("dt"))), util.Collections.emptyMap[String, String]) newCatalog } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala index ddd08185527e8..40114d063aada 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, PartitionsAlreadyExistException} -import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference} +import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference, Transform} import org.apache.spark.sql.types.{IntegerType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -43,7 +43,7 @@ class SupportsPartitionManagementSuite extends SparkFunSuite { .add("id", IntegerType) .add("data", StringType) .add("dt", StringType), - Array(LogicalExpressions.identity(ref("dt"))), + Array[Transform](LogicalExpressions.identity(ref("dt"))), util.Collections.emptyMap[String, String]) newCatalog } @@ -164,7 +164,8 @@ class SupportsPartitionManagementSuite extends SparkFunSuite { .add("col0", IntegerType) .add("part0", IntegerType) .add("part1", StringType), - Array(LogicalExpressions.identity(ref("part0")), LogicalExpressions.identity(ref("part1"))), + Array[Transform]( + LogicalExpressions.identity(ref("part0")), LogicalExpressions.identity(ref("part1"))), util.Collections.emptyMap[String, String]) val partTable = table.asInstanceOf[InMemoryPartitionTable] diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala index d9eb0892d1389..e6afd852af1aa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala @@ -471,34 +471,22 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { assert(source1.existenceDefaultValues(1) == UTF8String.fromString("abc")) assert(source1.existenceDefaultValues(2) == null) - // Positive test: StructType.defaultValues works because the existence default value parses and - // resolves successfully, then evaluates to a non-literal expression: this is constant-folded at - // reference time. - val source2 = StructType( - Array(StructField("c1", IntegerType, true, - new MetadataBuilder() - .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "1 + 1") - .putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "1 + 1") - .build()))) - val error = "fails to parse as a valid literal value" - assert(source2.existenceDefaultValues.size == 1) - assert(source2.existenceDefaultValues(0) == 2) - // Negative test: StructType.defaultValues fails because the existence default value fails to // parse. - val source3 = StructType(Array( + val error = "fails to parse as a valid literal value" + val source2 = StructType(Array( StructField("c1", IntegerType, true, new MetadataBuilder() .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "invalid") .putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "invalid") .build()))) assert(intercept[AnalysisException] { - source3.existenceDefaultValues + source2.existenceDefaultValues }.getMessage.contains(error)) // Negative test: StructType.defaultValues fails because the existence default value fails to // resolve. - val source4 = StructType(Array( + val source3 = StructType(Array( StructField("c1", IntegerType, true, new MetadataBuilder() .putString( @@ -509,7 +497,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { "(SELECT 'abc' FROM missingtable)") .build()))) assert(intercept[AnalysisException] { - source4.existenceDefaultValues + source3.existenceDefaultValues }.getMessage.contains(error)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala index 3f9eb5c8084e6..9ac536e5d8aeb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, CreateTable, LogicalPlan, ReplaceColumns, ReplaceTable} +import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, Column, CreateTable, LogicalPlan, ReplaceColumns, ReplaceTable} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableChangeColumnCommand, CreateDataSourceTableCommand, CreateTableCommand} @@ -32,18 +32,18 @@ object ReplaceCharWithVarchar extends Rule[LogicalPlan] { plan.resolveOperators { // V2 commands case cmd: CreateTable => - cmd.copy(tableSchema = replaceCharWithVarcharInSchema(cmd.tableSchema)) + cmd.copy(columns = cmd.columns.map(replaceCharWithVarcharInColumn)) case cmd: ReplaceTable => - cmd.copy(tableSchema = replaceCharWithVarcharInSchema(cmd.tableSchema)) + cmd.copy(columns = cmd.columns.map(replaceCharWithVarcharInColumn)) case cmd: AddColumns => cmd.copy(columnsToAdd = cmd.columnsToAdd.map { col => - col.copy(dataType = CharVarcharUtils.replaceCharWithVarchar(col.dataType)) + col.copy(column = replaceCharWithVarcharInColumn(col.column)) }) case cmd: AlterColumn => cmd.copy(dataType = cmd.dataType.map(CharVarcharUtils.replaceCharWithVarchar)) case cmd: ReplaceColumns => cmd.copy(columnsToAdd = cmd.columnsToAdd.map { col => - col.copy(dataType = CharVarcharUtils.replaceCharWithVarchar(col.dataType)) + col.copy(column = replaceCharWithVarcharInColumn(col.column)) }) // V1 commands @@ -61,11 +61,11 @@ object ReplaceCharWithVarchar extends Rule[LogicalPlan] { } } - private def replaceCharWithVarcharInSchema(schema: StructType): StructType = { - CharVarcharUtils.replaceCharWithVarchar(schema).asInstanceOf[StructType] + private def replaceCharWithVarcharInColumn(col: Column): Column = { + col.copy(dataType = CharVarcharUtils.replaceCharWithVarchar(col.dataType)) } private def replaceCharWithVarcharInTableMeta(tbl: CatalogTable): CatalogTable = { - tbl.copy(schema = replaceCharWithVarcharInSchema(tbl.schema)) + tbl.copy(schema = CharVarcharUtils.replaceCharWithVarchar(tbl.schema).asInstanceOf[StructType]) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 7b2d5015840cb..4e7e4833cf9c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -23,10 +23,10 @@ import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} +import org.apache.spark.sql.catalyst.optimizer.ConstantFolding import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, ResolveDefaultColumns => DefaultCols} -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ +import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, ResolveDefaultColumns => DefaultColumnUtil} import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, LookupCatalog, SupportsNamespaces, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} @@ -50,17 +50,26 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case AddColumns(ResolvedV1TableIdentifier(ident), cols) => - cols.foreach { c => + case a @ AddColumns(ResolvedV1TableIdentifier(ident), colsToAdd) if a.resolved => + colsToAdd.foreach { c => if (c.name.length > 1) { throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( Seq(ident.catalog.get, ident.database.get, ident.table), "ADD COLUMN with qualified column") } - if (!c.nullable) { + if (!c.column.nullable) { throw QueryCompilationErrors.addColumnWithV1TableCannotSpecifyNotNullError } } + // Check default values before converting to v1 command. + DefaultColumnUtil.checkDefaultValuesInPlan(a, isForV1 = true) + val cols = if (colsToAdd.exists(_.column.defaultValue.isDefined)) { + // Do a constant-folding, as we need to store the expression SQL string which should be in + // its simplest form. + ConstantFolding(a).asInstanceOf[AddColumns].columnsToAdd.map(_.column) + } else { + colsToAdd.map(_.column) + } AlterTableAddColumnsCommand(ident, cols.map(convertToStructField)) case ReplaceColumns(ResolvedV1TableIdentifier(ident), _) => @@ -69,7 +78,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) "REPLACE COLUMNS") case a @ AlterColumn(ResolvedTable(catalog, ident, table: V1Table, _), _, _, _, _, _, _) - if isSessionCatalog(catalog) => + if isSessionCatalog(catalog) && a.resolved => if (a.column.name.length > 1) { throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( Seq(catalog.name, ident.namespace()(0), ident.name), @@ -95,14 +104,21 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) quoteIfNeeded(colName), table) } } - // Add the current default column value string (if any) to the column metadata. - a.setDefaultExpression.map { c => builder.putString(CURRENT_DEFAULT_COLUMN_METADATA_KEY, c) } val newColumn = StructField( colName, dataType, nullable = true, builder.build()) - AlterTableChangeColumnCommand(table.catalogTable.identifier, colName, newColumn) + // Check default values before converting to v1 command. + DefaultColumnUtil.checkDefaultValuesInPlan(a, isForV1 = true) + val defaultValue = if (a.defaultExpression.isDefined) { + // Do a constant-folding, as we need to store the expression SQL string which should be in + // its simplest form. + ConstantFolding(a).asInstanceOf[AlterColumn].defaultExpression + } else { + a.defaultExpression + } + AlterTableChangeColumnCommand(table.catalogTable.identifier, colName, newColumn, defaultValue) case RenameColumn(ResolvedV1TableIdentifier(ident), _, _) => throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( @@ -167,12 +183,22 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) // For CREATE TABLE [AS SELECT], we should use the v1 command if the catalog is resolved to the // session catalog and the table provider is not v2. - case c @ CreateTable(ResolvedV1Identifier(ident), _, _, _, _) => + case c @ CreateTable(ResolvedV1Identifier(ident), _, _, _, _) if c.resolved => val (storageFormat, provider) = getStorageFormatAndProvider( c.tableSpec.provider, c.tableSpec.options, c.tableSpec.location, c.tableSpec.serde, ctas = false) if (!isV2Provider(provider)) { - constructV1TableCmd(None, c.tableSpec, ident, c.tableSchema, c.partitioning, + // Check default values before converting to v1 command. + DefaultColumnUtil.checkDefaultValuesInPlan(c, isForV1 = true) + val cols = if (c.columns.exists(_.defaultValue.isDefined)) { + // Do a constant-folding, as we need to store the expression SQL string which should be in + // its simplest form. + ConstantFolding(c).asInstanceOf[CreateTable].columns + } else { + c.columns + } + constructV1TableCmd(None, c.tableSpec, ident, + StructType(cols.map(convertToStructField)), c.partitioning, c.ignoreIfExists, storageFormat, provider) } else { c @@ -599,13 +625,9 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) } } - private def convertToStructField(col: QualifiedColType): StructField = { - val builder = new MetadataBuilder - col.comment.foreach(builder.putString("comment", _)) - col.default.map { - value: String => builder.putString(DefaultCols.CURRENT_DEFAULT_COLUMN_METADATA_KEY, value) - } - StructField(col.name.head, col.dataType, nullable = true, builder.build()) + private def convertToStructField(col: Column): StructField = { + assert(col.resolved) + CatalogV2Util.v2ColumnToStructField(col.toV2Column) } private def isV2Provider(provider: String): Boolean = { 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 fdd4f10c793a0..491664ffc4887 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 @@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.sql.{Row, SparkSession} @@ -35,9 +36,8 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.plans.logical.{DefaultValueExpression, DropDefaultColumnValue, LogicalPlan} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.SupportsNamespaces._ @@ -365,7 +365,8 @@ case class AlterTableUnsetPropertiesCommand( case class AlterTableChangeColumnCommand( tableName: TableIdentifier, columnName: String, - newColumn: StructField) extends LeafRunnableCommand { + newColumn: StructField, + newDefaultValue: Option[Expression] = None) extends LeafRunnableCommand { // TODO: support change column name/dataType/metadata/position. override def run(sparkSession: SparkSession): Seq[Row] = { @@ -385,20 +386,15 @@ case class AlterTableChangeColumnCommand( val newDataSchema = table.dataSchema.fields.map { field => if (field.name == originColumn.name) { // Create a new column from the origin column with the new comment. - val withNewComment: StructField = - addComment(field, newColumn.getComment) + val withNewComment: StructField = addComment(field, newColumn.getComment) // Create a new column from the origin column with the new current default value. - if (newColumn.getCurrentDefaultValue().isDefined) { - if (newColumn.getCurrentDefaultValue().get.nonEmpty) { - val result: StructField = - addCurrentDefaultValue(withNewComment, newColumn.getCurrentDefaultValue()) - // Check that the proposed default value parses and analyzes correctly, and that the - // type of the resulting expression is equivalent or coercible to the destination column - // type. - ResolveDefaultColumns.analyze(result, "ALTER TABLE ALTER COLUMN") - result - } else { - withNewComment.clearCurrentDefaultValue() + if (newDefaultValue.isDefined) { + newDefaultValue.get match { + case DropDefaultColumnValue => withNewComment.clearCurrentDefaultValue() + case d: DefaultValueExpression => + withNewComment.withCurrentDefaultValue(d.originalSQL) + case other => throw SparkException.internalError( + "Unexpected expression in AlterTableChangeColumnCommand.defaultExpression: " + other) } } else { withNewComment @@ -425,11 +421,6 @@ case class AlterTableChangeColumnCommand( private def addComment(column: StructField, comment: Option[String]): StructField = comment.map(column.withComment).getOrElse(column) - // Add the current default value to a column, if default value is empty, return the original - // column. - private def addCurrentDefaultValue(column: StructField, value: Option[String]): StructField = - value.map(column.withCurrentDefaultValue).getOrElse(column) - // Compare a [[StructField]] to another, return true if they have the same column // name(by resolver) and dataType. private def columnEqual( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index f6266bcb33f62..f47e8ef667227 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -231,19 +231,17 @@ case class AlterTableAddColumnsCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val catalogTable = verifyAlterTableAddColumn(sparkSession.sessionState.conf, catalog, table) - val colsWithProcessedDefaults = - constantFoldCurrentDefaultsToExistDefaults(sparkSession, catalogTable.provider) CommandUtils.uncacheTableOrView(sparkSession, table.quotedString) catalog.refreshTable(table) SchemaUtils.checkColumnNameDuplication( - (colsWithProcessedDefaults ++ catalogTable.schema).map(_.name), + (colsToAdd ++ catalogTable.schema).map(_.name), conf.caseSensitiveAnalysis) - DDLUtils.checkTableColumns(catalogTable, StructType(colsWithProcessedDefaults)) + DDLUtils.checkTableColumns(catalogTable, StructType(colsToAdd)) val existingSchema = CharVarcharUtils.getRawSchema(catalogTable.dataSchema) - catalog.alterTableDataSchema(table, StructType(existingSchema ++ colsWithProcessedDefaults)) + catalog.alterTableDataSchema(table, StructType(existingSchema ++ colsToAdd)) Seq.empty[Row] } @@ -279,24 +277,6 @@ case class AlterTableAddColumnsCommand( } catalogTable } - - /** - * ALTER TABLE ADD COLUMNS commands may optionally specify a DEFAULT expression for any column. - * In that case, this method evaluates its originally specified value and then stores the result - * in a separate column metadata entry, then returns the updated column definitions. - */ - private def constantFoldCurrentDefaultsToExistDefaults( - sparkSession: SparkSession, tableProvider: Option[String]): Seq[StructField] = { - colsToAdd.map { col: StructField => - if (col.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY)) { - val foldedStructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( - StructType(Array(col)), tableProvider, "ALTER TABLE ADD COLUMNS", true) - foldedStructType.fields(0) - } else { - col - } - } - } } 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 8b985e82963e8..b8025c3966672 100644 --- 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 @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 -import org.apache.spark.sql.catalyst.util.{ResolveDefaultColumns, V2ExpressionBuilder} +import org.apache.spark.sql.catalyst.util.V2ExpressionBuilder import org.apache.spark.sql.connector.catalog.SupportsRead import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.expressions.{Expression => V2Expression, NullOrdering, SortDirection, SortOrder => V2SortOrder, SortValue} @@ -61,7 +61,7 @@ import org.apache.spark.unsafe.types.UTF8String * Note that, this rule must be run after `PreprocessTableCreation` and * `PreprocessTableInsertion`. */ -case class DataSourceAnalysis(analyzer: Analyzer) extends Rule[LogicalPlan] { +object DataSourceAnalysis extends Rule[LogicalPlan] { def resolver: Resolver = conf.resolver @@ -133,11 +133,7 @@ case class DataSourceAnalysis(analyzer: Analyzer) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case CreateTable(tableDesc, mode, None) if DDLUtils.isDatasourceTable(tableDesc) => - val newSchema: StructType = - ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( - tableDesc.schema, tableDesc.provider, "CREATE TABLE", false) - val newTableDesc = tableDesc.copy(schema = newSchema) - CreateDataSourceTableCommand(newTableDesc, ignoreIfExists = mode == SaveMode.Ignore) + CreateDataSourceTableCommand(tableDesc, ignoreIfExists = mode == SaveMode.Ignore) case CreateTable(tableDesc, mode, Some(query)) if query.resolved && DDLUtils.isDatasourceTable(tableDesc) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index dc4fed49c1cfd..6fd47a534e9a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -244,7 +244,6 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi case create: V2CreateTablePlan if create.childrenResolved => val schema = create.tableSchema val partitioning = create.partitioning - val identifier = create.tableName val isCaseSensitive = conf.caseSensitiveAnalysis // Check that columns are not duplicated in the schema val flattenedSchema = SchemaUtils.explodeNestedFieldNames(schema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala index abc6bc60d96a3..550578443283f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala @@ -23,15 +23,14 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.TableSpec -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, Identifier, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.StructType case class CreateTableExec( catalog: TableCatalog, identifier: Identifier, - tableSchema: StructType, + columns: Array[Column], partitioning: Seq[Transform], tableSpec: TableSpec, ignoreIfExists: Boolean) extends LeafV2CommandExec { @@ -42,7 +41,7 @@ case class CreateTableExec( override protected def run(): Seq[InternalRow] = { if (!catalog.tableExists(identifier)) { try { - catalog.createTable(identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + catalog.createTable(identifier, columns, partitioning.toArray, tableProperties.asJava) } catch { case _: TableAlreadyExistsException if ignoreIfExists => logWarning(s"Table ${identifier.quoted} was created concurrently. Ignoring.") 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 29f0da1158ff5..81afa36d7f275 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 @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{And, Attribute, DynamicPruning import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.{toPrettySQL, ResolveDefaultColumns, V2ExpressionBuilder} +import org.apache.spark.sql.catalyst.util.{toPrettySQL, V2ExpressionBuilder} import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsDeleteV2, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, Table, TableCapability, TableCatalog, TruncatableTable} import org.apache.spark.sql.connector.catalog.index.SupportsIndex import org.apache.spark.sql.connector.expressions.{FieldReference, LiteralValue} @@ -45,7 +45,6 @@ import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, LogicalRe import org.apache.spark.sql.execution.streaming.continuous.{WriteToContinuousDataSource, WriteToContinuousDataSourceExec} import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.sources.{BaseRelation, TableScan} -import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.storage.StorageLevel @@ -172,12 +171,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat } WriteToDataSourceV2Exec(writer, invalidateCacheFunc, planLater(query), customMetrics) :: Nil - case CreateTable(ResolvedIdentifier(catalog, ident), schema, partitioning, + case CreateTable(ResolvedIdentifier(catalog, ident), columns, partitioning, tableSpec, ifNotExists) => - val newSchema: StructType = - ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( - schema, tableSpec.provider, "CREATE TABLE", false) - CreateTableExec(catalog.asTableCatalog, ident, newSchema, + val v2Columns = columns.map(_.toV2Column).toArray + CreateTableExec(catalog.asTableCatalog, ident, v2Columns, partitioning, qualifyLocInTableSpec(tableSpec), ifNotExists) :: Nil case CreateTableAsSelect(ResolvedIdentifier(catalog, ident), parts, query, tableSpec, @@ -196,16 +193,14 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case RefreshTable(r: ResolvedTable) => RefreshTableExec(r.catalog, r.identifier, recacheTable(r)) :: Nil - case ReplaceTable(ResolvedIdentifier(catalog, ident), schema, parts, tableSpec, orCreate) => - val newSchema: StructType = - ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( - schema, tableSpec.provider, "CREATE TABLE", false) + case ReplaceTable(ResolvedIdentifier(catalog, ident), columns, parts, tableSpec, orCreate) => + val v2Columns = columns.map(_.toV2Column).toArray catalog match { case staging: StagingTableCatalog => - AtomicReplaceTableExec(staging, ident, newSchema, parts, + AtomicReplaceTableExec(staging, ident, v2Columns, parts, qualifyLocInTableSpec(tableSpec), orCreate = orCreate, invalidateCache) :: Nil case _ => - ReplaceTableExec(catalog.asTableCatalog, ident, newSchema, parts, + ReplaceTableExec(catalog.asTableCatalog, ident, v2Columns, parts, qualifyLocInTableSpec(tableSpec), orCreate = orCreate, invalidateCache) :: Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 8b0098f14fedc..14b41c5c14a75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -23,8 +23,9 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, ResolveDefaultColumns} +import org.apache.spark.sql.catalyst.util.quoteIfNeeded import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsMetadataColumns, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.expressions.IdentityTransform case class DescribeTableExec( @@ -37,6 +38,7 @@ case class DescribeTableExec( addPartitioning(rows) if (isExtended) { + addColumnDefaultValue(rows) addMetadataColumns(rows) addTableDetails(rows) } @@ -68,17 +70,23 @@ case class DescribeTableExec( case (key, value) => key + "=" + value }.mkString("[", ",", "]") rows += toCatalystRow("Table Properties", properties, "") - - // If any columns have default values, append them to the result. - ResolveDefaultColumns.getDescribeMetadata(table.schema).foreach { row => - rows += toCatalystRow(row._1, row._2, row._3) - } } private def addSchema(rows: ArrayBuffer[InternalRow]): Unit = { - rows ++= table.schema.map{ column => + rows ++= table.columns.map { column => toCatalystRow( - column.name, column.dataType.simpleString, column.getComment().orNull) + column.name, column.dataType.simpleString, column.comment()) + } + } + + private def addColumnDefaultValue(rows: ArrayBuffer[InternalRow]): Unit = { + if (table.columns().exists(_.defaultValue() != null)) { + rows += emptyRow() + rows += toCatalystRow("# Column Default Values", "", "") + rows ++= table.columns.filter(_.defaultValue() != null).map { column => + toCatalystRow( + column.name, column.dataType.simpleString, column.defaultValue().getSql) + } } } @@ -104,10 +112,11 @@ case class DescribeTableExec( rows ++= table.partitioning .map(_.asInstanceOf[IdentityTransform].ref.fieldNames()) .map { fieldNames => - val nestedField = table.schema.findNestedField(fieldNames) + val schema = table.columns().asSchema + val nestedField = schema.findNestedField(fieldNames) assert(nestedField.isDefined, s"Not found the partition column ${fieldNames.map(quoteIfNeeded).mkString(".")} " + - s"in the table schema ${table.schema().catalogString}.") + s"in the table schema ${schema.catalogString}.") nestedField.get }.map { case (path, field) => toCatalystRow( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala index 0bd25064e353c..3cb1a74417db1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -90,8 +90,9 @@ trait FileDataSourceV2 extends TableProvider with DataSourceRegister { private var t: Table = null override def inferSchema(options: CaseInsensitiveStringMap): StructType = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ if (t == null) t = getTable(options) - t.schema() + t.columns.asSchema } // TODO: implement a light-weight partition inference which only looks at the path of one leaf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index ea221980fed85..55d97577d5781 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -23,16 +23,15 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.TableSpec -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, StagedTable, StagingTableCatalog, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, Identifier, StagedTable, StagingTableCatalog, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils case class ReplaceTableExec( catalog: TableCatalog, ident: Identifier, - tableSchema: StructType, + columns: Array[Column], partitioning: Seq[Transform], tableSpec: TableSpec, orCreate: Boolean, @@ -48,7 +47,7 @@ case class ReplaceTableExec( } else if (!orCreate) { throw QueryCompilationErrors.cannotReplaceMissingTableError(ident) } - catalog.createTable(ident, tableSchema, partitioning.toArray, tableProperties.asJava) + catalog.createTable(ident, columns, partitioning.toArray, tableProperties.asJava) Seq.empty } @@ -58,7 +57,7 @@ case class ReplaceTableExec( case class AtomicReplaceTableExec( catalog: StagingTableCatalog, identifier: Identifier, - tableSchema: StructType, + columns: Array[Column], partitioning: Seq[Transform], tableSpec: TableSpec, orCreate: Boolean, @@ -73,11 +72,11 @@ case class AtomicReplaceTableExec( } val staged = if (orCreate) { catalog.stageCreateOrReplace( - identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + identifier, columns, partitioning.toArray, tableProperties.asJava) } else if (catalog.tableExists(identifier)) { try { catalog.stageReplace( - identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + identifier, columns, partitioning.toArray, tableProperties.asJava) } catch { case e: NoSuchTableException => throw QueryCompilationErrors.cannotReplaceMissingTableError(identifier, Some(e)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala index ec40ad70b7964..5712159ddc800 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala @@ -59,7 +59,8 @@ case class ShowCreateTableExec( } private def showTableDataColumns(table: Table, builder: StringBuilder): Unit = { - val columns = CharVarcharUtils.getRawSchema(table.schema(), conf).fields.map(_.toDDL) + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + val columns = CharVarcharUtils.getRawSchema(table.columns.asSchema, conf).fields.map(_.toDDL) builder ++= concatByMultiLines(columns) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index b9afe71d243ea..2f3c1471e5e6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -26,7 +26,7 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.{FunctionIdentifier, SQLConfHelper, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTableType, CatalogUtils, SessionCatalog} -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, FunctionCatalog, Identifier, NamespaceChange, SupportsNamespaces, Table, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, Column, FunctionCatalog, Identifier, NamespaceChange, SupportsNamespaces, Table, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.catalog.NamespaceChange.RemoveProperty import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.connector.expressions.Transform @@ -92,6 +92,15 @@ class V2SessionCatalog(catalog: SessionCatalog) catalog.refreshTable(ident.asTableIdentifier) } + override def createTable( + ident: Identifier, + columns: Array[Column], + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + createTable(ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties) + } + + // TODO: remove it when no tests calling this deprecated method. override def createTable( ident: Identifier, schema: StructType, @@ -150,8 +159,7 @@ class V2SessionCatalog(catalog: SessionCatalog) } val properties = CatalogV2Util.applyPropertiesChanges(catalogTable.properties, changes) - val schema = CatalogV2Util.applySchemaChanges( - catalogTable.schema, changes, catalogTable.provider, "ALTER TABLE") + val schema = CatalogV2Util.applySchemaChanges(catalogTable.schema, changes) val comment = properties.get(TableCatalog.PROP_COMMENT) val owner = properties.getOrElse(TableCatalog.PROP_OWNER, catalogTable.owner) val location = properties.get(TableCatalog.PROP_LOCATION).map(CatalogUtils.stringToURI) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 490b708222396..c53c603ffaa08 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -86,8 +86,9 @@ case class CreateTableAsSelectExec( throw QueryCompilationErrors.tableAlreadyExistsError(ident) } - val schema = CharVarcharUtils.getRawSchema(query.schema, conf).asNullable - val table = catalog.createTable(ident, schema, + val columns = CatalogV2Util.structTypeToV2Columns( + CharVarcharUtils.getRawSchema(query.schema, conf).asNullable) + val table = catalog.createTable(ident, columns, partitioning.toArray, properties.asJava) writeToTable(catalog, table, writeOptions, ident) } @@ -125,9 +126,10 @@ case class AtomicCreateTableAsSelectExec( throw QueryCompilationErrors.tableAlreadyExistsError(ident) } - val schema = CharVarcharUtils.getRawSchema(query.schema, conf).asNullable + val columns = CatalogV2Util.structTypeToV2Columns( + CharVarcharUtils.getRawSchema(query.schema, conf).asNullable) val stagedTable = catalog.stageCreate( - ident, schema, partitioning.toArray, properties.asJava) + ident, columns, partitioning.toArray, properties.asJava) writeToTable(catalog, stagedTable, writeOptions, ident) } @@ -174,9 +176,10 @@ case class ReplaceTableAsSelectExec( } else if (!orCreate) { throw QueryCompilationErrors.cannotReplaceMissingTableError(ident) } - val schema = CharVarcharUtils.getRawSchema(query.schema, conf).asNullable + val columns = CatalogV2Util.structTypeToV2Columns( + CharVarcharUtils.getRawSchema(query.schema, conf).asNullable) val table = catalog.createTable( - ident, schema, partitioning.toArray, properties.asJava) + ident, columns, partitioning.toArray, properties.asJava) writeToTable(catalog, table, writeOptions, ident) } @@ -210,18 +213,19 @@ case class AtomicReplaceTableAsSelectExec( val properties = CatalogV2Util.convertTableProperties(tableSpec) override protected def run(): Seq[InternalRow] = { - val schema = CharVarcharUtils.getRawSchema(query.schema, conf).asNullable + val columns = CatalogV2Util.structTypeToV2Columns( + CharVarcharUtils.getRawSchema(query.schema, conf).asNullable) if (catalog.tableExists(ident)) { val table = catalog.loadTable(ident) invalidateCache(catalog, table, ident) } val staged = if (orCreate) { catalog.stageCreateOrReplace( - ident, schema, partitioning.toArray, properties.asJava) + ident, columns, partitioning.toArray, properties.asJava) } else if (catalog.tableExists(ident)) { try { catalog.stageReplace( - ident, schema, partitioning.toArray, properties.asJava) + ident, columns, partitioning.toArray, properties.asJava) } catch { case e: NoSuchTableException => throw QueryCompilationErrors.cannotReplaceMissingTableError(ident, Some(e)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala index d9a3a074ce6a1..1ab88cd41d875 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala @@ -86,7 +86,10 @@ class TextSocketTable(host: String, port: Int, numPartitions: Int, includeTimest } override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = () => new Scan { - override def readSchema(): StructType = schema() + override def readSchema(): StructType = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + columns.asSchema + } override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = { new TextSocketMicroBatchStream(host, port, numPartitions) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index f17d0c3dd2e51..9dd6516f9fb7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -195,7 +195,7 @@ abstract class BaseSessionStateBuilder( DetectAmbiguousSelfJoin +: PreprocessTableCreation(session) +: PreprocessTableInsertion +: - DataSourceAnalysis(this) +: + DataSourceAnalysis +: ApplyCharTypePadding +: ReplaceCharWithVarchar +: customPostHocResolutionRules diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 79b0084da23ad..205fc6d51d588 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -313,9 +313,11 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { val columns = sparkSession.sessionState.executePlan(plan).analyzed match { case ResolvedTable(_, _, table, _) => + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val (partitionColumnNames, bucketSpecOpt) = table.partitioning.toSeq.convertTransforms val bucketColumnNames = bucketSpecOpt.map(_.bucketColumnNames).getOrElse(Nil) - schemaToColumns(table.schema(), partitionColumnNames.contains, bucketColumnNames.contains) + schemaToColumns( + table.columns.asSchema, partitionColumnNames.contains, bucketColumnNames.contains) case ResolvedPersistentView(_, _, schema) => schemaToColumns(schema) @@ -630,7 +632,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { val plan = CreateTable( name = UnresolvedIdentifier(ident), - tableSchema = schema, + columns = schema.toColumns, partitioning = Seq(), tableSpec = tableSpec, ignoreIfExists = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index d4621468f846e..13f7695947e7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -180,11 +180,12 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ table match { case _: SupportsRead if table.supportsAny(MICRO_BATCH_READ, CONTINUOUS_READ) => + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ Dataset.ofRows( sparkSession, StreamingRelationV2( Some(provider), source, table, dsOptions, - table.schema.toAttributes, None, None, v1Relation)) + table.columns.asSchema.toAttributes, None, None, v1Relation)) // fallback to v1 // TODO (SPARK-27483): we should move this fallback logic to an analyzer rule. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index f1d2e351ebda1..a6b8b20da4143 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -299,7 +299,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { false) val cmd = CreateTable( UnresolvedIdentifier(originalMultipartIdentifier), - df.schema.asNullable, + df.schema.asNullable.toColumns, partitioningColumns.getOrElse(Nil).asTransforms.toSeq, tableSpec, ignoreIfExists = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index 835566238c9c1..d0cb8141ad71b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -110,7 +110,7 @@ class InMemoryTableSessionCatalog extends TestV2SessionCatalogBase[InMemoryTable Option(tables.get(ident)) match { case Some(table) => val properties = CatalogV2Util.applyPropertiesChanges(table.properties, changes) - val schema = CatalogV2Util.applySchemaChanges(table.schema, changes, None, "ALTER TABLE") + val schema = CatalogV2Util.applySchemaChanges(table.schema, changes) // fail if the last column in the schema was dropped if (schema.fields.isEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index 158e1634d58c5..3678f29ab49a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, ReplaceTableAsSelect} import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTableCatalog} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf @@ -178,7 +179,7 @@ class DataSourceV2DataFrameSuite testCatalog.createTable( Identifier.of(Array(), "table_name"), new StructType().add("i", "interval"), - Array.empty, Collections.emptyMap[String, String]) + Array.empty[Transform], Collections.emptyMap[String, String]) val df = sql(s"select interval 1 millisecond as i") val v2Writer = df.writeTo("testcat.table_name") val e1 = intercept[AnalysisException](v2Writer.append()) 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 38bd24356f159..e309d4a95dc70 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 @@ -31,10 +31,11 @@ import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableExceptio import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.ColumnStat import org.apache.spark.sql.catalyst.statsEstimation.StatsEstimationTestBase -import org.apache.spark.sql.catalyst.util.{DateTimeUtils, ResolveDefaultColumns} -import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.connector.catalog.{Column => ColumnV2, _} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.CatalogV2Util.withDefaultOwnership +import org.apache.spark.sql.connector.expressions.LiteralValue import org.apache.spark.sql.errors.QueryErrorsBase import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -45,7 +46,7 @@ import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG_IMPLEMENTATION} import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.SimpleScanSource -import org.apache.spark.sql.types.{LongType, MetadataBuilder, StringType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.unsafe.types.UTF8String @@ -585,7 +586,7 @@ class DataSourceV2SQLSuiteV1Filter assert(maybeReplacedTable === table, "Table should not have changed.") } - test("ReplaceTable: Erases the table contents and changes the metadata.") { + test("ReplaceTable: Erases the table contents and changes the metadata") { spark.sql(s"CREATE TABLE testcat.table_name USING $v2Source AS SELECT id, data FROM source") val testCatalog = catalog("testcat").asTableCatalog @@ -598,14 +599,11 @@ class DataSourceV2SQLSuiteV1Filter assert(replaced.asInstanceOf[InMemoryTable].rows.isEmpty, "Replaced table should have no rows after committing.") - assert(replaced.schema().fields.length === 1, + assert(replaced.columns.length === 1, "Replaced table should have new schema.") - val actual = replaced.schema().fields(0) - val expected = StructField("id", LongType, nullable = false, - new MetadataBuilder().putString( - ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "41 + 1") - .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "CAST(42 AS BIGINT)") - .build()) + val actual = replaced.columns.head + val expected = ColumnV2.create("id", LongType, false, null, + new ColumnDefaultValue("41 + 1", LiteralValue(42, IntegerType)), null) assert(actual === expected, "Replaced table should have new schema with DEFAULT column metadata.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuiteBase.scala index 14b951e66db36..781e0f96eaf28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuiteBase.scala @@ -24,6 +24,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row} import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTable, InMemoryRowLevelOperationTableCatalog} import org.apache.spark.sql.connector.expressions.LogicalExpressions._ +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.{DeleteFromTableExec, ReplaceDataExec, WriteDeltaExec} @@ -564,7 +565,8 @@ abstract class DeleteFromTableSuiteBase protected def createTable(schemaString: String): Unit = { val schema = StructType.fromDDL(schemaString) - catalog.createTable(ident, schema, Array(identity(reference(Seq("dep")))), extraTableProps) + catalog.createTable( + ident, schema, Array[Transform](identity(reference(Seq("dep")))), extraTableProps) } protected def createAndInitTable(schemaString: String, jsonData: String): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala index 0a0aaa8021996..46586c622db79 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.catalog.CatalogTableType -import org.apache.spark.sql.connector.catalog.{DelegatingCatalogExtension, Identifier, Table, TableCatalog, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, DelegatingCatalogExtension, Identifier, Table, TableCatalog, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType @@ -64,6 +64,15 @@ private[connector] trait TestV2SessionCatalogBase[T <: Table] extends Delegating } } + override def createTable( + ident: Identifier, + columns: Array[Column], + partitions: Array[Transform], + properties: java.util.Map[String, String]): Table = { + createTable(ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties) + } + + // TODO: remove it when no tests calling this deprecated method. override def createTable( ident: Identifier, schema: StructType, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala index 44cd4f0f9b31d..4f57555544e90 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.connector -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, CreateTablePartitioningValidationSuite, ResolvedTable, TestRelation2, TestTable2, UnresolvedFieldName, UnresolvedFieldPosition, UnresolvedIdentifier} -import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, AlterTableCommand, CreateTableAsSelect, DropColumns, LogicalPlan, QualifiedColType, RenameColumn, ReplaceColumns, ReplaceTableAsSelect, TableSpec} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, CreateTablePartitioningValidationSuite, ResolvedTable, RootTableSchema, TestRelation2, TestTable2, UnresolvedFieldName, UnresolvedFieldPosition, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, AlterTableCommand, Column, CreateTableAsSelect, DropColumns, LogicalPlan, QualifiedColType, RenameColumn, ReplaceColumns, ReplaceTableAsSelect, TableSpec} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition @@ -152,7 +152,8 @@ class V2CommandsCaseSensitivitySuite AddColumns( table, Seq(QualifiedColType( - Some(UnresolvedFieldName(field.init)), field.last, LongType, true, None, None, None))), + UnresolvedFieldName(field.init), + Column(field.last, LongType, true)))), Seq("Missing field " + field.head) ) } @@ -161,15 +162,11 @@ class V2CommandsCaseSensitivitySuite test("AlterTable: add column resolution - positional") { Seq("ID", "iD").foreach { ref => val alter = AddColumns( - table, - Seq(QualifiedColType( - None, - "f", - LongType, - true, - None, - Some(UnresolvedFieldPosition(ColumnPosition.after(ref))), - None))) + table, + Seq(QualifiedColType( + RootTableSchema, + Column("f", LongType, true), + Some(UnresolvedFieldPosition(ColumnPosition.after(ref)))))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { assertAnalysisErrorClass( @@ -184,23 +181,17 @@ class V2CommandsCaseSensitivitySuite test("AlterTable: add column resolution - column position referencing new column") { val alter = AddColumns( - table, - Seq(QualifiedColType( - None, - "x", - LongType, - true, - None, - Some(UnresolvedFieldPosition(ColumnPosition.after("id"))), - None), + table, + Seq( QualifiedColType( - None, - "y", - LongType, - true, - None, - Some(UnresolvedFieldPosition(ColumnPosition.after("X"))), - None))) + RootTableSchema, + Column("x", LongType, true), + Some(UnresolvedFieldPosition(ColumnPosition.after("id")))), + QualifiedColType( + RootTableSchema, + Column("y", LongType, true), + Some(UnresolvedFieldPosition(ColumnPosition.after("X")))) + )) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { assertAnalysisErrorClass( @@ -215,15 +206,11 @@ class V2CommandsCaseSensitivitySuite test("AlterTable: add column resolution - nested positional") { Seq("X", "Y").foreach { ref => val alter = AddColumns( - table, - Seq(QualifiedColType( - Some(UnresolvedFieldName(Seq("point"))), - "z", - LongType, - true, - None, - Some(UnresolvedFieldPosition(ColumnPosition.after(ref))), - None))) + table, + Seq(QualifiedColType( + UnresolvedFieldName(Seq("point")), + Column("z", LongType, true), + Some(UnresolvedFieldPosition(ColumnPosition.after(ref)))))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { assertAnalysisErrorClass( @@ -238,23 +225,16 @@ class V2CommandsCaseSensitivitySuite test("AlterTable: add column resolution - column position referencing new nested column") { val alter = AddColumns( - table, - Seq(QualifiedColType( - Some(UnresolvedFieldName(Seq("point"))), - "z", - LongType, - true, - None, - None, - None), + table, + Seq( + QualifiedColType( + UnresolvedFieldName(Seq("point")), + Column("z", LongType, true)), QualifiedColType( - Some(UnresolvedFieldName(Seq("point"))), - "zz", - LongType, - true, - None, - Some(UnresolvedFieldPosition(ColumnPosition.after("Z"))), - None))) + UnresolvedFieldName(Seq("point")), + Column("zz", LongType, true), + Some(UnresolvedFieldPosition(ColumnPosition.after("Z")))) + )) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { assertAnalysisErrorClass( @@ -270,22 +250,14 @@ class V2CommandsCaseSensitivitySuite assertAnalysisErrorClass( AddColumns( table, - Seq(QualifiedColType( - Some(UnresolvedFieldName(Seq("point"))), - "z", - LongType, - true, - None, - None, - None), - QualifiedColType( - Some(UnresolvedFieldName(Seq("point"))), - "Z", - LongType, - true, - None, - None, - None))), + Seq( + QualifiedColType( + UnresolvedFieldName(Seq("point")), + Column("z", LongType, true)), + QualifiedColType( + UnresolvedFieldName(Seq("point")), + Column("Z", LongType, true)) + )), "COLUMN_ALREADY_EXISTS", Map("columnName" -> toSQLId("point.z")), caseSensitive = false) @@ -296,13 +268,9 @@ class V2CommandsCaseSensitivitySuite AddColumns( table, Seq(QualifiedColType( - None, - "ID", - LongType, - true, - None, - Some(UnresolvedFieldPosition(ColumnPosition.after("id"))), - None))), + RootTableSchema, + Column("ID", LongType, true), + Some(UnresolvedFieldPosition(ColumnPosition.after("id")))))), Seq("Cannot add column, because ID already exists in root"), expectErrorOnCaseSensitive = false) } @@ -374,8 +342,9 @@ class V2CommandsCaseSensitivitySuite assertAnalysisErrorClass( ReplaceColumns( table, - Seq(QualifiedColType(None, "f", LongType, true, None, None, None), - QualifiedColType(None, "F", LongType, true, None, None, None))), + Seq( + QualifiedColType(RootTableSchema, Column("f", LongType, true)), + QualifiedColType(RootTableSchema, Column("F", LongType, true)))), "COLUMN_ALREADY_EXISTS", Map("columnName" -> toSQLId("f")), caseSensitive = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala index b262e405d4ef6..f7905daa20a5e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala @@ -974,7 +974,7 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase } test("continuous mode allows unspecified distribution and empty ordering") { - catalog.createTable(ident, schema, Array.empty, emptyProps) + catalog.createTable(ident, schema, Array.empty[Transform], emptyProps) withTempDir { checkpointDir => val inputData = ContinuousMemoryStream[(Long, String)] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 00d8101df837a..98c18bbaf62f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -34,17 +34,17 @@ import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.catalyst.plans.logical.{AlterColumn, AnalysisOnlyCommand, AppendData, Assignment, CreateTable, CreateTableAsSelect, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, InsertIntoStatement, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, OverwriteByExpression, OverwritePartitionsDynamic, Project, SetTableLocation, SetTableProperties, ShowTableProperties, SubqueryAlias, UnsetTableProperties, UpdateAction, UpdateTable} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns import org.apache.spark.sql.connector.FakeV2Provider -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, SupportsDelete, Table, TableCapability, TableCatalog, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Column, ColumnDefaultValue, Identifier, SupportsDelete, Table, TableCapability, TableCatalog, V1Table} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME -import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.expressions.{LiteralValue, Transform} import org.apache.spark.sql.execution.datasources.{CreateTable => CreateTableV1} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode} import org.apache.spark.sql.sources.SimpleScanSource -import org.apache.spark.sql.types.{BooleanType, CharType, DoubleType, IntegerType, LongType, MetadataBuilder, StringType, StructField, StructType} +import org.apache.spark.sql.types.{BooleanType, CharType, DoubleType, IntegerType, LongType, StringType, StructField, StructType, VarcharType} +import org.apache.spark.unsafe.types.UTF8String class PlanResolutionSuite extends AnalysisTest { import CatalystSqlParser._ @@ -54,21 +54,24 @@ class PlanResolutionSuite extends AnalysisTest { private val table: Table = { val t = mock(classOf[SupportsDelete]) - when(t.schema()).thenReturn(new StructType().add("i", "int").add("s", "string")) + when(t.columns()).thenReturn( + Array(Column.create("i", IntegerType), Column.create("s", StringType))) when(t.partitioning()).thenReturn(Array.empty[Transform]) t } private val table1: Table = { val t = mock(classOf[Table]) - when(t.schema()).thenReturn(new StructType().add("s", "string").add("i", "int")) + when(t.columns()).thenReturn( + Array(Column.create("s", StringType), Column.create("i", IntegerType))) when(t.partitioning()).thenReturn(Array.empty[Transform]) t } private val table2: Table = { val t = mock(classOf[Table]) - when(t.schema()).thenReturn(new StructType().add("i", "int").add("x", "string")) + when(t.columns()).thenReturn( + Array(Column.create("i", IntegerType), Column.create("x", StringType))) when(t.partitioning()).thenReturn(Array.empty[Transform]) t } @@ -76,53 +79,46 @@ class PlanResolutionSuite extends AnalysisTest { private val tableWithAcceptAnySchemaCapability: Table = { val t = mock(classOf[Table]) when(t.name()).thenReturn("v2TableWithAcceptAnySchemaCapability") - when(t.schema()).thenReturn(new StructType().add("i", "int")) + when(t.columns()).thenReturn(Array(Column.create("i", IntegerType))) when(t.capabilities()).thenReturn(Collections.singleton(TableCapability.ACCEPT_ANY_SCHEMA)) t } private val charVarcharTable: Table = { val t = mock(classOf[Table]) - when(t.schema()).thenReturn(new StructType().add("c1", "char(5)").add("c2", "varchar(5)")) + when(t.columns()).thenReturn( + Array(Column.create("c1", CharType(5)), Column.create("c2", VarcharType(5)))) when(t.partitioning()).thenReturn(Array.empty[Transform]) t } private val defaultValues: Table = { val t = mock(classOf[Table]) - when(t.schema()).thenReturn( - new StructType() - .add("i", BooleanType, true, - new MetadataBuilder() - .putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "true") - .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "true").build()) - .add("s", IntegerType, true, - new MetadataBuilder() - .putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "42") - .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "42").build())) + val default1 = new ColumnDefaultValue("true", LiteralValue(true, BooleanType)) + val default2 = new ColumnDefaultValue("42", LiteralValue(42, IntegerType)) + when(t.columns()).thenReturn(Array( + Column.create("i", BooleanType, true, null, default1, null), + Column.create("s", IntegerType, true, null, default2, null))) when(t.partitioning()).thenReturn(Array.empty[Transform]) t } private val defaultValues2: Table = { val t = mock(classOf[Table]) - when(t.schema()).thenReturn( - new StructType() - .add("i", StringType) - .add("e", StringType, true, - new MetadataBuilder() - .putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "'abc'") - .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "'abc'").build())) + val default = new ColumnDefaultValue( + "'abc'", LiteralValue(UTF8String.fromString("abc"), StringType)) + when(t.columns()).thenReturn(Array( + Column.create("i", StringType), + Column.create("e", StringType, true, null, default, null))) when(t.partitioning()).thenReturn(Array.empty[Transform]) t } private val tableWithColumnNamedDefault: Table = { val t = mock(classOf[Table]) - when(t.schema()).thenReturn( - new StructType() - .add("s", StringType) - .add("default", StringType)) + when(t.columns()).thenReturn(Array( + Column.create("s", StringType), + Column.create("default", StringType))) when(t.partitioning()).thenReturn(Array.empty[Transform]) t } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala index d0169bde40ff5..33e2fc46ccba4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala @@ -23,6 +23,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.QueryTest import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTableCatalog} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.functions.lit import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType @@ -51,7 +52,7 @@ class InMemoryTableMetricSuite testCatalog.createTable( Identifier.of(Array(), "table_name"), new StructType().add("i", "int"), - Array.empty, Collections.emptyMap[String, String]) + Array.empty[Transform], Collections.emptyMap[String, String]) func("testcat.table_name") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index 2a441157f9da4..8f5996438e202 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -38,6 +39,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap abstract class V2SessionCatalogBaseSuite extends SharedSparkSession with BeforeAndAfter { val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] + val emptyTrans: Array[Transform] = Array.empty val schema: StructType = new StructType() .add("id", IntegerType) .add("data", StringType) @@ -95,13 +97,13 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(catalog.listTables(Array("ns")).isEmpty) - catalog.createTable(ident1, schema, Array.empty, emptyProps) + catalog.createTable(ident1, schema, emptyTrans, emptyProps) assert(catalog.listTables(Array("ns")).toSet == Set(ident1)) assert(catalog.listTables(Array("ns2")).isEmpty) - catalog.createTable(ident3, schema, Array.empty, emptyProps) - catalog.createTable(ident2, schema, Array.empty, emptyProps) + catalog.createTable(ident3, schema, emptyTrans, emptyProps) + catalog.createTable(ident2, schema, emptyTrans, emptyProps) assert(catalog.listTables(Array("ns")).toSet == Set(ident1, ident2)) assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) @@ -123,7 +125,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) assert(parsed == Seq("db", "test_table")) @@ -141,7 +143,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) - val table = catalog.createTable(testIdent, schema, Array.empty, properties) + val table = catalog.createTable(testIdent, schema, emptyTrans, properties) val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) assert(parsed == Seq("db", "test_table")) @@ -156,13 +158,13 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) .map(part => quoteIdentifier(part)).mkString(".") val exc = intercept[TableAlreadyExistsException] { - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) } checkErrorTableAlreadyExists(exc, parsed) @@ -183,26 +185,26 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) // default location - val t1 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + val t1 = catalog.createTable(testIdent, schema, emptyTrans, properties).asInstanceOf[V1Table] assert(t1.catalogTable.location === spark.sessionState.catalog.defaultTablePath(testIdent.asTableIdentifier)) catalog.dropTable(testIdent) // relative path properties.put(TableCatalog.PROP_LOCATION, "relative/path") - val t2 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + val t2 = catalog.createTable(testIdent, schema, emptyTrans, properties).asInstanceOf[V1Table] assert(t2.catalogTable.location === makeQualifiedPathWithWarehouse("db.db/relative/path")) catalog.dropTable(testIdent) // absolute path without scheme properties.put(TableCatalog.PROP_LOCATION, "/absolute/path") - val t3 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + val t3 = catalog.createTable(testIdent, schema, emptyTrans, properties).asInstanceOf[V1Table] assert(t3.catalogTable.location.toString === "file:///absolute/path") catalog.dropTable(testIdent) // absolute path with scheme properties.put(TableCatalog.PROP_LOCATION, "file:/absolute/path") - val t4 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + val t4 = catalog.createTable(testIdent, schema, emptyTrans, properties).asInstanceOf[V1Table] assert(t4.catalogTable.location.toString === "file:/absolute/path") catalog.dropTable(testIdent) } @@ -212,7 +214,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) @@ -224,7 +226,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("loadTable") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) val loaded = catalog.loadTable(testIdent) assert(table.name == loaded.name) @@ -245,7 +247,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("invalidateTable") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) catalog.invalidateTable(testIdent) val loaded = catalog.loadTable(testIdent) @@ -266,7 +268,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add property") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(filterV2TableProperties(table.properties) == Map()) @@ -285,7 +287,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val properties = new util.HashMap[String, String]() properties.put("prop-1", "1") - val table = catalog.createTable(testIdent, schema, Array.empty, properties) + val table = catalog.createTable(testIdent, schema, emptyTrans, properties) assert(filterV2TableProperties(table.properties) == Map("prop-1" -> "1")) @@ -304,7 +306,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val properties = new util.HashMap[String, String]() properties.put("prop-1", "1") - val table = catalog.createTable(testIdent, schema, Array.empty, properties) + val table = catalog.createTable(testIdent, schema, emptyTrans, properties) assert(filterV2TableProperties(table.properties) == Map("prop-1" -> "1")) @@ -320,7 +322,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: remove missing property") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(filterV2TableProperties(table.properties) == Map()) @@ -336,7 +338,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add top-level column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -348,7 +350,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add required column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -361,7 +363,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add column with comment") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -378,7 +380,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -393,7 +395,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add column to primitive field fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -411,7 +413,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add field to missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -427,7 +429,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: update column data type") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -443,7 +445,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val originalSchema = new StructType() .add("id", IntegerType, nullable = false) .add("data", StringType) - val table = catalog.createTable(testIdent, originalSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, originalSchema, emptyTrans, emptyProps) assert(table.schema == originalSchema) @@ -457,7 +459,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: update missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -473,7 +475,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add comment") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -489,7 +491,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: replace comment") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -508,7 +510,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add comment to missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -524,7 +526,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: rename top-level column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -541,7 +543,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -560,7 +562,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -576,7 +578,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: rename missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -595,7 +597,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -612,7 +614,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: delete top-level column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -629,7 +631,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -645,7 +647,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: delete missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(table.schema == schema) @@ -667,7 +669,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) val tableSchema = schema.add("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) assert(table.schema == tableSchema) @@ -698,7 +700,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) // default location - val t1 = catalog.createTable(testIdent, schema, Array.empty, emptyProps).asInstanceOf[V1Table] + val t1 = catalog.createTable(testIdent, schema, emptyTrans, emptyProps).asInstanceOf[V1Table] assert(t1.catalogTable.location === spark.sessionState.catalog.defaultTablePath(testIdent.asTableIdentifier)) @@ -723,7 +725,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) @@ -750,7 +752,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) assert(!catalog.tableExists(testIdentNew)) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) catalog.renameTable(testIdent, testIdentNew) @@ -775,8 +777,8 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) assert(!catalog.tableExists(testIdentNew)) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) - catalog.createTable(testIdentNew, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdentNew, schema, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) assert(catalog.tableExists(testIdentNew)) @@ -795,7 +797,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) assert(!catalog.tableExists(testIdentNewOtherDb)) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) @@ -982,7 +984,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { assert(catalog.namespaceExists(testNs) === false) val exc = intercept[NoSuchDatabaseException] { - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) } assert(exc.getMessage.contains(testNs.quoted)) @@ -1016,7 +1018,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { val catalog = newCatalog() catalog.createNamespace(testNs, Map("property" -> "value").asJava) - catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdent, schema, emptyTrans, emptyProps) val exc = intercept[AnalysisException] { catalog.dropNamespace(testNs, cascade = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala index ba0dfa951781a..4da7dbb93e440 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.sources import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expression, Literal} import org.apache.spark.sql.catalyst.plans.SQLHelper @@ -69,7 +68,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with SQLHelper { Cast(e, dt, Option(SQLConf.get.sessionLocalTimeZone)) } } - val rule = DataSourceAnalysis(SimpleAnalyzer) + val rule = DataSourceAnalysis testRule( "convertStaticPartitions only handle INSERT having at least static partitions", caseSensitive) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index cc1d4ab3fcdf5..c1fbbdcfee6ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -1043,9 +1043,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-38336 INSERT INTO statements with tables with default columns: negative tests") { object Errors { - val COMMON_SUBSTRING = " has a DEFAULT value" + val COMMON_SUBSTRING = "has an invalid DEFAULT value" val COLUMN_DEFAULT_NOT_FOUND = "`default` cannot be resolved." - val BAD_SUBQUERY = "subquery expressions are not allowed in DEFAULT values" } // The default value fails to analyze. withTable("t") { @@ -1058,21 +1057,21 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { assert(intercept[AnalysisException] { sql("create table t(i boolean, s bigint default (select min(x) from badtable)) " + "using parquet") - }.getMessage.contains(Errors.BAD_SUBQUERY)) + }.getMessage.contains(Errors.COMMON_SUBSTRING)) } // The default value parses but refers to a table from the catalog. withTable("t", "other") { sql("create table other(x string) using parquet") assert(intercept[AnalysisException] { sql("create table t(i boolean, s bigint default (select min(x) from other)) using parquet") - }.getMessage.contains(Errors.BAD_SUBQUERY)) + }.getMessage.contains(Errors.COMMON_SUBSTRING)) } // The default value has an explicit alias. It fails to evaluate when inlined into the VALUES // list at the INSERT INTO time. withTable("t") { assert(intercept[AnalysisException] { sql("create table t(i boolean default (select false as alias), s bigint) using parquet") - }.getMessage.contains(Errors.BAD_SUBQUERY)) + }.getMessage.contains(Errors.COMMON_SUBSTRING)) } // Explicit default values may not participate in complex expressions in the VALUES list. withTable("t") { @@ -1424,8 +1423,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Negative tests") { object Errors { - val COMMON_SUBSTRING = " has a DEFAULT value" - val BAD_SUBQUERY = "subquery expressions are not allowed in DEFAULT values" + val COMMON_SUBSTRING = "has an invalid DEFAULT value" } // The default value fails to analyze. withTable("t") { @@ -1439,7 +1437,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { sql("create table t(i boolean) using parquet") assert(intercept[AnalysisException] { sql("alter table t add column s bigint default (select min(x) from badtable)") - }.getMessage.contains(Errors.BAD_SUBQUERY)) + }.getMessage.contains(Errors.COMMON_SUBSTRING)) } // The default value parses but refers to a table from the catalog. withTable("t", "other") { @@ -1447,14 +1445,14 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { sql("create table t(i boolean) using parquet") assert(intercept[AnalysisException] { sql("alter table t add column s bigint default (select min(x) from other)") - }.getMessage.contains(Errors.BAD_SUBQUERY)) + }.getMessage.contains(Errors.COMMON_SUBSTRING)) } // The default value parses but the type is not coercible. withTable("t") { sql("create table t(i boolean) using parquet") assert(intercept[AnalysisException] { sql("alter table t add column s bigint default false") - }.getMessage.contains("provided a value of incompatible type")) + }.getMessage.contains("The default value has an incompatible data type")) } // The default value is disabled per configuration. withTable("t") { @@ -1502,8 +1500,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-38838 INSERT INTO with defaults set by ALTER TABLE ALTER COLUMN: negative tests") { object Errors { - val COMMON_SUBSTRING = " has a DEFAULT value" - val BAD_SUBQUERY = "subquery expressions are not allowed in DEFAULT values" + val COMMON_SUBSTRING = "has an invalid DEFAULT value" } val createTable = "create table t(i boolean, s bigint) using parquet" withTable("t") { @@ -1515,16 +1512,16 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { // The default value analyzes to a table not in the catalog. assert(intercept[AnalysisException] { sql("alter table t alter column s set default (select min(x) from badtable)") - }.getMessage.contains(Errors.BAD_SUBQUERY)) + }.getMessage.contains(Errors.COMMON_SUBSTRING)) // The default value has an explicit alias. It fails to evaluate when inlined into the VALUES // list at the INSERT INTO time. assert(intercept[AnalysisException] { sql("alter table t alter column s set default (select 42 as alias)") - }.getMessage.contains(Errors.BAD_SUBQUERY)) + }.getMessage.contains(Errors.COMMON_SUBSTRING)) // The default value parses but the type is not coercible. assert(intercept[AnalysisException] { sql("alter table t alter column s set default false") - }.getMessage.contains("provided a value of incompatible type")) + }.getMessage.contains("The default value has an incompatible data type")) // The default value is disabled per configuration. withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") { val sqlText = "alter table t alter column s set default 41 + 1" @@ -1737,7 +1734,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-39359 Restrict DEFAULT columns to allowlist of supported data source types") { withSQLConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS.key -> "csv,json,orc") { - val unsupported = "DEFAULT values are not supported for target data source" + val unsupported = "does not support column default value" assert(intercept[AnalysisException] { sql(s"create table t(a string default 'abc') using parquet") }.getMessage.contains(unsupported)) @@ -1779,9 +1776,6 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } // Negative tests: provided array element types must match their corresponding DEFAULT // declarations, if applicable. - val incompatibleDefault = - "Failed to execute ALTER TABLE ADD COLUMNS command because the destination table column s " + - "has a DEFAULT value with type" Seq( Config( "parquet"), @@ -1797,7 +1791,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } assert(intercept[AnalysisException] { sql("alter table t add column s array default array('abc', 'def')") - }.getMessage.contains(incompatibleDefault)) + }.getMessage.contains("The default value has an incompatible data type")) } } } @@ -1832,9 +1826,6 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { // Negative tests: provided map element types must match their corresponding DEFAULT // declarations, if applicable. - val incompatibleDefault = - "Failed to execute ALTER TABLE ADD COLUMNS command because the destination table column s " + - "has a DEFAULT value with type" Seq( Config( "parquet"), @@ -1850,7 +1841,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } assert(intercept[AnalysisException] { sql("alter table t add column s struct default struct(42, 56)") - }.getMessage.contains(incompatibleDefault)) + }.getMessage.contains("The default value has an incompatible data type")) } } } @@ -1951,9 +1942,6 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } // Negative tests: provided map element types must match their corresponding DEFAULT // declarations, if applicable. - val incompatibleDefault = - "Failed to execute ALTER TABLE ADD COLUMNS command because the destination table column s " + - "has a DEFAULT value with type" Seq( Config( "parquet"), @@ -1969,7 +1957,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } assert(intercept[AnalysisException] { sql("alter table t add column s map default map(42, 56)") - }.getMessage.contains(incompatibleDefault)) + }.getMessage.contains("The default value has an incompatible data type")) } } } @@ -1982,47 +1970,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { ).foreach { query => assert(intercept[AnalysisException] { sql(query) - }.getMessage.contains( - QueryCompilationErrors.defaultValuesMayNotContainSubQueryExpressions().getMessage)) - } - } - - test("SPARK-39844 Restrict adding DEFAULT columns for existing tables to certain sources") { - Seq("csv", "json", "orc", "parquet").foreach { provider => - withTable("t1") { - // Set the allowlist of table providers to include the new table type for all SQL commands. - withSQLConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS.key -> provider) { - // It is OK to create a new table with a column DEFAULT value assigned if the table - // provider is in the allowlist. - sql(s"create table t1(a int default 42) using $provider") - // It is OK to add a new column to the table with a DEFAULT value to the existing table - // since this table provider is not yet present in the - // 'ADD_DEFAULT_COLUMN_EXISTING_TABLE_BANNED_PROVIDERS' denylist. - sql(s"alter table t1 add column (b string default 'abc')") - // Insert a row into the table and check that the assigned DEFAULT value is correct. - sql(s"insert into t1 values (42, default)") - checkAnswer(spark.table("t1"), Row(42, "abc")) - } - // Now update the allowlist of table providers to prohibit ALTER TABLE ADD COLUMN commands - // from assigning DEFAULT values. - withSQLConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS.key -> s"$provider*") { - assert(intercept[AnalysisException] { - // Try to add another column to the existing table again. This fails because the table - // provider is now in the denylist. - sql(s"alter table t1 add column (b string default 'abc')") - }.getMessage.contains( - QueryCompilationErrors.addNewDefaultColumnToExistingTableNotAllowed( - "ALTER TABLE ADD COLUMNS", provider).getMessage)) - withTable("t2") { - // It is still OK to create a new table with a column DEFAULT value assigned, even if - // the table provider is in the above denylist. - sql(s"create table t2(a int default 42) using $provider") - // Insert a row into the table and check that the assigned DEFAULT value is correct. - sql(s"insert into t2 values (default)") - checkAnswer(spark.table("t2"), Row(42)) - } - } - } + }.getMessage.contains("The default value must be a constant")) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index c4e0057ae952d..0c16d8dc1d0b6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -100,7 +100,7 @@ class HiveSessionStateBuilder( RelationConversions(catalog) +: PreprocessTableCreation(session) +: PreprocessTableInsertion +: - DataSourceAnalysis(this) +: + DataSourceAnalysis +: ApplyCharTypePadding +: HiveAnalysis +: ReplaceCharWithVarchar +: diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 13f2a865936f5..ea1e9a7e0486f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -869,7 +869,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter |SORTED BY (s1) |INTO 200 BUCKETS |STORED AS PARQUET - """.stripMargin + |""".stripMargin } else { """ |CREATE TABLE test1( @@ -880,14 +880,14 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter |CLUSTERED BY (v1) |SORTED BY (s1) |INTO 200 BUCKETS - """.stripMargin + |""".stripMargin } val insertString = """ |INSERT INTO test1 |SELECT * FROM VALUES(1,1,1) - """.stripMargin + |""".stripMargin val dropString = "DROP TABLE IF EXISTS test1" From f5bc509f3b90981eb86907d9d310681c5b6eeafd Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 15 Feb 2023 18:18:54 +0800 Subject: [PATCH 02/13] improve --- .../main/resources/error/error-classes.json | 5 ++ .../spark/sql/connector/catalog/Column.java | 8 +++ .../connector/catalog/ColumnDefaultValue.java | 15 ++++++ .../sql/connector/catalog/TableChange.java | 4 +- .../sql/catalyst/analysis/Analyzer.scala | 25 ++++++---- .../analysis/ResolveDefaultColumns.scala | 6 +-- .../util/ResolveDefaultColumnsUtil.scala | 26 ++++++---- .../sql/errors/QueryCompilationErrors.scala | 6 +++ .../apache/spark/sql/internal/SQLConf.scala | 11 ++-- .../apache/spark/sql/types/StructType.scala | 50 +++++++++++-------- .../sql/connector/DataSourceV2SQLSuite.scala | 4 +- .../spark/sql/sources/InsertSuite.scala | 39 +++++++++++++++ 12 files changed, 146 insertions(+), 53 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index f13dc3789646f..c42cbe6b4d1e7 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3636,6 +3636,11 @@ "Invalid DEFAULT value for column : fails to parse as a valid literal value." ] }, + "_LEGACY_ERROR_TEMP_1345" : { + "message" : [ + "Failed to execute ALTER TABLE ADD COLUMNS command because DEFAULT values are not supported for target data source with table provider: \"\"." + ] + }, "_LEGACY_ERROR_TEMP_1346" : { "message" : [ "Failed to execute command because DEFAULT values are not supported when adding new columns to previously existing target data source with table provider: \"\"." diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java index 08ce6629d7f1f..d566dadceea60 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java @@ -22,6 +22,14 @@ import org.apache.spark.sql.internal.connector.ColumnImpl; import org.apache.spark.sql.types.DataType; +/** + * An interface representing a column of a {@link Table}. It defines basic properties of a column, + * such as name and data type, as well as some advanced ones like default column value. + *

+ * Data Sources do not need to implement it. They should consume it in APIs like + * @{@link TableCatalog#createTable(Identifier, Column[], Transform[], Map)}, and report it in + * {@link Table#columns()} by calling the static `create` functions of this interface to create it. + */ public interface Column { static Column create(String name, DataType dataType) { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java index 9cf45f678a228..cd0b8ea2946b3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java @@ -22,6 +22,15 @@ import org.apache.spark.sql.connector.expressions.Literal; +/** + * A class representing the default value of a column. It contains both the SQL string and literal + * value of the user-specified default value expression. The SQL string should be re-evaluated for + * each table writing, which may produce different values if the default value expression is + * something like {@code CURRENT_DATE()}. The literal value is used to back-fill existing data if + * new columns with default value are added. Note: the back-fill can be lazy. The data sources can + * remember the column default value and let the reader fill the column value when reading existing + * data that do not have these new columns. + */ public class ColumnDefaultValue { private String sql; private Literal value; @@ -31,11 +40,17 @@ public ColumnDefaultValue(String sql, Literal value) { this.value = value; } + /** + * Returns the SQL string of the default value expression. + */ @Nonnull public String getSql() { return sql; } + /** + * Returns the default value literal. + */ @Nonnull public Literal getValue() { return value; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java index 29dc7fb825789..609cfab2d568e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java @@ -22,7 +22,6 @@ import javax.annotation.Nullable; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.connector.expressions.Literal; import org.apache.spark.sql.types.DataType; /** @@ -386,7 +385,7 @@ interface ColumnChange extends TableChange { /** * A TableChange to add a field. The implementation may need to back-fill all the existing data * to add this new column, or remember the column default value specified here and let the reader - * to fill the column value when reading these existing data that do not have this column. + * fill the column value when reading existing data that do not have this new column. *

* If the field already exists, the change must result in an {@link IllegalArgumentException}. * If the new field is nested and its parent does not exist or is not a struct, the change must @@ -697,6 +696,7 @@ public String[] fieldNames() { /** * Returns the column default value SQL string (Spark SQL dialect). The default value literal * is not provided as updating column default values does not need to back-fill existing data. + * Null means dropping the column default value. */ @Nullable public String newDefaultValue() { return newDefaultValue; } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 46deaf309e99f..03607b156d5be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -3677,13 +3677,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val columnsToDrop = a.columnsToDrop a.copy(columnsToDrop = columnsToDrop.flatMap(c => resolveFieldNamesOpt(table, c.name, c))) - case a: AlterTableCommand if a.table.resolved && hasUnresolvedFieldName(a) => - val table = a.table.asInstanceOf[ResolvedTable] - a.transformExpressions { - case u: UnresolvedFieldName => resolveFieldNames(table, u.name, u) - } - - case a @ AddColumns(r: ResolvedTable, cols) => + case a @ AddColumns(r: ResolvedTable, cols) if !a.resolved => // 'colsToAdd' keeps track of new columns being added. It stores a mapping from a // normalized parent name of fields to field names that belong to the parent. // For example, if we add columns "a.b.c", "a.b.d", and "a.c", 'colsToAdd' will become @@ -3714,11 +3708,14 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } val schema = r.table.columns.asSchema val resolvedCols = cols.map { col => - col.path match { + val path = col.path.transform { + case u: UnresolvedFieldName => resolveFieldNames(r, u.name, u) + }.asInstanceOf[FieldName] + path match { case RootTableSchema => // Adding to the root. Just need to resolve position. val resolvedPosition = resolvePosition(col, schema, Nil) - col.copy(position = resolvedPosition) + col.copy(path = path, position = resolvedPosition) case parent: ResolvedFieldName => val parentSchema = parent.field.dataType match { case s: StructType => s @@ -3726,8 +3723,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor col.name, parent.name, parent.origin) } val resolvedPosition = resolvePosition(col, parentSchema, parent.name) - col.copy(position = resolvedPosition) - // This should not happen. All `UnresolvedFieldName` should have been resolved before. + col.copy(path = path, position = resolvedPosition) + // This should not happen. All `UnresolvedFieldName` should have been resolved. case _ => col } } @@ -3755,6 +3752,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val resolved = a.copy(dataType = newDataType, position = newPosition) resolved.copyTagsFrom(a) resolved + + case a: AlterTableCommand if a.table.resolved && hasUnresolvedFieldName(a) => + val table = a.table.asInstanceOf[ResolvedTable] + a.transformExpressions { + case u: UnresolvedFieldName => resolveFieldNames(table, u.name, u) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultColumns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultColumns.scala index a04844c6526db..b09a2c4ef3eec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultColumns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultColumns.scala @@ -155,7 +155,7 @@ case class ResolveDefaultColumns(catalog: SessionCatalog) extends Rule[LogicalPl val schemaForTargetTable: Option[StructType] = getSchemaForTargetTable(u.table) schemaForTargetTable.map { schema => val defaultExpressions: Seq[Expression] = schema.fields.map { - case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f, "UPDATE") + case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f) case _ => Literal(null) } // Create a map from each column name in the target table to its DEFAULT expression. @@ -185,7 +185,7 @@ case class ResolveDefaultColumns(catalog: SessionCatalog) extends Rule[LogicalPl } } val defaultExpressions: Seq[Expression] = schema.fields.map { - case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f, "MERGE") + case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f) case _ => Literal(null) } val columnNamesToExpressions: Map[String, Expression] = @@ -349,7 +349,7 @@ case class ResolveDefaultColumns(catalog: SessionCatalog) extends Rule[LogicalPl input: LogicalPlan): Option[LogicalPlan] = { val schema = insertTableSchemaWithoutPartitionColumns val defaultExpressions: Seq[Expression] = schema.fields.map { - case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f, "INSERT") + case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f) case _ => Literal(null) } // Check the type of `input` and replace its expressions accordingly. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala index 780cc4e08565c..113a0f13ae234 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala @@ -75,14 +75,12 @@ object ResolveDefaultColumns { * * @param field represents the DEFAULT column value whose "default" metadata to parse * and analyze. - * @param statementType which type of statement we are running, such as INSERT; useful for errors. * @param metadataKey which key to look up from the column metadata; generally either * CURRENT_DEFAULT_COLUMN_METADATA_KEY or EXISTS_DEFAULT_COLUMN_METADATA_KEY. * @return Result of the analysis and constant-folding operation. */ def analyze( field: StructField, - statementType: String, metadataKey: String = CURRENT_DEFAULT_COLUMN_METADATA_KEY): Expression = { // Parse the expression. val colText: String = field.metadata.getString(metadataKey) @@ -129,7 +127,7 @@ object ResolveDefaultColumns { val defaultValue: Option[String] = field.getExistenceDefaultValue() defaultValue.map { text: String => val expr = try { - val expr = analyze(field, "", EXISTS_DEFAULT_COLUMN_METADATA_KEY) + val expr = analyze(field, EXISTS_DEFAULT_COLUMN_METADATA_KEY) expr match { case _: ExprLiteral | _: Cast => expr } @@ -211,11 +209,12 @@ object ResolveDefaultColumns { case cmd: AlterTableCommand => val table = cmd.resolvedTable + val provider = getTableProviderFromProp(table.table.properties()) + val isAddColumns = cmd.isInstanceOf[AddColumns] cmd.transformExpressionsDown { case q @ QualifiedColType(path, Column(name, dataType, _, _, Some(default), _), _) - if path.resolved => - checkTableProvider( - table.catalog, table.name, getTableProviderFromProp(table.table.properties())) + if path.resolved => + checkTableProvider(table.catalog, table.name, provider, isAddColumns) checkDefaultValue( default, table.name, @@ -236,17 +235,24 @@ object ResolveDefaultColumns { private def checkTableProvider( catalog: CatalogPlugin, tableName: String, - provider: Option[String]): Unit = { + provider: Option[String], + isAddColumns: Boolean = false): Unit = { // We only need to check table provider for the session catalog. Other custom v2 catalogs // can check table providers in their implementations of createTable, alterTable, etc. if (CatalogV2Util.isSessionCatalog(catalog)) { val conf = SQLConf.get - val allowedProviders: Array[String] = conf.getConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS) + val keywords: Array[String] = conf.getConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS) .toLowerCase().split(",").map(_.trim) - val providerName = provider.getOrElse(conf.defaultDataSourceName).toLowerCase() - if (!allowedProviders.contains(providerName)) { + val allowedTableProviders: Array[String] = keywords.map(_.stripSuffix("*")) + val addColumnExistingTableBannedProviders: Array[String] = + keywords.filter(_.endsWith("*")).map(_.stripSuffix("*")) + val providerName: String = provider.getOrElse(conf.defaultDataSourceName).toLowerCase() + if (!allowedTableProviders.contains(providerName)) { throw QueryCompilationErrors.defaultReferencesNotAllowedInDataSource(tableName) } + if (isAddColumns && addColumnExistingTableBannedProviders.contains(providerName)) { + throw QueryCompilationErrors.addNewDefaultColumnToExistingTableNotAllowed(providerName) + } } } 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 c927cad9d8211..7918ab7a30955 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 @@ -3156,6 +3156,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { "operation" -> "column default value")) } + def addNewDefaultColumnToExistingTableNotAllowed(dataSource: String): Throwable = { + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1345", + messageParameters = Map("dataSource" -> dataSource)) + } + def notConstantDefaultValueError( tableName: String, colName: Seq[String], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f2297a2bdaad7..9b8b97c28c394 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3100,11 +3100,12 @@ object SQLConf { buildConf("spark.sql.defaultColumn.allowedProviders") .internal() .doc("List of table providers wherein SQL commands are permitted to assign DEFAULT column " + - "values. Comma-separated list, whitespace ignored, case-insensitive. The data sources " + - "must support substituting in the provided values when the corresponding fields are not " + - "present in storage. This can happen when users create an external table and specify " + - "columns with default value, or add columns with default value to an existing non-empty " + - "table.") + "values. Comma-separated list, whitespace ignored, case-insensitive. If an asterisk " + + "appears after any table provider in this list, any command may assign DEFAULT column " + + "except `ALTER TABLE ... ADD COLUMN`. Otherwise, if no asterisk appears, all commands " + + "are permitted. This is useful because in order for such `ALTER TABLE ... ADD COLUMN` " + + "commands to work, the target data source must include support for substituting in the " + + "provided values when the corresponding fields are not present in storage.") .version("3.4.0") .stringConf .createWithDefault("csv,json,orc,parquet,hive") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 6786ffd745375..4c573532600d4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -25,7 +25,7 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.analysis.Resolver -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering, Literal} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} import org.apache.spark.sql.catalyst.plans.logical.{Column, DefaultValueExpression} import org.apache.spark.sql.catalyst.trees.Origin @@ -396,28 +396,38 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru protected[sql] def toAttributes: Seq[AttributeReference] = map(field => field.toAttribute) private[sql] def toColumns: Array[Column] = fields.map { f => - val defaultValue = f.getCurrentDefaultValue().map { sql => - val existDefaultOpt = f.getExistenceDefaultValue() - assert(existDefaultOpt.isDefined, "current and exist default must be both set or neither") - val e = CatalystSqlParser.parseExpression(f.getExistenceDefaultValue().get) - assert(e.resolved && e.foldable, - "exist default must be simple SQL string that is resolved and foldable after parsing, " + - "but got: " + existDefaultOpt.get) - DefaultValueExpression(e, sql) + def createColumn( + defaultValue: Option[DefaultValueExpression], + metadata: Metadata): Column = { + val metadataJSON = if (metadata == Metadata.empty) { + None + } else { + Some(metadata.json) + } + Column(f.name, f.dataType, f.nullable, f.getComment(), defaultValue, metadataJSON) } - - val cleanedMetadata = new MetadataBuilder() - .withMetadata(f.metadata) - .remove("comment") - .remove(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY) - .remove(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY) - .build() - val metadataInJSON = if (cleanedMetadata == Metadata.empty) { - None + if (f.getCurrentDefaultValue().isDefined && f.getExistenceDefaultValue().isDefined) { + val e = ResolveDefaultColumns.analyze( + f, ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY) + assert(e.resolved && e.foldable, + "exist default must be simple SQL string that is resolved and foldable, " + + "but got: " + f.getExistenceDefaultValue().get) + val defaultValue = Some(DefaultValueExpression( + Literal(e.eval(), f.dataType), f.getCurrentDefaultValue().get)) + val cleanedMetadata = new MetadataBuilder() + .withMetadata(f.metadata) + .remove("comment") + .remove(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY) + .remove(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY) + .build() + createColumn(defaultValue, cleanedMetadata) } else { - Some(cleanedMetadata.json) + val cleanedMetadata = new MetadataBuilder() + .withMetadata(f.metadata) + .remove("comment") + .build() + createColumn(None, cleanedMetadata) } - Column(f.name, f.dataType, f.nullable, f.getComment(), defaultValue, metadataInJSON) } def treeString: String = treeString(Int.MaxValue) 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 e309d4a95dc70..83fe2ba51b57a 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 @@ -46,7 +46,7 @@ import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG_IMPLEMENTATION} import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.SimpleScanSource -import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} +import org.apache.spark.sql.types.{LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.unsafe.types.UTF8String @@ -603,7 +603,7 @@ class DataSourceV2SQLSuiteV1Filter "Replaced table should have new schema.") val actual = replaced.columns.head val expected = ColumnV2.create("id", LongType, false, null, - new ColumnDefaultValue("41 + 1", LiteralValue(42, IntegerType)), null) + new ColumnDefaultValue("41 + 1", LiteralValue(42L, LongType)), null) assert(actual === expected, "Replaced table should have new schema with DEFAULT column metadata.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index c1fbbdcfee6ed..7288b9c11a0da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -1974,6 +1974,45 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } } + test("SPARK-39844 Restrict adding DEFAULT columns for existing tables to certain sources") { + Seq("csv", "json", "orc", "parquet").foreach { provider => + withTable("t1") { + // Set the allowlist of table providers to include the new table type for all SQL commands. + withSQLConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS.key -> provider) { + // It is OK to create a new table with a column DEFAULT value assigned if the table + // provider is in the allowlist. + sql(s"create table t1(a int default 42) using $provider") + // It is OK to add a new column to the table with a DEFAULT value to the existing table + // since this table provider is not yet present in the + // 'ADD_DEFAULT_COLUMN_EXISTING_TABLE_BANNED_PROVIDERS' denylist. + sql(s"alter table t1 add column (b string default 'abc')") + // Insert a row into the table and check that the assigned DEFAULT value is correct. + sql(s"insert into t1 values (42, default)") + checkAnswer(spark.table("t1"), Row(42, "abc")) + } + // Now update the allowlist of table providers to prohibit ALTER TABLE ADD COLUMN commands + // from assigning DEFAULT values. + withSQLConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS.key -> s"$provider*") { + assert(intercept[AnalysisException] { + // Try to add another column to the existing table again. This fails because the table + // provider is now in the denylist. + sql(s"alter table t1 add column (b string default 'abc')") + }.getMessage.contains( + QueryCompilationErrors.addNewDefaultColumnToExistingTableNotAllowed( + provider).getMessage)) + withTable("t2") { + // It is still OK to create a new table with a column DEFAULT value assigned, even if + // the table provider is in the above denylist. + sql(s"create table t2(a int default 42) using $provider") + // Insert a row into the table and check that the assigned DEFAULT value is correct. + sql(s"insert into t2 values (default)") + checkAnswer(spark.table("t2"), Row(42)) + } + } + } + } + } + test("Stop task set if FileAlreadyExistsException was thrown") { val tableName = "t" Seq(true, false).foreach { fastFail => From f07c99102fdd1268ed39ea8c1c2946eda59913ac Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 15 Feb 2023 18:21:59 +0800 Subject: [PATCH 03/13] revert for proto --- .../queries/function_udf_2.12.json | 8 ++++---- .../queries/function_udf_2.12.proto.bin | Bin 11257 -> 11257 bytes .../apache/spark/sql/types/StructField.scala | 8 ++++---- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.json b/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.json index 010dfa84e8596..76738354e15b7 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.json @@ -21,7 +21,7 @@ } }], "scalarScalaUdf": { - "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQyWQlE7Ce2cPkCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAABdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAAXNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwc3IARm9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJvZHVjdEVuY29kZXIcqKluUDodYQIAA0wABmNsc1RhZ3EAfgAPTAAGZmllbGRzcQB+AAJMAAZzY2hlbWF0ACdMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0VHlwZTt4cHNyACZzY2FsYS5yZWZsZWN0LkNsYXNzVGFnJEdlbmVyaWNDbGFzc1RhZwAAAAAAAAABAgABTAAMcnVudGltZUNsYXNzdAARTGphdmEvbGFuZy9DbGFzczt4cHZyAAxzY2FsYS5UdXBsZTHPDUf18JuzPAIAAUwAAl8xcQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcgAsc2NhbGEuY29sbGVjdGlvbi5pbW11dGFibGUuTGlzdFNlcmlhbGl6ZUVuZCSKXGNb91MLbQIAAHhweHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RUeXBllANIqWlTBxgCAAlJAAlfaGFzaENvZGVCAAhiaXRtYXAkMFoAGWhhc0V4aXN0ZW5jZURlZmF1bHRWYWx1ZXNbABZleGlzdGVuY2VEZWZhdWx0VmFsdWVzcQB+AAhbABhleGlzdGVuY2VEZWZhdWx0c0JpdG1hc2t0AAJbWkwADWZpZWxkTmFtZXNTZXR0ACBMc2NhbGEvY29sbGVjdGlvbi9pbW11dGFibGUvU2V0O1sABmZpZWxkc3QAKVtMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0RmllbGQ7TAALbmFtZVRvRmllbGR0ABZMc2NhbGEvY29sbGVjdGlvbi9NYXA7TAALbmFtZVRvSW5kZXhxAH4ANnhwAAAAAAAAcHBwdXIAKVtMb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQ7tWPEaGAaDUcCAAB4cAAAAAFzcgAmb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQrgSSJZ9l3nwIABFoACG51bGxhYmxlTAAIZGF0YVR5cGVxAH4AEEwACG1ldGFkYXRhcQB+ACRMAARuYW1lcQB+ABR4cABxAH4AGHEAfgApcQB+ACxwcA==", + "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQyWQlE7Ce2cPkCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAABdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAAXNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwc3IARm9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJvZHVjdEVuY29kZXIcqKluUDodYQIAA0wABmNsc1RhZ3EAfgAPTAAGZmllbGRzcQB+AAJMAAZzY2hlbWF0ACdMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0VHlwZTt4cHNyACZzY2FsYS5yZWZsZWN0LkNsYXNzVGFnJEdlbmVyaWNDbGFzc1RhZwAAAAAAAAABAgABTAAMcnVudGltZUNsYXNzdAARTGphdmEvbGFuZy9DbGFzczt4cHZyAAxzY2FsYS5UdXBsZTHPDUf18JuzPAIAAUwAAl8xcQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcgAsc2NhbGEuY29sbGVjdGlvbi5pbW11dGFibGUuTGlzdFNlcmlhbGl6ZUVuZCSKXGNb91MLbQIAAHhweHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RUeXBl9I34EN1tjkUCAAlJAAlfaGFzaENvZGVCAAhiaXRtYXAkMFoAGWhhc0V4aXN0ZW5jZURlZmF1bHRWYWx1ZXNbABZleGlzdGVuY2VEZWZhdWx0VmFsdWVzcQB+AAhbABhleGlzdGVuY2VEZWZhdWx0c0JpdG1hc2t0AAJbWkwADWZpZWxkTmFtZXNTZXR0ACBMc2NhbGEvY29sbGVjdGlvbi9pbW11dGFibGUvU2V0O1sABmZpZWxkc3QAKVtMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0RmllbGQ7TAALbmFtZVRvRmllbGR0ABZMc2NhbGEvY29sbGVjdGlvbi9NYXA7TAALbmFtZVRvSW5kZXhxAH4ANnhwAAAAAAAAcHBwdXIAKVtMb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQ7tWPEaGAaDUcCAAB4cAAAAAFzcgAmb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQrgSSJZ9l3nwIABFoACG51bGxhYmxlTAAIZGF0YVR5cGVxAH4AEEwACG1ldGFkYXRhcQB+ACRMAARuYW1lcQB+ABR4cABxAH4AGHEAfgApcQB+ACxwcA==", "nullable": true } } @@ -39,7 +39,7 @@ } }], "scalarScalaUdf": { - "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQzyD4NN4Grh74CAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAACdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAAnNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFzcgBGb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRQcm9kdWN0RW5jb2RlchyoqW5QOh1hAgADTAAGY2xzVGFncQB+AA9MAAZmaWVsZHNxAH4AAkwABnNjaGVtYXQAJ0xvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9TdHJ1Y3RUeXBlO3hwc3IAJnNjYWxhLnJlZmxlY3QuQ2xhc3NUYWckR2VuZXJpY0NsYXNzVGFnAAAAAAAAAAECAAFMAAxydW50aW1lQ2xhc3N0ABFMamF2YS9sYW5nL0NsYXNzO3hwdnIADHNjYWxhLlR1cGxlMi6UZn1bkvn1AgACTAACXzFxAH4AAUwAAl8ycQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NyACxzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0U2VyaWFsaXplRW5kJIpcY1v3UwttAgAAeHB4c3IAJW9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdFR5cGWUA0ipaVMHGAIACUkACV9oYXNoQ29kZUIACGJpdG1hcCQwWgAZaGFzRXhpc3RlbmNlRGVmYXVsdFZhbHVlc1sAFmV4aXN0ZW5jZURlZmF1bHRWYWx1ZXNxAH4ACFsAGGV4aXN0ZW5jZURlZmF1bHRzQml0bWFza3QAAltaTAANZmllbGROYW1lc1NldHQAIExzY2FsYS9jb2xsZWN0aW9uL2ltbXV0YWJsZS9TZXQ7WwAGZmllbGRzdAApW0xvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9TdHJ1Y3RGaWVsZDtMAAtuYW1lVG9GaWVsZHQAFkxzY2FsYS9jb2xsZWN0aW9uL01hcDtMAAtuYW1lVG9JbmRleHEAfgA4eHAAAAAAAABwcHB1cgApW0xvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RGaWVsZDu1Y8RoYBoNRwIAAHhwAAAAAnNyACZvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RGaWVsZCuBJIln2XefAgAEWgAIbnVsbGFibGVMAAhkYXRhVHlwZXEAfgAQTAAIbWV0YWRhdGFxAH4AJEwABG5hbWVxAH4AFHhwAHEAfgAYcQB+AClxAH4ALHNxAH4APABxAH4AGHEAfgApcQB+ADFwcA==", + "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQzyD4NN4Grh74CAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAACdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAAnNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFzcgBGb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRQcm9kdWN0RW5jb2RlchyoqW5QOh1hAgADTAAGY2xzVGFncQB+AA9MAAZmaWVsZHNxAH4AAkwABnNjaGVtYXQAJ0xvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9TdHJ1Y3RUeXBlO3hwc3IAJnNjYWxhLnJlZmxlY3QuQ2xhc3NUYWckR2VuZXJpY0NsYXNzVGFnAAAAAAAAAAECAAFMAAxydW50aW1lQ2xhc3N0ABFMamF2YS9sYW5nL0NsYXNzO3hwdnIADHNjYWxhLlR1cGxlMi6UZn1bkvn1AgACTAACXzFxAH4AAUwAAl8ycQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NyACxzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0U2VyaWFsaXplRW5kJIpcY1v3UwttAgAAeHB4c3IAJW9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdFR5cGX0jfgQ3W2ORQIACUkACV9oYXNoQ29kZUIACGJpdG1hcCQwWgAZaGFzRXhpc3RlbmNlRGVmYXVsdFZhbHVlc1sAFmV4aXN0ZW5jZURlZmF1bHRWYWx1ZXNxAH4ACFsAGGV4aXN0ZW5jZURlZmF1bHRzQml0bWFza3QAAltaTAANZmllbGROYW1lc1NldHQAIExzY2FsYS9jb2xsZWN0aW9uL2ltbXV0YWJsZS9TZXQ7WwAGZmllbGRzdAApW0xvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9TdHJ1Y3RGaWVsZDtMAAtuYW1lVG9GaWVsZHQAFkxzY2FsYS9jb2xsZWN0aW9uL01hcDtMAAtuYW1lVG9JbmRleHEAfgA4eHAAAAAAAABwcHB1cgApW0xvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RGaWVsZDu1Y8RoYBoNRwIAAHhwAAAAAnNyACZvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RGaWVsZCuBJIln2XefAgAEWgAIbnVsbGFibGVMAAhkYXRhVHlwZXEAfgAQTAAIbWV0YWRhdGFxAH4AJEwABG5hbWVxAH4AFHhwAHEAfgAYcQB+AClxAH4ALHNxAH4APABxAH4AGHEAfgApcQB+ADFwcA==", "nullable": true } } @@ -61,7 +61,7 @@ } }], "scalarScalaUdf": { - "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQ009cpyjjQtFMCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAADdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAA3NyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFxAH4AEXNyAEZvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByb2R1Y3RFbmNvZGVyHKipblA6HWECAANMAAZjbHNUYWdxAH4AD0wABmZpZWxkc3EAfgACTAAGc2NoZW1hdAAnTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL1N0cnVjdFR5cGU7eHBzcgAmc2NhbGEucmVmbGVjdC5DbGFzc1RhZyRHZW5lcmljQ2xhc3NUYWcAAAAAAAAAAQIAAUwADHJ1bnRpbWVDbGFzc3QAEUxqYXZhL2xhbmcvQ2xhc3M7eHB2cgAMc2NhbGEuVHVwbGUzibsxap5k00kCAANMAAJfMXEAfgABTAACXzJxAH4AAUwAAl8zcQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NxAH4AIwBxAH4AEXEAfgApdAACXzNxAH4AL3EAfgAvc3IALHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLkxpc3RTZXJpYWxpemVFbmQkilxjW/dTC20CAAB4cHhzcgAlb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0VHlwZZQDSKlpUwcYAgAJSQAJX2hhc2hDb2RlQgAIYml0bWFwJDBaABloYXNFeGlzdGVuY2VEZWZhdWx0VmFsdWVzWwAWZXhpc3RlbmNlRGVmYXVsdFZhbHVlc3EAfgAIWwAYZXhpc3RlbmNlRGVmYXVsdHNCaXRtYXNrdAACW1pMAA1maWVsZE5hbWVzU2V0dAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL1NldDtbAAZmaWVsZHN0AClbTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL1N0cnVjdEZpZWxkO0wAC25hbWVUb0ZpZWxkdAAWTHNjYWxhL2NvbGxlY3Rpb24vTWFwO0wAC25hbWVUb0luZGV4cQB+ADp4cAAAAAAAAHBwcHVyAClbTG9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdEZpZWxkO7VjxGhgGg1HAgAAeHAAAAADc3IAJm9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdEZpZWxkK4EkiWfZd58CAARaAAhudWxsYWJsZUwACGRhdGFUeXBlcQB+ABBMAAhtZXRhZGF0YXEAfgAkTAAEbmFtZXEAfgAUeHAAcQB+ABhxAH4AKXEAfgAsc3EAfgA+AHEAfgAYcQB+AClxAH4AMXNxAH4APgBxAH4AGHEAfgApcQB+ADNwcA==", + "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQ009cpyjjQtFMCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAADdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAAA3NyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFxAH4AEXNyAEZvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByb2R1Y3RFbmNvZGVyHKipblA6HWECAANMAAZjbHNUYWdxAH4AD0wABmZpZWxkc3EAfgACTAAGc2NoZW1hdAAnTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL1N0cnVjdFR5cGU7eHBzcgAmc2NhbGEucmVmbGVjdC5DbGFzc1RhZyRHZW5lcmljQ2xhc3NUYWcAAAAAAAAAAQIAAUwADHJ1bnRpbWVDbGFzc3QAEUxqYXZhL2xhbmcvQ2xhc3M7eHB2cgAMc2NhbGEuVHVwbGUzibsxap5k00kCAANMAAJfMXEAfgABTAACXzJxAH4AAUwAAl8zcQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NxAH4AIwBxAH4AEXEAfgApdAACXzNxAH4AL3EAfgAvc3IALHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLkxpc3RTZXJpYWxpemVFbmQkilxjW/dTC20CAAB4cHhzcgAlb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0VHlwZfSN+BDdbY5FAgAJSQAJX2hhc2hDb2RlQgAIYml0bWFwJDBaABloYXNFeGlzdGVuY2VEZWZhdWx0VmFsdWVzWwAWZXhpc3RlbmNlRGVmYXVsdFZhbHVlc3EAfgAIWwAYZXhpc3RlbmNlRGVmYXVsdHNCaXRtYXNrdAACW1pMAA1maWVsZE5hbWVzU2V0dAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL1NldDtbAAZmaWVsZHN0AClbTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL1N0cnVjdEZpZWxkO0wAC25hbWVUb0ZpZWxkdAAWTHNjYWxhL2NvbGxlY3Rpb24vTWFwO0wAC25hbWVUb0luZGV4cQB+ADp4cAAAAAAAAHBwcHVyAClbTG9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdEZpZWxkO7VjxGhgGg1HAgAAeHAAAAADc3IAJm9yZy5hcGFjaGUuc3Bhcmsuc3FsLnR5cGVzLlN0cnVjdEZpZWxkK4EkiWfZd58CAARaAAhudWxsYWJsZUwACGRhdGFUeXBlcQB+ABBMAAhtZXRhZGF0YXEAfgAkTAAEbmFtZXEAfgAUeHAAcQB+ABhxAH4AKXEAfgAsc3EAfgA+AHEAfgAYcQB+AClxAH4AMXNxAH4APgBxAH4AGHEAfgApcQB+ADNwcA==", "nullable": true } } @@ -87,7 +87,7 @@ } }], "scalarScalaUdf": { - "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQ1hQsS9jxAO/gCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAAEdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAABHNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFxAH4AEXEAfgARc3IARm9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJvZHVjdEVuY29kZXIcqKluUDodYQIAA0wABmNsc1RhZ3EAfgAPTAAGZmllbGRzcQB+AAJMAAZzY2hlbWF0ACdMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0VHlwZTt4cHNyACZzY2FsYS5yZWZsZWN0LkNsYXNzVGFnJEdlbmVyaWNDbGFzc1RhZwAAAAAAAAABAgABTAAMcnVudGltZUNsYXNzdAARTGphdmEvbGFuZy9DbGFzczt4cHZyAAxzY2FsYS5UdXBsZTTIfhC+GxSuCgIABEwAAl8xcQB+AAFMAAJfMnEAfgABTAACXzNxAH4AAUwAAl80cQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NxAH4AIwBxAH4AEXEAfgApdAACXzNxAH4AL3EAfgAvc3EAfgAjAHEAfgARcQB+ACl0AAJfNHEAfgAvcQB+AC9zcgAsc2NhbGEuY29sbGVjdGlvbi5pbW11dGFibGUuTGlzdFNlcmlhbGl6ZUVuZCSKXGNb91MLbQIAAHhweHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RUeXBllANIqWlTBxgCAAlJAAlfaGFzaENvZGVCAAhiaXRtYXAkMFoAGWhhc0V4aXN0ZW5jZURlZmF1bHRWYWx1ZXNbABZleGlzdGVuY2VEZWZhdWx0VmFsdWVzcQB+AAhbABhleGlzdGVuY2VEZWZhdWx0c0JpdG1hc2t0AAJbWkwADWZpZWxkTmFtZXNTZXR0ACBMc2NhbGEvY29sbGVjdGlvbi9pbW11dGFibGUvU2V0O1sABmZpZWxkc3QAKVtMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0RmllbGQ7TAALbmFtZVRvRmllbGR0ABZMc2NhbGEvY29sbGVjdGlvbi9NYXA7TAALbmFtZVRvSW5kZXhxAH4APHhwAAAAAAAAcHBwdXIAKVtMb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQ7tWPEaGAaDUcCAAB4cAAAAARzcgAmb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQrgSSJZ9l3nwIABFoACG51bGxhYmxlTAAIZGF0YVR5cGVxAH4AEEwACG1ldGFkYXRhcQB+ACRMAARuYW1lcQB+ABR4cABxAH4AGHEAfgApcQB+ACxzcQB+AEAAcQB+ABhxAH4AKXEAfgAxc3EAfgBAAHEAfgAYcQB+AClxAH4AM3NxAH4AQABxAH4AGHEAfgApcQB+ADVwcA==", + "payload": "rO0ABXNyAC1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jb25uZWN0LmNvbW1vbi5VZGZQYWNrZXR7DRDpFmMz1QIAA0wACGZ1bmN0aW9udAASTGphdmEvbGFuZy9PYmplY3Q7TAANaW5wdXRFbmNvZGVyc3QAFkxzY2FsYS9jb2xsZWN0aW9uL1NlcTtMAA1vdXRwdXRFbmNvZGVydAA4TG9yZy9hcGFjaGUvc3Bhcmsvc3FsL2NhdGFseXN0L2VuY29kZXJzL0Fnbm9zdGljRW5jb2Rlcjt4cHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC5UZXN0VURGcyQkYW5vbiQ1hQsS9jxAO/gCAAB4cHNyACRzY2FsYS5jb2xsZWN0aW9uLm11dGFibGUuQXJyYXlCdWZmZXIVOLBTg4KOcwIAA0kAC2luaXRpYWxTaXplSQAFc2l6ZTBbAAVhcnJheXQAE1tMamF2YS9sYW5nL09iamVjdDt4cAAAABAAAAAEdXIAE1tMamF2YS5sYW5nLk9iamVjdDuQzlifEHMpbAIAAHhwAAAABHNyAE1vcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJFByaW1pdGl2ZUxvbmdFbmNvZGVyJE3e2W1O/wUaAgAAeHIATG9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJpbWl0aXZlTGVhZkVuY29kZXLf4z0LufMQmwIAAHhyAENvcmcuYXBhY2hlLnNwYXJrLnNxbC5jYXRhbHlzdC5lbmNvZGVycy5BZ25vc3RpY0VuY29kZXJzJExlYWZFbmNvZGVyKtoOee2SqKMCAANaAAtpc1ByaW1pdGl2ZUwABmNsc1RhZ3QAGExzY2FsYS9yZWZsZWN0L0NsYXNzVGFnO0wACGRhdGFUeXBldAAlTG9yZy9hcGFjaGUvc3Bhcmsvc3FsL3R5cGVzL0RhdGFUeXBlO3hwAXNyACpzY2FsYS5yZWZsZWN0Lk1hbmlmZXN0RmFjdG9yeSRMb25nTWFuaWZlc3QAAAAAAAAAAQIAAHhyABxzY2FsYS5yZWZsZWN0LkFueVZhbE1hbmlmZXN0AAAAAAAAAAECAAFMAAh0b1N0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO3hwdAAETG9uZ3NyACRvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5Mb25nVHlwZSStAg1XC0z3OwIAAHhwcQB+ABFxAH4AEXEAfgARc3IARm9yZy5hcGFjaGUuc3Bhcmsuc3FsLmNhdGFseXN0LmVuY29kZXJzLkFnbm9zdGljRW5jb2RlcnMkUHJvZHVjdEVuY29kZXIcqKluUDodYQIAA0wABmNsc1RhZ3EAfgAPTAAGZmllbGRzcQB+AAJMAAZzY2hlbWF0ACdMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0VHlwZTt4cHNyACZzY2FsYS5yZWZsZWN0LkNsYXNzVGFnJEdlbmVyaWNDbGFzc1RhZwAAAAAAAAABAgABTAAMcnVudGltZUNsYXNzdAARTGphdmEvbGFuZy9DbGFzczt4cHZyAAxzY2FsYS5UdXBsZTTIfhC+GxSuCgIABEwAAl8xcQB+AAFMAAJfMnEAfgABTAACXzNxAH4AAUwAAl80cQB+AAF4cHNyADJzY2FsYS5jb2xsZWN0aW9uLmltbXV0YWJsZS5MaXN0JFNlcmlhbGl6YXRpb25Qcm94eQAAAAAAAAABAwAAeHBzcgBEb3JnLmFwYWNoZS5zcGFyay5zcWwuY2F0YWx5c3QuZW5jb2RlcnMuQWdub3N0aWNFbmNvZGVycyRFbmNvZGVyRmllbGTkpQPl0rttUgIABloACG51bGxhYmxlTAADZW5jcQB+AANMAAhtZXRhZGF0YXQAJUxvcmcvYXBhY2hlL3NwYXJrL3NxbC90eXBlcy9NZXRhZGF0YTtMAARuYW1lcQB+ABRMAApyZWFkTWV0aG9kdAAOTHNjYWxhL09wdGlvbjtMAAt3cml0ZU1ldGhvZHEAfgAleHAAcQB+ABFzcgAjb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuTWV0YWRhdGFtjWLvldkl+wIAA0kACV9oYXNoQ29kZVoACGJpdG1hcCQwTAADbWFwdAAgTHNjYWxhL2NvbGxlY3Rpb24vaW1tdXRhYmxlL01hcDt4cAAAAAAAc3IAKHNjYWxhLmNvbGxlY3Rpb24uaW1tdXRhYmxlLk1hcCRFbXB0eU1hcCSx6xuFbkKAywIAAHhwdAACXzFzcgALc2NhbGEuTm9uZSRGUCT2U8qUrAIAAHhyAAxzY2FsYS5PcHRpb27+aTf92w5mdAIAAHhwcQB+AC9zcQB+ACMAcQB+ABFxAH4AKXQAAl8ycQB+AC9xAH4AL3NxAH4AIwBxAH4AEXEAfgApdAACXzNxAH4AL3EAfgAvc3EAfgAjAHEAfgARcQB+ACl0AAJfNHEAfgAvcQB+AC9zcgAsc2NhbGEuY29sbGVjdGlvbi5pbW11dGFibGUuTGlzdFNlcmlhbGl6ZUVuZCSKXGNb91MLbQIAAHhweHNyACVvcmcuYXBhY2hlLnNwYXJrLnNxbC50eXBlcy5TdHJ1Y3RUeXBl9I34EN1tjkUCAAlJAAlfaGFzaENvZGVCAAhiaXRtYXAkMFoAGWhhc0V4aXN0ZW5jZURlZmF1bHRWYWx1ZXNbABZleGlzdGVuY2VEZWZhdWx0VmFsdWVzcQB+AAhbABhleGlzdGVuY2VEZWZhdWx0c0JpdG1hc2t0AAJbWkwADWZpZWxkTmFtZXNTZXR0ACBMc2NhbGEvY29sbGVjdGlvbi9pbW11dGFibGUvU2V0O1sABmZpZWxkc3QAKVtMb3JnL2FwYWNoZS9zcGFyay9zcWwvdHlwZXMvU3RydWN0RmllbGQ7TAALbmFtZVRvRmllbGR0ABZMc2NhbGEvY29sbGVjdGlvbi9NYXA7TAALbmFtZVRvSW5kZXhxAH4APHhwAAAAAAAAcHBwdXIAKVtMb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQ7tWPEaGAaDUcCAAB4cAAAAARzcgAmb3JnLmFwYWNoZS5zcGFyay5zcWwudHlwZXMuU3RydWN0RmllbGQrgSSJZ9l3nwIABFoACG51bGxhYmxlTAAIZGF0YVR5cGVxAH4AEEwACG1ldGFkYXRhcQB+ACRMAARuYW1lcQB+ABR4cABxAH4AGHEAfgApcQB+ACxzcQB+AEAAcQB+ABhxAH4AKXEAfgAxc3EAfgBAAHEAfgAYcQB+AClxAH4AM3NxAH4AQABxAH4AGHEAfgApcQB+ADVwcA==", "nullable": true } } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_udf_2.12.proto.bin index fe3efd5577b102c6e877216128a839d78c5433b1..aa1971d3112fcbdacea1120789b10ca3a9ff1026 100644 GIT binary patch delta 78 ycmewv{xf{TUT%&ry*~u*=JvU6Zj={g+ Date: Wed, 15 Feb 2023 18:24:16 +0800 Subject: [PATCH 04/13] fix javadoc --- .../org/apache/spark/sql/connector/catalog/Column.java | 7 +++++-- .../spark/sql/connector/catalog/ColumnDefaultValue.java | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java index d566dadceea60..e0fc1c321fac9 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java @@ -17,8 +17,10 @@ package org.apache.spark.sql.connector.catalog; +import java.util.Map; import javax.annotation.Nullable; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.internal.connector.ColumnImpl; import org.apache.spark.sql.types.DataType; @@ -27,8 +29,9 @@ * such as name and data type, as well as some advanced ones like default column value. *

* Data Sources do not need to implement it. They should consume it in APIs like - * @{@link TableCatalog#createTable(Identifier, Column[], Transform[], Map)}, and report it in - * {@link Table#columns()} by calling the static `create` functions of this interface to create it. + * {@link TableCatalog#createTable(Identifier, Column[], Transform[], Map)}, and report it in + * {@link Table#columns()} by calling the static {@code create} functions of this interface to + * create it. */ public interface Column { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java index cd0b8ea2946b3..52f65e4877e0b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java @@ -25,7 +25,7 @@ /** * A class representing the default value of a column. It contains both the SQL string and literal * value of the user-specified default value expression. The SQL string should be re-evaluated for - * each table writing, which may produce different values if the default value expression is + * each table writing command, which may produce different values if the default value expression is * something like {@code CURRENT_DATE()}. The literal value is used to back-fill existing data if * new columns with default value are added. Note: the back-fill can be lazy. The data sources can * remember the column default value and let the reader fill the column value when reading existing From d1979e828d4ae9d7e38a89e0088de1b76255f7f3 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 15 Feb 2023 20:25:57 +0800 Subject: [PATCH 05/13] more improvement --- .../main/resources/error/error-classes.json | 5 +++ .../connector/catalog/ColumnDefaultValue.java | 2 +- .../util/ResolveDefaultColumnsUtil.scala | 34 ++++++++++++------- .../sql/errors/QueryCompilationErrors.scala | 12 +++++++ .../analysis/ResolveSessionCatalog.scala | 13 ++----- .../spark/sql/sources/InsertSuite.scala | 19 ++++++----- 6 files changed, 54 insertions(+), 31 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index c42cbe6b4d1e7..13ce7223b6ed5 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -791,6 +791,11 @@ "Cannot CREATE/REPLACE/ALTER TABLE because column has an invalid DEFAULT value: ." ], "subClass" : { + "SUBQUERY_EXPRESSION" : { + "message" : [ + "Subquery expressions are not allowed in the default value." + ] + }, "INCOMPATIBLE_DATA_TYPE" : { "message" : [ "The default value has an incompatible data type with the column:", diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java index 52f65e4877e0b..7aceb31e57163 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java @@ -41,7 +41,7 @@ public ColumnDefaultValue(String sql, Literal value) { } /** - * Returns the SQL string of the default value expression. + * Returns the SQL string (Spark SQL dialect) of the default value expression. */ @Nonnull public String getSql() { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala index 113a0f13ae234..22edeeb55400b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.{Literal => ExprLiteral} +import org.apache.spark.sql.catalyst.optimizer.ConstantFolding import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION @@ -262,7 +263,10 @@ object ResolveDefaultColumns { colName: Seq[String], targetType: DataType, isForV1: Boolean): Unit = { - if (default.resolved) { + if (default.containsPattern(PLAN_EXPRESSION)) { + throw QueryCompilationErrors.hasSubqueryDefaultValueError( + tblName, colName, default.originalSQL) + } else if (default.resolved) { if (!default.child.foldable) { throw QueryCompilationErrors.notConstantDefaultValueError( tblName, colName, default.originalSQL) @@ -271,25 +275,31 @@ object ResolveDefaultColumns { throw QueryCompilationErrors.incompatibleTypeDefaultValueError( tblName, colName, targetType, default.child, default.originalSQL) } - } else { + // Check passes. + } else if (default.references.nonEmpty) { // Ideally we should let the rest of `CheckAnalysis` to report errors about why the default // expression is unresolved. But we should report a better error here if the default - // expression references columns or contains subquery expressions, which means it's not a - // constant for sure. - if (default.references.nonEmpty || default.containsPattern(PLAN_EXPRESSION)) { - throw QueryCompilationErrors.notConstantDefaultValueError( - tblName, colName, default.originalSQL) - } + // expression references columns, which means it's not a constant for sure. + throw QueryCompilationErrors.notConstantDefaultValueError( + tblName, colName, default.originalSQL) + } else if (isForV1) { // When converting to v1 commands, the plan is not fully resolved and we can't do a complete // analysis check. There is no "rest of CheckAnalysis" to report better errors and we must // fail here. This is temporary and we can remove it when using v2 commands by default. - if (isForV1) { - throw QueryCompilationErrors.notConstantDefaultValueError( - tblName, colName, default.originalSQL) - } + throw QueryCompilationErrors.notConstantDefaultValueError( + tblName, colName, default.originalSQL) } } + /** + * Applies constant folding for DDL commands. This is used when converting v2 commands to v1 + * commands, where we don't have a chance to go through the optimizer but the default value + * framework requires a literal value of the default value expression. + */ + def contantFoldDDLCommand[T <: LogicalPlan](cmd: T): T = { + ConstantFolding(cmd).asInstanceOf[T] + } + /** * This is an Analyzer for processing default column values using built-in functions only. */ 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 7918ab7a30955..9ed00282ba600 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 @@ -3162,6 +3162,18 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { messageParameters = Map("dataSource" -> dataSource)) } + def hasSubqueryDefaultValueError( + tableName: String, + colName: Seq[String], + defaultValue: String): Throwable = { + new AnalysisException( + errorClass = "INVALID_COLUMN_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + messageParameters = Map( + "tableName" -> tableName, + "colName" -> toSQLId(colName), + "defaultValue" -> defaultValue)) + } + def notConstantDefaultValueError( tableName: String, colName: Seq[String], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 4e7e4833cf9c8..13aef437b0769 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} -import org.apache.spark.sql.catalyst.optimizer.ConstantFolding import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, ResolveDefaultColumns => DefaultColumnUtil} @@ -64,9 +63,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) // Check default values before converting to v1 command. DefaultColumnUtil.checkDefaultValuesInPlan(a, isForV1 = true) val cols = if (colsToAdd.exists(_.column.defaultValue.isDefined)) { - // Do a constant-folding, as we need to store the expression SQL string which should be in - // its simplest form. - ConstantFolding(a).asInstanceOf[AddColumns].columnsToAdd.map(_.column) + DefaultColumnUtil.contantFoldDDLCommand(a).columnsToAdd.map(_.column) } else { colsToAdd.map(_.column) } @@ -112,9 +109,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) // Check default values before converting to v1 command. DefaultColumnUtil.checkDefaultValuesInPlan(a, isForV1 = true) val defaultValue = if (a.defaultExpression.isDefined) { - // Do a constant-folding, as we need to store the expression SQL string which should be in - // its simplest form. - ConstantFolding(a).asInstanceOf[AlterColumn].defaultExpression + DefaultColumnUtil.contantFoldDDLCommand(a).defaultExpression } else { a.defaultExpression } @@ -191,9 +186,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) // Check default values before converting to v1 command. DefaultColumnUtil.checkDefaultValuesInPlan(c, isForV1 = true) val cols = if (c.columns.exists(_.defaultValue.isDefined)) { - // Do a constant-folding, as we need to store the expression SQL string which should be in - // its simplest form. - ConstantFolding(c).asInstanceOf[CreateTable].columns + DefaultColumnUtil.contantFoldDDLCommand(c).columns } else { c.columns } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 7288b9c11a0da..83d77a83d624e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -1045,6 +1045,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { object Errors { val COMMON_SUBSTRING = "has an invalid DEFAULT value" val COLUMN_DEFAULT_NOT_FOUND = "`default` cannot be resolved." + val BAD_SUBQUERY = "Subquery expressions are not allowed in the default value" } // The default value fails to analyze. withTable("t") { @@ -1057,21 +1058,21 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { assert(intercept[AnalysisException] { sql("create table t(i boolean, s bigint default (select min(x) from badtable)) " + "using parquet") - }.getMessage.contains(Errors.COMMON_SUBSTRING)) + }.getMessage.contains(Errors.BAD_SUBQUERY)) } // The default value parses but refers to a table from the catalog. withTable("t", "other") { sql("create table other(x string) using parquet") assert(intercept[AnalysisException] { sql("create table t(i boolean, s bigint default (select min(x) from other)) using parquet") - }.getMessage.contains(Errors.COMMON_SUBSTRING)) + }.getMessage.contains(Errors.BAD_SUBQUERY)) } // The default value has an explicit alias. It fails to evaluate when inlined into the VALUES // list at the INSERT INTO time. withTable("t") { assert(intercept[AnalysisException] { sql("create table t(i boolean default (select false as alias), s bigint) using parquet") - }.getMessage.contains(Errors.COMMON_SUBSTRING)) + }.getMessage.contains(Errors.BAD_SUBQUERY)) } // Explicit default values may not participate in complex expressions in the VALUES list. withTable("t") { @@ -1424,6 +1425,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Negative tests") { object Errors { val COMMON_SUBSTRING = "has an invalid DEFAULT value" + val BAD_SUBQUERY = "Subquery expressions are not allowed in the default value" } // The default value fails to analyze. withTable("t") { @@ -1437,7 +1439,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { sql("create table t(i boolean) using parquet") assert(intercept[AnalysisException] { sql("alter table t add column s bigint default (select min(x) from badtable)") - }.getMessage.contains(Errors.COMMON_SUBSTRING)) + }.getMessage.contains(Errors.BAD_SUBQUERY)) } // The default value parses but refers to a table from the catalog. withTable("t", "other") { @@ -1445,7 +1447,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { sql("create table t(i boolean) using parquet") assert(intercept[AnalysisException] { sql("alter table t add column s bigint default (select min(x) from other)") - }.getMessage.contains(Errors.COMMON_SUBSTRING)) + }.getMessage.contains(Errors.BAD_SUBQUERY)) } // The default value parses but the type is not coercible. withTable("t") { @@ -1501,6 +1503,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-38838 INSERT INTO with defaults set by ALTER TABLE ALTER COLUMN: negative tests") { object Errors { val COMMON_SUBSTRING = "has an invalid DEFAULT value" + val BAD_SUBQUERY = "Subquery expressions are not allowed in the default value" } val createTable = "create table t(i boolean, s bigint) using parquet" withTable("t") { @@ -1512,12 +1515,12 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { // The default value analyzes to a table not in the catalog. assert(intercept[AnalysisException] { sql("alter table t alter column s set default (select min(x) from badtable)") - }.getMessage.contains(Errors.COMMON_SUBSTRING)) + }.getMessage.contains(Errors.BAD_SUBQUERY)) // The default value has an explicit alias. It fails to evaluate when inlined into the VALUES // list at the INSERT INTO time. assert(intercept[AnalysisException] { sql("alter table t alter column s set default (select 42 as alias)") - }.getMessage.contains(Errors.COMMON_SUBSTRING)) + }.getMessage.contains(Errors.BAD_SUBQUERY)) // The default value parses but the type is not coercible. assert(intercept[AnalysisException] { sql("alter table t alter column s set default false") @@ -1970,7 +1973,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { ).foreach { query => assert(intercept[AnalysisException] { sql(query) - }.getMessage.contains("The default value must be a constant")) + }.getMessage.contains("Subquery expressions are not allowed in the default value")) } } From 9ec993bd576c2dd4b44b03328c344deb71334e8c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 15 Feb 2023 20:50:34 +0800 Subject: [PATCH 06/13] rename --- .../sql/catalyst/parser/AstBuilder.scala | 16 +-- .../plans/logical/v2AlterTableCommands.scala | 4 +- .../catalyst/plans/logical/v2Commands.scala | 12 +-- .../util/ResolveDefaultColumnsUtil.scala | 20 ++-- .../apache/spark/sql/types/StructType.scala | 8 +- .../sql/catalyst/parser/DDLParserSuite.scala | 98 ++++++++++--------- .../analysis/ReplaceCharWithVarchar.scala | 4 +- .../analysis/ResolveSessionCatalog.scala | 2 +- .../V2CommandsCaseSensitivitySuite.scala | 26 ++--- 9 files changed, 100 insertions(+), 90 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index b95fe5c7ba6f6..e25d3a0544332 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2995,7 +2995,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Create a [[StructType]] from a number of CREATE TABLE column definitions. */ override def visitCreateOrReplaceTableColTypeList( - ctx: CreateOrReplaceTableColTypeListContext): Seq[Column] = withOrigin(ctx) { + ctx: CreateOrReplaceTableColTypeListContext): Seq[ColumnDefinition] = withOrigin(ctx) { ctx.createOrReplaceTableColType().asScala.map(visitCreateOrReplaceTableColType).toSeq } @@ -3003,7 +3003,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Create a top level column from a CREATE TABLE column definition. */ override def visitCreateOrReplaceTableColType( - ctx: CreateOrReplaceTableColTypeContext): Column = withOrigin(ctx) { + ctx: CreateOrReplaceTableColTypeContext): ColumnDefinition = withOrigin(ctx) { import ctx._ val name: String = colName.getText @@ -3038,7 +3038,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } } - Column( + ColumnDefinition( name = name, dataType = typedVisit[DataType](ctx.dataType), nullable = nullable, @@ -3239,7 +3239,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * types like `i INT`, which should be appended to the existing table schema. */ type TableClauses = ( - Seq[Transform], Seq[Column], Option[BucketSpec], Map[String, String], + Seq[Transform], Seq[ColumnDefinition], Option[BucketSpec], Map[String, String], Map[String, String], Option[String], Option[String], Option[SerdeInfo]) /** @@ -3267,13 +3267,13 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Parse a list of transforms or columns. */ override def visitPartitionFieldList( - ctx: PartitionFieldListContext): (Seq[Transform], Seq[Column]) = withOrigin(ctx) { + ctx: PartitionFieldListContext): (Seq[Transform], Seq[ColumnDefinition]) = withOrigin(ctx) { val (transforms, columns) = ctx.fields.asScala.map { case transform: PartitionTransformContext => (Some(visitPartitionTransform(transform)), None) case field: PartitionColumnContext => val f = visitColType(field.colType) - val col = Column(f.name, f.dataType, f.nullable, f.getComment(), defaultValue = None) + val col = ColumnDefinition(f.name, f.dataType, f.nullable, f.getComment(), None) (None, Some(col)) }.unzip @@ -3729,7 +3729,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit private def partitionExpressions( partTransforms: Seq[Transform], - partCols: Seq[Column], + partCols: Seq[ColumnDefinition], ctx: ParserRuleContext): Seq[Transform] = { if (partTransforms.nonEmpty) { if (partCols.nonEmpty) { @@ -3981,7 +3981,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit path = if (name.length > 1) UnresolvedFieldName(name.init) else RootTableSchema, position = Option(ctx.colPosition).map( pos => UnresolvedFieldPosition(typedVisit[ColumnPosition](pos))), - column = Column( + column = ColumnDefinition( name = name.last, dataType = typedVisit[DataType](ctx.dataType), nullable = ctx.NULL == null, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala index 39b3211bb819d..ae0f2ea71db5c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala @@ -107,7 +107,7 @@ case class UnsetTableProperties( */ case class QualifiedColType( path: FieldName, - column: Column, + column: ColumnDefinition, position: Option[FieldPosition] = None) extends Expression with Unevaluable { def name: Seq[String] = path.name :+ column.name override def children: Seq[Expression] = path +: column +: position.toSeq @@ -121,7 +121,7 @@ case class QualifiedColType( newChildren: IndexedSeq[Expression]): Expression = { copy( newChildren(0).asInstanceOf[FieldName], - newChildren(1).asInstanceOf[Column], + newChildren(1).asInstanceOf[ColumnDefinition], newChildren.drop(2).headOption.map(_.asInstanceOf[FieldPosition])) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 247af2c058e16..50df441982481 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -396,7 +396,7 @@ trait V2CreateTablePlan extends LogicalPlan { def name: LogicalPlan def tableSpec: TableSpec def partitioning: Seq[Transform] - def columns: Seq[Column] + def columns: Seq[ColumnDefinition] def tableSchema: StructType = StructType(columns.map { col => // Schema only cares about the tree structure with name and data type. StructField(col.name, col.dataType, col.nullable) @@ -436,7 +436,7 @@ case class DefaultValueExpression(child: Expression, originalSQL: String) * Column definition for tables. This is an expression so that analyzer can resolve the default * value expression in DDL commands automatically. */ -case class Column( +case class ColumnDefinition( name: String, dataType: DataType, nullable: Boolean = true, @@ -466,7 +466,7 @@ case class Column( */ case class CreateTable( name: LogicalPlan, - columns: Seq[Column], + columns: Seq[ColumnDefinition], partitioning: Seq[Transform], tableSpec: TableSpec, ignoreIfExists: Boolean) extends UnaryCommand with V2CreateTablePlan { @@ -494,7 +494,7 @@ case class CreateTableAsSelect( analyzedQuery: Option[LogicalPlan] = None) extends BinaryCommand with V2CreateTablePlan with KeepAnalyzedQuery { - override def columns: Seq[Column] = query.schema.toColumns + override def columns: Seq[ColumnDefinition] = query.schema.toColumns override def tableSchema: StructType = query.schema override def left: LogicalPlan = name override def right: LogicalPlan = query @@ -529,7 +529,7 @@ case class CreateTableAsSelect( */ case class ReplaceTable( name: LogicalPlan, - columns: Seq[Column], + columns: Seq[ColumnDefinition], partitioning: Seq[Transform], tableSpec: TableSpec, orCreate: Boolean) extends UnaryCommand with V2CreateTablePlan { @@ -560,7 +560,7 @@ case class ReplaceTableAsSelect( analyzedQuery: Option[LogicalPlan] = None) extends BinaryCommand with V2CreateTablePlan with KeepAnalyzedQuery { - override def columns: Seq[Column] = query.schema.toColumns + override def columns: Seq[ColumnDefinition] = query.schema.toColumns override def tableSchema: StructType = query.schema override def left: LogicalPlan = name override def right: LogicalPlan = query diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala index 22edeeb55400b..7f08e226c2f50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala @@ -201,11 +201,12 @@ object ResolveDefaultColumns { case cmd: V2CreateTablePlan if cmd.columns.exists(_.defaultValue.isDefined) => val ident = cmd.resolvedName checkTableProvider(ident.catalog, ident.name, cmd.tableSpec.provider) - cmd.columns.filter(_.defaultValue.isDefined).foreach { col => - val Column(name, dataType, _, _, Some(default), _) = col - // CREATE/REPLACE TABLE only has top-level columns - val colName = Seq(name) - checkDefaultValue(default, ident.name, colName, dataType, isForV1) + cmd.columns.foreach { col => + if (col.defaultValue.isDefined) { + // CREATE/REPLACE TABLE only has top-level columns + val colName = Seq(col.name) + checkDefaultValue(col.defaultValue.get, ident.name, colName, col.dataType, isForV1) + } } case cmd: AlterTableCommand => @@ -213,14 +214,13 @@ object ResolveDefaultColumns { val provider = getTableProviderFromProp(table.table.properties()) val isAddColumns = cmd.isInstanceOf[AddColumns] cmd.transformExpressionsDown { - case q @ QualifiedColType(path, Column(name, dataType, _, _, Some(default), _), _) - if path.resolved => + case q @ QualifiedColType(path, col, _) if col.defaultValue.isDefined && path.resolved => checkTableProvider(table.catalog, table.name, provider, isAddColumns) checkDefaultValue( - default, + col.defaultValue.get, table.name, - path.name :+ name, - dataType, + path.name :+ col.name, + col.dataType, isForV1) q } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 4c573532600d4..4c13d9db546e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -27,7 +27,7 @@ import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering, Literal} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} -import org.apache.spark.sql.catalyst.plans.logical.{Column, DefaultValueExpression} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnDefinition, DefaultValueExpression} import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.catalyst.types.{PhysicalDataType, PhysicalStructType} import org.apache.spark.sql.catalyst.util.{truncatedString, ResolveDefaultColumns, StringUtils} @@ -395,16 +395,16 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru protected[sql] def toAttributes: Seq[AttributeReference] = map(field => field.toAttribute) - private[sql] def toColumns: Array[Column] = fields.map { f => + private[sql] def toColumns: Array[ColumnDefinition] = fields.map { f => def createColumn( defaultValue: Option[DefaultValueExpression], - metadata: Metadata): Column = { + metadata: Metadata): ColumnDefinition = { val metadataJSON = if (metadata == Metadata.empty) { None } else { Some(metadata.json) } - Column(f.name, f.dataType, f.nullable, f.getComment(), defaultValue, metadataJSON) + ColumnDefinition(f.name, f.dataType, f.nullable, f.getComment(), defaultValue, metadataJSON) } if (f.getCurrentDefaultValue().isDefined && f.getExistenceDefaultValue().isDefined) { val e = ResolveDefaultColumns.analyze( 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 7bf86390d8b47..d5d0d4860e32c 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 @@ -47,8 +47,8 @@ class DDLParserSuite extends AnalysisTest { val expectedTableSpec = TableSpec( Seq("my_tab"), Some(Seq( - Column("a", IntegerType, comment = Some("test")), - Column("b", StringType, nullable = false) + ColumnDefinition("a", IntegerType, comment = Some("test")), + ColumnDefinition("b", StringType, nullable = false) )), Seq.empty[Transform], Map.empty[String, String], @@ -75,7 +75,7 @@ class DDLParserSuite extends AnalysisTest { sql, TableSpec( Seq("my_tab"), - Some(Seq(Column("a", IntegerType), Column("b", StringType))), + Some(Seq(ColumnDefinition("a", IntegerType), ColumnDefinition("b", StringType))), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -93,7 +93,9 @@ class DDLParserSuite extends AnalysisTest { "USING parquet PARTITIONED BY (a)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("a", IntegerType, comment = Some("test")), Column("b", StringType))), + Some(Seq( + ColumnDefinition("a", IntegerType, comment = Some("test")), + ColumnDefinition("b", StringType))), Seq(IdentityTransform(FieldReference("a"))), Map.empty[String, String], Some("parquet"), @@ -135,9 +137,9 @@ class DDLParserSuite extends AnalysisTest { val expectedTableSpec = TableSpec( Seq("my_tab"), Some(Seq( - Column("a", IntegerType), - Column("b", StringType), - Column("ts", TimestampType) + ColumnDefinition("a", IntegerType), + ColumnDefinition("b", StringType), + ColumnDefinition("ts", TimestampType) )), Seq( IdentityTransform(FieldReference("a")), @@ -170,7 +172,7 @@ class DDLParserSuite extends AnalysisTest { val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("a", IntegerType), Column("b", StringType))), + Some(Seq(ColumnDefinition("a", IntegerType), ColumnDefinition("b", StringType))), List(bucket(5, Array(FieldReference.column("a")), Array(FieldReference.column("b")))), Map.empty[String, String], Some("parquet"), @@ -189,7 +191,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("a", IntegerType), Column("b", StringType))), + Some(Seq(ColumnDefinition("a", IntegerType), ColumnDefinition("b", StringType))), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -209,7 +211,7 @@ class DDLParserSuite extends AnalysisTest { " TBLPROPERTIES('test' = 'test')" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("a", IntegerType), Column("b", StringType))), + Some(Seq(ColumnDefinition("a", IntegerType), ColumnDefinition("b", StringType))), Seq.empty[Transform], Map("test" -> "test"), Some("parquet"), @@ -227,7 +229,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("a", IntegerType), Column("b", StringType))), + Some(Seq(ColumnDefinition("a", IntegerType), ColumnDefinition("b", StringType))), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -245,7 +247,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE 1m.2g(a INT) USING parquet" val expectedTableSpec = TableSpec( Seq("1m", "2g"), - Some(Seq(Column("a", IntegerType))), + Some(Seq(ColumnDefinition("a", IntegerType))), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -263,7 +265,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab (id bigint) PARTITIONED BY (part string)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("id", LongType), Column("part", StringType))), + Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -281,7 +283,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab PARTITIONED BY (part string)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("part", StringType))), + Some(Seq(ColumnDefinition("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -299,7 +301,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab (id bigint) USING parquet PARTITIONED BY (part string)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("id", LongType), Column("part", StringType))), + Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], Some("parquet"), @@ -372,7 +374,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("id", LongType), Column("part", StringType))), + Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -397,7 +399,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("id", LongType), Column("part", StringType))), + Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -454,7 +456,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("id", LongType), Column("part", StringType))), + Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -505,7 +507,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("id", LongType), Column("part", StringType))), + Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -528,7 +530,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(Column("id", LongType), Column("part", StringType))), + Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -1034,7 +1036,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMN x int"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType)) + Seq(QualifiedColType(RootTableSchema, ColumnDefinition("x", IntegerType)) ))) } @@ -1043,8 +1045,8 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS x int, y string"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType)), - QualifiedColType(RootTableSchema, Column("y", StringType)) + Seq(QualifiedColType(RootTableSchema, ColumnDefinition("x", IntegerType)), + QualifiedColType(RootTableSchema, ColumnDefinition("y", StringType)) ))) } @@ -1053,7 +1055,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS x int"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType)) + Seq(QualifiedColType(RootTableSchema, ColumnDefinition("x", IntegerType)) ))) } @@ -1062,7 +1064,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS (x int)"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType)) + Seq(QualifiedColType(RootTableSchema, ColumnDefinition("x", IntegerType)) ))) } @@ -1071,7 +1073,8 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS (x int COMMENT 'doc')"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType, comment = Some("doc"))) + Seq(QualifiedColType( + RootTableSchema, ColumnDefinition("x", IntegerType, comment = Some("doc"))) ))) } @@ -1080,7 +1083,8 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMN x int NOT NULL"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType, nullable = false)) + Seq(QualifiedColType( + RootTableSchema, ColumnDefinition("x", IntegerType, nullable = false)) ))) } @@ -1089,7 +1093,8 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMN x int COMMENT 'doc'"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType(RootTableSchema, Column("x", IntegerType, comment = Some("doc"))) + Seq(QualifiedColType( + RootTableSchema, ColumnDefinition("x", IntegerType, comment = Some("doc"))) ))) } @@ -1100,7 +1105,7 @@ class DDLParserSuite extends AnalysisTest { UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq(QualifiedColType( RootTableSchema, - Column("x", IntegerType), + ColumnDefinition("x", IntegerType), Some(UnresolvedFieldPosition(first()))) ))) @@ -1110,7 +1115,7 @@ class DDLParserSuite extends AnalysisTest { UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq(QualifiedColType( RootTableSchema, - Column("x", IntegerType), + ColumnDefinition("x", IntegerType), Some(UnresolvedFieldPosition(after("y")))) ))) } @@ -1122,7 +1127,7 @@ class DDLParserSuite extends AnalysisTest { UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq(QualifiedColType( UnresolvedFieldName(Seq("x", "y")), - Column("z", IntegerType, comment = Some("doc"))) + ColumnDefinition("z", IntegerType, comment = Some("doc"))) ))) } @@ -1134,10 +1139,10 @@ class DDLParserSuite extends AnalysisTest { Seq( QualifiedColType( UnresolvedFieldName(Seq("x", "y")), - Column("z", IntegerType, comment = Some("doc"))), + ColumnDefinition("z", IntegerType, comment = Some("doc"))), QualifiedColType( UnresolvedFieldName(Seq("a")), - Column("b", StringType), + ColumnDefinition("b", StringType), Some(UnresolvedFieldPosition(first()))) ))) } @@ -1376,21 +1381,24 @@ class DDLParserSuite extends AnalysisTest { parsePlan(sql1), ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), - Seq(QualifiedColType(RootTableSchema, Column("x", StringType))))) + Seq(QualifiedColType(RootTableSchema, ColumnDefinition("x", StringType))))) comparePlans( parsePlan(sql2), ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), - Seq(QualifiedColType(RootTableSchema, Column("x", StringType, comment = Some("x1")))))) + Seq(QualifiedColType( + RootTableSchema, ColumnDefinition("x", StringType, comment = Some("x1")))))) comparePlans( parsePlan(sql3), ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), Seq( - QualifiedColType(RootTableSchema, Column("x", StringType, comment = Some("x1"))), - QualifiedColType(RootTableSchema, Column("y", IntegerType)) + QualifiedColType( + RootTableSchema, ColumnDefinition("x", StringType, comment = Some("x1"))), + QualifiedColType( + RootTableSchema, ColumnDefinition("y", IntegerType)) ))) comparePlans( @@ -1398,8 +1406,10 @@ class DDLParserSuite extends AnalysisTest { ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), Seq( - QualifiedColType(RootTableSchema, Column("x", StringType, comment = Some("x1"))), - QualifiedColType(RootTableSchema, Column("y", IntegerType, comment = Some("y1"))) + QualifiedColType( + RootTableSchema, ColumnDefinition("x", StringType, comment = Some("x1"))), + QualifiedColType( + RootTableSchema, ColumnDefinition("y", IntegerType, comment = Some("y1"))) ))) val sql5 = "ALTER TABLE table_name PARTITION (a='1') REPLACE COLUMNS (x string)" @@ -2400,7 +2410,7 @@ class DDLParserSuite extends AnalysisTest { private case class TableSpec( name: Seq[String], - columns: Option[Seq[Column]], + columns: Option[Seq[ColumnDefinition]], partitioning: Seq[Transform], properties: Map[String, String], provider: Option[String], @@ -2488,7 +2498,7 @@ class DDLParserSuite extends AnalysisTest { val sql = "CREATE TABLE 1m.2g(a INT)" val expectedTableSpec = TableSpec( Seq("1m", "2g"), - Some(Seq(Column("a", IntegerType))), + Some(Seq(ColumnDefinition("a", IntegerType))), Seq.empty[Transform], Map.empty[String, String], None, @@ -2535,8 +2545,8 @@ class DDLParserSuite extends AnalysisTest { test("SPARK-38335: Implement parser support for DEFAULT values for columns in tables") { // These CREATE/REPLACE TABLE statements should parse successfully. val columnsWithDefaultValue = Seq( - Column("a", IntegerType), - Column( + ColumnDefinition("a", IntegerType), + ColumnDefinition( "b", StringType, nullable = false, @@ -2563,7 +2573,7 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE t1 ADD COLUMN x int NOT NULL DEFAULT 42"), AddColumns(UnresolvedTable(Seq("t1"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType(RootTableSchema, Column( + Seq(QualifiedColType(RootTableSchema, ColumnDefinition( "x", IntegerType, nullable = false, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala index 9ac536e5d8aeb..eb1785c877763 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, Column, CreateTable, LogicalPlan, ReplaceColumns, ReplaceTable} +import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, ColumnDefinition, CreateTable, LogicalPlan, ReplaceColumns, ReplaceTable} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableChangeColumnCommand, CreateDataSourceTableCommand, CreateTableCommand} @@ -61,7 +61,7 @@ object ReplaceCharWithVarchar extends Rule[LogicalPlan] { } } - private def replaceCharWithVarcharInColumn(col: Column): Column = { + private def replaceCharWithVarcharInColumn(col: ColumnDefinition): ColumnDefinition = { col.copy(dataType = CharVarcharUtils.replaceCharWithVarchar(col.dataType)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 13aef437b0769..3efb5496242c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -618,7 +618,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) } } - private def convertToStructField(col: Column): StructField = { + private def convertToStructField(col: ColumnDefinition): StructField = { assert(col.resolved) CatalogV2Util.v2ColumnToStructField(col.toV2Column) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala index 4f57555544e90..7e30d70075b7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.connector import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, CreateTablePartitioningValidationSuite, ResolvedTable, RootTableSchema, TestRelation2, TestTable2, UnresolvedFieldName, UnresolvedFieldPosition, UnresolvedIdentifier} -import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, AlterTableCommand, Column, CreateTableAsSelect, DropColumns, LogicalPlan, QualifiedColType, RenameColumn, ReplaceColumns, ReplaceTableAsSelect, TableSpec} +import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, AlterTableCommand, ColumnDefinition, CreateTableAsSelect, DropColumns, LogicalPlan, QualifiedColType, RenameColumn, ReplaceColumns, ReplaceTableAsSelect, TableSpec} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition @@ -153,7 +153,7 @@ class V2CommandsCaseSensitivitySuite table, Seq(QualifiedColType( UnresolvedFieldName(field.init), - Column(field.last, LongType, true)))), + ColumnDefinition(field.last, LongType, true)))), Seq("Missing field " + field.head) ) } @@ -165,7 +165,7 @@ class V2CommandsCaseSensitivitySuite table, Seq(QualifiedColType( RootTableSchema, - Column("f", LongType, true), + ColumnDefinition("f", LongType, true), Some(UnresolvedFieldPosition(ColumnPosition.after(ref)))))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { @@ -185,11 +185,11 @@ class V2CommandsCaseSensitivitySuite Seq( QualifiedColType( RootTableSchema, - Column("x", LongType, true), + ColumnDefinition("x", LongType, true), Some(UnresolvedFieldPosition(ColumnPosition.after("id")))), QualifiedColType( RootTableSchema, - Column("y", LongType, true), + ColumnDefinition("y", LongType, true), Some(UnresolvedFieldPosition(ColumnPosition.after("X")))) )) Seq(true, false).foreach { caseSensitive => @@ -209,7 +209,7 @@ class V2CommandsCaseSensitivitySuite table, Seq(QualifiedColType( UnresolvedFieldName(Seq("point")), - Column("z", LongType, true), + ColumnDefinition("z", LongType, true), Some(UnresolvedFieldPosition(ColumnPosition.after(ref)))))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { @@ -229,10 +229,10 @@ class V2CommandsCaseSensitivitySuite Seq( QualifiedColType( UnresolvedFieldName(Seq("point")), - Column("z", LongType, true)), + ColumnDefinition("z", LongType, true)), QualifiedColType( UnresolvedFieldName(Seq("point")), - Column("zz", LongType, true), + ColumnDefinition("zz", LongType, true), Some(UnresolvedFieldPosition(ColumnPosition.after("Z")))) )) Seq(true, false).foreach { caseSensitive => @@ -253,10 +253,10 @@ class V2CommandsCaseSensitivitySuite Seq( QualifiedColType( UnresolvedFieldName(Seq("point")), - Column("z", LongType, true)), + ColumnDefinition("z", LongType, true)), QualifiedColType( UnresolvedFieldName(Seq("point")), - Column("Z", LongType, true)) + ColumnDefinition("Z", LongType, true)) )), "COLUMN_ALREADY_EXISTS", Map("columnName" -> toSQLId("point.z")), @@ -269,7 +269,7 @@ class V2CommandsCaseSensitivitySuite table, Seq(QualifiedColType( RootTableSchema, - Column("ID", LongType, true), + ColumnDefinition("ID", LongType, true), Some(UnresolvedFieldPosition(ColumnPosition.after("id")))))), Seq("Cannot add column, because ID already exists in root"), expectErrorOnCaseSensitive = false) @@ -343,8 +343,8 @@ class V2CommandsCaseSensitivitySuite ReplaceColumns( table, Seq( - QualifiedColType(RootTableSchema, Column("f", LongType, true)), - QualifiedColType(RootTableSchema, Column("F", LongType, true)))), + QualifiedColType(RootTableSchema, ColumnDefinition("f", LongType, true)), + QualifiedColType(RootTableSchema, ColumnDefinition("F", LongType, true)))), "COLUMN_ALREADY_EXISTS", Map("columnName" -> toSQLId("f")), caseSensitive = false) From 17f7ac802d5ca8aef6dd31794956af66d8d16470 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 16 Feb 2023 11:03:35 +0800 Subject: [PATCH 07/13] revert refactor --- .../main/resources/error/error-classes.json | 31 +-- .../sql/catalyst/analysis/Analyzer.scala | 45 ++-- .../sql/catalyst/analysis/CheckAnalysis.scala | 10 +- .../analysis/ResolveDefaultColumns.scala | 6 +- .../catalyst/analysis/v2ResolutionPlans.scala | 6 - .../sql/catalyst/parser/AstBuilder.scala | 135 +++++----- .../catalyst/plans/logical/statements.scala | 31 +++ .../plans/logical/v2AlterTableCommands.scala | 81 ++---- .../catalyst/plans/logical/v2Commands.scala | 95 ++----- .../util/ResolveDefaultColumnsUtil.scala | 248 +++++++++--------- .../sql/connector/catalog/CatalogV2Util.scala | 68 ++++- .../sql/errors/QueryCompilationErrors.scala | 54 +--- .../apache/spark/sql/types/StructType.scala | 40 +-- .../catalog/ExternalCatalogSuite.scala | 37 ++- .../catalog/SessionCatalogSuite.scala | 67 +++++ .../sql/catalyst/parser/DDLParserSuite.scala | 197 +++++++------- .../catalog/InMemoryTableCatalog.scala | 2 +- .../spark/sql/types/StructTypeSuite.scala | 22 +- .../analysis/ReplaceCharWithVarchar.scala | 16 +- .../analysis/ResolveSessionCatalog.scala | 51 ++-- .../spark/sql/execution/command/ddl.scala | 35 ++- .../spark/sql/execution/command/tables.scala | 26 +- .../datasources/DataSourceStrategy.scala | 10 +- .../datasources/v2/DataSourceV2Strategy.scala | 19 +- .../datasources/v2/DescribeTableExec.scala | 29 +- .../datasources/v2/V2SessionCatalog.scala | 3 +- .../internal/BaseSessionStateBuilder.scala | 2 +- .../spark/sql/internal/CatalogImpl.scala | 6 +- .../sql/streaming/DataStreamWriter.scala | 2 +- ...SourceV2DataFrameSessionCatalogSuite.scala | 2 +- .../V2CommandsCaseSensitivitySuite.scala | 125 +++++---- .../sql/sources/DataSourceAnalysisSuite.scala | 3 +- .../spark/sql/sources/InsertSuite.scala | 38 ++- .../sql/hive/HiveSessionStateBuilder.scala | 2 +- 34 files changed, 803 insertions(+), 741 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 13ce7223b6ed5..f7e4086263d63 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -786,30 +786,6 @@ "The expected format is ByteString, but was ()." ] }, - "INVALID_COLUMN_DEFAULT_VALUE" : { - "message" : [ - "Cannot CREATE/REPLACE/ALTER TABLE because column has an invalid DEFAULT value: ." - ], - "subClass" : { - "SUBQUERY_EXPRESSION" : { - "message" : [ - "Subquery expressions are not allowed in the default value." - ] - }, - "INCOMPATIBLE_DATA_TYPE" : { - "message" : [ - "The default value has an incompatible data type with the column:", - "Default value: ", - "Column: " - ] - }, - "NOT_CONSTANT" : { - "message" : [ - "The default value must be a constant, or can be folded to a constant." - ] - } - } - }, "INVALID_COLUMN_NAME_AS_PATH" : { "message" : [ "The datasource cannot save the column because its name contains some characters that are not allowed in file paths. Please, use an alias to rename it." @@ -3643,7 +3619,7 @@ }, "_LEGACY_ERROR_TEMP_1345" : { "message" : [ - "Failed to execute ALTER TABLE ADD COLUMNS command because DEFAULT values are not supported for target data source with table provider: \"\"." + "Failed to execute command because DEFAULT values are not supported for target data source with table provider: \"\"." ] }, "_LEGACY_ERROR_TEMP_1346" : { @@ -3651,6 +3627,11 @@ "Failed to execute command because DEFAULT values are not supported when adding new columns to previously existing target data source with table provider: \"\"." ] }, + "_LEGACY_ERROR_TEMP_1347" : { + "message" : [ + "Failed to execute command because subquery expressions are not allowed in DEFAULT values." + ] + }, "_LEGACY_ERROR_TEMP_2000" : { "message" : [ ". If necessary set to false to bypass this error." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 03607b156d5be..d14667801d446 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -3677,6 +3677,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val columnsToDrop = a.columnsToDrop a.copy(columnsToDrop = columnsToDrop.flatMap(c => resolveFieldNamesOpt(table, c.name, c))) + case a: AlterTableCommand if a.table.resolved && hasUnresolvedFieldName(a) => + val table = a.table.asInstanceOf[ResolvedTable] + a.transformExpressions { + case u: UnresolvedFieldName => resolveFieldNames(table, u.name, u) + } + case a @ AddColumns(r: ResolvedTable, cols) if !a.resolved => // 'colsToAdd' keeps track of new columns being added. It stores a mapping from a // normalized parent name of fields to field names that belong to the parent. @@ -3697,43 +3703,40 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor ResolvedFieldPosition(ColumnPosition.after(colName)) case None => throw QueryCompilationErrors.referenceColNotFoundForAlterTableChangesError( - col.column.name, allFields) + col.colName, allFields) } case _ => ResolvedFieldPosition(u.position) } case resolved => resolved } - colsToAdd(resolvedParentName) = fieldsAdded :+ col.column.name + colsToAdd(resolvedParentName) = fieldsAdded :+ col.colName resolvedPosition } val schema = r.table.columns.asSchema val resolvedCols = cols.map { col => - val path = col.path.transform { - case u: UnresolvedFieldName => resolveFieldNames(r, u.name, u) - }.asInstanceOf[FieldName] - path match { - case RootTableSchema => - // Adding to the root. Just need to resolve position. - val resolvedPosition = resolvePosition(col, schema, Nil) - col.copy(path = path, position = resolvedPosition) - case parent: ResolvedFieldName => - val parentSchema = parent.field.dataType match { + col.path match { + case Some(parent: UnresolvedFieldName) => + // Adding a nested field, need to resolve the parent column and position. + val resolvedParent = resolveFieldNames(r, parent.name, parent) + val parentSchema = resolvedParent.field.dataType match { case s: StructType => s case _ => throw QueryCompilationErrors.invalidFieldName( col.name, parent.name, parent.origin) } - val resolvedPosition = resolvePosition(col, parentSchema, parent.name) - col.copy(path = path, position = resolvedPosition) - // This should not happen. All `UnresolvedFieldName` should have been resolved. - case _ => col + val resolvedPosition = resolvePosition(col, parentSchema, resolvedParent.name) + col.copy(path = Some(resolvedParent), position = resolvedPosition) + case _ => + // Adding to the root. Just need to resolve position. + val resolvedPosition = resolvePosition(col, schema, Nil) + col.copy(position = resolvedPosition) } } val resolved = a.copy(columnsToAdd = resolvedCols) resolved.copyTagsFrom(a) resolved - case a @ AlterColumn(table: ResolvedTable, - ResolvedFieldName(path, field), dataType, _, _, position, _) => + case a @ AlterColumn( + table: ResolvedTable, ResolvedFieldName(path, field), dataType, _, _, position, _) => val newDataType = dataType.flatMap { dt => // Hive style syntax provides the column type, even if it may not have changed. val existing = CharVarcharUtils.getRawType(field.metadata).getOrElse(field.dataType) @@ -3752,12 +3755,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val resolved = a.copy(dataType = newDataType, position = newPosition) resolved.copyTagsFrom(a) resolved - - case a: AlterTableCommand if a.table.resolved && hasUnresolvedFieldName(a) => - val table = a.table.asInstanceOf[ResolvedTable] - a.transformExpressions { - case u: UnresolvedFieldName => resolveFieldNames(table, u.name, u) - } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 98a52bde25927..e95c21ad985e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.trees.TreePattern.{LATERAL_COLUMN_ALIAS_REFERENCE, UNRESOLVED_WINDOW_EXPRESSION} -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, ResolveDefaultColumns => DefaultColumnUtil, StringUtils, TypeUtils} +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils, TypeUtils} import org.apache.spark.sql.connector.catalog.{LookupCatalog, SupportsPartitionManagement} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} import org.apache.spark.sql.internal.SQLConf @@ -240,9 +240,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB // general unresolved check below to throw a more tailored error message. ResolveReferencesInAggregate.checkUnresolvedGroupByAll(operator) - // Early checks for column default values, to produce better error messages - DefaultColumnUtil.checkDefaultValuesInPlan(operator) - getAllExpressions(operator).foreach(_.foreachUp { case a: Attribute if !a.resolved => failUnresolvedAttribute(operator, a, "UNRESOLVED_COLUMN") @@ -594,9 +591,8 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case create: V2CreateTablePlan => val references = create.partitioning.flatMap(_.references).toSet - val tableSchema = create.tableSchema val badReferences = references.map(_.fieldNames).flatMap { column => - tableSchema.findNestedField(column) match { + create.tableSchema.findNestedField(column) match { case Some(_) => None case _ => @@ -610,7 +606,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB messageParameters = Map("cols" -> badReferences.mkString(", "))) } - tableSchema.foreach(f => TypeUtils.failWithIntervalType(f.dataType)) + create.tableSchema.foreach(f => TypeUtils.failWithIntervalType(f.dataType)) case write: V2WriteCommand if write.resolved => write.query.schema.foreach(f => TypeUtils.failWithIntervalType(f.dataType)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultColumns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultColumns.scala index b09a2c4ef3eec..a04844c6526db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultColumns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultColumns.scala @@ -155,7 +155,7 @@ case class ResolveDefaultColumns(catalog: SessionCatalog) extends Rule[LogicalPl val schemaForTargetTable: Option[StructType] = getSchemaForTargetTable(u.table) schemaForTargetTable.map { schema => val defaultExpressions: Seq[Expression] = schema.fields.map { - case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f) + case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f, "UPDATE") case _ => Literal(null) } // Create a map from each column name in the target table to its DEFAULT expression. @@ -185,7 +185,7 @@ case class ResolveDefaultColumns(catalog: SessionCatalog) extends Rule[LogicalPl } } val defaultExpressions: Seq[Expression] = schema.fields.map { - case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f) + case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f, "MERGE") case _ => Literal(null) } val columnNamesToExpressions: Map[String, Expression] = @@ -349,7 +349,7 @@ case class ResolveDefaultColumns(catalog: SessionCatalog) extends Rule[LogicalPl input: LogicalPlan): Option[LogicalPlan] = { val schema = insertTableSchemaWithoutPartitionColumns val defaultExpressions: Seq[Expression] = schema.fields.map { - case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f) + case f if f.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => analyze(f, "INSERT") case _ => Literal(null) } // Check the type of `input` and replace its expressions accordingly. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 412cc9cb11212..2d26e281607ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -102,7 +102,6 @@ sealed trait FieldName extends LeafExpression with Unevaluable { } case class UnresolvedFieldName(name: Seq[String]) extends FieldName { - assert(name.length > 0) override lazy val resolved = false } @@ -196,10 +195,6 @@ case class ResolvedFieldName(path: Seq[String], field: StructField) extends Fiel def name: Seq[String] = path :+ field.name } -case object RootTableSchema extends FieldName { - def name: Seq[String] = Nil -} - case class ResolvedFieldPosition(position: ColumnPosition) extends FieldPosition @@ -251,7 +246,6 @@ case class ResolvedIdentifier( catalog: CatalogPlugin, identifier: Identifier) extends LeafNodeWithoutStats { override def output: Seq[Attribute] = Nil - def name: String = (catalog.name +: identifier.namespace() :+ identifier.name()).quoted } // A fake v2 catalog to hold temp views. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index e25d3a0544332..aea496b872bc2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils} +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils, ResolveDefaultColumns} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, getZoneId, stringToDate, stringToTimestamp, stringToTimestampWithoutTimeZone} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog} import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition @@ -2955,14 +2955,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Create top level table schema. */ protected def createSchema(ctx: CreateOrReplaceTableColTypeListContext): StructType = { - val columns = Option(ctx).toArray.flatMap(visitCreateOrReplaceTableColTypeList) - StructType(columns.map { c => - if (c.defaultValue.isDefined) { - throw QueryParsingErrors.defaultColumnNotImplementedYetError(ctx) - } - val field = StructField(c.name, c.dataType, c.nullable) - c.comment.map(field.withComment).getOrElse(field) - }) + StructType(Option(ctx).toArray.flatMap(visitCreateOrReplaceTableColTypeList)) } /** @@ -2995,18 +2988,17 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Create a [[StructType]] from a number of CREATE TABLE column definitions. */ override def visitCreateOrReplaceTableColTypeList( - ctx: CreateOrReplaceTableColTypeListContext): Seq[ColumnDefinition] = withOrigin(ctx) { + ctx: CreateOrReplaceTableColTypeListContext): Seq[StructField] = withOrigin(ctx) { ctx.createOrReplaceTableColType().asScala.map(visitCreateOrReplaceTableColType).toSeq } /** - * Create a top level column from a CREATE TABLE column definition. + * Create a top level [[StructField]] from a CREATE TABLE column definition. */ override def visitCreateOrReplaceTableColType( - ctx: CreateOrReplaceTableColTypeContext): ColumnDefinition = withOrigin(ctx) { + ctx: CreateOrReplaceTableColTypeContext): StructField = withOrigin(ctx) { import ctx._ - val name: String = colName.getText // Check that no duplicates exist among any CREATE TABLE column options specified. var nullable = true var defaultExpression: Option[DefaultExpressionContext] = None @@ -3015,35 +3007,49 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit if (option.NULL != null) { if (!nullable) { throw QueryParsingErrors.duplicateCreateTableColumnOption( - option, name, "NOT NULL") + option, colName.getText, "NOT NULL") } nullable = false } Option(option.defaultExpression()).foreach { expr => - if (!conf.getConf(SQLConf.ENABLE_DEFAULT_COLUMNS)) { - throw QueryParsingErrors.defaultColumnNotEnabledError(ctx) - } if (defaultExpression.isDefined) { throw QueryParsingErrors.duplicateCreateTableColumnOption( - option, name, "DEFAULT") + option, colName.getText, "DEFAULT") } defaultExpression = Some(expr) } Option(option.commentSpec()).foreach { spec => if (commentSpec.isDefined) { throw QueryParsingErrors.duplicateCreateTableColumnOption( - option, name, "COMMENT") + option, colName.getText, "COMMENT") } commentSpec = Some(spec) } } - ColumnDefinition( + val builder = new MetadataBuilder + // Add comment to metadata + commentSpec.map(visitCommentSpec).foreach { + builder.putString("comment", _) + } + // Add the 'DEFAULT expression' clause in the column definition, if any, to the column metadata. + defaultExpression.map(visitDefaultExpression).foreach { field => + if (conf.getConf(SQLConf.ENABLE_DEFAULT_COLUMNS)) { + // Add default to metadata + builder.putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, field) + builder.putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, field) + } else { + throw QueryParsingErrors.defaultColumnNotEnabledError(ctx) + } + } + + val name: String = colName.getText + + StructField( name = name, dataType = typedVisit[DataType](ctx.dataType), nullable = nullable, - comment = commentSpec.map(visitCommentSpec), - defaultValue = defaultExpression.map(visitDefaultExpression)) + metadata = builder.build()) } /** @@ -3094,17 +3100,20 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit string(visitStringLit(ctx.stringLit)) } - override def visitDefaultExpression(ctx: DefaultExpressionContext): DefaultValueExpression = { - withOrigin(ctx) { - val exprCtx = ctx.expression() - // Extract the raw expression text so that we can save the user provided text. This will be - // used in error reporting. Note: `exprCtx.getText` returns a string without spaces, so we - // need to get the text from the underlying char stream instead. - val start = exprCtx.getStart.getStartIndex - val end = exprCtx.getStop.getStopIndex - val originalSQL = exprCtx.getStart.getInputStream.getText(new Interval(start, end)) - DefaultValueExpression(expression(exprCtx), originalSQL) - } + /** + * Create a default string. + */ + override def visitDefaultExpression(ctx: DefaultExpressionContext): String = withOrigin(ctx) { + val exprCtx = ctx.expression() + // Make sure it can be converted to Catalyst expressions. + expression(exprCtx) + // Extract the raw expression text so that we can save the user provided text. We don't + // use `Expression.sql` to avoid storing incorrect text caused by bugs in any expression's + // `sql` method. Note: `exprCtx.getText` returns a string without spaces, so we need to + // get the text from the underlying char stream instead. + val start = exprCtx.getStart.getStartIndex + val end = exprCtx.getStop.getStopIndex + exprCtx.getStart.getInputStream.getText(new Interval(start, end)) } /** @@ -3239,7 +3248,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * types like `i INT`, which should be appended to the existing table schema. */ type TableClauses = ( - Seq[Transform], Seq[ColumnDefinition], Option[BucketSpec], Map[String, String], + Seq[Transform], Seq[StructField], Option[BucketSpec], Map[String, String], Map[String, String], Option[String], Option[String], Option[SerdeInfo]) /** @@ -3267,14 +3276,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Parse a list of transforms or columns. */ override def visitPartitionFieldList( - ctx: PartitionFieldListContext): (Seq[Transform], Seq[ColumnDefinition]) = withOrigin(ctx) { + ctx: PartitionFieldListContext): (Seq[Transform], Seq[StructField]) = withOrigin(ctx) { val (transforms, columns) = ctx.fields.asScala.map { case transform: PartitionTransformContext => (Some(visitPartitionTransform(transform)), None) case field: PartitionColumnContext => - val f = visitColType(field.colType) - val col = ColumnDefinition(f.name, f.dataType, f.nullable, f.getComment(), None) - (None, Some(col)) + (None, Some(visitColType(field.colType))) }.unzip (transforms.flatten.toSeq, columns.flatten.toSeq) @@ -3729,13 +3736,13 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit private def partitionExpressions( partTransforms: Seq[Transform], - partCols: Seq[ColumnDefinition], + partCols: Seq[StructField], ctx: ParserRuleContext): Seq[Transform] = { if (partTransforms.nonEmpty) { if (partCols.nonEmpty) { val references = partTransforms.map(_.describe()).mkString(", ") val columns = partCols - .map(column => s"${column.name} ${column.dataType.simpleString}") + .map(field => s"${field.name} ${field.dataType.simpleString}") .mkString(", ") operationNotAllowed( s"""PARTITION BY: Cannot mix partition expressions and partition columns: @@ -3823,8 +3830,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit case _ => // Note: table schema includes both the table columns list and the partition columns // with data type. + val schema = StructType(columns ++ partCols) CreateTable(UnresolvedIdentifier(table), - columns ++ partCols, partitioning, tableSpec, ignoreIfExists = ifNotExists) + schema, partitioning, tableSpec, ignoreIfExists = ifNotExists) } } @@ -3892,8 +3900,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit case _ => // Note: table schema includes both the table columns list and the partition columns // with data type. + val schema = StructType(columns ++ partCols) ReplaceTable(UnresolvedIdentifier(table), - columns ++ partCols, partitioning, tableSpec, orCreate = orCreate) + schema, partitioning, tableSpec, orCreate = orCreate) } } @@ -3973,21 +3982,20 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit override def visitQualifiedColTypeWithPosition( ctx: QualifiedColTypeWithPositionContext): QualifiedColType = withOrigin(ctx) { val name = typedVisit[Seq[String]](ctx.name) + // Add the 'DEFAULT expression' clause in the column definition, if any, to the column metadata. val defaultExpr = Option(ctx.defaultExpression()).map(visitDefaultExpression) if (defaultExpr.isDefined && !conf.getConf(SQLConf.ENABLE_DEFAULT_COLUMNS)) { throw QueryParsingErrors.defaultColumnNotEnabledError(ctx) } QualifiedColType( - path = if (name.length > 1) UnresolvedFieldName(name.init) else RootTableSchema, + path = if (name.length > 1) Some(UnresolvedFieldName(name.init)) else None, + colName = name.last, + dataType = typedVisit[DataType](ctx.dataType), + nullable = ctx.NULL == null, + comment = Option(ctx.commentSpec()).map(visitCommentSpec), position = Option(ctx.colPosition).map( pos => UnresolvedFieldPosition(typedVisit[ColumnPosition](pos))), - column = ColumnDefinition( - name = name.last, - dataType = typedVisit[DataType](ctx.dataType), - nullable = ctx.NULL == null, - comment = Option(ctx.commentSpec()).map(visitCommentSpec), - defaultValue = defaultExpr - )) + default = defaultExpr) } /** @@ -4032,8 +4040,6 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * ALTER TABLE table1 ALTER COLUMN a.b.c SET NOT NULL * ALTER TABLE table1 ALTER COLUMN a.b.c DROP NOT NULL * ALTER TABLE table1 ALTER COLUMN a.b.c COMMENT 'new comment' - * ALTER TABLE table1 ALTER COLUMN a.b.c SET DEFAULT value - * ALTER TABLE table1 ALTER COLUMN a.b.c DROP DEFAULT * ALTER TABLE table1 ALTER COLUMN a.b.c FIRST * ALTER TABLE table1 ALTER COLUMN a.b.c AFTER x * }}} @@ -4070,18 +4076,19 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } else { None } - val defaultExpression: Option[Expression] = if (action.defaultExpression != null) { - Some(visitDefaultExpression(action.defaultExpression)) - } else if (action.dropDefault != null) { - Some(DropDefaultColumnValue) - } else { - None - } - if (defaultExpression.isDefined && !conf.getConf(SQLConf.ENABLE_DEFAULT_COLUMNS)) { + val setDefaultExpression: Option[String] = + if (action.defaultExpression != null) { + Option(action.defaultExpression()).map(visitDefaultExpression) + } else if (action.dropDefault != null) { + Some("") + } else { + None + } + if (setDefaultExpression.isDefined && !conf.getConf(SQLConf.ENABLE_DEFAULT_COLUMNS)) { throw QueryParsingErrors.defaultColumnNotEnabledError(ctx) } - assert(Seq(dataType, nullable, comment, position, defaultExpression) + assert(Seq(dataType, nullable, comment, position, setDefaultExpression) .count(_.nonEmpty) == 1) AlterColumn( @@ -4091,7 +4098,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit nullable = nullable, comment = comment, position = position, - defaultExpression = defaultExpression) + setDefaultExpression = setDefaultExpression) } /** @@ -4127,7 +4134,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit comment = Option(ctx.colType().commentSpec()).map(visitCommentSpec), position = Option(ctx.colPosition).map( pos => UnresolvedFieldPosition(typedVisit[ColumnPosition](pos))), - defaultExpression = None) + setDefaultExpression = None) } override def visitHiveReplaceColumns( @@ -4147,7 +4154,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit "Column position", "REPLACE COLUMNS", ctx) } val col = typedVisit[QualifiedColType](colType) - if (col.path != RootTableSchema) { + if (col.path.isDefined) { throw QueryParsingErrors.operationInHiveStyleCommandUnsupportedError( "Replacing with a nested column", "REPLACE COLUMNS", ctx) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 9fbb5892e405f..897869ab4f256 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -17,9 +17,14 @@ package org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.analysis.{FieldName, FieldPosition} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.trees.{LeafLike, UnaryLike} +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns +import org.apache.spark.sql.connector.catalog.ColumnDefaultValue +import org.apache.spark.sql.connector.expressions.LiteralValue import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.types.DataType /** * A logical plan node that contains exactly what was parsed from SQL. @@ -119,6 +124,32 @@ object SerdeInfo { } } +/** + * Column data as parsed by ALTER TABLE ... (ADD|REPLACE) COLUMNS. + */ +case class QualifiedColType( + path: Option[FieldName], + colName: String, + dataType: DataType, + nullable: Boolean, + comment: Option[String], + position: Option[FieldPosition], + default: Option[String]) { + def name: Seq[String] = path.map(_.name).getOrElse(Nil) :+ colName + + def resolved: Boolean = path.forall(_.resolved) && position.forall(_.resolved) + + def getV2Default: ColumnDefaultValue = { + default.map { sql => + val e = ResolveDefaultColumns.analyze(colName, dataType, sql, "ALTER TABLE") + assert(e.resolved && e.foldable, + "exist default must be simple SQL string that is resolved and foldable, " + + "but got: " + sql) + new ColumnDefaultValue(sql, LiteralValue(e.eval(), dataType)) + }.orNull + } +} + /** * An INSERT INTO statement, as parsed from SQL. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala index ae0f2ea71db5c..eb9d45f06ec79 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala @@ -17,14 +17,12 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.analysis.{FieldName, FieldPosition, ResolvedTable, RootTableSchema} +import org.apache.spark.sql.catalyst.analysis.{FieldName, FieldPosition} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression, Unevaluable} import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.connector.catalog.{TableCatalog, TableChange} import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.{DataType, NullType} +import org.apache.spark.sql.types.DataType /** * The base trait for commands that need to alter a v2 table with [[TableChange]]s. @@ -33,10 +31,6 @@ trait AlterTableCommand extends UnaryCommand { def changes: Seq[TableChange] def table: LogicalPlan final override def child: LogicalPlan = table - def resolvedTable: ResolvedTable = { - assert(childrenResolved) - table.asInstanceOf[ResolvedTable] - } } /** @@ -102,30 +96,6 @@ case class UnsetTableProperties( copy(table = newChild) } -/** - * Column data as parsed by ALTER TABLE ... (ADD|REPLACE) COLUMNS. - */ -case class QualifiedColType( - path: FieldName, - column: ColumnDefinition, - position: Option[FieldPosition] = None) extends Expression with Unevaluable { - def name: Seq[String] = path.name :+ column.name - override def children: Seq[Expression] = path +: column +: position.toSeq - - override def dataType: DataType = throw SparkException.internalError( - "QualifiedColType.dataType should not be called.") - override def nullable: Boolean = throw SparkException.internalError( - "QualifiedColType.nullable should not be called.") - - override protected def withNewChildrenInternal( - newChildren: IndexedSeq[Expression]): Expression = { - copy( - newChildren(0).asInstanceOf[FieldName], - newChildren(1).asInstanceOf[ColumnDefinition], - newChildren.drop(2).headOption.map(_.asInstanceOf[FieldPosition])) - } -} - /** * The logical plan of the ALTER TABLE ... ADD COLUMNS command. */ @@ -133,22 +103,24 @@ case class AddColumns( table: LogicalPlan, columnsToAdd: Seq[QualifiedColType]) extends AlterTableCommand { columnsToAdd.foreach { c => - TypeUtils.failWithIntervalType(c.column.dataType) + TypeUtils.failWithIntervalType(c.dataType) } + override lazy val resolved: Boolean = table.resolved && columnsToAdd.forall(_.resolved) + override def changes: Seq[TableChange] = { columnsToAdd.map { col => - require(col.path.resolved, + require(col.path.forall(_.resolved), "FieldName should be resolved before it's converted to TableChange.") require(col.position.forall(_.resolved), "FieldPosition should be resolved before it's converted to TableChange.") TableChange.addColumn( col.name.toArray, - col.column.dataType, - col.column.nullable, - col.column.comment.orNull, + col.dataType, + col.nullable, + col.comment.orNull, col.position.map(_.position).orNull, - col.column.defaultValue.map(_.toV2).orNull) + col.getV2Default) } } @@ -163,9 +135,11 @@ case class ReplaceColumns( table: LogicalPlan, columnsToAdd: Seq[QualifiedColType]) extends AlterTableCommand { columnsToAdd.foreach { c => - TypeUtils.failWithIntervalType(c.column.dataType) + TypeUtils.failWithIntervalType(c.dataType) } + override lazy val resolved: Boolean = table.resolved && columnsToAdd.forall(_.resolved) + override def changes: Seq[TableChange] = { // REPLACE COLUMNS deletes all the existing columns and adds new columns specified. require(table.resolved) @@ -174,15 +148,15 @@ case class ReplaceColumns( TableChange.deleteColumn(Array(name), false /* ifExists */) } val addChanges = columnsToAdd.map { col => - assert(col.path == RootTableSchema) + assert(col.path.isEmpty) assert(col.position.isEmpty) TableChange.addColumn( col.name.toArray, - col.column.dataType, - col.column.nullable, - col.column.comment.orNull, + col.dataType, + col.nullable, + col.comment.orNull, null, - col.column.defaultValue.map(_.toV2).orNull) + col.getV2Default) } deleteChanges ++ addChanges } @@ -225,12 +199,6 @@ case class RenameColumn( copy(table = newChild) } -// A fake expression to indicate a drop column default value action in `AlterColumn`. -case object DropDefaultColumnValue extends LeafExpression with Unevaluable { - override def nullable: Boolean = true - override def dataType: DataType = NullType -} - /** * The logical plan of the ALTER TABLE ... ALTER COLUMN command. */ @@ -241,9 +209,7 @@ case class AlterColumn( nullable: Option[Boolean], comment: Option[String], position: Option[FieldPosition], - defaultExpression: Option[Expression]) extends AlterTableCommand { - assert(column != RootTableSchema, "AlterTable.column must be a real (nested) column.") - + setDefaultExpression: Option[String]) extends AlterTableCommand { override def changes: Seq[TableChange] = { require(column.resolved, "FieldName should be resolved before it's converted to TableChange.") val colName = column.name.toArray @@ -261,13 +227,8 @@ case class AlterColumn( "FieldPosition should be resolved before it's converted to TableChange.") TableChange.updateColumnPosition(colName, newPosition.position) } - val defaultValueChange = defaultExpression.map { - case DropDefaultColumnValue => - TableChange.updateColumnDefaultValue(colName, null) - case d: DefaultValueExpression => - TableChange.updateColumnDefaultValue(colName, d.originalSQL) - case other => throw SparkException.internalError( - "Unexpected expression in AlterColumn.defaultExpression: " + other) + val defaultValueChange = setDefaultExpression.map { newDefaultExpression => + TableChange.updateColumnDefaultValue(colName, newDefaultExpression) } typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange ++ defaultValueChange } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 50df441982481..9508b2fb99336 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -17,21 +17,19 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, EliminateSubqueryAliases, FieldName, NamedRelation, PartitionSpec, ResolvedIdentifier, UnresolvedException} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.FunctionResource -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, Literal, MetadataAttribute, NamedExpression, UnaryExpression, Unevaluable, V2ExpressionUtils} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, MetadataAttribute, NamedExpression, Unevaluable, V2ExpressionUtils} import org.apache.spark.sql.catalyst.plans.DescribeCommandSchema import org.apache.spark.sql.catalyst.trees.BinaryLike import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, RowDeltaUtils, WriteDeltaProjections} -import org.apache.spark.sql.connector.catalog.{Column => V2Column, ColumnDefaultValue} -import org.apache.spark.sql.connector.expressions.{LiteralValue, Transform} +import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.expressions.filter.Predicate import org.apache.spark.sql.connector.write.{DeltaWrite, RowLevelOperation, RowLevelOperationTable, SupportsDelta, Write} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.internal.connector.ColumnImpl import org.apache.spark.sql.types.{BooleanType, DataType, IntegerType, MetadataBuilder, StringType, StructField, StructType} // For v2 DML commands, it may end up with the v1 fallback code path and need to build a DataFrame @@ -393,18 +391,9 @@ case class WriteDelta( /** A trait used for logical plan nodes that create or replace V2 table definitions. */ trait V2CreateTablePlan extends LogicalPlan { - def name: LogicalPlan - def tableSpec: TableSpec + def tableName: Identifier def partitioning: Seq[Transform] - def columns: Seq[ColumnDefinition] - def tableSchema: StructType = StructType(columns.map { col => - // Schema only cares about the tree structure with name and data type. - StructField(col.name, col.dataType, col.nullable) - }) - def resolvedName: ResolvedIdentifier = { - assert(childrenResolved) - name.asInstanceOf[ResolvedIdentifier] - } + def tableSchema: StructType /** * Creates a copy of this node with the new partitioning transforms. This method is used to @@ -413,66 +402,23 @@ trait V2CreateTablePlan extends LogicalPlan { def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan } -/** - * A fake expression to hold the column default value expression and its original SQL text. - */ -case class DefaultValueExpression(child: Expression, originalSQL: String) - extends UnaryExpression with Unevaluable { - override def dataType: DataType = child.dataType - override protected def withNewChildInternal(newChild: Expression): Expression = - copy(child = newChild) - - // Convert the default expression to ColumnDefaultValue, which is required by DS v2 APIs. - def toV2: ColumnDefaultValue = child match { - case Literal(value, dataType) => - new ColumnDefaultValue(originalSQL, LiteralValue(value, dataType)) - // Analyzer makes sure the column default value is a constant. - case other => throw SparkException.internalError( - "Default value must be a literal, but got " + other) - } -} - -/** - * Column definition for tables. This is an expression so that analyzer can resolve the default - * value expression in DDL commands automatically. - */ -case class ColumnDefinition( - name: String, - dataType: DataType, - nullable: Boolean = true, - comment: Option[String] = None, - defaultValue: Option[DefaultValueExpression] = None, - metadataInJSON: Option[String] = None) extends Expression with Unevaluable { - override def children: Seq[Expression] = defaultValue.toSeq - - override protected def withNewChildrenInternal( - newChildren: IndexedSeq[Expression]): Expression = { - copy(defaultValue = newChildren.headOption.map(_.asInstanceOf[DefaultValueExpression])) - } - - def toV2Column: V2Column = { - ColumnImpl( - name, - dataType, - nullable, - comment.orNull, - defaultValue.map(_.toV2).orNull, - metadataInJSON.orNull) - } -} - /** * Create a new table with a v2 catalog. */ case class CreateTable( name: LogicalPlan, - columns: Seq[ColumnDefinition], + tableSchema: StructType, partitioning: Seq[Transform], tableSpec: TableSpec, ignoreIfExists: Boolean) extends UnaryCommand with V2CreateTablePlan { override def child: LogicalPlan = name + override def tableName: Identifier = { + assert(child.resolved) + child.asInstanceOf[ResolvedIdentifier].identifier + } + override protected def withNewChildInternal(newChild: LogicalPlan): V2CreateTablePlan = copy(name = newChild) @@ -494,11 +440,15 @@ case class CreateTableAsSelect( analyzedQuery: Option[LogicalPlan] = None) extends BinaryCommand with V2CreateTablePlan with KeepAnalyzedQuery { - override def columns: Seq[ColumnDefinition] = query.schema.toColumns override def tableSchema: StructType = query.schema override def left: LogicalPlan = name override def right: LogicalPlan = query + override def tableName: Identifier = { + assert(left.resolved) + left.asInstanceOf[ResolvedIdentifier].identifier + } + override lazy val resolved: Boolean = childrenResolved && { // the table schema is created from the query schema, so the only resolution needed is to check // that the columns referenced by the table's partitioning exist in the query schema @@ -529,13 +479,18 @@ case class CreateTableAsSelect( */ case class ReplaceTable( name: LogicalPlan, - columns: Seq[ColumnDefinition], + tableSchema: StructType, partitioning: Seq[Transform], tableSpec: TableSpec, orCreate: Boolean) extends UnaryCommand with V2CreateTablePlan { override def child: LogicalPlan = name + override def tableName: Identifier = { + assert(child.resolved) + child.asInstanceOf[ResolvedIdentifier].identifier + } + override protected def withNewChildInternal(newChild: LogicalPlan): V2CreateTablePlan = copy(name = newChild) @@ -560,7 +515,6 @@ case class ReplaceTableAsSelect( analyzedQuery: Option[LogicalPlan] = None) extends BinaryCommand with V2CreateTablePlan with KeepAnalyzedQuery { - override def columns: Seq[ColumnDefinition] = query.schema.toColumns override def tableSchema: StructType = query.schema override def left: LogicalPlan = name override def right: LogicalPlan = query @@ -572,6 +526,11 @@ case class ReplaceTableAsSelect( references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) } + override def tableName: Identifier = { + assert(name.resolved) + name.asInstanceOf[ResolvedIdentifier].identifier + } + override def storeAnalyzedQuery(): Command = copy(analyzedQuery = Some(query)) override protected def withNewChildrenInternal( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala index 7f08e226c2f50..be7d74b078232 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.util -import java.util.{Map => JMap} - import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException @@ -28,10 +26,10 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, InMemoryCatalog, import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.{Literal => ExprLiteral} import org.apache.spark.sql.catalyst.optimizer.ConstantFolding -import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, FunctionCatalog, Identifier, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogManager, FunctionCatalog, Identifier} import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf @@ -70,34 +68,143 @@ object ResolveDefaultColumns { val CURRENT_DEFAULT_COLUMN_NAME = "DEFAULT" /** - * Parses and analyzes the DEFAULT column text in `field`. The default value has already been - * validated in CREATE/REPLACE/ALTER TABLE commands. We don't need to validate it again when - * reading it out. + * Finds "current default" expressions in CREATE/REPLACE TABLE columns and constant-folds them. + * + * The results are stored in the "exists default" metadata of the same columns. For example, in + * the event of this statement: + * + * CREATE TABLE T(a INT, b INT DEFAULT 5 + 5) + * + * This method constant-folds the "current default" value, stored in the CURRENT_DEFAULT metadata + * of the "b" column, to "10", storing the result in the "exists default" value within the + * EXISTS_DEFAULT metadata of that same column. Meanwhile the "current default" metadata of this + * "b" column retains its original value of "5 + 5". + * + * The reason for constant-folding the EXISTS_DEFAULT is to make the end-user visible behavior the + * same, after executing an ALTER TABLE ADD COLUMNS command with DEFAULT value, as if the system + * had performed an exhaustive backfill of the provided value to all previously existing rows in + * the table instead. We choose to avoid doing such a backfill because it would be a + * time-consuming and costly operation. Instead, we elect to store the EXISTS_DEFAULT in the + * column metadata for future reference when querying data out of the data source. In turn, each + * data source then takes responsibility to provide the constant-folded value in the + * EXISTS_DEFAULT metadata for such columns where the value is not present in storage. + * + * @param tableSchema represents the names and types of the columns of the statement to process. + * @param tableProvider provider of the target table to store default values for, if any. + * @param statementType name of the statement being processed, such as INSERT; useful for errors. + * @param addNewColumnToExistingTable true if the statement being processed adds a new column to + * a table that already exists. + * @return a copy of `tableSchema` with field metadata updated with the constant-folded values. + */ + def constantFoldCurrentDefaultsToExistDefaults( + tableSchema: StructType, + tableProvider: Option[String], + statementType: String, + addNewColumnToExistingTable: Boolean): StructType = { + if (SQLConf.get.enableDefaultColumns) { + val keywords: Array[String] = + SQLConf.get.getConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS) + .toLowerCase().split(",").map(_.trim) + val allowedTableProviders: Array[String] = + keywords.map(_.stripSuffix("*")) + val addColumnExistingTableBannedProviders: Array[String] = + keywords.filter(_.endsWith("*")).map(_.stripSuffix("*")) + val givenTableProvider: String = tableProvider.getOrElse("").toLowerCase() + val newFields: Seq[StructField] = tableSchema.fields.map { field => + if (field.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY)) { + // Make sure that the target table has a provider that supports default column values. + if (!allowedTableProviders.contains(givenTableProvider)) { + throw QueryCompilationErrors + .defaultReferencesNotAllowedInDataSource(statementType, givenTableProvider) + } + if (addNewColumnToExistingTable && + givenTableProvider.nonEmpty && + addColumnExistingTableBannedProviders.contains(givenTableProvider)) { + throw QueryCompilationErrors + .addNewDefaultColumnToExistingTableNotAllowed(statementType, givenTableProvider) + } + val analyzed: Expression = analyze(field, statementType) + val newMetadata: Metadata = new MetadataBuilder().withMetadata(field.metadata) + .putString(EXISTS_DEFAULT_COLUMN_METADATA_KEY, analyzed.sql).build() + field.copy(metadata = newMetadata) + } else { + field + } + } + StructType(newFields) + } else { + tableSchema + } + } + + /** + * Parses and analyzes the DEFAULT column text in `field`, returning an error upon failure. * * @param field represents the DEFAULT column value whose "default" metadata to parse * and analyze. + * @param statementType which type of statement we are running, such as INSERT; useful for errors. * @param metadataKey which key to look up from the column metadata; generally either * CURRENT_DEFAULT_COLUMN_METADATA_KEY or EXISTS_DEFAULT_COLUMN_METADATA_KEY. * @return Result of the analysis and constant-folding operation. */ def analyze( field: StructField, + statementType: String, metadataKey: String = CURRENT_DEFAULT_COLUMN_METADATA_KEY): Expression = { + analyze(field.name, field.dataType, field.metadata.getString(metadataKey), statementType) + } + + /** + * Parses and analyzes the DEFAULT column SQL string, returning an error upon failure. + * + * @return Result of the analysis and constant-folding operation. + */ + def analyze( + colName: String, + dataType: DataType, + defaultSQL: String, + statementType: String): Expression = { // Parse the expression. - val colText: String = field.metadata.getString(metadataKey) - val parser = new CatalystSqlParser() - val parsed: Expression = parser.parseExpression(colText) + lazy val parser = new CatalystSqlParser() + val parsed: Expression = try { + parser.parseExpression(defaultSQL) + } catch { + case ex: ParseException => + throw new AnalysisException( + s"Failed to execute $statementType command because the destination table column " + + s"$colName has a DEFAULT value of $defaultSQL which fails to parse as a valid " + + s"expression: ${ex.getMessage}") + } + // Check invariants before moving on to analysis. + if (parsed.containsPattern(PLAN_EXPRESSION)) { + throw QueryCompilationErrors.defaultValuesMayNotContainSubQueryExpressions() + } // Analyze the parse result. - val analyzer: Analyzer = DefaultColumnAnalyzer - val plan = analyzer.execute(Project(Seq(Alias(parsed, field.name)()), OneRowRelation())) + val plan = try { + val analyzer: Analyzer = DefaultColumnAnalyzer + val analyzed = analyzer.execute(Project(Seq(Alias(parsed, colName)()), OneRowRelation())) + analyzer.checkAnalysis(analyzed) + ConstantFolding(analyzed) + } catch { + case ex: AnalysisException => + throw new AnalysisException( + s"Failed to execute $statementType command because the destination table column " + + s"$colName has a DEFAULT value of $defaultSQL which fails to resolve as a valid " + + s"expression: ${ex.getMessage}") + } val analyzed: Expression = plan.collectFirst { case Project(Seq(a: Alias), OneRowRelation()) => a.child }.get // Perform implicit coercion from the provided expression type to the required column type. - if (field.dataType == analyzed.dataType) { + if (dataType == analyzed.dataType) { analyzed + } else if (Cast.canUpCast(analyzed.dataType, dataType)) { + Cast(analyzed, dataType) } else { - Cast(analyzed, field.dataType) + throw new AnalysisException( + s"Failed to execute $statementType command because the destination table column " + + s"$colName has a DEFAULT value with type $dataType, but the " + + s"statement provided a value of incompatible type ${analyzed.dataType}") } } /** @@ -128,7 +235,7 @@ object ResolveDefaultColumns { val defaultValue: Option[String] = field.getExistenceDefaultValue() defaultValue.map { text: String => val expr = try { - val expr = analyze(field, EXISTS_DEFAULT_COLUMN_METADATA_KEY) + val expr = analyze(field, "", EXISTS_DEFAULT_COLUMN_METADATA_KEY) expr match { case _: ExprLiteral | _: Cast => expr } @@ -189,117 +296,6 @@ object ResolveDefaultColumns { rows.toSeq } - def checkDefaultValuesInPlan(plan: LogicalPlan, isForV1: Boolean = false): Unit = { - plan match { - // Do not check anything if the children are not resolved yet. - case _ if !plan.childrenResolved => - case AlterColumn(t: ResolvedTable, col: ResolvedFieldName, _, _, _, _, - Some(default: DefaultValueExpression)) => - checkTableProvider(t.catalog, t.name, getTableProviderFromProp(t.table.properties())) - checkDefaultValue(default, t.name, col.name, col.field.dataType, isForV1) - - case cmd: V2CreateTablePlan if cmd.columns.exists(_.defaultValue.isDefined) => - val ident = cmd.resolvedName - checkTableProvider(ident.catalog, ident.name, cmd.tableSpec.provider) - cmd.columns.foreach { col => - if (col.defaultValue.isDefined) { - // CREATE/REPLACE TABLE only has top-level columns - val colName = Seq(col.name) - checkDefaultValue(col.defaultValue.get, ident.name, colName, col.dataType, isForV1) - } - } - - case cmd: AlterTableCommand => - val table = cmd.resolvedTable - val provider = getTableProviderFromProp(table.table.properties()) - val isAddColumns = cmd.isInstanceOf[AddColumns] - cmd.transformExpressionsDown { - case q @ QualifiedColType(path, col, _) if col.defaultValue.isDefined && path.resolved => - checkTableProvider(table.catalog, table.name, provider, isAddColumns) - checkDefaultValue( - col.defaultValue.get, - table.name, - path.name :+ col.name, - col.dataType, - isForV1) - q - } - - case _ => - } - } - - private def getTableProviderFromProp(props: JMap[String, String]): Option[String] = { - Option(props.get(TableCatalog.PROP_PROVIDER)) - } - - private def checkTableProvider( - catalog: CatalogPlugin, - tableName: String, - provider: Option[String], - isAddColumns: Boolean = false): Unit = { - // We only need to check table provider for the session catalog. Other custom v2 catalogs - // can check table providers in their implementations of createTable, alterTable, etc. - if (CatalogV2Util.isSessionCatalog(catalog)) { - val conf = SQLConf.get - val keywords: Array[String] = conf.getConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS) - .toLowerCase().split(",").map(_.trim) - val allowedTableProviders: Array[String] = keywords.map(_.stripSuffix("*")) - val addColumnExistingTableBannedProviders: Array[String] = - keywords.filter(_.endsWith("*")).map(_.stripSuffix("*")) - val providerName: String = provider.getOrElse(conf.defaultDataSourceName).toLowerCase() - if (!allowedTableProviders.contains(providerName)) { - throw QueryCompilationErrors.defaultReferencesNotAllowedInDataSource(tableName) - } - if (isAddColumns && addColumnExistingTableBannedProviders.contains(providerName)) { - throw QueryCompilationErrors.addNewDefaultColumnToExistingTableNotAllowed(providerName) - } - } - } - - private def checkDefaultValue( - default: DefaultValueExpression, - tblName: String, - colName: Seq[String], - targetType: DataType, - isForV1: Boolean): Unit = { - if (default.containsPattern(PLAN_EXPRESSION)) { - throw QueryCompilationErrors.hasSubqueryDefaultValueError( - tblName, colName, default.originalSQL) - } else if (default.resolved) { - if (!default.child.foldable) { - throw QueryCompilationErrors.notConstantDefaultValueError( - tblName, colName, default.originalSQL) - } - if (!Cast.canUpCast(default.child.dataType, targetType)) { - throw QueryCompilationErrors.incompatibleTypeDefaultValueError( - tblName, colName, targetType, default.child, default.originalSQL) - } - // Check passes. - } else if (default.references.nonEmpty) { - // Ideally we should let the rest of `CheckAnalysis` to report errors about why the default - // expression is unresolved. But we should report a better error here if the default - // expression references columns, which means it's not a constant for sure. - throw QueryCompilationErrors.notConstantDefaultValueError( - tblName, colName, default.originalSQL) - } else if (isForV1) { - // When converting to v1 commands, the plan is not fully resolved and we can't do a complete - // analysis check. There is no "rest of CheckAnalysis" to report better errors and we must - // fail here. This is temporary and we can remove it when using v2 commands by default. - throw QueryCompilationErrors.notConstantDefaultValueError( - tblName, colName, default.originalSQL) - } - } - - /** - * Applies constant folding for DDL commands. This is used when converting v2 commands to v1 - * commands, where we don't have a chance to go through the optimizer but the default value - * framework requires a literal value of the default value expression. - */ - def contantFoldDDLCommand[T <: LogicalPlan](cmd: T): T = { - ConstantFolding(cmd).asInstanceOf[T] - } - /** * This is an Analyzer for processing default column values using built-in functions only. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index b2b805ce16bb2..0df10a3012539 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -25,10 +25,12 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.analysis.{AsOfTimestamp, AsOfVersion, NamedRelation, NoSuchDatabaseException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchTableException, TimeTravelSpec} import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{SerdeInfo, TableSpec} +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.connector.catalog.functions.UnboundFunction +import org.apache.spark.sql.connector.expressions.LiteralValue import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.types.{ArrayType, MapType, Metadata, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, MapType, Metadata, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils @@ -133,7 +135,9 @@ private[sql] object CatalogV2Util { */ def applySchemaChanges( schema: StructType, - changes: Seq[TableChange]): StructType = { + changes: Seq[TableChange], + tableProvider: Option[String], + statementType: String): StructType = { changes.foldLeft(schema) { (schema, change) => change match { case add: AddColumn => @@ -143,7 +147,7 @@ private[sql] object CatalogV2Util { val fieldWithDefault: StructField = encodeDefaultValue(add.defaultValue(), field) val fieldWithComment: StructField = Option(add.comment).map(fieldWithDefault.withComment).getOrElse(fieldWithDefault) - addField(schema, fieldWithComment, add.position()) + addField(schema, fieldWithComment, add.position(), tableProvider, statementType, true) case names => replace(schema, names.init, parent => parent.dataType match { case parentType: StructType => @@ -153,7 +157,8 @@ private[sql] object CatalogV2Util { Option(add.comment).map(fieldWithDefault.withComment) .getOrElse(fieldWithDefault) Some(parent.copy(dataType = - addField(parentType, fieldWithComment, add.position()))) + addField(parentType, fieldWithComment, add.position(), tableProvider, + statementType, true))) case _ => throw new IllegalArgumentException(s"Not a struct: ${names.init.last}") }) @@ -183,7 +188,8 @@ private[sql] object CatalogV2Util { throw new IllegalArgumentException("Field not found: " + name) } val withFieldRemoved = StructType(struct.fields.filter(_ != oldField)) - addField(withFieldRemoved, oldField, update.position()) + addField(withFieldRemoved, oldField, update.position(), tableProvider, statementType, + false) } update.fieldNames() match { @@ -203,10 +209,8 @@ private[sql] object CatalogV2Util { // The new DEFAULT value string will be non-empty for any DDL commands that set the // default value, such as "ALTER TABLE t ALTER COLUMN c SET DEFAULT ..." (this is // enforced by the parser). On the other hand, commands that drop the default value such - // as "ALTER TABLE t ALTER COLUMN c DROP DEFAULT" will set this string to null. - // Note: we should only update the "current default", as the previous "exist default" - // should still be applied when reading existing data files without the column. - if (update.newDefaultValue() != null) { + // as "ALTER TABLE t ALTER COLUMN c DROP DEFAULT" will set this string to empty. + if (update.newDefaultValue().nonEmpty) { Some(field.withCurrentDefaultValue(update.newDefaultValue())) } else { Some(field.clearCurrentDefaultValue) @@ -225,8 +229,11 @@ private[sql] object CatalogV2Util { private def addField( schema: StructType, field: StructField, - position: ColumnPosition): StructType = { - if (position == null) { + position: ColumnPosition, + tableProvider: Option[String], + statementType: String, + addNewColumnToExistingTable: Boolean): StructType = { + val newSchema: StructType = if (position == null) { schema.add(field) } else if (position.isInstanceOf[First]) { StructType(field +: schema.fields) @@ -239,6 +246,8 @@ private[sql] object CatalogV2Util { val (before, after) = schema.fields.splitAt(fieldIndex + 1) StructType(before ++ (field +: after)) } + constantFoldCurrentDefaultsToExistDefaults( + newSchema, tableProvider, statementType, addNewColumnToExistingTable) } private def replace( @@ -427,7 +436,7 @@ private[sql] object CatalogV2Util { StructType(columns.map(v2ColumnToStructField)) } - def v2ColumnToStructField(col: Column): StructField = { + private def v2ColumnToStructField(col: Column): StructField = { val metadata = Option(col.metadataInJSON()).map(Metadata.fromJson).getOrElse(Metadata.empty) var f = StructField(col.name(), col.dataType(), col.nullable(), metadata) Option(col.comment()).foreach { comment => @@ -456,6 +465,39 @@ private[sql] object CatalogV2Util { } def structTypeToV2Columns(schema: StructType): Array[Column] = { - schema.toColumns.map(_.toV2Column) + schema.fields.map(structFieldToV2Column) + } + + private def structFieldToV2Column(f: StructField): Column = { + def createV2Column(defaultValue: ColumnDefaultValue, metadata: Metadata): Column = { + val metadataJSON = if (metadata == Metadata.empty) { + null + } else { + metadata.json + } + Column.create( + f.name, f.dataType, f.nullable, f.getComment().orNull, defaultValue, metadataJSON) + } + if (f.getCurrentDefaultValue().isDefined && f.getExistenceDefaultValue().isDefined) { + val e = analyze(f, EXISTS_DEFAULT_COLUMN_METADATA_KEY) + assert(e.resolved && e.foldable, + "exist default must be simple SQL string that is resolved and foldable, " + + "but got: " + f.getExistenceDefaultValue().get) + val defaultValue = new ColumnDefaultValue( + f.getCurrentDefaultValue().get, LiteralValue(e.eval(), f.dataType)) + val cleanedMetadata = new MetadataBuilder() + .withMetadata(f.metadata) + .remove("comment") + .remove(CURRENT_DEFAULT_COLUMN_METADATA_KEY) + .remove(EXISTS_DEFAULT_COLUMN_METADATA_KEY) + .build() + createV2Column(defaultValue, cleanedMetadata) + } else { + val cleanedMetadata = new MetadataBuilder() + .withMetadata(f.metadata) + .remove("comment") + .build() + createV2Column(null, cleanedMetadata) + } } } 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 9ed00282ba600..74dd5879061fe 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 @@ -3148,58 +3148,28 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { "defaultValue" -> defaultValue)) } - def defaultReferencesNotAllowedInDataSource(tableName: String): Throwable = { - new AnalysisException( - errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", - messageParameters = Map( - "tableName" -> tableName, - "operation" -> "column default value")) - } - - def addNewDefaultColumnToExistingTableNotAllowed(dataSource: String): Throwable = { + def defaultReferencesNotAllowedInDataSource( + statementType: String, dataSource: String): Throwable = { new AnalysisException( errorClass = "_LEGACY_ERROR_TEMP_1345", - messageParameters = Map("dataSource" -> dataSource)) - } - - def hasSubqueryDefaultValueError( - tableName: String, - colName: Seq[String], - defaultValue: String): Throwable = { - new AnalysisException( - errorClass = "INVALID_COLUMN_DEFAULT_VALUE.SUBQUERY_EXPRESSION", messageParameters = Map( - "tableName" -> tableName, - "colName" -> toSQLId(colName), - "defaultValue" -> defaultValue)) + "statementType" -> statementType, + "dataSource" -> dataSource)) } - def notConstantDefaultValueError( - tableName: String, - colName: Seq[String], - defaultValue: String): Throwable = { + def addNewDefaultColumnToExistingTableNotAllowed( + statementType: String, dataSource: String): Throwable = { new AnalysisException( - errorClass = "INVALID_COLUMN_DEFAULT_VALUE.NOT_CONSTANT", + errorClass = "_LEGACY_ERROR_TEMP_1346", messageParameters = Map( - "tableName" -> tableName, - "colName" -> toSQLId(colName), - "defaultValue" -> defaultValue)) + "statementType" -> statementType, + "dataSource" -> dataSource)) } - def incompatibleTypeDefaultValueError( - tableName: String, - colName: Seq[String], - colType: DataType, - defaultExpression: Expression, - defaultValue: String): Throwable = { + def defaultValuesMayNotContainSubQueryExpressions(): Throwable = { new AnalysisException( - errorClass = "INVALID_COLUMN_DEFAULT_VALUE.INCOMPATIBLE_DATA_TYPE", - messageParameters = Map( - "tableName" -> tableName, - "colName" -> toSQLId(colName), - "defaultValue" -> defaultValue, - "defaultValueType" -> toSQLType(defaultExpression.dataType), - "colType" -> toSQLType(colType))) + errorClass = "_LEGACY_ERROR_TEMP_1347", + messageParameters = Map.empty) } def nullableColumnOrFieldError(name: Seq[String]): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 4c13d9db546e1..a9c3829a7211d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -25,12 +25,11 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.analysis.Resolver -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering, Literal} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} -import org.apache.spark.sql.catalyst.plans.logical.{ColumnDefinition, DefaultValueExpression} import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.catalyst.types.{PhysicalDataType, PhysicalStructType} -import org.apache.spark.sql.catalyst.util.{truncatedString, ResolveDefaultColumns, StringUtils} +import org.apache.spark.sql.catalyst.util.{truncatedString, StringUtils} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} @@ -395,41 +394,6 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru protected[sql] def toAttributes: Seq[AttributeReference] = map(field => field.toAttribute) - private[sql] def toColumns: Array[ColumnDefinition] = fields.map { f => - def createColumn( - defaultValue: Option[DefaultValueExpression], - metadata: Metadata): ColumnDefinition = { - val metadataJSON = if (metadata == Metadata.empty) { - None - } else { - Some(metadata.json) - } - ColumnDefinition(f.name, f.dataType, f.nullable, f.getComment(), defaultValue, metadataJSON) - } - if (f.getCurrentDefaultValue().isDefined && f.getExistenceDefaultValue().isDefined) { - val e = ResolveDefaultColumns.analyze( - f, ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY) - assert(e.resolved && e.foldable, - "exist default must be simple SQL string that is resolved and foldable, " + - "but got: " + f.getExistenceDefaultValue().get) - val defaultValue = Some(DefaultValueExpression( - Literal(e.eval(), f.dataType), f.getCurrentDefaultValue().get)) - val cleanedMetadata = new MetadataBuilder() - .withMetadata(f.metadata) - .remove("comment") - .remove(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY) - .remove(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY) - .build() - createColumn(defaultValue, cleanedMetadata) - } else { - val cleanedMetadata = new MetadataBuilder() - .withMetadata(f.metadata) - .remove("comment") - .build() - createColumn(None, cleanedMetadata) - } - } - def treeString: String = treeString(Int.MaxValue) def treeString(maxDepth: Int): String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index fe5cc543e8495..32eb884942763 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, NoSuchDatabaseException, NoSuchFunctionException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -1047,11 +1048,37 @@ abstract class CatalogTestUtils { identifier = TableIdentifier(name, database), tableType = CatalogTableType.EXTERNAL, storage = storageFormat.copy(locationUri = Some(Utils.createTempDir().toURI)), - schema = new StructType() - .add("col1", "int") - .add("col2", "string") - .add("a", "int") - .add("b", "string"), + schema = if (defaultColumns) { + new StructType() + .add("col1", "int") + .add("col2", "string") + .add("a", IntegerType, nullable = true, + new MetadataBuilder().putString( + ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "42") + .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "41").build()) + .add("b", StringType, nullable = false, + new MetadataBuilder().putString( + ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "\"abc\"").build()) + // The default value fails to parse. + .add("c", LongType, nullable = false, + new MetadataBuilder().putString( + ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "_@#$%").build()) + // The default value fails to resolve. + .add("d", LongType, nullable = false, + new MetadataBuilder().putString( + ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, + "(select min(x) from badtable)").build()) + // The default value fails to coerce to the required type. + .add("e", BooleanType, nullable = false, + new MetadataBuilder().putString( + ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "41 + 1").build()) + } else { + new StructType() + .add("col1", "int") + .add("col2", "string") + .add("a", "int") + .add("b", "string") + }, provider = Some(defaultProvider), partitionColumnNames = Seq("a", "b"), bucketSpec = Some(BucketSpec(4, Seq("col1"), Nil))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 2593fdcee5a25..a7254865c1ed1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{LeafCommand, LogicalPlan, Project, Range, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER @@ -121,6 +122,72 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { assert(e.contains(s"`$name` is not a valid name for tables/databases.")) } + test("create table with default columns") { + def test: Unit = withBasicCatalog { catalog => + assert(catalog.externalCatalog.listTables("db1").isEmpty) + assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) + catalog.createTable(newTable( + "tbl3", Some("db1"), defaultColumns = true), ignoreIfExists = false) + catalog.createTable(newTable( + "tbl3", Some("db2"), defaultColumns = true), ignoreIfExists = false) + assert(catalog.externalCatalog.listTables("db1").toSet == Set("tbl3")) + assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2", "tbl3")) + // Inspect the default column values. + val db1tbl3 = catalog.externalCatalog.getTable("db1", "tbl3") + val currentDefault = ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY + + def findField(name: String, schema: StructType): StructField = + schema.fields.filter(_.name == name).head + val columnA: StructField = findField("a", db1tbl3.schema) + val columnB: StructField = findField("b", db1tbl3.schema) + val columnC: StructField = findField("c", db1tbl3.schema) + val columnD: StructField = findField("d", db1tbl3.schema) + val columnE: StructField = findField("e", db1tbl3.schema) + + val defaultValueColumnA: String = columnA.metadata.getString(currentDefault) + val defaultValueColumnB: String = columnB.metadata.getString(currentDefault) + val defaultValueColumnC: String = columnC.metadata.getString(currentDefault) + val defaultValueColumnD: String = columnD.metadata.getString(currentDefault) + val defaultValueColumnE: String = columnE.metadata.getString(currentDefault) + + assert(defaultValueColumnA == "42") + assert(defaultValueColumnB == "\"abc\"") + assert(defaultValueColumnC == "_@#$%") + assert(defaultValueColumnD == "(select min(x) from badtable)") + assert(defaultValueColumnE == "41 + 1") + + // Analyze the default column values. + val statementType = "CREATE TABLE" + assert(ResolveDefaultColumns.analyze(columnA, statementType).sql == "42") + assert(ResolveDefaultColumns + .analyze(columnA, statementType, ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY) + .sql == "41") + assert(ResolveDefaultColumns.analyze(columnB, statementType).sql == "'abc'") + assert(intercept[AnalysisException] { + ResolveDefaultColumns.analyze(columnC, statementType) + }.getMessage.contains("fails to parse as a valid expression")) + assert(intercept[AnalysisException] { + ResolveDefaultColumns.analyze(columnD, statementType) + }.getMessage.contains("subquery expressions are not allowed in DEFAULT values")) + assert(intercept[AnalysisException] { + ResolveDefaultColumns.analyze(columnE, statementType) + }.getMessage.contains("statement provided a value of incompatible type")) + + // Make sure that constant-folding default values does not take place when the feature is + // disabled. + withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") { + val result: StructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( + db1tbl3.schema, db1tbl3.provider, "CREATE TABLE", false) + val columnEWithFeatureDisabled: StructField = findField("e", result) + // No constant-folding has taken place to the EXISTS_DEFAULT metadata. + assert(!columnEWithFeatureDisabled.metadata.contains("EXISTS_DEFAULT")) + } + } + withSQLConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS.key -> "csv,hive,json,orc,parquet") { + test + } + } + test("create databases using invalid names") { withEmptyCatalog { catalog => testInvalidName( 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 d5d0d4860e32c..0efbd75ad93d0 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 @@ -23,11 +23,12 @@ import org.apache.spark.SparkThrowable import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions.{EqualTo, Hex, Literal} import org.apache.spark.sql.catalyst.plans.logical.{TableSpec => LogicalTableSpec, _} +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition.{after, first} import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.connector.expressions.LogicalExpressions.bucket import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{IntegerType, LongType, StringType, TimestampType} +import org.apache.spark.sql.types.{IntegerType, LongType, MetadataBuilder, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class DDLParserSuite extends AnalysisTest { @@ -46,10 +47,9 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab(a INT COMMENT 'test', b STRING NOT NULL) USING parquet" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq( - ColumnDefinition("a", IntegerType, comment = Some("test")), - ColumnDefinition("b", StringType, nullable = false) - )), + Some(new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType, nullable = false)), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -75,7 +75,7 @@ class DDLParserSuite extends AnalysisTest { sql, TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("a", IntegerType), ColumnDefinition("b", StringType))), + Some(new StructType().add("a", IntegerType).add("b", StringType)), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -93,9 +93,9 @@ class DDLParserSuite extends AnalysisTest { "USING parquet PARTITIONED BY (a)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq( - ColumnDefinition("a", IntegerType, comment = Some("test")), - ColumnDefinition("b", StringType))), + Some(new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType)), Seq(IdentityTransform(FieldReference("a"))), Map.empty[String, String], Some("parquet"), @@ -136,11 +136,10 @@ class DDLParserSuite extends AnalysisTest { """.stripMargin val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq( - ColumnDefinition("a", IntegerType), - ColumnDefinition("b", StringType), - ColumnDefinition("ts", TimestampType) - )), + Some(new StructType() + .add("a", IntegerType) + .add("b", StringType) + .add("ts", TimestampType)), Seq( IdentityTransform(FieldReference("a")), BucketTransform(LiteralValue(16, IntegerType), Seq(FieldReference("b"))), @@ -172,7 +171,7 @@ class DDLParserSuite extends AnalysisTest { val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("a", IntegerType), ColumnDefinition("b", StringType))), + Some(new StructType().add("a", IntegerType).add("b", StringType)), List(bucket(5, Array(FieldReference.column("a")), Array(FieldReference.column("b")))), Map.empty[String, String], Some("parquet"), @@ -191,7 +190,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("a", IntegerType), ColumnDefinition("b", StringType))), + Some(new StructType().add("a", IntegerType).add("b", StringType)), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -211,7 +210,7 @@ class DDLParserSuite extends AnalysisTest { " TBLPROPERTIES('test' = 'test')" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("a", IntegerType), ColumnDefinition("b", StringType))), + Some(new StructType().add("a", IntegerType).add("b", StringType)), Seq.empty[Transform], Map("test" -> "test"), Some("parquet"), @@ -228,15 +227,15 @@ class DDLParserSuite extends AnalysisTest { val createSql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" val expectedTableSpec = TableSpec( - Seq("my_tab"), - Some(Seq(ColumnDefinition("a", IntegerType), ColumnDefinition("b", StringType))), - Seq.empty[Transform], - Map.empty[String, String], - Some("parquet"), - Map.empty[String, String], - Some("/tmp/file"), - None, - None) + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + Map.empty[String, String], + Some("parquet"), + Map.empty[String, String], + Some("/tmp/file"), + None, + None) Seq(createSql, replaceSql).foreach { sql => testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } @@ -247,7 +246,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE 1m.2g(a INT) USING parquet" val expectedTableSpec = TableSpec( Seq("1m", "2g"), - Some(Seq(ColumnDefinition("a", IntegerType))), + Some(new StructType().add("a", IntegerType)), Seq.empty[Transform], Map.empty[String, String], Some("parquet"), @@ -265,7 +264,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab (id bigint) PARTITIONED BY (part string)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), + Some(new StructType().add("id", LongType).add("part", StringType)), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -283,7 +282,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab PARTITIONED BY (part string)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("part", StringType))), + Some(new StructType().add("part", StringType)), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -301,7 +300,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = "REPLACE TABLE my_tab (id bigint) USING parquet PARTITIONED BY (part string)" val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), + Some(new StructType().add("id", LongType).add("part", StringType)), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], Some("parquet"), @@ -374,7 +373,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), + Some(new StructType().add("id", LongType).add("part", StringType)), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -399,7 +398,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), + Some(new StructType().add("id", LongType).add("part", StringType)), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -456,7 +455,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), + Some(new StructType().add("id", LongType).add("part", StringType)), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -507,7 +506,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), + Some(new StructType().add("id", LongType).add("part", StringType)), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -530,7 +529,7 @@ class DDLParserSuite extends AnalysisTest { val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") val expectedTableSpec = TableSpec( Seq("my_tab"), - Some(Seq(ColumnDefinition("id", LongType), ColumnDefinition("part", StringType))), + Some(new StructType().add("id", LongType).add("part", StringType)), Seq(IdentityTransform(FieldReference("part"))), Map.empty[String, String], None, @@ -873,7 +872,7 @@ class DDLParserSuite extends AnalysisTest { sql, TableSpec( Seq("table_name"), - Some(Seq()), + Some(new StructType), Seq.empty[Transform], Map.empty[String, String], Some("json"), @@ -1036,7 +1035,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMN x int"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType(RootTableSchema, ColumnDefinition("x", IntegerType)) + Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None) ))) } @@ -1045,8 +1044,8 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS x int, y string"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(RootTableSchema, ColumnDefinition("x", IntegerType)), - QualifiedColType(RootTableSchema, ColumnDefinition("y", StringType)) + Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None), + QualifiedColType(None, "y", StringType, true, None, None, None) ))) } @@ -1055,7 +1054,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS x int"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(RootTableSchema, ColumnDefinition("x", IntegerType)) + Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None) ))) } @@ -1064,7 +1063,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS (x int)"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType(RootTableSchema, ColumnDefinition("x", IntegerType)) + Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None) ))) } @@ -1073,8 +1072,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMNS (x int COMMENT 'doc')"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", None), - Seq(QualifiedColType( - RootTableSchema, ColumnDefinition("x", IntegerType, comment = Some("doc"))) + Seq(QualifiedColType(None, "x", IntegerType, true, Some("doc"), None, None) ))) } @@ -1083,8 +1081,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMN x int NOT NULL"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType( - RootTableSchema, ColumnDefinition("x", IntegerType, nullable = false)) + Seq(QualifiedColType(None, "x", IntegerType, false, None, None, None) ))) } @@ -1093,8 +1090,7 @@ class DDLParserSuite extends AnalysisTest { parsePlan("ALTER TABLE table_name ADD COLUMN x int COMMENT 'doc'"), AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType( - RootTableSchema, ColumnDefinition("x", IntegerType, comment = Some("doc"))) + Seq(QualifiedColType(None, "x", IntegerType, true, Some("doc"), None, None) ))) } @@ -1104,9 +1100,13 @@ class DDLParserSuite extends AnalysisTest { AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq(QualifiedColType( - RootTableSchema, - ColumnDefinition("x", IntegerType), - Some(UnresolvedFieldPosition(first()))) + None, + "x", + IntegerType, + true, + None, + Some(UnresolvedFieldPosition(first())), + None) ))) comparePlans( @@ -1114,9 +1114,13 @@ class DDLParserSuite extends AnalysisTest { AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq(QualifiedColType( - RootTableSchema, - ColumnDefinition("x", IntegerType), - Some(UnresolvedFieldPosition(after("y")))) + None, + "x", + IntegerType, + true, + None, + Some(UnresolvedFieldPosition(after("y"))), + None) ))) } @@ -1126,8 +1130,7 @@ class DDLParserSuite extends AnalysisTest { AddColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq(QualifiedColType( - UnresolvedFieldName(Seq("x", "y")), - ColumnDefinition("z", IntegerType, comment = Some("doc"))) + Some(UnresolvedFieldName(Seq("x", "y"))), "z", IntegerType, true, Some("doc"), None, None) ))) } @@ -1138,12 +1141,21 @@ class DDLParserSuite extends AnalysisTest { UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None), Seq( QualifiedColType( - UnresolvedFieldName(Seq("x", "y")), - ColumnDefinition("z", IntegerType, comment = Some("doc"))), + Some(UnresolvedFieldName(Seq("x", "y"))), + "z", + IntegerType, + true, + Some("doc"), + None, + None), QualifiedColType( - UnresolvedFieldName(Seq("a")), - ColumnDefinition("b", StringType), - Some(UnresolvedFieldPosition(first()))) + Some(UnresolvedFieldName(Seq("a"))), + "b", + StringType, + true, + None, + Some(UnresolvedFieldPosition(first())), + None) ))) } @@ -1381,24 +1393,21 @@ class DDLParserSuite extends AnalysisTest { parsePlan(sql1), ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), - Seq(QualifiedColType(RootTableSchema, ColumnDefinition("x", StringType))))) + Seq(QualifiedColType(None, "x", StringType, true, None, None, None)))) comparePlans( parsePlan(sql2), ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), - Seq(QualifiedColType( - RootTableSchema, ColumnDefinition("x", StringType, comment = Some("x1")))))) + Seq(QualifiedColType(None, "x", StringType, true, Some("x1"), None, None)))) comparePlans( parsePlan(sql3), ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), Seq( - QualifiedColType( - RootTableSchema, ColumnDefinition("x", StringType, comment = Some("x1"))), - QualifiedColType( - RootTableSchema, ColumnDefinition("y", IntegerType)) + QualifiedColType(None, "x", StringType, true, Some("x1"), None, None), + QualifiedColType(None, "y", IntegerType, true, None, None, None) ))) comparePlans( @@ -1406,10 +1415,8 @@ class DDLParserSuite extends AnalysisTest { ReplaceColumns( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", None), Seq( - QualifiedColType( - RootTableSchema, ColumnDefinition("x", StringType, comment = Some("x1"))), - QualifiedColType( - RootTableSchema, ColumnDefinition("y", IntegerType, comment = Some("y1"))) + QualifiedColType(None, "x", StringType, true, Some("x1"), None, None), + QualifiedColType(None, "y", IntegerType, true, Some("y1"), None, None) ))) val sql5 = "ALTER TABLE table_name PARTITION (a='1') REPLACE COLUMNS (x string)" @@ -2410,7 +2417,7 @@ class DDLParserSuite extends AnalysisTest { private case class TableSpec( name: Seq[String], - columns: Option[Seq[ColumnDefinition]], + schema: Option[StructType], partitioning: Seq[Transform], properties: Map[String, String], provider: Option[String], @@ -2426,7 +2433,7 @@ class DDLParserSuite extends AnalysisTest { case create: CreateTable => TableSpec( create.name.asInstanceOf[UnresolvedIdentifier].nameParts, - Some(create.columns), + Some(create.tableSchema), create.partitioning, create.tableSpec.properties, create.tableSpec.provider, @@ -2438,7 +2445,7 @@ class DDLParserSuite extends AnalysisTest { case replace: ReplaceTable => TableSpec( replace.name.asInstanceOf[UnresolvedIdentifier].nameParts, - Some(replace.columns), + Some(replace.tableSchema), replace.partitioning, replace.tableSpec.properties, replace.tableSpec.provider, @@ -2449,7 +2456,7 @@ class DDLParserSuite extends AnalysisTest { case ctas: CreateTableAsSelect => TableSpec( ctas.name.asInstanceOf[UnresolvedIdentifier].nameParts, - if (ctas.query.resolved) Some(ctas.columns) else None, + Some(ctas.query).filter(_.resolved).map(_.schema), ctas.partitioning, ctas.tableSpec.properties, ctas.tableSpec.provider, @@ -2461,7 +2468,7 @@ class DDLParserSuite extends AnalysisTest { case rtas: ReplaceTableAsSelect => TableSpec( rtas.name.asInstanceOf[UnresolvedIdentifier].nameParts, - if (rtas.query.resolved) Some(rtas.columns) else None, + Some(rtas.query).filter(_.resolved).map(_.schema), rtas.partitioning, rtas.tableSpec.properties, rtas.tableSpec.provider, @@ -2498,7 +2505,7 @@ class DDLParserSuite extends AnalysisTest { val sql = "CREATE TABLE 1m.2g(a INT)" val expectedTableSpec = TableSpec( Seq("1m", "2g"), - Some(Seq(ColumnDefinition("a", IntegerType))), + Some(new StructType().add("a", IntegerType)), Seq.empty[Transform], Map.empty[String, String], None, @@ -2544,40 +2551,34 @@ class DDLParserSuite extends AnalysisTest { test("SPARK-38335: Implement parser support for DEFAULT values for columns in tables") { // These CREATE/REPLACE TABLE statements should parse successfully. - val columnsWithDefaultValue = Seq( - ColumnDefinition("a", IntegerType), - ColumnDefinition( - "b", - StringType, - nullable = false, - defaultValue = Some(DefaultValueExpression(Literal("abc"), "'abc'"))) - ) + val schemaWithDefaultColumn = new StructType() + .add("a", IntegerType, true) + .add("b", StringType, false, + new MetadataBuilder() + .putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "\"abc\"") + .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "\"abc\"").build()) val createTableResult = - CreateTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithDefaultValue, + CreateTable(UnresolvedIdentifier(Seq("my_tab")), schemaWithDefaultColumn, Seq.empty[Transform], LogicalTableSpec(Map.empty[String, String], Some("parquet"), Map.empty[String, String], None, None, None, false), false) // Parse the CREATE TABLE statement twice, swapping the order of the NOT NULL and DEFAULT // options, to make sure that the parser accepts any ordering of these options. comparePlans(parsePlan( - "CREATE TABLE my_tab(a INT, b STRING NOT NULL DEFAULT 'abc') USING parquet"), + "CREATE TABLE my_tab(a INT, b STRING NOT NULL DEFAULT \"abc\") USING parquet"), createTableResult) comparePlans(parsePlan( - "CREATE TABLE my_tab(a INT, b STRING DEFAULT 'abc' NOT NULL) USING parquet"), + "CREATE TABLE my_tab(a INT, b STRING DEFAULT \"abc\" NOT NULL) USING parquet"), createTableResult) comparePlans(parsePlan("REPLACE TABLE my_tab(a INT, " + - "b STRING NOT NULL DEFAULT 'abc') USING parquet"), - ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithDefaultValue, + "b STRING NOT NULL DEFAULT \"abc\") USING parquet"), + ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), schemaWithDefaultColumn, Seq.empty[Transform], LogicalTableSpec(Map.empty[String, String], Some("parquet"), Map.empty[String, String], None, None, None, false), false)) // These ALTER TABLE statements should parse successfully. comparePlans( parsePlan("ALTER TABLE t1 ADD COLUMN x int NOT NULL DEFAULT 42"), AddColumns(UnresolvedTable(Seq("t1"), "ALTER TABLE ... ADD COLUMN", None), - Seq(QualifiedColType(RootTableSchema, ColumnDefinition( - "x", - IntegerType, - nullable = false, - defaultValue = Some(DefaultValueExpression(Literal(42), "42"))))))) + Seq(QualifiedColType(None, "x", IntegerType, false, None, None, Some("42"))))) comparePlans( parsePlan("ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT 42"), AlterColumn( @@ -2587,7 +2588,7 @@ class DDLParserSuite extends AnalysisTest { None, None, None, - Some(DefaultValueExpression(Literal(42), "42")))) + Some("42"))) // It is possible to pass an empty string default value using quotes. comparePlans( parsePlan("ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT ''"), @@ -2598,7 +2599,7 @@ class DDLParserSuite extends AnalysisTest { None, None, None, - Some(DefaultValueExpression(Literal(""), "''")))) + Some("''"))) // It is not possible to pass an empty string default value without using quotes. // This results in a parsing error. val sql1 = "ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT " @@ -2623,7 +2624,7 @@ class DDLParserSuite extends AnalysisTest { None, None, None, - Some(DropDefaultColumnValue))) + Some(""))) // Make sure that the parser returns an exception when the feature is disabled. withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") { val sql = "CREATE TABLE my_tab(a INT, b STRING NOT NULL DEFAULT \"abc\") USING parquet" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala index 302df6dd11edc..50bea2b8d2f27 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala @@ -130,7 +130,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { override def alterTable(ident: Identifier, changes: TableChange*): Table = { val table = loadTable(ident).asInstanceOf[InMemoryTable] val properties = CatalogV2Util.applyPropertiesChanges(table.properties, changes) - val schema = CatalogV2Util.applySchemaChanges(table.schema, changes) + val schema = CatalogV2Util.applySchemaChanges(table.schema, changes, None, "ALTER TABLE") // fail if the last column in the schema was dropped if (schema.fields.isEmpty) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala index e6afd852af1aa..d9eb0892d1389 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala @@ -471,22 +471,34 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { assert(source1.existenceDefaultValues(1) == UTF8String.fromString("abc")) assert(source1.existenceDefaultValues(2) == null) + // Positive test: StructType.defaultValues works because the existence default value parses and + // resolves successfully, then evaluates to a non-literal expression: this is constant-folded at + // reference time. + val source2 = StructType( + Array(StructField("c1", IntegerType, true, + new MetadataBuilder() + .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "1 + 1") + .putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "1 + 1") + .build()))) + val error = "fails to parse as a valid literal value" + assert(source2.existenceDefaultValues.size == 1) + assert(source2.existenceDefaultValues(0) == 2) + // Negative test: StructType.defaultValues fails because the existence default value fails to // parse. - val error = "fails to parse as a valid literal value" - val source2 = StructType(Array( + val source3 = StructType(Array( StructField("c1", IntegerType, true, new MetadataBuilder() .putString(ResolveDefaultColumns.EXISTS_DEFAULT_COLUMN_METADATA_KEY, "invalid") .putString(ResolveDefaultColumns.CURRENT_DEFAULT_COLUMN_METADATA_KEY, "invalid") .build()))) assert(intercept[AnalysisException] { - source2.existenceDefaultValues + source3.existenceDefaultValues }.getMessage.contains(error)) // Negative test: StructType.defaultValues fails because the existence default value fails to // resolve. - val source3 = StructType(Array( + val source4 = StructType(Array( StructField("c1", IntegerType, true, new MetadataBuilder() .putString( @@ -497,7 +509,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { "(SELECT 'abc' FROM missingtable)") .build()))) assert(intercept[AnalysisException] { - source3.existenceDefaultValues + source4.existenceDefaultValues }.getMessage.contains(error)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala index eb1785c877763..3f9eb5c8084e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ReplaceCharWithVarchar.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, ColumnDefinition, CreateTable, LogicalPlan, ReplaceColumns, ReplaceTable} +import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, CreateTable, LogicalPlan, ReplaceColumns, ReplaceTable} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableChangeColumnCommand, CreateDataSourceTableCommand, CreateTableCommand} @@ -32,18 +32,18 @@ object ReplaceCharWithVarchar extends Rule[LogicalPlan] { plan.resolveOperators { // V2 commands case cmd: CreateTable => - cmd.copy(columns = cmd.columns.map(replaceCharWithVarcharInColumn)) + cmd.copy(tableSchema = replaceCharWithVarcharInSchema(cmd.tableSchema)) case cmd: ReplaceTable => - cmd.copy(columns = cmd.columns.map(replaceCharWithVarcharInColumn)) + cmd.copy(tableSchema = replaceCharWithVarcharInSchema(cmd.tableSchema)) case cmd: AddColumns => cmd.copy(columnsToAdd = cmd.columnsToAdd.map { col => - col.copy(column = replaceCharWithVarcharInColumn(col.column)) + col.copy(dataType = CharVarcharUtils.replaceCharWithVarchar(col.dataType)) }) case cmd: AlterColumn => cmd.copy(dataType = cmd.dataType.map(CharVarcharUtils.replaceCharWithVarchar)) case cmd: ReplaceColumns => cmd.copy(columnsToAdd = cmd.columnsToAdd.map { col => - col.copy(column = replaceCharWithVarcharInColumn(col.column)) + col.copy(dataType = CharVarcharUtils.replaceCharWithVarchar(col.dataType)) }) // V1 commands @@ -61,11 +61,11 @@ object ReplaceCharWithVarchar extends Rule[LogicalPlan] { } } - private def replaceCharWithVarcharInColumn(col: ColumnDefinition): ColumnDefinition = { - col.copy(dataType = CharVarcharUtils.replaceCharWithVarchar(col.dataType)) + private def replaceCharWithVarcharInSchema(schema: StructType): StructType = { + CharVarcharUtils.replaceCharWithVarchar(schema).asInstanceOf[StructType] } private def replaceCharWithVarcharInTableMeta(tbl: CatalogTable): CatalogTable = { - tbl.copy(schema = CharVarcharUtils.replaceCharWithVarchar(tbl.schema).asInstanceOf[StructType]) + tbl.copy(schema = replaceCharWithVarcharInSchema(tbl.schema)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 3efb5496242c2..7b2d5015840cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -25,7 +25,8 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, ResolveDefaultColumns => DefaultColumnUtil} +import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, ResolveDefaultColumns => DefaultCols} +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, LookupCatalog, SupportsNamespaces, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} @@ -49,24 +50,17 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case a @ AddColumns(ResolvedV1TableIdentifier(ident), colsToAdd) if a.resolved => - colsToAdd.foreach { c => + case AddColumns(ResolvedV1TableIdentifier(ident), cols) => + cols.foreach { c => if (c.name.length > 1) { throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( Seq(ident.catalog.get, ident.database.get, ident.table), "ADD COLUMN with qualified column") } - if (!c.column.nullable) { + if (!c.nullable) { throw QueryCompilationErrors.addColumnWithV1TableCannotSpecifyNotNullError } } - // Check default values before converting to v1 command. - DefaultColumnUtil.checkDefaultValuesInPlan(a, isForV1 = true) - val cols = if (colsToAdd.exists(_.column.defaultValue.isDefined)) { - DefaultColumnUtil.contantFoldDDLCommand(a).columnsToAdd.map(_.column) - } else { - colsToAdd.map(_.column) - } AlterTableAddColumnsCommand(ident, cols.map(convertToStructField)) case ReplaceColumns(ResolvedV1TableIdentifier(ident), _) => @@ -75,7 +69,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) "REPLACE COLUMNS") case a @ AlterColumn(ResolvedTable(catalog, ident, table: V1Table, _), _, _, _, _, _, _) - if isSessionCatalog(catalog) && a.resolved => + if isSessionCatalog(catalog) => if (a.column.name.length > 1) { throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( Seq(catalog.name, ident.namespace()(0), ident.name), @@ -101,19 +95,14 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) quoteIfNeeded(colName), table) } } + // Add the current default column value string (if any) to the column metadata. + a.setDefaultExpression.map { c => builder.putString(CURRENT_DEFAULT_COLUMN_METADATA_KEY, c) } val newColumn = StructField( colName, dataType, nullable = true, builder.build()) - // Check default values before converting to v1 command. - DefaultColumnUtil.checkDefaultValuesInPlan(a, isForV1 = true) - val defaultValue = if (a.defaultExpression.isDefined) { - DefaultColumnUtil.contantFoldDDLCommand(a).defaultExpression - } else { - a.defaultExpression - } - AlterTableChangeColumnCommand(table.catalogTable.identifier, colName, newColumn, defaultValue) + AlterTableChangeColumnCommand(table.catalogTable.identifier, colName, newColumn) case RenameColumn(ResolvedV1TableIdentifier(ident), _, _) => throw QueryCompilationErrors.operationOnlySupportedWithV2TableError( @@ -178,20 +167,12 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) // For CREATE TABLE [AS SELECT], we should use the v1 command if the catalog is resolved to the // session catalog and the table provider is not v2. - case c @ CreateTable(ResolvedV1Identifier(ident), _, _, _, _) if c.resolved => + case c @ CreateTable(ResolvedV1Identifier(ident), _, _, _, _) => val (storageFormat, provider) = getStorageFormatAndProvider( c.tableSpec.provider, c.tableSpec.options, c.tableSpec.location, c.tableSpec.serde, ctas = false) if (!isV2Provider(provider)) { - // Check default values before converting to v1 command. - DefaultColumnUtil.checkDefaultValuesInPlan(c, isForV1 = true) - val cols = if (c.columns.exists(_.defaultValue.isDefined)) { - DefaultColumnUtil.contantFoldDDLCommand(c).columns - } else { - c.columns - } - constructV1TableCmd(None, c.tableSpec, ident, - StructType(cols.map(convertToStructField)), c.partitioning, + constructV1TableCmd(None, c.tableSpec, ident, c.tableSchema, c.partitioning, c.ignoreIfExists, storageFormat, provider) } else { c @@ -618,9 +599,13 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) } } - private def convertToStructField(col: ColumnDefinition): StructField = { - assert(col.resolved) - CatalogV2Util.v2ColumnToStructField(col.toV2Column) + private def convertToStructField(col: QualifiedColType): StructField = { + val builder = new MetadataBuilder + col.comment.foreach(builder.putString("comment", _)) + col.default.map { + value: String => builder.putString(DefaultCols.CURRENT_DEFAULT_COLUMN_METADATA_KEY, value) + } + StructField(col.name.head, col.dataType, nullable = true, builder.build()) } private def isV2Provider(provider: String): Boolean = { 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 491664ffc4887..fdd4f10c793a0 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 @@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} -import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.sql.{Row, SparkSession} @@ -36,8 +35,9 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.plans.logical.{DefaultValueExpression, DropDefaultColumnValue, LogicalPlan} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.SupportsNamespaces._ @@ -365,8 +365,7 @@ case class AlterTableUnsetPropertiesCommand( case class AlterTableChangeColumnCommand( tableName: TableIdentifier, columnName: String, - newColumn: StructField, - newDefaultValue: Option[Expression] = None) extends LeafRunnableCommand { + newColumn: StructField) extends LeafRunnableCommand { // TODO: support change column name/dataType/metadata/position. override def run(sparkSession: SparkSession): Seq[Row] = { @@ -386,15 +385,20 @@ case class AlterTableChangeColumnCommand( val newDataSchema = table.dataSchema.fields.map { field => if (field.name == originColumn.name) { // Create a new column from the origin column with the new comment. - val withNewComment: StructField = addComment(field, newColumn.getComment) + val withNewComment: StructField = + addComment(field, newColumn.getComment) // Create a new column from the origin column with the new current default value. - if (newDefaultValue.isDefined) { - newDefaultValue.get match { - case DropDefaultColumnValue => withNewComment.clearCurrentDefaultValue() - case d: DefaultValueExpression => - withNewComment.withCurrentDefaultValue(d.originalSQL) - case other => throw SparkException.internalError( - "Unexpected expression in AlterTableChangeColumnCommand.defaultExpression: " + other) + if (newColumn.getCurrentDefaultValue().isDefined) { + if (newColumn.getCurrentDefaultValue().get.nonEmpty) { + val result: StructField = + addCurrentDefaultValue(withNewComment, newColumn.getCurrentDefaultValue()) + // Check that the proposed default value parses and analyzes correctly, and that the + // type of the resulting expression is equivalent or coercible to the destination column + // type. + ResolveDefaultColumns.analyze(result, "ALTER TABLE ALTER COLUMN") + result + } else { + withNewComment.clearCurrentDefaultValue() } } else { withNewComment @@ -421,6 +425,11 @@ case class AlterTableChangeColumnCommand( private def addComment(column: StructField, comment: Option[String]): StructField = comment.map(column.withComment).getOrElse(column) + // Add the current default value to a column, if default value is empty, return the original + // column. + private def addCurrentDefaultValue(column: StructField, value: Option[String]): StructField = + value.map(column.withCurrentDefaultValue).getOrElse(column) + // Compare a [[StructField]] to another, return true if they have the same column // name(by resolver) and dataType. private def columnEqual( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index f47e8ef667227..f6266bcb33f62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -231,17 +231,19 @@ case class AlterTableAddColumnsCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val catalogTable = verifyAlterTableAddColumn(sparkSession.sessionState.conf, catalog, table) + val colsWithProcessedDefaults = + constantFoldCurrentDefaultsToExistDefaults(sparkSession, catalogTable.provider) CommandUtils.uncacheTableOrView(sparkSession, table.quotedString) catalog.refreshTable(table) SchemaUtils.checkColumnNameDuplication( - (colsToAdd ++ catalogTable.schema).map(_.name), + (colsWithProcessedDefaults ++ catalogTable.schema).map(_.name), conf.caseSensitiveAnalysis) - DDLUtils.checkTableColumns(catalogTable, StructType(colsToAdd)) + DDLUtils.checkTableColumns(catalogTable, StructType(colsWithProcessedDefaults)) val existingSchema = CharVarcharUtils.getRawSchema(catalogTable.dataSchema) - catalog.alterTableDataSchema(table, StructType(existingSchema ++ colsToAdd)) + catalog.alterTableDataSchema(table, StructType(existingSchema ++ colsWithProcessedDefaults)) Seq.empty[Row] } @@ -277,6 +279,24 @@ case class AlterTableAddColumnsCommand( } catalogTable } + + /** + * ALTER TABLE ADD COLUMNS commands may optionally specify a DEFAULT expression for any column. + * In that case, this method evaluates its originally specified value and then stores the result + * in a separate column metadata entry, then returns the updated column definitions. + */ + private def constantFoldCurrentDefaultsToExistDefaults( + sparkSession: SparkSession, tableProvider: Option[String]): Seq[StructField] = { + colsToAdd.map { col: StructField => + if (col.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY)) { + val foldedStructType = ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( + StructType(Array(col)), tableProvider, "ALTER TABLE ADD COLUMNS", true) + foldedStructType.fields(0) + } else { + col + } + } + } } 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 b8025c3966672..8b985e82963e8 100644 --- 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 @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 -import org.apache.spark.sql.catalyst.util.V2ExpressionBuilder +import org.apache.spark.sql.catalyst.util.{ResolveDefaultColumns, V2ExpressionBuilder} import org.apache.spark.sql.connector.catalog.SupportsRead import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.expressions.{Expression => V2Expression, NullOrdering, SortDirection, SortOrder => V2SortOrder, SortValue} @@ -61,7 +61,7 @@ import org.apache.spark.unsafe.types.UTF8String * Note that, this rule must be run after `PreprocessTableCreation` and * `PreprocessTableInsertion`. */ -object DataSourceAnalysis extends Rule[LogicalPlan] { +case class DataSourceAnalysis(analyzer: Analyzer) extends Rule[LogicalPlan] { def resolver: Resolver = conf.resolver @@ -133,7 +133,11 @@ object DataSourceAnalysis extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case CreateTable(tableDesc, mode, None) if DDLUtils.isDatasourceTable(tableDesc) => - CreateDataSourceTableCommand(tableDesc, ignoreIfExists = mode == SaveMode.Ignore) + val newSchema: StructType = + ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( + tableDesc.schema, tableDesc.provider, "CREATE TABLE", false) + val newTableDesc = tableDesc.copy(schema = newSchema) + CreateDataSourceTableCommand(newTableDesc, ignoreIfExists = mode == SaveMode.Ignore) case CreateTable(tableDesc, mode, Some(query)) if query.resolved && DDLUtils.isDatasourceTable(tableDesc) => 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 81afa36d7f275..91f8d688dbcda 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 @@ -31,8 +31,9 @@ import org.apache.spark.sql.catalyst.expressions.{And, Attribute, DynamicPruning import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.{toPrettySQL, V2ExpressionBuilder} +import org.apache.spark.sql.catalyst.util.{toPrettySQL, ResolveDefaultColumns, V2ExpressionBuilder} import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsDeleteV2, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, Table, TableCapability, TableCatalog, TruncatableTable} +import org.apache.spark.sql.connector.catalog.CatalogV2Util.structTypeToV2Columns import org.apache.spark.sql.connector.catalog.index.SupportsIndex import org.apache.spark.sql.connector.expressions.{FieldReference, LiteralValue} import org.apache.spark.sql.connector.expressions.filter.{And => V2And, Not => V2Not, Or => V2Or, Predicate} @@ -45,6 +46,7 @@ import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, LogicalRe import org.apache.spark.sql.execution.streaming.continuous.{WriteToContinuousDataSource, WriteToContinuousDataSourceExec} import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.sources.{BaseRelation, TableScan} +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.storage.StorageLevel @@ -171,10 +173,12 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat } WriteToDataSourceV2Exec(writer, invalidateCacheFunc, planLater(query), customMetrics) :: Nil - case CreateTable(ResolvedIdentifier(catalog, ident), columns, partitioning, + case CreateTable(ResolvedIdentifier(catalog, ident), schema, partitioning, tableSpec, ifNotExists) => - val v2Columns = columns.map(_.toV2Column).toArray - CreateTableExec(catalog.asTableCatalog, ident, v2Columns, + val newSchema: StructType = + ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( + schema, tableSpec.provider, "CREATE TABLE", false) + CreateTableExec(catalog.asTableCatalog, ident, structTypeToV2Columns(newSchema), partitioning, qualifyLocInTableSpec(tableSpec), ifNotExists) :: Nil case CreateTableAsSelect(ResolvedIdentifier(catalog, ident), parts, query, tableSpec, @@ -193,8 +197,11 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case RefreshTable(r: ResolvedTable) => RefreshTableExec(r.catalog, r.identifier, recacheTable(r)) :: Nil - case ReplaceTable(ResolvedIdentifier(catalog, ident), columns, parts, tableSpec, orCreate) => - val v2Columns = columns.map(_.toV2Column).toArray + case ReplaceTable(ResolvedIdentifier(catalog, ident), schema, parts, tableSpec, orCreate) => + val newSchema: StructType = + ResolveDefaultColumns.constantFoldCurrentDefaultsToExistDefaults( + schema, tableSpec.provider, "CREATE TABLE", false) + val v2Columns = structTypeToV2Columns(newSchema) catalog match { case staging: StagingTableCatalog => AtomicReplaceTableExec(staging, ident, v2Columns, parts, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 14b41c5c14a75..8b0098f14fedc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -23,9 +23,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.util.quoteIfNeeded +import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, ResolveDefaultColumns} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsMetadataColumns, Table, TableCatalog} -import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.expressions.IdentityTransform case class DescribeTableExec( @@ -38,7 +37,6 @@ case class DescribeTableExec( addPartitioning(rows) if (isExtended) { - addColumnDefaultValue(rows) addMetadataColumns(rows) addTableDetails(rows) } @@ -70,23 +68,17 @@ case class DescribeTableExec( case (key, value) => key + "=" + value }.mkString("[", ",", "]") rows += toCatalystRow("Table Properties", properties, "") - } - private def addSchema(rows: ArrayBuffer[InternalRow]): Unit = { - rows ++= table.columns.map { column => - toCatalystRow( - column.name, column.dataType.simpleString, column.comment()) + // If any columns have default values, append them to the result. + ResolveDefaultColumns.getDescribeMetadata(table.schema).foreach { row => + rows += toCatalystRow(row._1, row._2, row._3) } } - private def addColumnDefaultValue(rows: ArrayBuffer[InternalRow]): Unit = { - if (table.columns().exists(_.defaultValue() != null)) { - rows += emptyRow() - rows += toCatalystRow("# Column Default Values", "", "") - rows ++= table.columns.filter(_.defaultValue() != null).map { column => - toCatalystRow( - column.name, column.dataType.simpleString, column.defaultValue().getSql) - } + private def addSchema(rows: ArrayBuffer[InternalRow]): Unit = { + rows ++= table.schema.map{ column => + toCatalystRow( + column.name, column.dataType.simpleString, column.getComment().orNull) } } @@ -112,11 +104,10 @@ case class DescribeTableExec( rows ++= table.partitioning .map(_.asInstanceOf[IdentityTransform].ref.fieldNames()) .map { fieldNames => - val schema = table.columns().asSchema - val nestedField = schema.findNestedField(fieldNames) + val nestedField = table.schema.findNestedField(fieldNames) assert(nestedField.isDefined, s"Not found the partition column ${fieldNames.map(quoteIfNeeded).mkString(".")} " + - s"in the table schema ${schema.catalogString}.") + s"in the table schema ${table.schema().catalogString}.") nestedField.get }.map { case (path, field) => toCatalystRow( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 2f3c1471e5e6e..461e948b02979 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -159,7 +159,8 @@ class V2SessionCatalog(catalog: SessionCatalog) } val properties = CatalogV2Util.applyPropertiesChanges(catalogTable.properties, changes) - val schema = CatalogV2Util.applySchemaChanges(catalogTable.schema, changes) + val schema = CatalogV2Util.applySchemaChanges( + catalogTable.schema, changes, catalogTable.provider, "ALTER TABLE") val comment = properties.get(TableCatalog.PROP_COMMENT) val owner = properties.getOrElse(TableCatalog.PROP_OWNER, catalogTable.owner) val location = properties.get(TableCatalog.PROP_LOCATION).map(CatalogUtils.stringToURI) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 9dd6516f9fb7e..f17d0c3dd2e51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -195,7 +195,7 @@ abstract class BaseSessionStateBuilder( DetectAmbiguousSelfJoin +: PreprocessTableCreation(session) +: PreprocessTableInsertion +: - DataSourceAnalysis +: + DataSourceAnalysis(this) +: ApplyCharTypePadding +: ReplaceCharWithVarchar +: customPostHocResolutionRules diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 205fc6d51d588..79b0084da23ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -313,11 +313,9 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { val columns = sparkSession.sessionState.executePlan(plan).analyzed match { case ResolvedTable(_, _, table, _) => - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val (partitionColumnNames, bucketSpecOpt) = table.partitioning.toSeq.convertTransforms val bucketColumnNames = bucketSpecOpt.map(_.bucketColumnNames).getOrElse(Nil) - schemaToColumns( - table.columns.asSchema, partitionColumnNames.contains, bucketColumnNames.contains) + schemaToColumns(table.schema(), partitionColumnNames.contains, bucketColumnNames.contains) case ResolvedPersistentView(_, _, schema) => schemaToColumns(schema) @@ -632,7 +630,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { val plan = CreateTable( name = UnresolvedIdentifier(ident), - columns = schema.toColumns, + tableSchema = schema, partitioning = Seq(), tableSpec = tableSpec, ignoreIfExists = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index a6b8b20da4143..f1d2e351ebda1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -299,7 +299,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { false) val cmd = CreateTable( UnresolvedIdentifier(originalMultipartIdentifier), - df.schema.asNullable.toColumns, + df.schema.asNullable, partitioningColumns.getOrElse(Nil).asTransforms.toSeq, tableSpec, ignoreIfExists = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index d0cb8141ad71b..835566238c9c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -110,7 +110,7 @@ class InMemoryTableSessionCatalog extends TestV2SessionCatalogBase[InMemoryTable Option(tables.get(ident)) match { case Some(table) => val properties = CatalogV2Util.applyPropertiesChanges(table.properties, changes) - val schema = CatalogV2Util.applySchemaChanges(table.schema, changes) + val schema = CatalogV2Util.applySchemaChanges(table.schema, changes, None, "ALTER TABLE") // fail if the last column in the schema was dropped if (schema.fields.isEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala index 7e30d70075b7a..44cd4f0f9b31d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.connector -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, CreateTablePartitioningValidationSuite, ResolvedTable, RootTableSchema, TestRelation2, TestTable2, UnresolvedFieldName, UnresolvedFieldPosition, UnresolvedIdentifier} -import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, AlterTableCommand, ColumnDefinition, CreateTableAsSelect, DropColumns, LogicalPlan, QualifiedColType, RenameColumn, ReplaceColumns, ReplaceTableAsSelect, TableSpec} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, CreateTablePartitioningValidationSuite, ResolvedTable, TestRelation2, TestTable2, UnresolvedFieldName, UnresolvedFieldPosition, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, AlterTableCommand, CreateTableAsSelect, DropColumns, LogicalPlan, QualifiedColType, RenameColumn, ReplaceColumns, ReplaceTableAsSelect, TableSpec} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition @@ -152,8 +152,7 @@ class V2CommandsCaseSensitivitySuite AddColumns( table, Seq(QualifiedColType( - UnresolvedFieldName(field.init), - ColumnDefinition(field.last, LongType, true)))), + Some(UnresolvedFieldName(field.init)), field.last, LongType, true, None, None, None))), Seq("Missing field " + field.head) ) } @@ -162,11 +161,15 @@ class V2CommandsCaseSensitivitySuite test("AlterTable: add column resolution - positional") { Seq("ID", "iD").foreach { ref => val alter = AddColumns( - table, - Seq(QualifiedColType( - RootTableSchema, - ColumnDefinition("f", LongType, true), - Some(UnresolvedFieldPosition(ColumnPosition.after(ref)))))) + table, + Seq(QualifiedColType( + None, + "f", + LongType, + true, + None, + Some(UnresolvedFieldPosition(ColumnPosition.after(ref))), + None))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { assertAnalysisErrorClass( @@ -181,17 +184,23 @@ class V2CommandsCaseSensitivitySuite test("AlterTable: add column resolution - column position referencing new column") { val alter = AddColumns( - table, - Seq( - QualifiedColType( - RootTableSchema, - ColumnDefinition("x", LongType, true), - Some(UnresolvedFieldPosition(ColumnPosition.after("id")))), + table, + Seq(QualifiedColType( + None, + "x", + LongType, + true, + None, + Some(UnresolvedFieldPosition(ColumnPosition.after("id"))), + None), QualifiedColType( - RootTableSchema, - ColumnDefinition("y", LongType, true), - Some(UnresolvedFieldPosition(ColumnPosition.after("X")))) - )) + None, + "y", + LongType, + true, + None, + Some(UnresolvedFieldPosition(ColumnPosition.after("X"))), + None))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { assertAnalysisErrorClass( @@ -206,11 +215,15 @@ class V2CommandsCaseSensitivitySuite test("AlterTable: add column resolution - nested positional") { Seq("X", "Y").foreach { ref => val alter = AddColumns( - table, - Seq(QualifiedColType( - UnresolvedFieldName(Seq("point")), - ColumnDefinition("z", LongType, true), - Some(UnresolvedFieldPosition(ColumnPosition.after(ref)))))) + table, + Seq(QualifiedColType( + Some(UnresolvedFieldName(Seq("point"))), + "z", + LongType, + true, + None, + Some(UnresolvedFieldPosition(ColumnPosition.after(ref))), + None))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { assertAnalysisErrorClass( @@ -225,16 +238,23 @@ class V2CommandsCaseSensitivitySuite test("AlterTable: add column resolution - column position referencing new nested column") { val alter = AddColumns( - table, - Seq( - QualifiedColType( - UnresolvedFieldName(Seq("point")), - ColumnDefinition("z", LongType, true)), + table, + Seq(QualifiedColType( + Some(UnresolvedFieldName(Seq("point"))), + "z", + LongType, + true, + None, + None, + None), QualifiedColType( - UnresolvedFieldName(Seq("point")), - ColumnDefinition("zz", LongType, true), - Some(UnresolvedFieldPosition(ColumnPosition.after("Z")))) - )) + Some(UnresolvedFieldName(Seq("point"))), + "zz", + LongType, + true, + None, + Some(UnresolvedFieldPosition(ColumnPosition.after("Z"))), + None))) Seq(true, false).foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { assertAnalysisErrorClass( @@ -250,14 +270,22 @@ class V2CommandsCaseSensitivitySuite assertAnalysisErrorClass( AddColumns( table, - Seq( - QualifiedColType( - UnresolvedFieldName(Seq("point")), - ColumnDefinition("z", LongType, true)), - QualifiedColType( - UnresolvedFieldName(Seq("point")), - ColumnDefinition("Z", LongType, true)) - )), + Seq(QualifiedColType( + Some(UnresolvedFieldName(Seq("point"))), + "z", + LongType, + true, + None, + None, + None), + QualifiedColType( + Some(UnresolvedFieldName(Seq("point"))), + "Z", + LongType, + true, + None, + None, + None))), "COLUMN_ALREADY_EXISTS", Map("columnName" -> toSQLId("point.z")), caseSensitive = false) @@ -268,9 +296,13 @@ class V2CommandsCaseSensitivitySuite AddColumns( table, Seq(QualifiedColType( - RootTableSchema, - ColumnDefinition("ID", LongType, true), - Some(UnresolvedFieldPosition(ColumnPosition.after("id")))))), + None, + "ID", + LongType, + true, + None, + Some(UnresolvedFieldPosition(ColumnPosition.after("id"))), + None))), Seq("Cannot add column, because ID already exists in root"), expectErrorOnCaseSensitive = false) } @@ -342,9 +374,8 @@ class V2CommandsCaseSensitivitySuite assertAnalysisErrorClass( ReplaceColumns( table, - Seq( - QualifiedColType(RootTableSchema, ColumnDefinition("f", LongType, true)), - QualifiedColType(RootTableSchema, ColumnDefinition("F", LongType, true)))), + Seq(QualifiedColType(None, "f", LongType, true, None, None, None), + QualifiedColType(None, "F", LongType, true, None, None, None))), "COLUMN_ALREADY_EXISTS", Map("columnName" -> toSQLId("f")), caseSensitive = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala index 4da7dbb93e440..ba0dfa951781a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expression, Literal} import org.apache.spark.sql.catalyst.plans.SQLHelper @@ -68,7 +69,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with SQLHelper { Cast(e, dt, Option(SQLConf.get.sessionLocalTimeZone)) } } - val rule = DataSourceAnalysis + val rule = DataSourceAnalysis(SimpleAnalyzer) testRule( "convertStaticPartitions only handle INSERT having at least static partitions", caseSensitive) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 83d77a83d624e..cc1d4ab3fcdf5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -1043,9 +1043,9 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-38336 INSERT INTO statements with tables with default columns: negative tests") { object Errors { - val COMMON_SUBSTRING = "has an invalid DEFAULT value" + val COMMON_SUBSTRING = " has a DEFAULT value" val COLUMN_DEFAULT_NOT_FOUND = "`default` cannot be resolved." - val BAD_SUBQUERY = "Subquery expressions are not allowed in the default value" + val BAD_SUBQUERY = "subquery expressions are not allowed in DEFAULT values" } // The default value fails to analyze. withTable("t") { @@ -1424,8 +1424,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Negative tests") { object Errors { - val COMMON_SUBSTRING = "has an invalid DEFAULT value" - val BAD_SUBQUERY = "Subquery expressions are not allowed in the default value" + val COMMON_SUBSTRING = " has a DEFAULT value" + val BAD_SUBQUERY = "subquery expressions are not allowed in DEFAULT values" } // The default value fails to analyze. withTable("t") { @@ -1454,7 +1454,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { sql("create table t(i boolean) using parquet") assert(intercept[AnalysisException] { sql("alter table t add column s bigint default false") - }.getMessage.contains("The default value has an incompatible data type")) + }.getMessage.contains("provided a value of incompatible type")) } // The default value is disabled per configuration. withTable("t") { @@ -1502,8 +1502,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-38838 INSERT INTO with defaults set by ALTER TABLE ALTER COLUMN: negative tests") { object Errors { - val COMMON_SUBSTRING = "has an invalid DEFAULT value" - val BAD_SUBQUERY = "Subquery expressions are not allowed in the default value" + val COMMON_SUBSTRING = " has a DEFAULT value" + val BAD_SUBQUERY = "subquery expressions are not allowed in DEFAULT values" } val createTable = "create table t(i boolean, s bigint) using parquet" withTable("t") { @@ -1524,7 +1524,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { // The default value parses but the type is not coercible. assert(intercept[AnalysisException] { sql("alter table t alter column s set default false") - }.getMessage.contains("The default value has an incompatible data type")) + }.getMessage.contains("provided a value of incompatible type")) // The default value is disabled per configuration. withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") { val sqlText = "alter table t alter column s set default 41 + 1" @@ -1737,7 +1737,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-39359 Restrict DEFAULT columns to allowlist of supported data source types") { withSQLConf(SQLConf.DEFAULT_COLUMN_ALLOWED_PROVIDERS.key -> "csv,json,orc") { - val unsupported = "does not support column default value" + val unsupported = "DEFAULT values are not supported for target data source" assert(intercept[AnalysisException] { sql(s"create table t(a string default 'abc') using parquet") }.getMessage.contains(unsupported)) @@ -1779,6 +1779,9 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } // Negative tests: provided array element types must match their corresponding DEFAULT // declarations, if applicable. + val incompatibleDefault = + "Failed to execute ALTER TABLE ADD COLUMNS command because the destination table column s " + + "has a DEFAULT value with type" Seq( Config( "parquet"), @@ -1794,7 +1797,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } assert(intercept[AnalysisException] { sql("alter table t add column s array default array('abc', 'def')") - }.getMessage.contains("The default value has an incompatible data type")) + }.getMessage.contains(incompatibleDefault)) } } } @@ -1829,6 +1832,9 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { // Negative tests: provided map element types must match their corresponding DEFAULT // declarations, if applicable. + val incompatibleDefault = + "Failed to execute ALTER TABLE ADD COLUMNS command because the destination table column s " + + "has a DEFAULT value with type" Seq( Config( "parquet"), @@ -1844,7 +1850,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } assert(intercept[AnalysisException] { sql("alter table t add column s struct default struct(42, 56)") - }.getMessage.contains("The default value has an incompatible data type")) + }.getMessage.contains(incompatibleDefault)) } } } @@ -1945,6 +1951,9 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } // Negative tests: provided map element types must match their corresponding DEFAULT // declarations, if applicable. + val incompatibleDefault = + "Failed to execute ALTER TABLE ADD COLUMNS command because the destination table column s " + + "has a DEFAULT value with type" Seq( Config( "parquet"), @@ -1960,7 +1969,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } assert(intercept[AnalysisException] { sql("alter table t add column s map default map(42, 56)") - }.getMessage.contains("The default value has an incompatible data type")) + }.getMessage.contains(incompatibleDefault)) } } } @@ -1973,7 +1982,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { ).foreach { query => assert(intercept[AnalysisException] { sql(query) - }.getMessage.contains("Subquery expressions are not allowed in the default value")) + }.getMessage.contains( + QueryCompilationErrors.defaultValuesMayNotContainSubQueryExpressions().getMessage)) } } @@ -2002,7 +2012,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { sql(s"alter table t1 add column (b string default 'abc')") }.getMessage.contains( QueryCompilationErrors.addNewDefaultColumnToExistingTableNotAllowed( - provider).getMessage)) + "ALTER TABLE ADD COLUMNS", provider).getMessage)) withTable("t2") { // It is still OK to create a new table with a column DEFAULT value assigned, even if // the table provider is in the above denylist. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 0c16d8dc1d0b6..c4e0057ae952d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -100,7 +100,7 @@ class HiveSessionStateBuilder( RelationConversions(catalog) +: PreprocessTableCreation(session) +: PreprocessTableInsertion +: - DataSourceAnalysis +: + DataSourceAnalysis(this) +: ApplyCharTypePadding +: HiveAnalysis +: ReplaceCharWithVarchar +: From 7aa6d440d8d5ed3689fa1eba922359a7722afd09 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 18 Feb 2023 01:51:46 +0800 Subject: [PATCH 08/13] address comments --- .../org/apache/spark/sql/connector/catalog/Column.java | 2 ++ .../sql/connector/catalog/ColumnDefaultValue.java | 2 ++ .../spark/sql/connector/catalog/CatalogV2Util.scala | 10 ++++++++++ .../scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- .../spark/sql/internal/connector/ColumnImpl.scala | 2 +- 5 files changed, 16 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java index e0fc1c321fac9..d2c8f25e73904 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Column.java @@ -20,6 +20,7 @@ import java.util.Map; import javax.annotation.Nullable; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.internal.connector.ColumnImpl; import org.apache.spark.sql.types.DataType; @@ -33,6 +34,7 @@ * {@link Table#columns()} by calling the static {@code create} functions of this interface to * create it. */ +@Evolving public interface Column { static Column create(String name, DataType dataType) { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java index 7aceb31e57163..367dcd3605c05 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java @@ -20,6 +20,7 @@ import java.util.Objects; import javax.annotation.Nonnull; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.connector.expressions.Literal; /** @@ -31,6 +32,7 @@ * remember the column default value and let the reader fill the column value when reading existing * data that do not have these new columns. */ +@Evolving public class ColumnDefaultValue { private String sql; private Literal value; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 0df10a3012539..8fb132066bfcf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -432,6 +432,11 @@ private[sql] object CatalogV2Util { .asTableCatalog } + /** + * Converts DS v2 columns to StructType, which encodes column comment and default value to + * StructField metadata. This is mainly used to define the schema of v2 scan, w.r.t. the columns + * of the v2 table. + */ def v2ColumnsToStructType(columns: Array[Column]): StructType = { StructType(columns.map(v2ColumnToStructField)) } @@ -464,6 +469,11 @@ private[sql] object CatalogV2Util { }.getOrElse(f) } + /** + * Converts a StructType to DS v2 columns, which decodes the StructField metadata to v2 column + * comment and default value. This is mainly used to generate DS v2 columns from table schema in + * DDL commands, so that Spark can pass DS v2 columns to DS v2 createTable and related APIs. + */ def structTypeToV2Columns(schema: StructType): Array[Column] = { schema.fields.map(structFieldToV2Column) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9b8b97c28c394..8d8aacbc9cbcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3108,7 +3108,7 @@ object SQLConf { "provided values when the corresponding fields are not present in storage.") .version("3.4.0") .stringConf - .createWithDefault("csv,json,orc,parquet,hive") + .createWithDefault("csv,json,orc,parquet") val JSON_GENERATOR_WRITE_NULL_IF_WITH_DEFAULT_VALUE = buildConf("spark.sql.jsonGenerator.writeNullIfWithDefaultValue") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala index 642246144ff2e..5ab3f83eeae56 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ColumnImpl.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.internal.connector import org.apache.spark.sql.connector.catalog.{Column, ColumnDefaultValue} import org.apache.spark.sql.types.DataType -// The default implementation of v2 column. +// The standard concrete implementation of data source V2 column. case class ColumnImpl( name: String, dataType: DataType, From 6a0c8a0263961d22e7df9ccf343735d1eddbc174 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 18 Feb 2023 01:54:21 +0800 Subject: [PATCH 09/13] Update sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala Co-authored-by: Daniel Tenedorio --- .../apache/spark/sql/catalyst/plans/logical/statements.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 897869ab4f256..9c639a4bce69f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -143,7 +143,7 @@ case class QualifiedColType( default.map { sql => val e = ResolveDefaultColumns.analyze(colName, dataType, sql, "ALTER TABLE") assert(e.resolved && e.foldable, - "exist default must be simple SQL string that is resolved and foldable, " + + "The existence default value must be a simple SQL string that is resolved and foldable, " + "but got: " + sql) new ColumnDefaultValue(sql, LiteralValue(e.eval(), dataType)) }.orNull From 32728c7e17bd2f4bf37364a3e4ea2c8cefa1f818 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 18 Feb 2023 01:54:40 +0800 Subject: [PATCH 10/13] Update sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala Co-authored-by: Daniel Tenedorio --- .../org/apache/spark/sql/connector/catalog/CatalogV2Util.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 8fb132066bfcf..9b481356fa603 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -491,7 +491,7 @@ private[sql] object CatalogV2Util { if (f.getCurrentDefaultValue().isDefined && f.getExistenceDefaultValue().isDefined) { val e = analyze(f, EXISTS_DEFAULT_COLUMN_METADATA_KEY) assert(e.resolved && e.foldable, - "exist default must be simple SQL string that is resolved and foldable, " + + "The existence default value must be a simple SQL string that is resolved and foldable, " + "but got: " + f.getExistenceDefaultValue().get) val defaultValue = new ColumnDefaultValue( f.getCurrentDefaultValue().get, LiteralValue(e.eval(), f.dataType)) From 88ce6848720328629487c1319ff25a8ce99918cc Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 18 Feb 2023 18:18:51 +0800 Subject: [PATCH 11/13] improve doc --- .../sql/connector/catalog/ColumnDefaultValue.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java index 367dcd3605c05..b8e75c11c813a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ColumnDefaultValue.java @@ -43,7 +43,11 @@ public ColumnDefaultValue(String sql, Literal value) { } /** - * Returns the SQL string (Spark SQL dialect) of the default value expression. + * Returns the SQL string (Spark SQL dialect) of the default value expression. This is the + * original string contents of the SQL expression specified at the time the column was created in + * a CREATE TABLE, REPLACE TABLE, or ADD COLUMN command. For example, for + * "CREATE TABLE t (col INT DEFAULT 40 + 2)", this returns the string literal "40 + 2" (without + * quotation marks). */ @Nonnull public String getSql() { @@ -51,7 +55,9 @@ public String getSql() { } /** - * Returns the default value literal. + * Returns the default value literal. This is the literal value corresponding to + * {@link #getSql()}. For the example in the doc of {@link #getSql()}, this returns a literal + * integer with a value of 42. */ @Nonnull public Literal getValue() { From 3e81760d5f5e73feefaa85c0b817ed2034a6502c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 19 Feb 2023 00:45:06 +0800 Subject: [PATCH 12/13] fix compile --- .../apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala index 18f656748ac17..dfd4ebf4db1a0 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.analysis.{caseSensitiveResolution, Analyzer import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.connect.planner.SparkConnectPlanner import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, InMemoryCatalog} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -75,7 +76,7 @@ class ProtoToParsedPlanTestSuite extends SparkFunSuite with SharedSparkSession { inMemoryCatalog.createTable( Identifier.of(Array("tempdb"), "myTable"), new StructType().add("id", "long"), - Array.empty, + Array.empty[Transform], util.Collections.emptyMap()) val catalogManager = new CatalogManager( From 9cd3dd288cdbb1b4f69673199190c7c09e97ec86 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 20 Feb 2023 10:29:58 +0800 Subject: [PATCH 13/13] fix compile --- .../spark/sql/connect/ProtoToParsedPlanTestSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala index dfd4ebf4db1a0..841017ae6c000 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala @@ -68,16 +68,17 @@ class ProtoToParsedPlanTestSuite extends SparkFunSuite with SharedSparkSession { protected val inputFilePath: Path = baseResourcePath.resolve("queries") protected val goldenFilePath: Path = baseResourcePath.resolve("explain-results") + private val emptyProps: util.Map[String, String] = util.Collections.emptyMap() private val analyzer = { val inMemoryCatalog = new InMemoryCatalog inMemoryCatalog.initialize("primary", CaseInsensitiveStringMap.empty()) - inMemoryCatalog.createNamespace(Array("tempdb"), util.Collections.emptyMap()) + inMemoryCatalog.createNamespace(Array("tempdb"), emptyProps) inMemoryCatalog.createTable( Identifier.of(Array("tempdb"), "myTable"), new StructType().add("id", "long"), Array.empty[Transform], - util.Collections.emptyMap()) + emptyProps) val catalogManager = new CatalogManager( inMemoryCatalog,