diff --git a/.github/workflows/connectors_test.yaml b/.github/workflows/connectors_test.yaml index cec3c722b7e..b7c1fe33daa 100644 --- a/.github/workflows/connectors_test.yaml +++ b/.github/workflows/connectors_test.yaml @@ -16,7 +16,7 @@ jobs: distribution: 'zulu' java-version: '8' - name: Cache Scala, SBT - uses: actions/cache@v2 + uses: actions/cache@v4 with: path: | ~/.sbt diff --git a/.scalafmt.conf b/.scalafmt.conf new file mode 100644 index 00000000000..00a1dbbca51 --- /dev/null +++ b/.scalafmt.conf @@ -0,0 +1,48 @@ +# Copyright (2025) The Delta Lake Project Authors. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +align = none +align.openParenDefnSite = false +align.openParenCallSite = false +align.tokens = [] +importSelectors = "singleLine" +optIn.configStyleArguments = false +continuationIndent { + callSite = 2 + defnSite = 4 +} +danglingParentheses { + defnSite = false + callSite = false +} +docstrings { + style = Asterisk + wrap = no +} +literals.hexDigits = upper +maxColumn = 100 +newlines { + beforeCurlyLambdaParams = false + source = keep +} +rewrite.rules = [Imports] +rewrite.imports.sort = scalastyle +rewrite.imports.groups = [ + ["java\\..*"], + ["scala\\..*"], + ["io\\.delta\\..*"], + ["org\\.apache\\.spark\\.sql\\.delta.*"] +] +runner.dialect = scala212 +version = 3.8.6 diff --git a/build.sbt b/build.sbt index d18fb793bd3..94ddfe4a738 100644 --- a/build.sbt +++ b/build.sbt @@ -157,11 +157,26 @@ lazy val commonSettings = Seq( unidocSourceFilePatterns := Nil, ) -// enforce java code style -def javafmtCheckSettings() = Seq( +//////////////////////////// +// START: Code Formatting // +//////////////////////////// + +/** Enforce java code style on compile. */ +def javafmtCheckSettings(): Seq[Def.Setting[Task[CompileAnalysis]]] = Seq( (Compile / compile) := ((Compile / compile) dependsOn (Compile / javafmtCheckAll)).value ) +/** Enforce scala code style on compile. */ +def scalafmtCheckSettings(): Seq[Def.Setting[Task[CompileAnalysis]]] = Seq( + (Compile / compile) := ((Compile / compile) dependsOn (Compile / scalafmtCheckAll)).value, +) + +// TODO: define fmtAll and fmtCheckAll tasks that run both scala and java fmts/checks + +////////////////////////// +// END: Code Formatting // +////////////////////////// + /** * Note: we cannot access sparkVersion.value here, since that can only be used within a task or * setting macro. @@ -233,7 +248,7 @@ def runTaskOnlyOnSparkMaster[T]( } lazy val connectCommon = (project in file("spark-connect/common")) - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "delta-connect-common", commonSettings, @@ -272,7 +287,7 @@ lazy val connectCommon = (project in file("spark-connect/common")) ) lazy val connectClient = (project in file("spark-connect/client")) - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .dependsOn(connectCommon % "compile->compile;test->test;provided->provided") .settings( name := "delta-connect-client", @@ -361,7 +376,7 @@ lazy val connectClient = (project in file("spark-connect/client")) lazy val connectServer = (project in file("spark-connect/server")) .dependsOn(connectCommon % "compile->compile;test->test;provided->provided") .dependsOn(spark % "compile->compile;test->test;provided->provided") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "delta-connect-server", commonSettings, @@ -405,7 +420,7 @@ lazy val connectServer = (project in file("spark-connect/server")) lazy val spark = (project in file("spark")) .dependsOn(storage) .enablePlugins(Antlr4Plugin) - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "delta-spark", commonSettings, @@ -493,7 +508,7 @@ lazy val spark = (project in file("spark")) lazy val contribs = (project in file("contribs")) .dependsOn(spark % "compile->compile;test->test;provided->provided") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "delta-contribs", commonSettings, @@ -532,7 +547,7 @@ lazy val contribs = (project in file("contribs")) lazy val sharing = (project in file("sharing")) .dependsOn(spark % "compile->compile;test->test;provided->provided") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "delta-sharing-spark", commonSettings, @@ -558,12 +573,14 @@ lazy val sharing = (project in file("sharing")) ).configureUnidoc() lazy val kernelApi = (project in file("kernel/kernel-api")) + .enablePlugins(ScalafmtPlugin) .settings( name := "delta-kernel-api", commonSettings, scalaStyleSettings, javaOnlyReleaseSettings, javafmtCheckSettings, + scalafmtCheckSettings, Test / javaOptions ++= Seq("-ea"), libraryDependencies ++= Seq( "org.roaringbitmap" % "RoaringBitmap" % "0.9.25", @@ -638,6 +655,7 @@ lazy val kernelApi = (project in file("kernel/kernel-api")) ).configureUnidoc(docTitle = "Delta Kernel") lazy val kernelDefaults = (project in file("kernel/kernel-defaults")) + .enablePlugins(ScalafmtPlugin) .dependsOn(kernelApi) .dependsOn(kernelApi % "test->test") .dependsOn(storage) @@ -650,6 +668,7 @@ lazy val kernelDefaults = (project in file("kernel/kernel-defaults")) scalaStyleSettings, javaOnlyReleaseSettings, javafmtCheckSettings, + scalafmtCheckSettings, Test / javaOptions ++= Seq("-ea"), libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client-runtime" % hadoopVersion, @@ -682,7 +701,7 @@ lazy val kernelDefaults = (project in file("kernel/kernel-defaults")) // TODO unidoc // TODO(scott): figure out a better way to include tests in this project lazy val storage = (project in file("storage")) - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "delta-storage", commonSettings, @@ -707,7 +726,7 @@ lazy val storage = (project in file("storage")) lazy val storageS3DynamoDB = (project in file("storage-s3-dynamodb")) .dependsOn(storage % "compile->compile;test->test;provided->provided") .dependsOn(spark % "test->test") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "delta-storage-s3-dynamodb", commonSettings, @@ -733,7 +752,7 @@ val icebergSparkRuntimeArtifactName = { lazy val testDeltaIcebergJar = (project in file("testDeltaIcebergJar")) // delta-iceberg depends on delta-spark! So, we need to include it during our test. .dependsOn(spark % "test") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "test-delta-iceberg-jar", commonSettings, @@ -763,7 +782,7 @@ val deltaIcebergSparkIncludePrefixes = Seq( // scalastyle:off println lazy val iceberg = (project in file("iceberg")) .dependsOn(spark % "compile->compile;test->test;provided->provided") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "delta-iceberg", commonSettings, @@ -833,7 +852,7 @@ lazy val generateIcebergJarsTask = TaskKey[Unit]("generateIcebergJars", "Generat lazy val icebergShaded = (project in file("icebergShaded")) .dependsOn(spark % "provided") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "iceberg-shaded", commonSettings, @@ -864,7 +883,7 @@ lazy val icebergShaded = (project in file("icebergShaded")) lazy val hudi = (project in file("hudi")) .dependsOn(spark % "compile->compile;test->test;provided->provided") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "delta-hudi", commonSettings, @@ -916,7 +935,7 @@ lazy val hudi = (project in file("hudi")) lazy val hive = (project in file("connectors/hive")) .dependsOn(standaloneCosmetic) - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "delta-hive", commonSettings, @@ -933,7 +952,7 @@ lazy val hive = (project in file("connectors/hive")) lazy val hiveAssembly = (project in file("connectors/hive-assembly")) .dependsOn(hive) - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "delta-hive-assembly", Compile / unmanagedJars += (hive / Compile / packageBin / packageBin).value, @@ -960,7 +979,7 @@ lazy val hiveAssembly = (project in file("connectors/hive-assembly")) lazy val hiveTest = (project in file("connectors/hive-test")) .dependsOn(goldenTables % "test") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "hive-test", // Make the project use the assembly jar to ensure we are testing the assembly jar that users @@ -993,7 +1012,7 @@ lazy val hiveTest = (project in file("connectors/hive-test")) lazy val hiveMR = (project in file("connectors/hive-mr")) .dependsOn(hiveTest % "test->test") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "hive-mr", commonSettings, @@ -1020,7 +1039,7 @@ lazy val hiveMR = (project in file("connectors/hive-mr")) lazy val hiveTez = (project in file("connectors/hive-tez")) .dependsOn(hiveTest % "test->test") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "hive-tez", commonSettings, @@ -1064,7 +1083,7 @@ lazy val hiveTez = (project in file("connectors/hive-tez")) lazy val hive2MR = (project in file("connectors/hive2-mr")) .dependsOn(goldenTables % "test") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "hive2-mr", commonSettings, @@ -1095,7 +1114,7 @@ lazy val hive2MR = (project in file("connectors/hive2-mr")) lazy val hive2Tez = (project in file("connectors/hive2-tez")) .dependsOn(goldenTables % "test") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "hive2-tez", commonSettings, @@ -1162,7 +1181,7 @@ lazy val hive2Tez = (project in file("connectors/hive2-tez")) */ lazy val standaloneCosmetic = project .dependsOn(storage) // this doesn't impact the output artifact (jar), only the pom.xml dependencies - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "delta-standalone", commonSettings, @@ -1182,7 +1201,7 @@ lazy val standaloneCosmetic = project lazy val testStandaloneCosmetic = (project in file("connectors/testStandaloneCosmetic")) .dependsOn(standaloneCosmetic) .dependsOn(goldenTables % "test") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "test-standalone-cosmetic", commonSettings, @@ -1199,7 +1218,7 @@ lazy val testStandaloneCosmetic = (project in file("connectors/testStandaloneCos * except `ParquetSchemaConverter` are working without `parquet-hadoop` in testStandaloneCosmetic`. */ lazy val testParquetUtilsWithStandaloneCosmetic = project.dependsOn(standaloneCosmetic) - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "test-parquet-utils-with-standalone-cosmetic", commonSettings, @@ -1223,7 +1242,7 @@ def scalaCollectionPar(version: String) = version match { * create a separate project to skip the shading. */ lazy val standaloneParquet = (project in file("connectors/standalone-parquet")) - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .dependsOn(standaloneWithoutParquetUtils) .settings( name := "delta-standalone-parquet", @@ -1238,7 +1257,7 @@ lazy val standaloneParquet = (project in file("connectors/standalone-parquet")) /** A dummy project to allow `standaloneParquet` depending on the shaded standalone jar. */ lazy val standaloneWithoutParquetUtils = project - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "delta-standalone-without-parquet-utils", commonSettings, @@ -1251,7 +1270,7 @@ lazy val standaloneWithoutParquetUtils = project lazy val standalone = (project in file("connectors/standalone")) .dependsOn(storage % "compile->compile;provided->provided") .dependsOn(goldenTables % "test") - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "delta-standalone-original", commonSettings, @@ -1376,7 +1395,7 @@ lazy val compatibility = (project in file("connectors/oss-compatibility-tests")) lazy val goldenTables = (project in file("connectors/golden-tables")) .dependsOn(spark % "test") // depends on delta-spark - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings( name := "golden-tables", commonSettings, @@ -1403,7 +1422,7 @@ def sqlDeltaImportScalaVersion(scalaBinaryVersion: String): String = { lazy val sqlDeltaImport = (project in file("connectors/sql-delta-import")) .dependsOn(spark) - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "sql-delta-import", commonSettings, @@ -1435,7 +1454,7 @@ lazy val flink = (project in file("connectors/flink")) .dependsOn(standaloneCosmetic % "provided") .dependsOn(kernelApi) .dependsOn(kernelDefaults) - .disablePlugins(JavaFormatterPlugin) + .disablePlugins(JavaFormatterPlugin, ScalafmtPlugin) .settings ( name := "delta-flink", commonSettings, diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000000.crc.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000000.crc.crc new file mode 100644 index 00000000000..ebff5f46514 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000000.crc.crc differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000000.json.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 00000000000..b277b514f73 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000000.json.crc differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000001.crc.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000001.crc.crc new file mode 100644 index 00000000000..df4e0288890 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000001.crc.crc differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000001.json.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000001.json.crc new file mode 100644 index 00000000000..4d2fb531f46 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000001.json.crc differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000002.crc.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000002.crc.crc new file mode 100644 index 00000000000..bfc205a6dee Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000002.crc.crc differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000002.json.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000002.json.crc new file mode 100644 index 00000000000..05384150035 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/.00000000000000000002.json.crc differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000000.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000000.crc new file mode 100644 index 00000000000..e0b1bd686c2 --- /dev/null +++ b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000000.crc @@ -0,0 +1 @@ +{"txnId":"bda32d72-442d-4705-9a8c-16093eb31744","tableSizeBytes":452,"numFiles":1,"numMetadata":1,"numProtocol":1,"setTransactions":[],"domainMetadata":[],"metadata":{"id":"da00fe29-8b6e-4f3b-b91f-a3729283bc1a","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"month\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["month"],"configuration":{"delta.enableChangeDataFeed":"true"},"createdTime":1740185389028},"protocol":{"minReaderVersion":1,"minWriterVersion":7,"writerFeatures":["changeDataFeed","appendOnly","invariants"]},"allFiles":[{"path":"month=1/part-00000-22d25ea7-a383-44df-ad22-6b06d871b547.c000.snappy.parquet","partitionValues":{"month":"1"},"size":452,"modificationTime":1740185390672,"dataChange":false,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}]} diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000000.json b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..84507c06f03 --- /dev/null +++ b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1740185390903,"operation":"CREATE TABLE AS SELECT","operationParameters":{"partitionBy":"[\"month\"]","clusterBy":"[]","description":null,"isManaged":"false","properties":"{\"delta.enableChangeDataFeed\":\"true\"}"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputRows":"1","numOutputBytes":"452"},"engineInfo":"Apache-Spark/3.5.3 Delta-Lake/3.4.0-SNAPSHOT","txnId":"bda32d72-442d-4705-9a8c-16093eb31744"}} +{"metaData":{"id":"da00fe29-8b6e-4f3b-b91f-a3729283bc1a","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"month\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["month"],"configuration":{"delta.enableChangeDataFeed":"true"},"createdTime":1740185389028}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":7,"writerFeatures":["changeDataFeed","appendOnly","invariants"]}} +{"add":{"path":"month=1/part-00000-22d25ea7-a383-44df-ad22-6b06d871b547.c000.snappy.parquet","partitionValues":{"month":"1"},"size":452,"modificationTime":1740185390672,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}} diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000001.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000001.crc new file mode 100644 index 00000000000..67170724936 --- /dev/null +++ b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000001.crc @@ -0,0 +1 @@ +{"txnId":"0d7d28b8-55c2-4d8b-b48e-88b22c90aed1","tableSizeBytes":904,"numFiles":2,"numMetadata":1,"numProtocol":1,"setTransactions":[],"domainMetadata":[],"metadata":{"id":"da00fe29-8b6e-4f3b-b91f-a3729283bc1a","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"month\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["month"],"configuration":{"delta.enableChangeDataFeed":"true"},"createdTime":1740185389028},"protocol":{"minReaderVersion":1,"minWriterVersion":7,"writerFeatures":["changeDataFeed","appendOnly","invariants"]},"allFiles":[{"path":"month=2/part-00000-cc2a9650-0450-4879-9757-873b7f544510.c000.snappy.parquet","partitionValues":{"month":"2"},"size":452,"modificationTime":1740185395663,"dataChange":false,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"},{"path":"month=1/part-00000-22d25ea7-a383-44df-ad22-6b06d871b547.c000.snappy.parquet","partitionValues":{"month":"1"},"size":452,"modificationTime":1740185390672,"dataChange":false,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}]} diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000001.json b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000..d956ee99b35 --- /dev/null +++ b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000001.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1740185395669,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputRows":"1","numOutputBytes":"452"},"engineInfo":"Apache-Spark/3.5.3 Delta-Lake/3.4.0-SNAPSHOT","txnId":"0d7d28b8-55c2-4d8b-b48e-88b22c90aed1"}} +{"add":{"path":"month=2/part-00000-cc2a9650-0450-4879-9757-873b7f544510.c000.snappy.parquet","partitionValues":{"month":"2"},"size":452,"modificationTime":1740185395663,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"}} diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000002.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000002.crc new file mode 100644 index 00000000000..cb7d67984c2 --- /dev/null +++ b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000002.crc @@ -0,0 +1 @@ +{"txnId":"79b3e3aa-82dc-4c18-b95e-8b50089b55c7","tableSizeBytes":904,"numFiles":2,"numMetadata":1,"numProtocol":1,"setTransactions":[],"domainMetadata":[],"metadata":{"id":"da00fe29-8b6e-4f3b-b91f-a3729283bc1a","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"month\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["month"],"configuration":{"delta.enableChangeDataFeed":"true"},"createdTime":1740185389028},"protocol":{"minReaderVersion":1,"minWriterVersion":7,"writerFeatures":["changeDataFeed","appendOnly","invariants"]},"allFiles":[{"path":"month=2/part-00000-129a0441-5f41-4e46-be33-fd0289e53614.c000.snappy.parquet","partitionValues":{"month":"2"},"size":452,"modificationTime":1740185397380,"dataChange":false,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"},{"path":"month=1/part-00000-c5babbd8-6013-484c-818f-22d546976866.c000.snappy.parquet","partitionValues":{"month":"1"},"size":452,"modificationTime":1740185397384,"dataChange":false,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}]} diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000002.json b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000002.json new file mode 100644 index 00000000000..41d3ef238c9 --- /dev/null +++ b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/_delta_log/00000000000000000002.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1740185397394,"operation":"OPTIMIZE","operationParameters":{"predicate":"[]","zOrderBy":"[\"id\"]","clusterBy":"[]","auto":false},"readVersion":1,"isolationLevel":"SnapshotIsolation","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"2","numRemovedBytes":"904","p25FileSize":"452","numDeletionVectorsRemoved":"0","minFileSize":"452","numAddedFiles":"2","maxFileSize":"452","p75FileSize":"452","p50FileSize":"452","numAddedBytes":"904"},"engineInfo":"Apache-Spark/3.5.3 Delta-Lake/3.4.0-SNAPSHOT","txnId":"79b3e3aa-82dc-4c18-b95e-8b50089b55c7"}} +{"add":{"path":"month=1/part-00000-c5babbd8-6013-484c-818f-22d546976866.c000.snappy.parquet","partitionValues":{"month":"1"},"size":452,"modificationTime":1740185397384,"dataChange":false,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}} +{"remove":{"path":"month=1/part-00000-22d25ea7-a383-44df-ad22-6b06d871b547.c000.snappy.parquet","deletionTimestamp":1740185396708,"dataChange":false,"extendedFileMetadata":true,"partitionValues":{"month":"1"},"size":452,"stats":"{\"numRecords\":1}"}} +{"add":{"path":"month=2/part-00000-129a0441-5f41-4e46-be33-fd0289e53614.c000.snappy.parquet","partitionValues":{"month":"2"},"size":452,"modificationTime":1740185397380,"dataChange":false,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"}} +{"remove":{"path":"month=2/part-00000-cc2a9650-0450-4879-9757-873b7f544510.c000.snappy.parquet","deletionTimestamp":1740185396708,"dataChange":false,"extendedFileMetadata":true,"partitionValues":{"month":"2"},"size":452,"stats":"{\"numRecords\":1}"}} diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/.part-00000-22d25ea7-a383-44df-ad22-6b06d871b547.c000.snappy.parquet.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/.part-00000-22d25ea7-a383-44df-ad22-6b06d871b547.c000.snappy.parquet.crc new file mode 100644 index 00000000000..e72389a1212 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/.part-00000-22d25ea7-a383-44df-ad22-6b06d871b547.c000.snappy.parquet.crc differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/.part-00000-c5babbd8-6013-484c-818f-22d546976866.c000.snappy.parquet.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/.part-00000-c5babbd8-6013-484c-818f-22d546976866.c000.snappy.parquet.crc new file mode 100644 index 00000000000..e72389a1212 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/.part-00000-c5babbd8-6013-484c-818f-22d546976866.c000.snappy.parquet.crc differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/part-00000-22d25ea7-a383-44df-ad22-6b06d871b547.c000.snappy.parquet b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/part-00000-22d25ea7-a383-44df-ad22-6b06d871b547.c000.snappy.parquet new file mode 100644 index 00000000000..f5bd9a0ce91 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/part-00000-22d25ea7-a383-44df-ad22-6b06d871b547.c000.snappy.parquet differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/part-00000-c5babbd8-6013-484c-818f-22d546976866.c000.snappy.parquet b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/part-00000-c5babbd8-6013-484c-818f-22d546976866.c000.snappy.parquet new file mode 100644 index 00000000000..f5bd9a0ce91 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=1/part-00000-c5babbd8-6013-484c-818f-22d546976866.c000.snappy.parquet differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/.part-00000-129a0441-5f41-4e46-be33-fd0289e53614.c000.snappy.parquet.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/.part-00000-129a0441-5f41-4e46-be33-fd0289e53614.c000.snappy.parquet.crc new file mode 100644 index 00000000000..6ce716b4059 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/.part-00000-129a0441-5f41-4e46-be33-fd0289e53614.c000.snappy.parquet.crc differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/.part-00000-cc2a9650-0450-4879-9757-873b7f544510.c000.snappy.parquet.crc b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/.part-00000-cc2a9650-0450-4879-9757-873b7f544510.c000.snappy.parquet.crc new file mode 100644 index 00000000000..6ce716b4059 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/.part-00000-cc2a9650-0450-4879-9757-873b7f544510.c000.snappy.parquet.crc differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/part-00000-129a0441-5f41-4e46-be33-fd0289e53614.c000.snappy.parquet b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/part-00000-129a0441-5f41-4e46-be33-fd0289e53614.c000.snappy.parquet new file mode 100644 index 00000000000..f8b82293468 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/part-00000-129a0441-5f41-4e46-be33-fd0289e53614.c000.snappy.parquet differ diff --git a/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/part-00000-cc2a9650-0450-4879-9757-873b7f544510.c000.snappy.parquet b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/part-00000-cc2a9650-0450-4879-9757-873b7f544510.c000.snappy.parquet new file mode 100644 index 00000000000..f8b82293468 Binary files /dev/null and b/connectors/golden-tables/src/main/resources/golden/commit-info-containing-arbitrary-operationParams-types/month=2/part-00000-cc2a9650-0450-4879-9757-873b7f544510.c000.snappy.parquet differ diff --git a/connectors/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala b/connectors/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala index f1efeb21bbc..d2bb4a41af6 100644 --- a/connectors/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala +++ b/connectors/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala @@ -1663,6 +1663,24 @@ class GoldenTables extends QueryTest with SharedSparkSession { Row(0, 0) :: Nil, schema) } + + generateGoldenTable("commit-info-containing-arbitrary-operationParams-types") { tablePath => + spark.sql( + f""" + |CREATE TABLE delta.`$tablePath` + |USING DELTA + |PARTITIONED BY (month) + |TBLPROPERTIES (delta.enableChangeDataFeed = true) + |AS + |SELECT 1 AS id, 1 AS month""".stripMargin) + + // Add some data + spark.sql("INSERT INTO delta.`%s` VALUES (2, 2)".format(tablePath)) + + // Run optimize that generates a commitInfo with arbitrary value types + // operationParameters + spark.sql("OPTIMIZE delta.`%s` ZORDER BY id".format(tablePath)) + } } case class TestStruct(f1: String, f2: Long) diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConversionTransaction.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConversionTransaction.scala index 81d8513243f..bebdcc72cab 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConversionTransaction.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConversionTransaction.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.conf.Configuration -import shadedForDelta.org.apache.iceberg.{AppendFiles, DeleteFiles, OverwriteFiles, PendingUpdate, RewriteFiles, Transaction => IcebergTransaction} +import shadedForDelta.org.apache.iceberg.{AppendFiles, DeleteFiles, OverwriteFiles, PartitionSpec, PendingUpdate, RewriteFiles, Transaction => IcebergTransaction} import shadedForDelta.org.apache.iceberg.ExpireSnapshots import shadedForDelta.org.apache.iceberg.mapping.MappingUtil import shadedForDelta.org.apache.iceberg.mapping.NameMappingParser @@ -64,7 +64,8 @@ class IcebergConversionTransaction( protected val postCommitSnapshot: Snapshot, protected val tableOp: IcebergTableOp = WRITE_TABLE, protected val lastConvertedIcebergSnapshotId: Option[Long] = None, - protected val lastConvertedDeltaVersion: Option[Long] = None) extends DeltaLogging { + protected val lastConvertedDeltaVersion: Option[Long] = None + ) extends DeltaLogging { /////////////////////////// // Nested Helper Classes // @@ -100,7 +101,7 @@ class IcebergConversionTransaction( convertDeltaAddFileToIcebergDataFile( add, tablePath, - partitionSpec, + currentPartitionSpec, logicalToPhysicalPartitionNames, statsParser, postCommitSnapshot @@ -141,7 +142,7 @@ class IcebergConversionTransaction( convertDeltaAddFileToIcebergDataFile( add, tablePath, - partitionSpec, + currentPartitionSpec, logicalToPhysicalPartitionNames, statsParser, postCommitSnapshot @@ -154,7 +155,7 @@ class IcebergConversionTransaction( convertDeltaRemoveFileToIcebergDataFile( remove, tablePath, - partitionSpec, + currentPartitionSpec, logicalToPhysicalPartitionNames, postCommitSnapshot) ) @@ -178,7 +179,7 @@ class IcebergConversionTransaction( convertDeltaRemoveFileToIcebergDataFile( f, tablePath, - partitionSpec, + currentPartitionSpec, logicalToPhysicalPartitionNames, postCommitSnapshot) }.toSet.asJava @@ -188,7 +189,7 @@ class IcebergConversionTransaction( convertDeltaAddFileToIcebergDataFile( f, tablePath, - partitionSpec, + currentPartitionSpec, logicalToPhysicalPartitionNames, statsParser, postCommitSnapshot @@ -212,8 +213,15 @@ class IcebergConversionTransaction( protected val tablePath = postCommitSnapshot.deltaLog.dataPath protected val icebergSchema = convertDeltaSchemaToIcebergSchema(postCommitSnapshot.metadata.schema) + // Initial partition spec converted from Delta protected val partitionSpec = createPartitionSpec(icebergSchema, postCommitSnapshot.metadata.partitionColumns) + + // Current partition spec from iceberg table + def currentPartitionSpec: PartitionSpec = { + Some(txn.table()).map(_.spec()).getOrElse(partitionSpec) + } + private val logicalToPhysicalPartitionNames = getPartitionPhysicalNameMapping(postCommitSnapshot.metadata.partitionSchema) diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergTransactionUtils.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergTransactionUtils.scala index d70a2ce3080..81d4b5bc393 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergTransactionUtils.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergTransactionUtils.scala @@ -35,6 +35,7 @@ import shadedForDelta.org.apache.iceberg.{DataFile, DataFiles, FileFormat, Parti import shadedForDelta.org.apache.iceberg.Metrics import shadedForDelta.org.apache.iceberg.StructLike import shadedForDelta.org.apache.iceberg.TableProperties +import shadedForDelta.org.apache.iceberg.util.DateTimeUtil // scalastyle:off import.ordering.noEmptyLine import shadedForDelta.org.apache.iceberg.catalog.{Namespace, TableIdentifier => IcebergTableIdentifier} @@ -239,7 +240,8 @@ object IcebergTransactionUtils case _: DecimalType => new java.math.BigDecimal(partitionVal) case _: BinaryType => ByteBuffer.wrap(partitionVal.getBytes("UTF-8")) case _: TimestampNTZType => - java.sql.Timestamp.valueOf(partitionVal).getNanos/1000.asInstanceOf[Long] + DateTimeUtil.isoTimestampToMicros( + partitionVal.replace(" ", "T")) case _: TimestampType => try { getMicrosSinceEpoch(partitionVal) @@ -257,7 +259,8 @@ object IcebergTransactionUtils } private def getMicrosSinceEpoch(instant: String): Long = { - Instant.parse(instant).getNano/1000.asInstanceOf[Long] + DateTimeUtil.microsFromInstant( + Instant.parse(instant)) } private def getMetricsForIcebergDataFile( diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java index 0eb816e61d1..8d1a6d96965 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java @@ -20,6 +20,7 @@ import io.delta.kernel.engine.Engine; import io.delta.kernel.types.StructType; import java.util.List; +import java.util.Optional; /** * Represents the snapshot of a Delta table. @@ -61,6 +62,14 @@ public interface Snapshot { */ StructType getSchema(); + /** + * Returns the configuration for the provided {@code domain} if it exists in the snapshot. Returns + * empty if the {@code domain} is not present in the snapshot. + * + * @return the configuration for the provided domain if it exists + */ + Optional getDomainMetadata(String domain); + /** * Create a scan builder to construct a {@link Scan} to read data from this snapshot. * diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java index b2658bb21f9..457f73f89c1 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java @@ -117,7 +117,7 @@ static CloseableIterator transformLogicalData( // Note: `partitionValues` are not used as of now in this API, but taking the partition // values as input forces the connector to not pass data from multiple partitions this // API in a single call. - StructType tableSchema = getLogicalSchema(engine, transactionState); + StructType tableSchema = getLogicalSchema(transactionState); List partitionColNames = getPartitionColumnsList(transactionState); validateAndSanitizePartitionValues(tableSchema, partitionColNames, partitionValues); @@ -167,7 +167,7 @@ static CloseableIterator transformLogicalData( */ static DataWriteContext getWriteContext( Engine engine, Row transactionState, Map partitionValues) { - StructType tableSchema = getLogicalSchema(engine, transactionState); + StructType tableSchema = getLogicalSchema(transactionState); List partitionColNames = getPartitionColumnsList(transactionState); partitionValues = @@ -209,7 +209,7 @@ static CloseableIterator generateAppendActions( return fileStatusIter.map( dataFileStatus -> { if (isIcebergCompatV2Enabled) { - IcebergCompatV2Utils.validDataFileStatus(dataFileStatus); + IcebergCompatV2Utils.validateDataFileStatus(dataFileStatus); } AddFile addFileRow = AddFile.convertDataFileStatus( diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java b/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java index d1f7447d87c..2e907178519 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java @@ -18,6 +18,7 @@ import io.delta.kernel.annotation.Evolving; import io.delta.kernel.engine.Engine; import io.delta.kernel.exceptions.ConcurrentTransactionException; +import io.delta.kernel.exceptions.DomainDoesNotExistException; import io.delta.kernel.exceptions.InvalidConfigurationValueException; import io.delta.kernel.exceptions.UnknownConfigurationException; import io.delta.kernel.internal.TableConfig; @@ -91,6 +92,40 @@ TransactionBuilder withTransactionId( */ TransactionBuilder withMaxRetries(int maxRetries); + /** + * Commit the provided domain metadata as part of this transaction. If this is called more than + * once with the same {@code domain} the latest provided {@code config} will be committed in the + * transaction. Only user-controlled domains are allowed (aka. domains with a `delta.` prefix are + * not allowed). Adding and removing a domain with the same identifier in the same txn is not + * allowed. + * + *

See the Delta protocol for more information on how to use domain metadata Domain + * Metadata. + * + *

Please note using this API will automatically upgrade the protocol of the table to support + * Domain Metadata if it is not already supported. See + * How does Delta Lake manage feature compatibility? for more details. This may break existing + * writers that do not support the Domain Metadata feature; readers will be unaffected. + * + * @param domain the domain identifier + * @param config configuration string for this domain + * @return updated {@link TransactionBuilder} instance + */ + TransactionBuilder withDomainMetadata(String domain, String config); + + /** + * Mark the domain metadata with identifier {@code domain} as removed in this transaction. If this + * domain does not exist in the latest version of the table will throw a {@link + * DomainDoesNotExistException} upon calling {@link TransactionBuilder#build(Engine)}. Adding and + * removing a domain with the same identifier in one txn is not allowed. + * + * @param domain the domain identifier for the domain to remove + * @return updated {@link TransactionBuilder} instance + */ + TransactionBuilder withDomainMetadataRemoved(String domain); + /** * Build the transaction. Also validates the given info to ensure that a valid transaction can be * created. @@ -101,6 +136,8 @@ TransactionBuilder withTransactionId( * @throws InvalidConfigurationValueException if the value of the property is invalid. * @throws UnknownConfigurationException if any of the properties are unknown to {@link * TableConfig}. + * @throws DomainDoesNotExistException if removing a domain that does not exist in the latest + * version of the table */ Transaction build(Engine engine); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/DomainDoesNotExistException.java b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/DomainDoesNotExistException.java new file mode 100644 index 00000000000..31821f9b519 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/DomainDoesNotExistException.java @@ -0,0 +1,30 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.exceptions; + +import io.delta.kernel.annotation.Evolving; + +/** Thrown when attempting to remove a domain metadata that does not exist in the read snapshot. */ +@Evolving +public class DomainDoesNotExistException extends KernelException { + public DomainDoesNotExistException(String tablePath, String domain, long snapshotVersion) { + super( + String.format( + "%s: Cannot remove domain metadata with identifier %s because it does not exist in the " + + "read snapshot at version %s", + tablePath, domain, snapshotVersion)); + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/hook/PostCommitHook.java b/kernel/kernel-api/src/main/java/io/delta/kernel/hook/PostCommitHook.java index 173168ff1a0..50162b065fd 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/hook/PostCommitHook.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/hook/PostCommitHook.java @@ -33,7 +33,14 @@ enum PostCommitHookType { * perform this operation, reading previous checkpoint + logs is required to construct a new * checkpoint, with latency scaling based on log size (typically seconds to minutes). */ - CHECKPOINT + CHECKPOINT, + /** + * Writes a checksum file at the version committed by the transaction. This hook is present when + * all required table statistics (e.g. table size) for checksum file are known when a + * transaction commits. This operation has a minimal latency with no requirement of reading + * previous checkpoint or logs. + */ + CHECKSUM_SIMPLE } /** Invokes the post commit operation whose implementation must be thread safe. */ diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java index e5e990c500d..4baf15d5d46 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java @@ -140,51 +140,52 @@ public static KernelException invalidVersionRange(long startVersion, long endVer } /* ------------------------ PROTOCOL EXCEPTIONS ----------------------------- */ - public static KernelException unsupportedTableFeature(String feature) { + public static KernelException unsupportedReaderProtocol( + String tablePath, int tableReaderVersion) { String message = String.format( - "Unsupported Delta table feature: table requires feature \"%s\" " + "Unsupported Delta protocol reader version: table `%s` requires reader version %s " + "which is unsupported by this version of Delta Kernel.", - feature); + tablePath, tableReaderVersion); return new KernelException(message); } - public static KernelException unsupportedReaderProtocol( - String tablePath, int tableReaderVersion) { + public static KernelException unsupportedWriterProtocol( + String tablePath, int tableWriterVersion) { String message = String.format( - "Unsupported Delta protocol reader version: table `%s` requires reader version %s " + "Unsupported Delta protocol writer version: table `%s` requires writer version %s " + "which is unsupported by this version of Delta Kernel.", - tablePath, tableReaderVersion); + tablePath, tableWriterVersion); return new KernelException(message); } - public static KernelException unsupportedReaderFeature( - String tablePath, Set unsupportedFeatures) { + public static KernelException unsupportedTableFeature(String feature) { String message = String.format( - "Unsupported Delta reader features: table `%s` requires reader table features [%s] " + "Unsupported Delta table feature: table requires feature \"%s\" " + "which is unsupported by this version of Delta Kernel.", - tablePath, String.join(", ", unsupportedFeatures)); + feature); return new KernelException(message); } - public static KernelException unsupportedWriterProtocol( - String tablePath, int tableWriterVersion) { + public static KernelException unsupportedReaderFeatures( + String tablePath, Set readerFeatures) { String message = String.format( - "Unsupported Delta protocol writer version: table `%s` requires writer version %s " + "Unsupported Delta reader features: table `%s` requires reader table features [%s] " + "which is unsupported by this version of Delta Kernel.", - tablePath, tableWriterVersion); + tablePath, String.join(", ", readerFeatures)); return new KernelException(message); } - public static KernelException unsupportedWriterFeature(String tablePath, String writerFeature) { + public static KernelException unsupportedWriterFeatures( + String tablePath, Set writerFeatures) { String message = String.format( "Unsupported Delta writer feature: table `%s` requires writer table feature \"%s\" " + "which is unsupported by this version of Delta Kernel.", - tablePath, writerFeature); + tablePath, writerFeatures); return new KernelException(message); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/IcebergCompatV2Utils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/IcebergCompatV2Utils.java index 999eb036125..81fc251aeea 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/IcebergCompatV2Utils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/IcebergCompatV2Utils.java @@ -27,7 +27,7 @@ private IcebergCompatV2Utils() {} * * @param dataFileStatus The {@link DataFileStatus} to validate. */ - public static void validDataFileStatus(DataFileStatus dataFileStatus) { + public static void validateDataFileStatus(DataFileStatus dataFileStatus) { if (!dataFileStatus.getStatistics().isPresent()) { // presence of stats means always has a non-null `numRecords` throw DeltaErrors.missingNumRecordsStatsForIcebergCompatV2(dataFileStatus); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java index be0497d3e41..f5ea016c37d 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java @@ -25,10 +25,10 @@ import io.delta.kernel.internal.actions.DomainMetadata; import io.delta.kernel.internal.actions.Metadata; import io.delta.kernel.internal.actions.Protocol; +import io.delta.kernel.internal.checksum.CRCInfo; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.metrics.SnapshotQueryContext; import io.delta.kernel.internal.metrics.SnapshotReportImpl; -import io.delta.kernel.internal.replay.CRCInfo; import io.delta.kernel.internal.replay.CreateCheckpointIterator; import io.delta.kernel.internal.replay.LogReplay; import io.delta.kernel.internal.snapshot.LogSegment; @@ -117,6 +117,13 @@ public StructType getSchema() { return getMetadata().getSchema(); } + @Override + public Optional getDomainMetadata(String domain) { + return Optional.ofNullable(getDomainMetadataMap().get(domain)) + .filter(dm -> !dm.isRemoved()) // only consider active domain metadatas (not tombstones) + .map(DomainMetadata::getConfiguration); + } + @Override public ScanBuilder getScanBuilder() { return new ScanBuilderImpl( diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java index 0176caf84bd..4d4273f6f5f 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java @@ -197,7 +197,7 @@ public CloseableIterator getChanges( for (int rowId = 0; rowId < protocolVector.getSize(); rowId++) { if (!protocolVector.isNullAt(rowId)) { Protocol protocol = Protocol.fromColumnVector(protocolVector, rowId); - TableFeatures.validateReadSupportedTable(protocol, getDataPath().toString()); + TableFeatures.validateKernelCanReadTheTable(protocol, getDataPath().toString()); } } if (shouldDropProtocolColumn) { diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java index 5a7436b8cac..eea9eb457ce 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java @@ -25,9 +25,11 @@ import static io.delta.kernel.internal.util.VectorUtils.buildArrayValue; import static io.delta.kernel.internal.util.VectorUtils.stringStringMapValue; import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toSet; import io.delta.kernel.*; import io.delta.kernel.engine.Engine; +import io.delta.kernel.exceptions.DomainDoesNotExistException; import io.delta.kernel.exceptions.TableNotFoundException; import io.delta.kernel.internal.actions.*; import io.delta.kernel.internal.fs.Path; @@ -36,6 +38,7 @@ import io.delta.kernel.internal.replay.LogReplay; import io.delta.kernel.internal.snapshot.LogSegment; import io.delta.kernel.internal.snapshot.SnapshotHint; +import io.delta.kernel.internal.tablefeatures.TableFeature; import io.delta.kernel.internal.tablefeatures.TableFeatures; import io.delta.kernel.internal.util.ColumnMapping; import io.delta.kernel.internal.util.ColumnMapping.ColumnMappingMode; @@ -54,6 +57,8 @@ public class TransactionBuilderImpl implements TransactionBuilder { private final TableImpl table; private final String engineInfo; private final Operation operation; + private final Map domainMetadatasAdded = new HashMap<>(); + private final Set domainMetadatasRemoved = new HashSet<>(); private Optional schema = Optional.empty(); private Optional> partitionColumns = Optional.empty(); private Optional setTxnOpt = Optional.empty(); @@ -111,6 +116,31 @@ public TransactionBuilder withMaxRetries(int maxRetries) { return this; } + @Override + public TransactionBuilder withDomainMetadata(String domain, String config) { + checkArgument( + DomainMetadata.isUserControlledDomain(domain), + "Setting a system-controlled domain is not allowed: " + domain); + checkArgument( + !domainMetadatasRemoved.contains(domain), + "Cannot add a domain that is removed in this transaction"); + // we override any existing value + domainMetadatasAdded.put(domain, new DomainMetadata(domain, config, false /* removed */)); + return this; + } + + @Override + public TransactionBuilder withDomainMetadataRemoved(String domain) { + checkArgument( + DomainMetadata.isUserControlledDomain(domain), + "Removing a system-controlled domain is not allowed: " + domain); + checkArgument( + !domainMetadatasAdded.containsKey(domain), + "Cannot remove a domain that is added in this transaction"); + domainMetadatasRemoved.add(domain); + return this; + } + @Override public Transaction build(Engine engine) { SnapshotImpl snapshot; @@ -137,31 +167,28 @@ public Transaction build(Engine engine) { boolean shouldUpdateProtocol = false; Metadata metadata = snapshot.getMetadata(); Protocol protocol = snapshot.getProtocol(); - if (tableProperties.isPresent()) { - Map validatedProperties = - TableConfig.validateDeltaProperties(tableProperties.get()); - Map newProperties = - metadata.filterOutUnchangedProperties(validatedProperties); - - ColumnMapping.verifyColumnMappingChange( - metadata.getConfiguration(), newProperties, isNewTable); - - if (!newProperties.isEmpty()) { - shouldUpdateMetadata = true; - metadata = metadata.withNewConfiguration(newProperties); - } + Map validatedProperties = + TableConfig.validateDeltaProperties(tableProperties.orElse(Collections.emptyMap())); + Map newProperties = metadata.filterOutUnchangedProperties(validatedProperties); - Set newWriterFeatures = - TableFeatures.extractAutomaticallyEnabledWriterFeatures(metadata, protocol); - if (!newWriterFeatures.isEmpty()) { - logger.info("Automatically enabling writer features: {}", newWriterFeatures); - shouldUpdateProtocol = true; - List oldWriterFeatures = protocol.getWriterFeatures(); - protocol = protocol.withNewWriterFeatures(newWriterFeatures); - List curWriterFeatures = protocol.getWriterFeatures(); - checkArgument(!Objects.equals(oldWriterFeatures, curWriterFeatures)); - TableFeatures.validateWriteSupportedTable(protocol, metadata, table.getPath(engine)); - } + ColumnMapping.verifyColumnMappingChange(metadata.getConfiguration(), newProperties, isNewTable); + + if (!newProperties.isEmpty()) { + shouldUpdateMetadata = true; + metadata = metadata.withNewConfiguration(newProperties); + } + + Optional>> newProtocolAndFeatures = + TableFeatures.autoUpgradeProtocolBasedOnMetadata( + metadata, !domainMetadatasAdded.isEmpty(), protocol); + if (newProtocolAndFeatures.isPresent()) { + logger.info( + "Automatically enabling table features: {}", + newProtocolAndFeatures.get()._2.stream().map(TableFeature::featureName).collect(toSet())); + + shouldUpdateProtocol = true; + protocol = newProtocolAndFeatures.get()._1; + TableFeatures.validateKernelCanWriteToTable(protocol, metadata, table.getPath(engine)); } return new TransactionImpl( @@ -177,14 +204,15 @@ public Transaction build(Engine engine) { shouldUpdateMetadata, shouldUpdateProtocol, maxRetries, - table.getClock()); + table.getClock(), + getDomainMetadatasToCommit(snapshot)); } /** Validate the given parameters for the transaction. */ private void validate(Engine engine, SnapshotImpl snapshot, boolean isNewTable) { String tablePath = table.getPath(engine); // Validate the table has no features that Kernel doesn't yet support writing into it. - TableFeatures.validateWriteSupportedTable( + TableFeatures.validateKernelCanWriteToTable( snapshot.getProtocol(), snapshot.getMetadata(), tablePath); if (!isNewTable) { @@ -288,10 +316,47 @@ private Metadata getInitialMetadata() { } private Protocol getInitialProtocol() { - return new Protocol( - DEFAULT_READ_VERSION, - DEFAULT_WRITE_VERSION, - null /* readerFeatures */, - null /* writerFeatures */); + return new Protocol(DEFAULT_READ_VERSION, DEFAULT_WRITE_VERSION); + } + + /** + * Returns a list of the domain metadatas to commit. This consists of the domain metadatas added + * in the transaction using {@link TransactionBuilder#withDomainMetadata(String, String)} and the + * tombstones for the domain metadatas removed in the transaction using {@link + * TransactionBuilder#withDomainMetadataRemoved(String)}. + */ + private List getDomainMetadatasToCommit(SnapshotImpl snapshot) { + // Add all domain metadatas added in the transaction + List finalDomainMetadatas = new ArrayList<>(domainMetadatasAdded.values()); + + // Generate the tombstones for the removed domain metadatas + Map snapshotDomainMetadataMap = snapshot.getDomainMetadataMap(); + for (String domainName : domainMetadatasRemoved) { + // Note: we know domainName is not already in finalDomainMetadatas because we do not allow + // removing and adding a domain with the same identifier in a single txn! + if (snapshotDomainMetadataMap.containsKey(domainName)) { + DomainMetadata domainToRemove = snapshotDomainMetadataMap.get(domainName); + if (domainToRemove.isRemoved()) { + // If the domain is already removed we throw an error to avoid any inconsistencies or + // ambiguity. The snapshot read by the connector is inconsistent with the snapshot + // loaded here as the domain to remove no longer exists. + throw new DomainDoesNotExistException( + table.getDataPath().toString(), domainName, snapshot.getVersion()); + } + finalDomainMetadatas.add(domainToRemove.removed()); + } else { + // We must throw an error if the domain does not exist. Otherwise, there could be unexpected + // behavior within conflict resolution. For example, consider the following + // 1. Table has no domains set in V0 + // 2. txnA is started and wants to remove domain "foo" + // 3. txnB is started and adds domain "foo" and commits V1 before txnA + // 4. txnA needs to perform conflict resolution against the V1 commit from txnB + // Conflict resolution should fail but since the domain does not exist we cannot create + // a tombstone to mark it as removed and correctly perform conflict resolution. + throw new DomainDoesNotExistException( + table.getDataPath().toString(), domainName, snapshot.getVersion()); + } + } + return finalDomainMetadatas; } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java index 1dba7458002..7e0919783df 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java @@ -29,9 +29,12 @@ import io.delta.kernel.expressions.Column; import io.delta.kernel.hook.PostCommitHook; import io.delta.kernel.internal.actions.*; +import io.delta.kernel.internal.annotation.VisibleForTesting; +import io.delta.kernel.internal.checksum.CRCInfo; import io.delta.kernel.internal.data.TransactionStateRow; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.hook.CheckpointHook; +import io.delta.kernel.internal.hook.ChecksumSimpleHook; import io.delta.kernel.internal.metrics.TransactionMetrics; import io.delta.kernel.internal.metrics.TransactionReportImpl; import io.delta.kernel.internal.replay.ConflictChecker; @@ -39,6 +42,7 @@ import io.delta.kernel.internal.rowtracking.RowTracking; import io.delta.kernel.internal.tablefeatures.TableFeatures; import io.delta.kernel.internal.util.*; +import io.delta.kernel.metrics.TransactionMetricsResult; import io.delta.kernel.metrics.TransactionReport; import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterable; @@ -69,7 +73,7 @@ public class TransactionImpl implements Transaction { private final Optional setTxnOpt; private final boolean shouldUpdateProtocol; private final Clock clock; - private List domainMetadatas = new ArrayList<>(); + private List domainMetadatas; private Metadata metadata; private boolean shouldUpdateMetadata; private int maxRetries; @@ -89,7 +93,8 @@ public TransactionImpl( boolean shouldUpdateMetadata, boolean shouldUpdateProtocol, int maxRetries, - Clock clock) { + Clock clock, + List domainMetadatas) { this.isNewTable = isNewTable; this.dataPath = dataPath; this.logPath = logPath; @@ -103,6 +108,7 @@ public TransactionImpl( this.shouldUpdateProtocol = shouldUpdateProtocol; this.maxRetries = maxRetries; this.clock = clock; + this.domainMetadatas = domainMetadatas; } @Override @@ -129,6 +135,7 @@ public Optional getSetTxnOpt() { * * @param domainMetadatas List of domain metadata to be added to the transaction. */ + @VisibleForTesting public void addDomainMetadatas(List domainMetadatas) { this.domainMetadatas.addAll(domainMetadatas); } @@ -342,6 +349,7 @@ private TransactionCommitResult doCommit( dataAndMetadataActions.map( action -> { transactionMetrics.totalActionsCounter.increment(); + // TODO: handle RemoveFiles. if (!action.isNullAt(ADD_FILE_ORDINAL)) { transactionMetrics.addFilesCounter.increment(); transactionMetrics.addFilesSizeInBytesCounter.increment( @@ -362,6 +370,10 @@ private TransactionCommitResult doCommit( postCommitHooks.add(new CheckpointHook(dataPath, commitAsVersion)); } + buildPostCommitCrcInfoIfCurrentCrcAvailable( + commitAsVersion, transactionMetrics.captureTransactionMetricsResult()) + .ifPresent(crcInfo -> postCommitHooks.add(new ChecksumSimpleHook(crcInfo, logPath))); + return new TransactionCommitResult(commitAsVersion, postCommitHooks); } catch (FileAlreadyExistsException e) { throw e; @@ -437,6 +449,36 @@ private void recordTransactionReport( engine.getMetricsReporters().forEach(reporter -> reporter.report(transactionReport)); } + private Optional buildPostCommitCrcInfoIfCurrentCrcAvailable( + long commitAtVersion, TransactionMetricsResult metricsResult) { + if (isNewTable) { + return Optional.of( + new CRCInfo( + commitAtVersion, + metadata, + protocol, + metricsResult.getTotalAddFilesSizeInBytes(), + metricsResult.getNumAddFiles(), + Optional.of(txnId.toString()))); + } + + return readSnapshot + .getCurrentCrcInfo() + // in the case of a conflicting txn and successful retry the readSnapshot may not be + // commitVersion - 1 + .filter(lastCrcInfo -> commitAtVersion == lastCrcInfo.getVersion() + 1) + .map( + lastCrcInfo -> + new CRCInfo( + commitAtVersion, + metadata, + protocol, + // TODO: handle RemoveFiles for calculating table size and num of files. + lastCrcInfo.getTableSizeBytes() + metricsResult.getTotalAddFilesSizeInBytes(), + lastCrcInfo.getNumFiles() + metricsResult.getNumAddFiles(), + Optional.of(txnId.toString()))); + } + /** * Get the part of the schema of the table that needs the statistics to be collected per file. * diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/CommitInfo.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/CommitInfo.java index 6b16e79e1bc..5051a3e405d 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/CommitInfo.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/CommitInfo.java @@ -18,6 +18,7 @@ import static io.delta.kernel.internal.DeltaErrors.wrapEngineExceptionThrowsIO; import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; import static io.delta.kernel.internal.util.VectorUtils.stringStringMapValue; +import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toMap; import io.delta.kernel.data.ColumnVector; @@ -125,22 +126,14 @@ public CommitInfo( boolean isBlindAppend, String txnId, Map operationMetrics) { - this.inCommitTimestamp = inCommitTimestamp; + this.inCommitTimestamp = requireNonNull(inCommitTimestamp); this.timestamp = timestamp; - this.engineInfo = engineInfo; - this.operation = operation; - this.operationParameters = Collections.unmodifiableMap(operationParameters); + this.engineInfo = requireNonNull(engineInfo); + this.operation = requireNonNull(operation); + this.operationParameters = Collections.unmodifiableMap(requireNonNull(operationParameters)); this.isBlindAppend = isBlindAppend; - this.txnId = txnId; - this.operationMetrics = operationMetrics; - } - - public Optional getInCommitTimestamp() { - return inCommitTimestamp; - } - - public void setInCommitTimestamp(Optional inCommitTimestamp) { - this.inCommitTimestamp = inCommitTimestamp; + this.txnId = requireNonNull(txnId); + this.operationMetrics = Collections.unmodifiableMap(requireNonNull(operationMetrics)); } public long getTimestamp() { @@ -155,6 +148,30 @@ public String getOperation() { return operation; } + public Map getOperationParameters() { + return operationParameters; + } + + public boolean getIsBlindAppend() { + return isBlindAppend; + } + + public String getTxnId() { + return txnId; + } + + public Optional getInCommitTimestamp() { + return inCommitTimestamp; + } + + public Map getOperationMetrics() { + return operationMetrics; + } + + public void setInCommitTimestamp(Optional inCommitTimestamp) { + this.inCommitTimestamp = inCommitTimestamp; + } + /** * Encode as a {@link Row} object with the schema {@link CommitInfo#FULL_SCHEMA}. * diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/DomainMetadata.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/DomainMetadata.java index d3972267c90..8585b62bf5b 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/DomainMetadata.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/DomainMetadata.java @@ -26,10 +26,19 @@ import io.delta.kernel.types.StringType; import io.delta.kernel.types.StructType; import java.util.HashMap; +import java.util.Locale; import java.util.Map; /** Delta log action representing an `DomainMetadata` action */ public class DomainMetadata { + + /** Whether the provided {@code domain} is a user-controlled domain */ + public static boolean isUserControlledDomain(String domain) { + // Domain identifiers are case-sensitive, but we don't want to allow users to set domains + // with prefixes like `DELTA.` either, so perform case-insensitive check for this purpose + return !domain.toLowerCase(Locale.ROOT).startsWith("delta."); + } + /** Full schema of the {@link DomainMetadata} action in the Delta Log. */ public static final StructType FULL_SCHEMA = new StructType() @@ -117,6 +126,11 @@ public Row toRow() { return new GenericRow(DomainMetadata.FULL_SCHEMA, domainMetadataMap); } + public DomainMetadata removed() { + checkArgument(!removed, "Cannot remove a domain metadata tombstone (already removed)"); + return new DomainMetadata(domain, configuration, true /* removed */); + } + @Override public String toString() { return String.format( diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java index 39897bfbafa..1e08823ca6f 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java @@ -15,10 +15,19 @@ */ package io.delta.kernel.internal.actions; -import static io.delta.kernel.internal.util.VectorUtils.buildArrayValue; +import static io.delta.kernel.internal.tablefeatures.TableFeatures.TABLE_FEATURES; +import static io.delta.kernel.internal.tablefeatures.TableFeatures.TABLE_FEATURES_MIN_WRITER_VERSION; +import static io.delta.kernel.internal.util.Preconditions.checkArgument; +import static io.delta.kernel.internal.util.VectorUtils.stringArrayValue; +import static java.lang.String.format; +import static java.util.Collections.emptySet; +import static java.util.Collections.unmodifiableSet; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toSet; import io.delta.kernel.data.*; import io.delta.kernel.internal.data.GenericRow; +import io.delta.kernel.internal.tablefeatures.TableFeature; import io.delta.kernel.internal.tablefeatures.TableFeatures; import io.delta.kernel.internal.util.Tuple2; import io.delta.kernel.internal.util.VectorUtils; @@ -27,9 +36,15 @@ import io.delta.kernel.types.StringType; import io.delta.kernel.types.StructType; import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class Protocol { + ///////////////////////////////////////////////////////////////////////////////////////////////// + /// Public static variables and methods /// + ///////////////////////////////////////////////////////////////////////////////////////////////// + public static Protocol fromColumnVector(ColumnVector vector, int rowId) { if (vector.isNullAt(rowId)) { return null; @@ -39,11 +54,11 @@ public static Protocol fromColumnVector(ColumnVector vector, int rowId) { vector.getChild(0).getInt(rowId), vector.getChild(1).getInt(rowId), vector.getChild(2).isNullAt(rowId) - ? Collections.emptyList() - : VectorUtils.toJavaList(vector.getChild(2).getArray(rowId)), + ? emptySet() + : new HashSet<>(VectorUtils.toJavaList(vector.getChild(2).getArray(rowId))), vector.getChild(3).isNullAt(rowId) - ? Collections.emptyList() - : VectorUtils.toJavaList(vector.getChild(3).getArray(rowId))); + ? emptySet() + : new HashSet<>(VectorUtils.toJavaList(vector.getChild(3).getArray(rowId)))); } public static final StructType FULL_SCHEMA = @@ -55,18 +70,30 @@ public static Protocol fromColumnVector(ColumnVector vector, int rowId) { private final int minReaderVersion; private final int minWriterVersion; - private final List readerFeatures; - private final List writerFeatures; + private final Set readerFeatures; + private final Set writerFeatures; + + // These are derived fields from minReaderVersion and minWriterVersion + private final boolean supportsReaderFeatures; + private final boolean supportsWriterFeatures; + + public Protocol(int minReaderVersion, int minWriterVersion) { + this(minReaderVersion, minWriterVersion, emptySet(), emptySet()); + } public Protocol( int minReaderVersion, int minWriterVersion, - List readerFeatures, - List writerFeatures) { + Set readerFeatures, + Set writerFeatures) { this.minReaderVersion = minReaderVersion; this.minWriterVersion = minWriterVersion; - this.readerFeatures = readerFeatures; - this.writerFeatures = writerFeatures; + this.readerFeatures = + unmodifiableSet(requireNonNull(readerFeatures, "readerFeatures cannot be null")); + this.writerFeatures = + unmodifiableSet(requireNonNull(writerFeatures, "writerFeatures cannot be null")); + this.supportsReaderFeatures = TableFeatures.supportsReaderFeatures(minReaderVersion); + this.supportsWriterFeatures = TableFeatures.supportsWriterFeatures(minWriterVersion); } public int getMinReaderVersion() { @@ -77,11 +104,11 @@ public int getMinWriterVersion() { return minWriterVersion; } - public List getReaderFeatures() { + public Set getReaderFeatures() { return readerFeatures; } - public List getWriterFeatures() { + public Set getWriterFeatures() { return writerFeatures; } @@ -96,8 +123,26 @@ public String toString() { return sb.toString(); } + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + Protocol protocol = (Protocol) o; + return minReaderVersion == protocol.minReaderVersion + && minWriterVersion == protocol.minWriterVersion + && Objects.equals(readerFeatures, protocol.readerFeatures) + && Objects.equals(writerFeatures, protocol.writerFeatures); + } + + @Override + public int hashCode() { + return Objects.hash(minReaderVersion, minWriterVersion, readerFeatures, writerFeatures); + } + /** - * Encode as a {@link Row} object with the schema {@link Protocol#FULL_SCHEMA}. + * Encode as a {@link Row} object with the schema {@link Protocol#FULL_SCHEMA}. Write any empty + * `readerFeatures` and `writerFeatures` as null. * * @return {@link Row} object with the schema {@link Protocol#FULL_SCHEMA} */ @@ -105,23 +150,380 @@ public Row toRow() { Map protocolMap = new HashMap<>(); protocolMap.put(0, minReaderVersion); protocolMap.put(1, minWriterVersion); - protocolMap.put(2, buildArrayValue(readerFeatures, StringType.STRING)); - protocolMap.put(3, buildArrayValue(writerFeatures, StringType.STRING)); + if (supportsReaderFeatures) { + protocolMap.put(2, stringArrayValue(new ArrayList<>(readerFeatures))); + } + if (supportsWriterFeatures) { + protocolMap.put(3, stringArrayValue(new ArrayList<>(writerFeatures))); + } return new GenericRow(Protocol.FULL_SCHEMA, protocolMap); } - public Protocol withNewWriterFeatures(Set writerFeatures) { - Tuple2 newProtocolVersions = - TableFeatures.minProtocolVersionFromAutomaticallyEnabledFeatures(writerFeatures); - List newWriterFeatures = new ArrayList<>(writerFeatures); - if (this.writerFeatures != null) { - newWriterFeatures.addAll(this.writerFeatures); + ///////////////////////////////////////////////////////////////////////////////////////////////// + /// Public methods related to table features interaction with the protocol /// + ///////////////////////////////////////////////////////////////////////////////////////////////// + /** + * Get the set of features that are implicitly supported by the protocol. Features are implicitly + * supported if the reader and/or writer version is less than the versions that supports the + * explicit features specified in `readerFeatures` and `writerFeatures` sets. Examples: + * + *

+ * + *

    + *
  • (minRV = 1, minWV = 7, readerFeatures=[], writerFeatures=[domainMetadata]) results in [] + *
  • (minRV = 1, minWV = 3) results in [appendOnly, invariants, checkConstraints] + *
  • (minRV = 3, minWV = 7, readerFeatures=[v2Checkpoint], writerFeatures=[v2Checkpoint]) + * results in [] + *
  • (minRV = 2, minWV = 6) results in [appendOnly, invariants, checkConstraints, + * changeDataFeed, generatedColumns, columnMapping, identityColumns] + *
+ */ + public Set getImplicitlySupportedFeatures() { + if (supportsReaderFeatures && supportsWriterFeatures) { + return emptySet(); + } else { + return TABLE_FEATURES.stream() + .filter(f -> !supportsReaderFeatures && f.minReaderVersion() <= minReaderVersion) + .filter(f -> !supportsWriterFeatures && f.minWriterVersion() <= minWriterVersion) + .collect(Collectors.toSet()); + } + } + + /** + * Get the set of features that are explicitly supported by the protocol. Features are explicitly + * supported if they are present in the `readerFeatures` and/or `writerFeatures` sets. Examples: + * + *

+ * + *

    + *
  • (minRV = 1, minWV = 7, writerFeatures=[appendOnly, invariants, checkConstraints]) results + * in [appendOnly, invariants, checkConstraints] + *
  • (minRV = 3, minWV = 7, readerFeatures = [columnMapping], writerFeatures=[columnMapping, + * invariants]) results in [columnMapping, invariants] + *
  • (minRV = 1, minWV = 2, readerFeatures = [], writerFeatures=[]) results in [] + *
+ */ + public Set getExplicitlySupportedFeatures() { + return Stream.of(readerFeatures, writerFeatures) + .flatMap(Set::stream) + .map(TableFeatures::getTableFeature) // if a feature is not known, will throw an exception + .collect(Collectors.toSet()); + } + + /** + * Get the set of features that are both implicitly and explicitly supported by the protocol. + * Usually, the protocol has either implicit or explicit features, but not both. This API provides + * a way to get all enabled features. + */ + public Set getImplicitlyAndExplicitlySupportedFeatures() { + Set supportedFeatures = new HashSet<>(); + supportedFeatures.addAll(getImplicitlySupportedFeatures()); + supportedFeatures.addAll(getExplicitlySupportedFeatures()); + return supportedFeatures; + } + + /** + * Get the set of reader writer features that are both implicitly and explicitly supported by the + * protocol. Usually, the protocol has either implicit or explicit features, but not both. This + * API provides a way to get all enabled reader writer features. It doesn't return any writer only + * features. + */ + public Set getImplicitlyAndExplicitlySupportedReaderWriterFeatures() { + return Stream.concat( + // implicit supported features + TABLE_FEATURES.stream() + .filter( + f -> + !supportsReaderFeatures + && f.minReaderVersion() <= this.getMinReaderVersion()), + // explicitly supported features + readerFeatures.stream().map(TableFeatures::getTableFeature)) + .collect(toSet()); + } + + /** Create a new {@link Protocol} object with the given {@link TableFeature} supported. */ + public Protocol withFeatures(Iterable newFeatures) { + Protocol result = this; + for (TableFeature feature : newFeatures) { + result = result.withFeature(feature); + } + return result; + } + + /** + * Get a new Protocol object that has `feature` supported. Writer-only features will be added to + * `writerFeatures` field, and reader-writer features will be added to `readerFeatures` and + * `writerFeatures` fields. + * + *

If `feature` is already implicitly supported in the current protocol's legacy reader or + * writer protocol version, the new protocol will not modify the original protocol version, i.e., + * the feature will not be explicitly added to the protocol's `readerFeatures` or + * `writerFeatures`. This is to avoid unnecessary protocol upgrade for feature that it already + * supports. + * + *

Examples: + * + *

    + *
  • current protocol (2, 5) and new feature to add 'invariants` result in (2, 5) as this + * protocol already supports 'invariants' implicitly. + *
  • current protocol is (1, 7, writerFeature='rowTracking,domainMetadata' and the new feature + * to add is 'appendOnly' results in (1, 7, + * writerFeature='rowTracking,domainMetadata,appendOnly') + *
  • current protocol is (1, 7, writerFeature='rowTracking,domainMetadata' and the new feature + * to add is 'columnMapping' results in throwing UnsupportedOperationException as + * 'columnMapping' requires higher reader version (2) than the current protocol's reader + * version (1). + *
+ */ + public Protocol withFeature(TableFeature feature) { + // Add required dependencies of the feature + Protocol protocolWithDependencies = withFeatures(feature.requiredFeatures()); + + if (feature.minReaderVersion() > protocolWithDependencies.minReaderVersion) { + throw new UnsupportedOperationException( + "TableFeature requires higher reader protocol version"); } + + if (feature.minWriterVersion() > protocolWithDependencies.minWriterVersion) { + throw new UnsupportedOperationException( + "TableFeature requires higher writer protocol version"); + } + + boolean shouldAddToReaderFeatures = + feature.isReaderWriterFeature() + && + // protocol already has support for `readerFeatures` set and the new feature + // can be explicitly added to the protocol's `readerFeatures` + supportsReaderFeatures; + + Set newReaderFeatures = protocolWithDependencies.readerFeatures; + Set newWriterFeatures = protocolWithDependencies.writerFeatures; + + if (shouldAddToReaderFeatures) { + newReaderFeatures = new HashSet<>(protocolWithDependencies.readerFeatures); + newReaderFeatures.add(feature.featureName()); + } + + if (supportsWriterFeatures) { + newWriterFeatures = new HashSet<>(protocolWithDependencies.writerFeatures); + newWriterFeatures.add(feature.featureName()); + } + return new Protocol( - newProtocolVersions._1, - newProtocolVersions._2, - this.readerFeatures == null ? null : new ArrayList<>(this.readerFeatures), + protocolWithDependencies.minReaderVersion, + protocolWithDependencies.minWriterVersion, + newReaderFeatures, newWriterFeatures); } + + /** + * Determine whether this protocol can be safely upgraded to a new protocol `to`. This means all + * features supported by this protocol are supported by `to`. + * + *

Examples regarding feature status: + * + *

    + *
  • `[appendOnly]` to `[appendOnly]` results in allowed. + *
  • `[appendOnly, changeDataFeed]` to `[appendOnly]` results in not allowed. + *
+ */ + public boolean canUpgradeTo(Protocol to) { + return to.getImplicitlyAndExplicitlySupportedFeatures() + .containsAll(this.getImplicitlyAndExplicitlySupportedFeatures()); + } + + /** + * Protocol normalization is the process of converting a table features protocol to the weakest + * possible form. This primarily refers to converting a table features protocol to a legacy + * protocol. A Table Features protocol can be represented with the legacy representation only when + * the features set of the former exactly matches a legacy protocol. + * + *

Normalization can also decrease the reader version of a table features protocol when it is + * higher than necessary. + * + *

For example: + * + *

    + *
  • (1, 7, AppendOnly, Invariants, CheckConstraints) results in (1, 3) + *
  • (3, 7, RowTracking) results in (1, 7, RowTracking) + *
+ */ + public Protocol normalized() { + // Normalization can only be applied to table feature protocols. + if (!isFeatureProtocol()) { + return this; + } + + Tuple2 versions = + TableFeatures.minimumRequiredVersions(getExplicitlySupportedFeatures()); + int minReaderVersion = versions._1; + int minWriterVersion = versions._2; + Protocol newProtocol = new Protocol(minReaderVersion, minWriterVersion); + + if (this.getImplicitlyAndExplicitlySupportedFeatures() + .equals(newProtocol.getImplicitlyAndExplicitlySupportedFeatures())) { + return newProtocol; + } else { + // means we have some feature that is added after table feature support. + // Whatever the feature (reader or readerWriter), it is always going to + // have minWriterVersion as 7. Required minReaderVersion + // should be based on the supported features. + return new Protocol(minReaderVersion, TABLE_FEATURES_MIN_WRITER_VERSION) + .withFeatures(getExplicitlySupportedFeatures()); + } + } + + /** + * Protocol denormalization is the process of converting a legacy protocol to the equivalent table + * features protocol. This is the inverse of protocol normalization. It can be used to allow + * operations on legacy protocols that yield results which cannot be represented anymore by a + * legacy protocol. For example + * + *
    + *
  • (1, 3) results in (1, 7, readerFeatures=[], writerFeatures=[appendOnly, invariants, + * checkConstraints]) + *
  • (2, 5) results in (2, 7, readerFeatures=[], writerFeatures=[appendOnly, invariants, + * checkConstraints, changeDataFeed, generatedColumns, columnMapping]) + *
+ */ + public Protocol denormalized() { + // Denormalization can only be applied to legacy protocols. + if (!isLegacyProtocol()) { + return this; + } + + Tuple2 versions = + TableFeatures.minimumRequiredVersions(getImplicitlySupportedFeatures()); + int minReaderVersion = versions._1; + + return new Protocol(minReaderVersion, TABLE_FEATURES_MIN_WRITER_VERSION) + .withFeatures(getImplicitlySupportedFeatures()); + } + + /** + * Helper method that applies both denormalization and normalization. This can be used to + * normalize invalid legacy protocols such as (2, 3), (1, 5). A legacy protocol is invalid when + * the version numbers are higher than required to support the implied feature set. + */ + public Protocol denormalizedNormalized() { + return this.denormalized().normalized(); + } + + /** + * Merge this protocol with multiple `protocols` to have the highest reader and writer versions + * plus all explicitly and implicitly supported features. + */ + public Protocol merge(Protocol... others) { + List protocols = new ArrayList<>(); + protocols.add(this); + protocols.addAll(Arrays.asList(others)); + + int mergedReaderVersion = + protocols.stream().mapToInt(Protocol::getMinReaderVersion).max().orElse(0); + + int mergedWriterVersion = + protocols.stream().mapToInt(Protocol::getMinWriterVersion).max().orElse(0); + + Set mergedReaderFeatures = + protocols.stream().flatMap(p -> p.readerFeatures.stream()).collect(Collectors.toSet()); + + Set mergedWriterFeatures = + protocols.stream().flatMap(p -> p.writerFeatures.stream()).collect(Collectors.toSet()); + + Set mergedImplicitFeatures = + protocols.stream() + .flatMap(p -> p.getImplicitlySupportedFeatures().stream()) + .collect(Collectors.toSet()); + + Protocol mergedProtocol = + new Protocol( + mergedReaderVersion, + mergedWriterVersion, + mergedReaderFeatures, + mergedWriterFeatures) + .withFeatures(mergedImplicitFeatures); + + // The merged protocol is always normalized in order to represent the protocol + // with the weakest possible form. This enables backward compatibility. + // This is preceded by a denormalization step. This allows to fix invalid legacy Protocols. + // For example, (2, 3) is normalized to (1, 3). This is because there is no legacy feature + // in the set with reader version 2 unless the writer version is at least 5. + return mergedProtocol.denormalizedNormalized(); + } + + /** Validate the protocol contents represents a valid state */ + protected void validate() { + checkArgument(minReaderVersion >= 1, "minReaderVersion should be at least 1"); + checkArgument(minWriterVersion >= 1, "minWriterVersion should be at least 1"); + + // expect the reader and writer features to be empty if the protocol version does not support + checkArgument( + readerFeatures.isEmpty() || supportsReaderFeatures, + "Reader features are not supported for the reader version: " + minReaderVersion); + checkArgument( + writerFeatures.isEmpty() || supportsWriterFeatures, + "Writer features are not supported for the writer version: " + minWriterVersion); + + // If reader versions are supported, expect the writer versions to be supported as well + // We don't have any reader only features. + if (supportsReaderFeatures) { + checkArgument( + supportsWriterFeatures, + "writer version doesn't support writer features: " + minWriterVersion); + } + + if (supportsWriterFeatures) { + // ensure that the reader version supports all the readerWriter features + Set supportedFeatures = getExplicitlySupportedFeatures(); + supportedFeatures.stream() + .filter(TableFeature::isReaderWriterFeature) + .forEach( + feature -> { + checkArgument( + feature.minReaderVersion() <= minReaderVersion, + format( + "Reader version %d does not support readerWriter feature %s", + minReaderVersion, feature.featureName())); + + if (supportsReaderFeatures) { + // if the protocol supports reader features, then it should be part of the + // readerFeatures + checkArgument( + readerFeatures.contains(feature.featureName()), + format( + "ReaderWriter feature %s is not present in readerFeatures", + feature.featureName())); + } + }); + } else { + // ensure we don't get (minReaderVersion, minWriterVersion) that satisfy the readerWriter + // feature version requirements. E.g. (1, 5) is invalid as writer version indicates + // columnMapping supported but reader version does not support it (requires 2). + TABLE_FEATURES.stream() + .filter(TableFeature::isReaderWriterFeature) + .forEach( + f -> { + if (f.minWriterVersion() <= minWriterVersion) { + checkArgument( + f.minReaderVersion() <= minReaderVersion, + format( + "Reader version %d does not support readerWriter feature %s", + minReaderVersion, f.featureName())); + } + }); + } + } + + /** is the protocol a legacy protocol, i.e before (3, 7) */ + private boolean isLegacyProtocol() { + return !supportsReaderFeatures && !supportsWriterFeatures; + } + + /** is the protocol a table feature protocol, i.e after (3, 7) */ + private boolean isFeatureProtocol() { + // checking for writer feature support is enough as we have + // writerOnly or readerWriter features, but no readerOnly features. + return supportsWriterFeatures; + } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checkpoints/Checkpointer.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checkpoints/Checkpointer.java index 8435ac69b26..b651881c25e 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checkpoints/Checkpointer.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checkpoints/Checkpointer.java @@ -19,7 +19,6 @@ import static io.delta.kernel.internal.TableConfig.EXPIRED_LOG_CLEANUP_ENABLED; import static io.delta.kernel.internal.TableConfig.LOG_RETENTION; import static io.delta.kernel.internal.snapshot.MetadataCleanup.cleanupExpiredLogs; -import static io.delta.kernel.internal.tablefeatures.TableFeatures.validateWriteSupportedTable; import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; import io.delta.kernel.data.ColumnarBatch; @@ -32,6 +31,7 @@ import io.delta.kernel.internal.actions.Metadata; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.replay.CreateCheckpointIterator; +import io.delta.kernel.internal.tablefeatures.TableFeatures; import io.delta.kernel.internal.util.*; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.utils.FileStatus; @@ -65,7 +65,7 @@ public static void checkpoint(Engine engine, Clock clock, SnapshotImpl snapshot) logger.info("{}: Starting checkpoint for version: {}", tablePath, version); // Check if writing to the given table protocol version/features is supported in Kernel - validateWriteSupportedTable( + TableFeatures.validateKernelCanWriteToTable( snapshot.getProtocol(), snapshot.getMetadata(), snapshot.getDataPath().toString()); final Path checkpointPath = FileNames.checkpointFileSingular(logPath, version); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/CRCInfo.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/CRCInfo.java new file mode 100644 index 00000000000..b4d72dc89ad --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/CRCInfo.java @@ -0,0 +1,164 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.checksum; + +import static io.delta.kernel.internal.util.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +import io.delta.kernel.data.ColumnVector; +import io.delta.kernel.data.ColumnarBatch; +import io.delta.kernel.data.Row; +import io.delta.kernel.internal.actions.Metadata; +import io.delta.kernel.internal.actions.Protocol; +import io.delta.kernel.internal.data.GenericRow; +import io.delta.kernel.internal.util.InternalUtils; +import io.delta.kernel.types.LongType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructType; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CRCInfo { + private static final Logger logger = LoggerFactory.getLogger(CRCInfo.class); + + // Constants for schema field names + private static final String TABLE_SIZE_BYTES = "tableSizeBytes"; + private static final String NUM_FILES = "numFiles"; + private static final String NUM_METADATA = "numMetadata"; + private static final String NUM_PROTOCOL = "numProtocol"; + private static final String METADATA = "metadata"; + private static final String PROTOCOL = "protocol"; + private static final String TXN_ID = "txnId"; + + public static final StructType CRC_FILE_SCHEMA = + new StructType() + .add(TABLE_SIZE_BYTES, LongType.LONG) + .add(NUM_FILES, LongType.LONG) + .add(NUM_METADATA, LongType.LONG) + .add(NUM_PROTOCOL, LongType.LONG) + .add(METADATA, Metadata.FULL_SCHEMA) + .add(PROTOCOL, Protocol.FULL_SCHEMA) + .add(TXN_ID, StringType.STRING, /*nullable*/ true); + + public static Optional fromColumnarBatch( + long version, ColumnarBatch batch, int rowId, String crcFilePath) { + // Read required fields. + Protocol protocol = + Protocol.fromColumnVector(batch.getColumnVector(getSchemaIndex(PROTOCOL)), rowId); + Metadata metadata = + Metadata.fromColumnVector(batch.getColumnVector(getSchemaIndex(METADATA)), rowId); + long tableSizeBytes = + InternalUtils.requireNonNull( + batch.getColumnVector(getSchemaIndex(TABLE_SIZE_BYTES)), rowId, TABLE_SIZE_BYTES) + .getLong(rowId); + long numFiles = + InternalUtils.requireNonNull( + batch.getColumnVector(getSchemaIndex(NUM_FILES)), rowId, NUM_FILES) + .getLong(rowId); + + // Read optional fields + ColumnVector txnIdColumnVector = batch.getColumnVector(getSchemaIndex(TXN_ID)); + Optional txnId = + txnIdColumnVector.isNullAt(rowId) + ? Optional.empty() + : Optional.of(txnIdColumnVector.getString(rowId)); + + // protocol and metadata are nullable per fromColumnVector's implementation. + if (protocol == null || metadata == null) { + logger.warn("Invalid checksum file missing protocol and/or metadata: {}", crcFilePath); + return Optional.empty(); + } + return Optional.of(new CRCInfo(version, metadata, protocol, tableSizeBytes, numFiles, txnId)); + } + + private final long version; + private final Metadata metadata; + private final Protocol protocol; + private final long tableSizeBytes; + private final long numFiles; + private final Optional txnId; + + public CRCInfo( + long version, + Metadata metadata, + Protocol protocol, + long tableSizeBytes, + long numFiles, + Optional txnId) { + checkArgument(tableSizeBytes >= 0); + checkArgument(numFiles >= 0); + this.version = version; + this.metadata = requireNonNull(metadata); + this.protocol = requireNonNull(protocol); + this.tableSizeBytes = tableSizeBytes; + this.numFiles = numFiles; + this.txnId = requireNonNull(txnId); + } + + /** The version of the Delta table that this CRCInfo represents. */ + public long getVersion() { + return version; + } + + /** The {@link Metadata} stored in this CRCInfo. */ + public Metadata getMetadata() { + return metadata; + } + + /** The {@link Protocol} stored in this CRCInfo. */ + public Protocol getProtocol() { + return protocol; + } + + public long getNumFiles() { + return numFiles; + } + + public long getTableSizeBytes() { + return tableSizeBytes; + } + + public Optional getTxnId() { + return txnId; + } + + /** + * Encode as a {@link Row} object with the schema {@link CRCInfo#CRC_FILE_SCHEMA}. + * + * @return {@link Row} object with the schema {@link CRCInfo#CRC_FILE_SCHEMA} + */ + public Row toRow() { + Map values = new HashMap<>(); + // Add required fields + values.put(getSchemaIndex(TABLE_SIZE_BYTES), tableSizeBytes); + values.put(getSchemaIndex(NUM_FILES), numFiles); + values.put(getSchemaIndex(NUM_METADATA), 1L); + values.put(getSchemaIndex(NUM_PROTOCOL), 1L); + values.put(getSchemaIndex(METADATA), metadata.toRow()); + values.put(getSchemaIndex(PROTOCOL), protocol.toRow()); + + // Add optional fields + txnId.ifPresent(txn -> values.put(getSchemaIndex(TXN_ID), txn)); + return new GenericRow(CRC_FILE_SCHEMA, values); + } + + private static int getSchemaIndex(String fieldName) { + return CRC_FILE_SCHEMA.indexOf(fieldName); + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ChecksumReader.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumReader.java similarity index 98% rename from kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ChecksumReader.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumReader.java index 314c456afe8..eeb651a34f5 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ChecksumReader.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumReader.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.internal.replay; +package io.delta.kernel.internal.checksum; import static io.delta.kernel.internal.util.FileNames.*; import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; @@ -96,7 +96,7 @@ private static Optional readChecksumFile(Engine engine, Path filePath) .getJsonHandler() .readJsonFiles( singletonCloseableIterator(FileStatus.of(filePath.toString())), - CRCInfo.FULL_SCHEMA, + CRCInfo.CRC_FILE_SCHEMA, Optional.empty())) { // We do this instead of iterating through the rows or using `getSingularRow` so we // can use the existing fromColumnVector methods in Protocol, Metadata, Format etc diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumWriter.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumWriter.java new file mode 100644 index 00000000000..caec865694a --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumWriter.java @@ -0,0 +1,58 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.checksum; + +import static io.delta.kernel.internal.DeltaErrors.wrapEngineExceptionThrowsIO; +import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; +import static java.util.Objects.requireNonNull; + +import io.delta.kernel.engine.Engine; +import io.delta.kernel.internal.fs.Path; +import io.delta.kernel.internal.util.FileNames; +import java.io.IOException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Writers for writing checksum files from a snapshot */ +public class ChecksumWriter { + + private static final Logger logger = LoggerFactory.getLogger(ChecksumWriter.class); + + private final Path logPath; + + public ChecksumWriter(Path logPath) { + this.logPath = requireNonNull(logPath); + } + + /** Writes a checksum file */ + public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { + Path newChecksumPath = FileNames.checksumFile(logPath, crcInfo.getVersion()); + logger.info("Writing checksum file to path: {}", newChecksumPath); + wrapEngineExceptionThrowsIO( + () -> { + engine + .getJsonHandler() + .writeJsonFileAtomically( + newChecksumPath.toString(), + singletonCloseableIterator(crcInfo.toRow()), + false /* overwrite */); + logger.info("Write checksum file `{}` succeeds", newChecksumPath); + return null; + }, + "Write checksum file `%s`", + newChecksumPath); + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java index 11c50a2784f..871c3ce6997 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java @@ -60,11 +60,10 @@ private TransactionStateRow(HashMap valueMap) { * Get the logical schema of the table from the transaction state {@link Row} returned by {@link * Transaction#getTransactionState(Engine)}} * - * @param engine {@link Engine} instance to use for parsing the schema * @param transactionState Transaction state state {@link Row} * @return Logical schema of the table as {@link StructType} */ - public static StructType getLogicalSchema(Engine engine, Row transactionState) { + public static StructType getLogicalSchema(Row transactionState) { String serializedSchema = transactionState.getString(COL_NAME_TO_ORDINAL.get("logicalSchemaString")); return DataTypeJsonSerDe.deserializeStructType(serializedSchema); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/hook/ChecksumSimpleHook.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/hook/ChecksumSimpleHook.java new file mode 100644 index 00000000000..bb4d0e8c19f --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/hook/ChecksumSimpleHook.java @@ -0,0 +1,53 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.hook; + +import static io.delta.kernel.internal.util.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +import io.delta.kernel.engine.Engine; +import io.delta.kernel.hook.PostCommitHook; +import io.delta.kernel.internal.checksum.CRCInfo; +import io.delta.kernel.internal.checksum.ChecksumWriter; +import io.delta.kernel.internal.fs.Path; +import java.io.IOException; + +/** + * A post-commit hook that writes a new checksum file at the version committed by the transaction. + * This hook performs a simple checksum operation without requiring previous checkpoint or log + * reading. + */ +public class ChecksumSimpleHook implements PostCommitHook { + + private final CRCInfo crcInfo; + private final Path logPath; + + public ChecksumSimpleHook(CRCInfo crcInfo, Path logPath) { + this.crcInfo = requireNonNull(crcInfo); + this.logPath = requireNonNull(logPath); + } + + @Override + public void threadSafeInvoke(Engine engine) throws IOException { + checkArgument(engine != null); + new ChecksumWriter(logPath).writeCheckSum(engine, crcInfo); + } + + @Override + public PostCommitHookType getType() { + return PostCommitHookType.CHECKSUM_SIMPLE; + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CRCInfo.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CRCInfo.java deleted file mode 100644 index 85b1094f477..00000000000 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CRCInfo.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Copyright (2025) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.delta.kernel.internal.replay; - -import static java.util.Objects.requireNonNull; - -import io.delta.kernel.data.ColumnarBatch; -import io.delta.kernel.internal.actions.Metadata; -import io.delta.kernel.internal.actions.Protocol; -import io.delta.kernel.types.StructType; -import java.util.Optional; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class CRCInfo { - private static final Logger logger = LoggerFactory.getLogger(CRCInfo.class); - - public static Optional fromColumnarBatch( - long version, ColumnarBatch batch, int rowId, String crcFilePath) { - Protocol protocol = Protocol.fromColumnVector(batch.getColumnVector(PROTOCOL_ORDINAL), rowId); - Metadata metadata = Metadata.fromColumnVector(batch.getColumnVector(METADATA_ORDINAL), rowId); - // protocol and metadata are nullable per fromColumnVector's implementation. - if (protocol == null || metadata == null) { - logger.warn("Invalid checksum file missing protocol and/or metadata: {}", crcFilePath); - return Optional.empty(); - } - return Optional.of(new CRCInfo(version, metadata, protocol)); - } - - // We can add additional fields later - public static final StructType FULL_SCHEMA = - new StructType().add("protocol", Protocol.FULL_SCHEMA).add("metadata", Metadata.FULL_SCHEMA); - - private static final int PROTOCOL_ORDINAL = 0; - private static final int METADATA_ORDINAL = 1; - - private final long version; - private final Metadata metadata; - private final Protocol protocol; - - protected CRCInfo(long version, Metadata metadata, Protocol protocol) { - this.version = version; - this.metadata = requireNonNull(metadata); - this.protocol = requireNonNull(protocol); - } - - /** The version of the Delta table that this CRCInfo represents. */ - public long getVersion() { - return version; - } - - /** The {@link Metadata} stored in this CRCInfo. */ - public Metadata getMetadata() { - return metadata; - } - - /** The {@link Protocol} stored in this CRCInfo. */ - public Protocol getProtocol() { - return protocol; - } -} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/LogReplay.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/LogReplay.java index 97cbf3ad637..9572b4c8392 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/LogReplay.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/LogReplay.java @@ -28,6 +28,8 @@ import io.delta.kernel.expressions.Predicate; import io.delta.kernel.internal.actions.*; import io.delta.kernel.internal.checkpoints.SidecarFile; +import io.delta.kernel.internal.checksum.CRCInfo; +import io.delta.kernel.internal.checksum.ChecksumReader; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.lang.Lazy; import io.delta.kernel.internal.metrics.ScanMetrics; @@ -286,7 +288,7 @@ protected Tuple2 loadTableProtocolAndMetadata( if (protocol != null) { // Stop since we have found the latest Protocol and Metadata. - TableFeatures.validateReadSupportedTable(protocol, dataPath.toString()); + TableFeatures.validateKernelCanReadTheTable(protocol, dataPath.toString()); return new Tuple2<>(protocol, metadata); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotHint.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotHint.java index 5423d786b3f..76b9f653e21 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotHint.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotHint.java @@ -18,7 +18,7 @@ import io.delta.kernel.internal.actions.Metadata; import io.delta.kernel.internal.actions.Protocol; -import io.delta.kernel.internal.replay.CRCInfo; +import io.delta.kernel.internal.checksum.CRCInfo; /** Contains summary information of a {@link io.delta.kernel.Snapshot}. */ public class SnapshotHint { diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeature.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeature.java index 4c13ccfcdad..d598db85fb1 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeature.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeature.java @@ -43,6 +43,9 @@ * protocol but its metadata requirements are not satisfied, then clients still have to understand * the feature (at least to the extent that they can read and preserve the existing data in the * table that uses the feature). + * + *

Important note: uses the default implementation of `equals` and `hashCode` methods. We expect + * that the feature instances are singletons, so we don't need to compare the fields. */ public abstract class TableFeature { @@ -220,7 +223,4 @@ private void validate() { checkArgument(minReaderVersion() == 0, "Writer-only feature must have minReaderVersion=0"); } } - - // Important note: uses the default implementation of `equals` and `hashCode` methods. - // We expect that the feature instances are singletons, so we don't need to compare the fields. } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java index 314df4d7dca..257b584d9e2 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java @@ -20,6 +20,7 @@ import static io.delta.kernel.internal.util.ColumnMapping.ColumnMappingMode.NONE; import static io.delta.kernel.types.TimestampNTZType.TIMESTAMP_NTZ; import static io.delta.kernel.types.VariantType.VARIANT; +import static java.util.stream.Collectors.toSet; import io.delta.kernel.exceptions.KernelException; import io.delta.kernel.internal.DeltaErrors; @@ -33,7 +34,7 @@ import io.delta.kernel.types.FieldMetadata; import io.delta.kernel.types.StructType; import java.util.*; -import java.util.stream.Collectors; +import java.util.stream.Stream; /** Contains utility methods related to the Delta table feature support in protocol. */ public class TableFeatures { @@ -109,7 +110,8 @@ private static class ChangeDataFeedFeature extends TableFeature.LegacyWriterFeat @Override public boolean hasKernelWriteSupport(Metadata metadata) { - return false; // TODO: yet to be implemented in Kernel + // writable if change data feed is disabled + return !TableConfig.CHANGE_DATA_FEED_ENABLED.fromMetadata(metadata); } @Override @@ -208,7 +210,11 @@ private static class RowTrackingFeature extends TableFeature.WriterFeature @Override public boolean metadataRequiresFeatureToBeEnabled(Protocol protocol, Metadata metadata) { - return TableConfig.ROW_TRACKING_ENABLED.fromMetadata(metadata); + if (TableConfig.ROW_TRACKING_ENABLED.fromMetadata(metadata)) { + throw new UnsupportedOperationException( + "Feature `rowTracking` is not yet supported in Kernel."); + } + return false; } @Override @@ -299,11 +305,6 @@ private static class TimestampNtzTableFeature extends TableFeature.ReaderWriterF super("timestampNtz", /* minReaderVersion = */ 3, /* minWriterVersion = */ 7); } - @Override - public boolean hasKernelWriteSupport(Metadata metadata) { - return false; // TODO: yet to be implemented in Kernel - } - @Override public boolean metadataRequiresFeatureToBeEnabled(Protocol protocol, Metadata metadata) { return hasTypeColumn(metadata.getSchema(), TIMESTAMP_NTZ); @@ -342,6 +343,15 @@ private static class VacuumProtocolCheckTableFeature extends TableFeature.Reader /// END: Define the {@link TableFeature}s /// ///////////////////////////////////////////////////////////////////////////////// + ///////////////////////////////////////////////////////////////////////////////// + /// Public static variables and methods /// + ///////////////////////////////////////////////////////////////////////////////// + /** Min reader version that supports reader features. */ + public static final int TABLE_FEATURES_MIN_READER_VERSION = 3; + + /** Min reader version that supports writer features. */ + public static final int TABLE_FEATURES_MIN_WRITER_VERSION = 7; + public static final List TABLE_FEATURES = Collections.unmodifiableList( Arrays.asList( @@ -384,272 +394,168 @@ public static TableFeature getTableFeature(String featureName) { return tableFeature; } - ///////////////////////////////////////////////////////////////////////////////// - /// Everything below will be removed once the Kernel upgrades to use the /// - /// above interfaces. /// - ///////////////////////////////////////////////////////////////////////////////// - private static final Set SUPPORTED_WRITER_FEATURES = - Collections.unmodifiableSet( - new HashSet() { - { - add("appendOnly"); - add("inCommitTimestamp"); - add("columnMapping"); - add("typeWidening-preview"); - add("typeWidening"); - add(DOMAIN_METADATA_FEATURE_NAME); - add(ROW_TRACKING_FEATURE_NAME); - } - }); - - private static final Set SUPPORTED_READER_FEATURES = - Collections.unmodifiableSet( - new HashSet() { - { - add("columnMapping"); - add("deletionVectors"); - add("timestampNtz"); - add("typeWidening-preview"); - add("typeWidening"); - add("vacuumProtocolCheck"); - add("variantType"); - add("variantType-preview"); - add("v2Checkpoint"); - } - }); - - public static final String DOMAIN_METADATA_FEATURE_NAME = "domainMetadata"; + /** Does reader version supports explicitly specifying reader feature set in protocol? */ + public static boolean supportsReaderFeatures(int minReaderVersion) { + return minReaderVersion >= TABLE_FEATURES_MIN_READER_VERSION; + } - public static final String ROW_TRACKING_FEATURE_NAME = "rowTracking"; + /** Does writer version supports explicitly specifying writer feature set in protocol? */ + public static boolean supportsWriterFeatures(int minWriterVersion) { + return minWriterVersion >= TABLE_FEATURES_MIN_WRITER_VERSION; + } - public static final String INVARIANTS_FEATURE_NAME = "invariants"; + /** Returns the minimum reader/writer versions required to support all provided features. */ + public static Tuple2 minimumRequiredVersions(Set features) { + int minReaderVersion = + features.stream().mapToInt(TableFeature::minReaderVersion).max().orElse(0); - /** The minimum writer version required to support table features. */ - public static final int TABLE_FEATURES_MIN_WRITER_VERSION = 7; + int minWriterVersion = + features.stream().mapToInt(TableFeature::minWriterVersion).max().orElse(0); - //////////////////// - // Helper Methods // - //////////////////// - - public static void validateReadSupportedTable(Protocol protocol, String tablePath) { - switch (protocol.getMinReaderVersion()) { - case 1: - break; - case 2: - break; - case 3: - List readerFeatures = protocol.getReaderFeatures(); - if (!SUPPORTED_READER_FEATURES.containsAll(readerFeatures)) { - Set unsupportedFeatures = new HashSet<>(readerFeatures); - unsupportedFeatures.removeAll(SUPPORTED_READER_FEATURES); - throw DeltaErrors.unsupportedReaderFeature(tablePath, unsupportedFeatures); - } - break; - default: - throw DeltaErrors.unsupportedReaderProtocol(tablePath, protocol.getMinReaderVersion()); - } + return new Tuple2<>(Math.max(minReaderVersion, 1), Math.max(minWriterVersion, 1)); } /** - * Utility method to validate whether the given table is supported for writing from Kernel. - * Currently, the support is as follows: - * - *

    - *
  • protocol writer version 1. - *
  • protocol writer version 2 only with appendOnly feature enabled. - *
  • protocol writer version 7 with {@code appendOnly}, {@code inCommitTimestamp}, {@code - * columnMapping}, {@code typeWidening}, {@code domainMetadata}, {@code rowTracking} feature - * enabled. - *
+ * Upgrade the current protocol to satisfy all auto-update capable features required by the given + * metadata. If the current protocol already satisfies the metadata requirements, return empty. * - * @param protocol Table protocol - * @param metadata Table metadata + * @param newMetadata the new metadata to be applied to the table. + * @param needDomainMetadataSupport whether the table needs to explicitly support domain metadata. + * @param currentProtocol the current protocol of the table. + * @return the upgraded protocol and the set of new features that were enabled in the upgrade. */ - public static void validateWriteSupportedTable( - Protocol protocol, Metadata metadata, String tablePath) { - int minWriterVersion = protocol.getMinWriterVersion(); - switch (minWriterVersion) { - case 1: - break; - case 2: - // Append-only and column invariants are the writer features added in version 2 - // Append-only is supported, but not the invariants - validateNoInvariants(metadata.getSchema()); - break; - case 3: - // Check constraints are added in version 3 - throw unsupportedWriterProtocol(tablePath, minWriterVersion); - case 4: - // CDF and generated columns are writer features added in version 4 - throw unsupportedWriterProtocol(tablePath, minWriterVersion); - case 5: - // Column mapping is the only one writer feature added in version 5 - throw unsupportedWriterProtocol(tablePath, minWriterVersion); - case 6: - // Identity is the only one writer feature added in version 6 - throw unsupportedWriterProtocol(tablePath, minWriterVersion); - case 7: - for (String writerFeature : protocol.getWriterFeatures()) { - if (writerFeature.equals(INVARIANTS_FEATURE_NAME)) { - // For version 7, we allow 'invariants' to be present in the protocol's writerFeatures - // to unblock certain use cases, provided that no invariants are defined in the schema. - validateNoInvariants(metadata.getSchema()); - } else if (!SUPPORTED_WRITER_FEATURES.contains(writerFeature)) { - throw unsupportedWriterFeature(tablePath, writerFeature); - } - } - - // Eventually we may have a way to declare and enforce dependencies between features. - // By putting this check for row tracking here, it makes it easier to spot that row - // tracking defines such a dependency that can be implicitly checked. - if (isRowTrackingSupported(protocol) && !isDomainMetadataSupported(protocol)) { - throw DeltaErrors.rowTrackingSupportedWithDomainMetadataUnsupported(); - } - break; - default: - throw unsupportedWriterProtocol(tablePath, minWriterVersion); + public static Optional>> autoUpgradeProtocolBasedOnMetadata( + Metadata newMetadata, boolean needDomainMetadataSupport, Protocol currentProtocol) { + + Set allNeededTableFeatures = + extractAllNeededTableFeatures(newMetadata, currentProtocol); + if (needDomainMetadataSupport) { + allNeededTableFeatures = + Stream.concat(allNeededTableFeatures.stream(), Stream.of(DOMAIN_METADATA_W_FEATURE)) + .collect(toSet()); + } + Protocol required = + new Protocol(TABLE_FEATURES_MIN_READER_VERSION, TABLE_FEATURES_MIN_WRITER_VERSION) + .withFeatures(allNeededTableFeatures) + .normalized(); + + // See if all the required features are already supported in the current protocol. + if (!required.canUpgradeTo(currentProtocol)) { + // `required` has one or more features that are not supported in `currentProtocol`. + Set newFeatures = + new HashSet<>(required.getImplicitlyAndExplicitlySupportedFeatures()); + newFeatures.removeAll(currentProtocol.getImplicitlyAndExplicitlySupportedFeatures()); + return Optional.of(new Tuple2<>(required.merge(currentProtocol), newFeatures)); + } else { + return Optional.empty(); } } - /** - * Given the automatically enabled features from Delta table metadata, returns the minimum - * required reader and writer version that satisfies all enabled table features in the metadata. - * - * @param enabledFeatures the automatically enabled features from the Delta table metadata - * @return the minimum required reader and writer version that satisfies all enabled table - */ - public static Tuple2 minProtocolVersionFromAutomaticallyEnabledFeatures( - Set enabledFeatures) { + /** Utility method to check if the table with given protocol is readable by the Kernel. */ + public static void validateKernelCanReadTheTable(Protocol protocol, String tablePath) { + if (protocol.getMinReaderVersion() > TABLE_FEATURES_MIN_READER_VERSION) { + throw DeltaErrors.unsupportedReaderProtocol(tablePath, protocol.getMinReaderVersion()); + } - int readerVersion = 0; - int writerVersion = 0; + Set unsupportedFeatures = + protocol.getImplicitlyAndExplicitlySupportedReaderWriterFeatures().stream() + .filter(f -> !f.hasKernelReadSupport()) + .collect(toSet()); - for (String feature : enabledFeatures) { - readerVersion = Math.max(readerVersion, getMinReaderVersion(feature)); - writerVersion = Math.max(writerVersion, getMinWriterVersion(feature)); + if (!unsupportedFeatures.isEmpty()) { + throw unsupportedReaderFeatures( + tablePath, unsupportedFeatures.stream().map(TableFeature::featureName).collect(toSet())); } - - return new Tuple2<>(readerVersion, writerVersion); } /** - * Extract the writer features that should be enabled automatically based on the metadata which - * are not already enabled. For example, the {@code inCommitTimestamp} feature should be enabled - * when the delta property name (delta.enableInCommitTimestamps) is set to true in the metadata if - * it is not already enabled. - * - * @param metadata the metadata of the table - * @param protocol the protocol of the table - * @return the writer features that should be enabled automatically + * Utility method to check if the table with given protocol and metadata is writable by the + * Kernel. */ - public static Set extractAutomaticallyEnabledWriterFeatures( - Metadata metadata, Protocol protocol) { - return TableFeatures.SUPPORTED_WRITER_FEATURES.stream() - .filter(f -> metadataRequiresWriterFeatureToBeEnabled(metadata, f)) - .filter( - f -> protocol.getWriterFeatures() == null || !protocol.getWriterFeatures().contains(f)) - .collect(Collectors.toSet()); - } + public static void validateKernelCanWriteToTable( + Protocol protocol, Metadata metadata, String tablePath) { - /** - * Checks if the table protocol supports the "domainMetadata" writer feature. - * - * @param protocol the protocol to check - * @return true if the "domainMetadata" feature is supported, false otherwise - */ - public static boolean isDomainMetadataSupported(Protocol protocol) { - return isWriterFeatureSupported(protocol, DOMAIN_METADATA_FEATURE_NAME); + validateKernelCanReadTheTable(protocol, tablePath); + + if (protocol.getMinWriterVersion() > TABLE_FEATURES_MIN_WRITER_VERSION) { + throw unsupportedWriterProtocol(tablePath, protocol.getMinWriterVersion()); + } + + Set unsupportedFeatures = + protocol.getImplicitlyAndExplicitlySupportedFeatures().stream() + .filter(f -> !f.hasKernelWriteSupport(metadata)) + .collect(toSet()); + + if (!unsupportedFeatures.isEmpty()) { + throw unsupportedWriterFeatures( + tablePath, unsupportedFeatures.stream().map(TableFeature::featureName).collect(toSet())); + } } - /** - * Check if the table protocol supports the "rowTracking" writer feature. - * - * @param protocol the protocol to check - * @return true if the protocol supports row tracking, false otherwise - */ public static boolean isRowTrackingSupported(Protocol protocol) { - return isWriterFeatureSupported(protocol, ROW_TRACKING_FEATURE_NAME); + return protocol.getImplicitlyAndExplicitlySupportedFeatures().contains(ROW_TRACKING_W_FEATURE); } - /** - * Get the minimum reader version required for a feature. - * - * @param feature the feature - * @return the minimum reader version required for the feature - */ - private static int getMinReaderVersion(String feature) { - switch (feature) { - case "inCommitTimestamp": - return 3; - default: - return 1; - } + public static boolean isDomainMetadataSupported(Protocol protocol) { + return protocol + .getImplicitlyAndExplicitlySupportedFeatures() + .contains(DOMAIN_METADATA_W_FEATURE); } + ///////////////////////////////////////////////////////////////////////////////// + /// Private methods /// + ///////////////////////////////////////////////////////////////////////////////// /** - * Get the minimum writer version required for a feature. - * - * @param feature the feature - * @return the minimum writer version required for the feature + * Extracts all table features (and their dependency features) that are enabled by the given + * metadata and supported in existing protocol. */ - private static int getMinWriterVersion(String feature) { - switch (feature) { - case "inCommitTimestamp": - return 7; - default: - return 2; - } + private static Set extractAllNeededTableFeatures( + Metadata newMetadata, Protocol currentProtocol) { + Set protocolSupportedFeatures = + currentProtocol.getImplicitlyAndExplicitlySupportedFeatures(); + + Set metadataEnabledFeatures = + TableFeatures.TABLE_FEATURES.stream() + .filter(f -> f instanceof FeatureAutoEnabledByMetadata) + .filter( + f -> + ((FeatureAutoEnabledByMetadata) f) + .metadataRequiresFeatureToBeEnabled(currentProtocol, newMetadata)) + .collect(toSet()); + + // Each feature may have dependencies that are not yet enabled in the protocol. + Set newFeatures = getDependencyFeatures(metadataEnabledFeatures); + return Stream.concat(protocolSupportedFeatures.stream(), newFeatures.stream()).collect(toSet()); } /** - * Determine whether a writer feature must be supported and enabled to satisfy the metadata - * requirements. - * - * @param metadata the table metadata - * @param feature the writer feature to check - * @return whether the writer feature must be enabled + * Returns the smallest set of table features that contains `features` and that also contains all + * dependencies of all features in the returned set. */ - private static boolean metadataRequiresWriterFeatureToBeEnabled( - Metadata metadata, String feature) { - switch (feature) { - case "inCommitTimestamp": - return TableConfig.IN_COMMIT_TIMESTAMPS_ENABLED.fromMetadata(metadata); - default: - return false; - } - } - - private static boolean isWriterFeatureSupported(Protocol protocol, String featureName) { - List writerFeatures = protocol.getWriterFeatures(); - if (writerFeatures == null) { - return false; - } - return writerFeatures.contains(featureName) - && protocol.getMinWriterVersion() >= TABLE_FEATURES_MIN_WRITER_VERSION; - } + private static Set getDependencyFeatures(Set features) { + Set requiredFeatures = new HashSet<>(features); + features.forEach(feature -> requiredFeatures.addAll(feature.requiredFeatures())); - private static void validateNoInvariants(StructType tableSchema) { - if (hasInvariants(tableSchema)) { - throw DeltaErrors.columnInvariantsNotSupported(); + if (features.equals(requiredFeatures)) { + return features; + } else { + return getDependencyFeatures(requiredFeatures); } } private static boolean hasInvariants(StructType tableSchema) { return !SchemaUtils.filterRecursively( tableSchema, - /* recurseIntoMapOrArrayElements = */ false, // invariants are not allowed in maps or - // arrays - // arrays + // invariants are not allowed in maps or arrays + /* recurseIntoMapOrArrayElements = */ false, /* stopOnFirstMatch */ true, /* filter */ field -> field.getMetadata().contains("delta.invariants")) .isEmpty(); } private static boolean hasCheckConstraints(Metadata metadata) { - return metadata.getConfiguration().entrySet().stream() - .findAny() - .map(entry -> entry.getKey().startsWith("delta.constraints.")) - .orElse(false); + return metadata.getConfiguration().keySet().stream() + .anyMatch(s -> s.startsWith("delta.constraints.")); } /** diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java index f380d3de7ac..a5c07dc2450 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java @@ -111,7 +111,8 @@ public static void validatePartitionColumns(StructType schema, List part || dataType instanceof StringType || dataType instanceof BinaryType || dataType instanceof DateType - || dataType instanceof TimestampType)) { + || dataType instanceof TimestampType + || dataType instanceof TimestampNTZType)) { throw unsupportedPartitionDataType(partitionCol, dataType); } }); @@ -329,7 +330,8 @@ protected static void validateSupportedType(DataType dataType) { || dataType instanceof StringType || dataType instanceof BinaryType || dataType instanceof DateType - || dataType instanceof TimestampType) { + || dataType instanceof TimestampType + || dataType instanceof TimestampNTZType) { // supported types return; } else if (dataType instanceof StructType) { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/CloseableIteratorSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/CloseableIteratorSuite.scala index 387112bb25f..fd119ca715d 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/CloseableIteratorSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/CloseableIteratorSuite.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import io.delta.kernel.internal.util.Utils import io.delta.kernel.utils.CloseableIterator import io.delta.kernel.utils.CloseableIterator.BreakableFilterResult + import org.scalatest.funsuite.AnyFunSuite class CloseableIteratorSuite extends AnyFunSuite { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/TransactionSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/TransactionSuite.scala index d56dfd8f346..70a56321acd 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/TransactionSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/TransactionSuite.scala @@ -15,6 +15,12 @@ */ package io.delta.kernel +import java.lang.{Long => JLong} +import java.util +import java.util.Optional + +import scala.collection.JavaConverters._ + import io.delta.kernel.Transaction.{generateAppendActions, transformLogicalData} import io.delta.kernel.data._ import io.delta.kernel.exceptions.KernelException @@ -30,12 +36,8 @@ import io.delta.kernel.internal.util.VectorUtils.stringStringMapValue import io.delta.kernel.test.{MockEngineUtils, VectorTestUtils} import io.delta.kernel.types.{LongType, StringType, StructType} import io.delta.kernel.utils.{CloseableIterator, DataFileStatistics, DataFileStatus} -import org.scalatest.funsuite.AnyFunSuite -import java.lang.{Long => JLong} -import java.util -import java.util.Optional -import scala.collection.JavaConverters._ +import org.scalatest.funsuite.AnyFunSuite class TransactionSuite extends AnyFunSuite with VectorTestUtils with MockEngineUtils { @@ -48,7 +50,7 @@ class TransactionSuite extends AnyFunSuite with VectorTestUtils with MockEngineU mockEngine(), testTxnState(testSchema, enableIcebergCompatV2 = icebergCompatV2Enabled), testData(includePartitionCols = false), - Map.empty[String, Literal].asJava /* partition values */) + Map.empty[String, Literal].asJava /* partition values */ ) transformedDateIter.map(_.getData).forEachRemaining(batch => { assert(batch.getSchema === testSchema) }) @@ -93,8 +95,7 @@ class TransactionSuite extends AnyFunSuite with VectorTestUtils with MockEngineU "file2" -> None // missing stats ), "Iceberg V2 compatibility requires statistics" // expected error message - ) - ).foreach { case (actionRows, expectedErrorMsg) => + )).foreach { case (actionRows, expectedErrorMsg) => if (icebergCompatV2Enabled) { val ex = intercept[KernelException] { generateAppendActions(engine, txnState, actionRows, testDataWriteContext()) @@ -111,8 +112,7 @@ class TransactionSuite extends AnyFunSuite with VectorTestUtils with MockEngineU // valid stats val dataFileStatuses = testDataFileStatuses( "file1" -> testStats(Some(10)), - "file2" -> testStats(Some(20)) - ) + "file2" -> testStats(Some(20))) var actStats: Seq[String] = Seq.empty generateAppendActions(engine, txnState, dataFileStatuses, testDataWriteContext()) .forEachRemaining { addActionRow => @@ -130,8 +130,8 @@ class TransactionSuite extends AnyFunSuite with VectorTestUtils with MockEngineU object TransactionSuite extends VectorTestUtils with MockEngineUtils { def testData(includePartitionCols: Boolean): CloseableIterator[FilteredColumnarBatch] = { toCloseableIterator( - Seq.range(0, 5).map(_ => testBatch(includePartitionCols)).asJava.iterator() - ).map(batch => new FilteredColumnarBatch(batch, Optional.empty())) + Seq.range(0, 5).map(_ => testBatch(includePartitionCols)).asJava.iterator()).map(batch => + new FilteredColumnarBatch(batch, Optional.empty())) } def testBatch(includePartitionCols: Boolean): ColumnarBatch = { @@ -187,9 +187,9 @@ object TransactionSuite extends VectorTestUtils with MockEngineUtils { } def testTxnState( - schema: StructType, - partitionCols: Seq[String] = Seq.empty, - enableIcebergCompatV2: Boolean = false): Row = { + schema: StructType, + partitionCols: Seq[String] = Seq.empty, + enableIcebergCompatV2: Boolean = false): Row = { val configurationMap = Map(ICEBERG_COMPAT_V2_ENABLED.getKey -> enableIcebergCompatV2.toString) val metadata = new Metadata( "id", @@ -217,7 +217,7 @@ object TransactionSuite extends VectorTestUtils with MockEngineUtils { } def testDataFileStatuses(fileNameStatsPairs: (String, Option[DataFileStatistics])*) - : CloseableIterator[DataFileStatus] = { + : CloseableIterator[DataFileStatus] = { toCloseableIterator( fileNameStatsPairs.map { case (fileName, statsOpt) => @@ -225,8 +225,7 @@ object TransactionSuite extends VectorTestUtils with MockEngineUtils { fileName, 23L, // size - arbitrary value as this is just for tests. 23L, // modificationTime - arbitrary value as this is just for tests. - Optional.ofNullable(statsOpt.orNull) - ) + Optional.ofNullable(statsOpt.orNull)) }.asJava.iterator()) } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/deletionvectors/Base85CodecSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/deletionvectors/Base85CodecSuite.scala index 827c9c20a68..2f31992a37a 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/deletionvectors/Base85CodecSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/deletionvectors/Base85CodecSuite.scala @@ -22,6 +22,7 @@ import java.util.UUID import scala.util.Random import io.delta.kernel.internal.deletionvectors.Base85Codec + import org.scalatest.funsuite.AnyFunSuite class Base85CodecSuite extends AnyFunSuite { @@ -65,13 +66,13 @@ class Base85CodecSuite extends AnyFunSuite { test("Z85 spec character map") { assert(Base85Codec.ENCODE_MAP.length === 85) val referenceBytes = Seq( - 0x00, 0x09, 0x98, 0x62, 0x0f, 0xc7, 0x99, 0x43, 0x1f, 0x85, - 0x9a, 0x24, 0x2f, 0x43, 0x9b, 0x05, 0x3f, 0x01, 0x9b, 0xe6, - 0x4e, 0xbf, 0x9c, 0xc7, 0x5e, 0x7d, 0x9d, 0xa8, 0x6e, 0x3b, - 0x9e, 0x89, 0x7d, 0xf9, 0x9f, 0x6a, 0x8d, 0xb7, 0xa0, 0x4b, - 0x9d, 0x75, 0xa1, 0x2c, 0xad, 0x33, 0xa2, 0x0d, 0xbc, 0xf1, - 0xa2, 0xee, 0xcc, 0xaf, 0xa3, 0xcf, 0xdc, 0x6d, 0xa4, 0xb0, - 0xec, 0x2b, 0xa5, 0x91, 0xfb, 0xe9, 0xa6, 0x72) + 0x00, 0x09, 0x98, 0x62, 0x0F, 0xC7, 0x99, 0x43, 0x1F, 0x85, + 0x9A, 0x24, 0x2F, 0x43, 0x9B, 0x05, 0x3F, 0x01, 0x9B, 0xE6, + 0x4E, 0xBF, 0x9C, 0xC7, 0x5E, 0x7D, 0x9D, 0xA8, 0x6E, 0x3B, + 0x9E, 0x89, 0x7D, 0xF9, 0x9F, 0x6A, 0x8D, 0xB7, 0xA0, 0x4B, + 0x9D, 0x75, 0xA1, 0x2C, 0xAD, 0x33, 0xA2, 0x0D, 0xBC, 0xF1, + 0xA2, 0xEE, 0xCC, 0xAF, 0xA3, 0xCF, 0xDC, 0x6D, 0xA4, 0xB0, + 0xEC, 0x2B, 0xA5, 0x91, 0xFB, 0xE9, 0xA6, 0x72) .map(_.toByte).toArray val referenceString = new String(Base85Codec.ENCODE_MAP, US_ASCII) val encodedString = Base85Codec.encodeBytes(referenceBytes) @@ -87,7 +88,8 @@ class Base85CodecSuite extends AnyFunSuite { Base85Codec.decodeBytes( illegalEncodedString, // This value is irrelevant, any value should cause the failure. - 3) // outputLength + 3 + ) // outputLength } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/deletionvectors/RoaringBitmapArraySuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/deletionvectors/RoaringBitmapArraySuite.scala index 9fccc37e6bb..9b015b80a81 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/deletionvectors/RoaringBitmapArraySuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/deletionvectors/RoaringBitmapArraySuite.scala @@ -17,6 +17,7 @@ package io.delta.kernel.deletionvectors import io.delta.kernel.internal.deletionvectors.RoaringBitmapArray + import org.scalatest.funsuite.AnyFunSuite class RoaringBitmapArraySuite extends AnyFunSuite { @@ -67,7 +68,7 @@ class RoaringBitmapArraySuite extends AnyFunSuite { // TODO need to implement serialize to copy over tests - /** + /* final val BITMAP2_NUMBER = Int.MaxValue.toLong * 3L for (serializationFormat <- RoaringBitmapArrayFormat.values) { @@ -91,5 +92,6 @@ class RoaringBitmapArraySuite extends AnyFunSuite { output.deserialize(buffer) assert(input === output) } - */ + */ + } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/expressions/ExpressionsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/expressions/ExpressionsSuite.scala index b38406e1b1b..c794064a5be 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/expressions/ExpressionsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/expressions/ExpressionsSuite.scala @@ -16,6 +16,7 @@ package io.delta.kernel.expressions import io.delta.kernel.types._ + import org.scalatest.funsuite.AnyFunSuite class ExpressionsSuite extends AnyFunSuite { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala index d3342577fce..90f6f108dda 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala @@ -20,28 +20,29 @@ import java.io.FileNotFoundException import scala.reflect.ClassTag import io.delta.kernel.exceptions.TableNotFoundException -import org.scalatest.funsuite.AnyFunSuite -import io.delta.kernel.utils.FileStatus import io.delta.kernel.test.MockFileSystemClientUtils +import io.delta.kernel.utils.FileStatus + +import org.scalatest.funsuite.AnyFunSuite class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { def checkGetActiveCommitAtTimestamp( - fileList: Seq[FileStatus], - timestamp: Long, - expectedVersion: Long, - mustBeRecreatable: Boolean = true, - canReturnLastCommit: Boolean = false, - canReturnEarliestCommit: Boolean = false): Unit = { + fileList: Seq[FileStatus], + timestamp: Long, + expectedVersion: Long, + mustBeRecreatable: Boolean = true, + canReturnLastCommit: Boolean = false, + canReturnEarliestCommit: Boolean = false): Unit = { val activeCommit = DeltaHistoryManager.getActiveCommitAtTimestamp( createMockFSListFromEngine(fileList), logPath, timestamp, mustBeRecreatable, canReturnLastCommit, - canReturnEarliestCommit - ) - assert(activeCommit.getVersion == expectedVersion, + canReturnEarliestCommit) + assert( + activeCommit.getVersion == expectedVersion, s"Expected version $expectedVersion but got $activeCommit for timestamp=$timestamp") if (mustBeRecreatable) { @@ -53,20 +54,20 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil timestamp, false, // mustBeRecreatable canReturnLastCommit, - canReturnEarliestCommit - ) - assert(activeCommit.getVersion == expectedVersion, + canReturnEarliestCommit) + assert( + activeCommit.getVersion == expectedVersion, s"Expected version $expectedVersion but got $activeCommit for timestamp=$timestamp") } } def checkGetActiveCommitAtTimestampError[T <: Throwable]( - fileList: Seq[FileStatus], - timestamp: Long, - expectedErrorMessageContains: String, - mustBeRecreatable: Boolean = true, - canReturnLastCommit: Boolean = false, - canReturnEarliestCommit: Boolean = false)(implicit classTag: ClassTag[T]): Unit = { + fileList: Seq[FileStatus], + timestamp: Long, + expectedErrorMessageContains: String, + mustBeRecreatable: Boolean = true, + canReturnLastCommit: Boolean = false, + canReturnEarliestCommit: Boolean = false)(implicit classTag: ClassTag[T]): Unit = { val e = intercept[T] { DeltaHistoryManager.getActiveCommitAtTimestamp( createMockFSListFromEngine(fileList), @@ -74,8 +75,7 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil timestamp, mustBeRecreatable, canReturnLastCommit, - canReturnEarliestCommit - ) + canReturnEarliestCommit) } assert(e.getMessage.contains(expectedErrorMessageContains)) } @@ -92,13 +92,11 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, -1, - DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, -1, 0, 0).getMessage - ) + DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, -1, 0, 0).getMessage) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 21, - DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 21, 20, 2).getMessage - ) + DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 21, 20, 2).getMessage) // Valid queries with canReturnLastCommit=true and canReturnEarliestCommit=true checkGetActiveCommitAtTimestamp(deltaFiles, -1, 0, canReturnEarliestCommit = true) checkGetActiveCommitAtTimestamp(deltaFiles, 21, 2, canReturnLastCommit = true) @@ -116,13 +114,11 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, -1, - DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, -1, 0, 0).getMessage - ) + DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, -1, 0, 0).getMessage) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 21, - DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 21, 20, 2).getMessage - ) + DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 21, 20, 2).getMessage) // Valid queries with canReturnLastCommit=true and canReturnEarliestCommit=true checkGetActiveCommitAtTimestamp(deltaFiles, -1, 0, canReturnEarliestCommit = true) checkGetActiveCommitAtTimestamp(deltaFiles, 21, 2, canReturnLastCommit = true) @@ -138,13 +134,11 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 8, - DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, 8, 20, 2).getMessage - ) + DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, 8, 20, 2).getMessage) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 31, - DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 31, 30, 3).getMessage - ) + DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 31, 30, 3).getMessage) // Valid queries with canReturnLastCommit=true and canReturnEarliestCommit=true checkGetActiveCommitAtTimestamp(deltaFiles, 8, 2, canReturnEarliestCommit = true) checkGetActiveCommitAtTimestamp(deltaFiles, 31, 3, canReturnLastCommit = true) @@ -158,13 +152,11 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 8, - DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, 8, 20, 2).getMessage - ) + DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, 8, 20, 2).getMessage) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 21, - DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 21, 20, 2).getMessage - ) + DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 21, 20, 2).getMessage) // Valid queries with canReturnLastCommit=true and canReturnEarliestCommit=true checkGetActiveCommitAtTimestamp(deltaFiles, 8, 2, canReturnEarliestCommit = true) checkGetActiveCommitAtTimestamp(deltaFiles, 21, 2, canReturnLastCommit = true) @@ -180,13 +172,11 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 8, - DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, 8, 20, 2).getMessage - ) + DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, 8, 20, 2).getMessage) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 31, - DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 31, 30, 3).getMessage - ) + DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 31, 30, 3).getMessage) // Valid queries with canReturnLastCommit=true and canReturnEarliestCommit=true checkGetActiveCommitAtTimestamp(deltaFiles, 8, 2, canReturnEarliestCommit = true) checkGetActiveCommitAtTimestamp(deltaFiles, 31, 3, canReturnLastCommit = true) @@ -202,8 +192,7 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil true, // mustBeRecreatable false, // canReturnLastCommit false // canReturnEarliestCommit - ) - ) + )) // Empty _delta_log directory intercept[TableNotFoundException]( DeltaHistoryManager.getActiveCommitAtTimestamp( @@ -213,8 +202,7 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil true, // mustBeRecreatable false, // canReturnLastCommit false // canReturnEarliestCommit - ) - ) + )) } // TODO: corrects commit timestamps for increasing commits (monotonizeCommitTimestamps)? @@ -225,37 +213,33 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFileStatuses(Seq(1L, 2L, 3L)), 25, - "No recreatable commits found" - ) + "No recreatable commits found") // Must have corresponding delta file for a checkpoint checkGetActiveCommitAtTimestampError[RuntimeException]( singularCheckpointFileStatuses(Seq(1L)) ++ deltaFileStatuses(Seq(2L, 3L)), 25, - "No recreatable commits found" - ) + "No recreatable commits found") // No commit files at all (only checkpoint files) checkGetActiveCommitAtTimestampError[RuntimeException]( singularCheckpointFileStatuses(Seq(1L)), 25, - "No commits found" - ) + "No commits found") // No delta files checkGetActiveCommitAtTimestampError[RuntimeException]( Seq("foo", "notdelta.parquet", "foo.json", "001.checkpoint.00f.oo0.parquet") .map(FileStatus.of(_, 10, 10)), 25, - "No delta files found in the directory" - ) + "No delta files found in the directory") // No complete checkpoint checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFileStatuses(Seq(2L, 3L)) ++ multiCheckpointFileStatuses(Seq(2L), 3).take(2), 25, - "No recreatable commits found" - ) + "No recreatable commits found") } test("getActiveCommitAtTimestamp: when mustBeRecreatable=false") { - Seq(deltaFileStatuses(Seq(1L, 2L, 3L)), // w/o checkpoint + Seq( + deltaFileStatuses(Seq(1L, 2L, 3L)), // w/o checkpoint singularCheckpointFileStatuses(Seq(2L)) ++ deltaFileStatuses(Seq(1L, 2L, 3L)) // w/checkpoint ).foreach { deltaFiles => // Valid queries @@ -269,19 +253,25 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil deltaFiles, -1, DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, -1, 10, 1).getMessage, - mustBeRecreatable = false - ) + mustBeRecreatable = false) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 31, DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 31, 30, 3).getMessage, - mustBeRecreatable = false - ) + mustBeRecreatable = false) // Valid queries with canReturnLastCommit=true and canReturnEarliestCommit=true checkGetActiveCommitAtTimestamp( - deltaFiles, 0, 1, mustBeRecreatable = false, canReturnEarliestCommit = true) + deltaFiles, + 0, + 1, + mustBeRecreatable = false, + canReturnEarliestCommit = true) checkGetActiveCommitAtTimestamp( - deltaFiles, 31, 3, mustBeRecreatable = false, canReturnLastCommit = true) + deltaFiles, + 31, + 3, + mustBeRecreatable = false, + canReturnLastCommit = true) } } @@ -296,8 +286,7 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil false, // mustBeRecreatable false, // canReturnLastCommit false // canReturnEarliestCommit - ) - ) + )) // Empty _delta_log directory intercept[TableNotFoundException]( DeltaHistoryManager.getActiveCommitAtTimestamp( @@ -307,23 +296,20 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil true, // mustBeRecreatable false, // canReturnLastCommit false // canReturnEarliestCommit - ) - ) + )) /* ---------- CORRUPT LISTINGS --------- */ // No commit files at all (only checkpoint files) checkGetActiveCommitAtTimestampError[RuntimeException]( singularCheckpointFileStatuses(Seq(1L)), 25, "No delta files found in the directory", - mustBeRecreatable = false - ) + mustBeRecreatable = false) // No delta files checkGetActiveCommitAtTimestampError[RuntimeException]( Seq("foo", "notdelta.parquet", "foo.json", "001.checkpoint.00f.oo0.parquet") .map(FileStatus.of(_, 10, 10)), 25, "No delta files found in the directory", - mustBeRecreatable = false - ) + mustBeRecreatable = false) } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala index d6b799bb62b..1b479ce693c 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala @@ -22,11 +22,11 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import io.delta.kernel.exceptions.{InvalidTableException, KernelException, TableNotFoundException} -import io.delta.kernel.internal.util.FileNames -import io.delta.kernel.utils.FileStatus import io.delta.kernel.internal.DeltaLogActionUtils.{getCommitFilesForVersionRange, listDeltaLogFilesAsIter, verifyDeltaVersions} import io.delta.kernel.internal.fs.Path +import io.delta.kernel.internal.util.FileNames import io.delta.kernel.test.MockFileSystemClientUtils +import io.delta.kernel.utils.FileStatus import org.scalatest.funsuite.AnyFunSuite @@ -48,23 +48,20 @@ class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtil getCommitFiles(Seq(1, 2, 3)), 1, 3, - dataPath - ) + dataPath) // Only one version provided verifyDeltaVersions( getCommitFiles(Seq(1)), 1, 1, - dataPath - ) + dataPath) // Non-contiguous versions intercept[InvalidTableException] { verifyDeltaVersions( getCommitFiles(Seq(1, 3, 4)), 1, 4, - dataPath - ) + dataPath) } // End-version or start-version not right intercept[KernelException] { @@ -72,16 +69,14 @@ class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtil getCommitFiles(Seq(1, 2, 3)), 0, 3, - dataPath - ) + dataPath) } intercept[KernelException] { verifyDeltaVersions( getCommitFiles(Seq(1, 2, 3)), 1, 4, - dataPath - ) + dataPath) } // Empty versions intercept[KernelException] { @@ -89,8 +84,7 @@ class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtil getCommitFiles(Seq()), 1, 4, - dataPath - ) + dataPath) } // Unsorted or duplicates (shouldn't be possible) intercept[InvalidTableException] { @@ -98,16 +92,14 @@ class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtil getCommitFiles(Seq(1, 1, 2)), 1, 4, - dataPath - ) + dataPath) } intercept[InvalidTableException] { verifyDeltaVersions( getCommitFiles(Seq(1, 4, 3, 2)), 1, 2, - dataPath - ) + dataPath) } } @@ -121,26 +113,23 @@ class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtil createMockFSListFromEngine(_ => throw new FileNotFoundException()), dataPath, 0, - 1 - ) + 1) } } def testGetCommitFilesExpectedError[T <: Throwable]( - testName: String, - files: Seq[FileStatus], - startVersion: Long = 1, - endVersion: Long = 3, - expectedErrorMessageContains: String - )(implicit classTag: ClassTag[T]): Unit = { + testName: String, + files: Seq[FileStatus], + startVersion: Long = 1, + endVersion: Long = 3, + expectedErrorMessageContains: String)(implicit classTag: ClassTag[T]): Unit = { test("getCommitFilesForVersionRange: " + testName) { val e = intercept[T] { getCommitFilesForVersionRange( createMockFSListFromEngine(files), dataPath, startVersion, - endVersion - ) + endVersion) } assert(e.getMessage.contains(expectedErrorMessageContains)) } @@ -149,100 +138,85 @@ class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtil testGetCommitFilesExpectedError[KernelException]( testName = "empty directory", files = Seq(), - expectedErrorMessageContains = "no log files found in the requested version range" - ) + expectedErrorMessageContains = "no log files found in the requested version range") testGetCommitFilesExpectedError[KernelException]( testName = "all versions less than startVersion", files = deltaFileStatuses(Seq(0)), - expectedErrorMessageContains = "no log files found in the requested version range" - ) + expectedErrorMessageContains = "no log files found in the requested version range") testGetCommitFilesExpectedError[KernelException]( testName = "all versions greater than endVersion", files = deltaFileStatuses(Seq(4, 5, 6)), - expectedErrorMessageContains = "no log files found in the requested version range" - ) + expectedErrorMessageContains = "no log files found in the requested version range") testGetCommitFilesExpectedError[InvalidTableException]( testName = "missing log files", files = deltaFileStatuses(Seq(1, 3)), - expectedErrorMessageContains = "versions are not contiguous" - ) + expectedErrorMessageContains = "versions are not contiguous") testGetCommitFilesExpectedError[KernelException]( testName = "start version not available", files = deltaFileStatuses(Seq(2, 3, 4, 5)), - expectedErrorMessageContains = "no log file found for version 1" - ) + expectedErrorMessageContains = "no log file found for version 1") testGetCommitFilesExpectedError[KernelException]( testName = "end version not available", files = deltaFileStatuses(Seq(0, 1, 2)), - expectedErrorMessageContains = "no log file found for version 3" - ) + expectedErrorMessageContains = "no log file found for version 3") testGetCommitFilesExpectedError[KernelException]( testName = "invalid start version", files = deltaFileStatuses(Seq(0, 1, 2)), startVersion = -1, - expectedErrorMessageContains = "Invalid version range" - ) + expectedErrorMessageContains = "Invalid version range") testGetCommitFilesExpectedError[KernelException]( testName = "invalid end version", files = deltaFileStatuses(Seq(0, 1, 2)), startVersion = 3, endVersion = 2, - expectedErrorMessageContains = "Invalid version range" - ) + expectedErrorMessageContains = "Invalid version range") def testGetCommitFiles( - testName: String, - files: Seq[FileStatus], - startVersion: Long = 1, - endVersion: Long = 3, - expectedCommitFiles: Seq[FileStatus] - ): Unit = { + testName: String, + files: Seq[FileStatus], + startVersion: Long = 1, + endVersion: Long = 3, + expectedCommitFiles: Seq[FileStatus]): Unit = { test("getCommitFilesForVersionRange: " + testName) { assert( getCommitFilesForVersionRange( createMockFSListFromEngine(files), dataPath, startVersion, - endVersion - ).asScala sameElements expectedCommitFiles - ) + endVersion).asScala sameElements expectedCommitFiles) } } testGetCommitFiles( testName = "basic case", files = deltaFileStatuses(Seq(0, 1, 2, 3, 4, 5)), - expectedCommitFiles = deltaFileStatuses(Seq(1, 2, 3)) - ) + expectedCommitFiles = deltaFileStatuses(Seq(1, 2, 3))) testGetCommitFiles( testName = "basic case with checkpoint file", files = deltaFileStatuses(Seq(0, 1, 2, 3, 4, 5)) ++ singularCheckpointFileStatuses(Seq(2)), - expectedCommitFiles = deltaFileStatuses(Seq(1, 2, 3)) - ) + expectedCommitFiles = deltaFileStatuses(Seq(1, 2, 3))) testGetCommitFiles( testName = "basic case with non-log files", files = deltaFileStatuses(Seq(0, 1, 2, 3, 4, 5)) ++ deltaFileStatuses(Seq(2)) .map(fs => FileStatus.of(fs.getPath + ".crc", fs.getSize, fs.getModificationTime)), - expectedCommitFiles = deltaFileStatuses(Seq(1, 2, 3)) - ) + expectedCommitFiles = deltaFileStatuses(Seq(1, 2, 3))) testGetCommitFiles( testName = "version range size 1", files = deltaFileStatuses(Seq(0, 1, 2, 3, 4, 5)), startVersion = 0, endVersion = 0, - expectedCommitFiles = deltaFileStatuses(Seq(0)) - ) + expectedCommitFiles = deltaFileStatuses(Seq(0))) ///////////////////////////// // listDeltaLogFiles tests // diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala index e8aa0666b59..8993e6002a4 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala @@ -20,7 +20,7 @@ import java.util.{Arrays, Collections, Optional} import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import io.delta.kernel.data.{ColumnVector, ColumnarBatch} +import io.delta.kernel.data.{ColumnarBatch, ColumnVector} import io.delta.kernel.exceptions.{InvalidTableException, TableNotFoundException} import io.delta.kernel.expressions.Predicate import io.delta.kernel.internal.checkpoints.{CheckpointInstance, CheckpointMetaData, SidecarFile} @@ -30,6 +30,7 @@ import io.delta.kernel.internal.util.{FileNames, Utils} import io.delta.kernel.test.{BaseMockJsonHandler, BaseMockParquetHandler, MockFileSystemClientUtils, MockListFromFileSystemClient, VectorTestUtils} import io.delta.kernel.types.StructType import io.delta.kernel.utils.{CloseableIterator, FileStatus} + import org.scalatest.funsuite.AnyFunSuite class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { @@ -81,7 +82,8 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { .map(f => (f.getPath, f.getSize, f.getModificationTime)).sortBy(_._1) val actualCheckpointStatuses = logSegment.getCheckpoints.asScala .map(f => (f.getPath, f.getSize, f.getModificationTime)).sortBy(_._1) - assert(expectedCheckpointStatuses sameElements actualCheckpointStatuses, + assert( + expectedCheckpointStatuses sameElements actualCheckpointStatuses, s"expected:\n$expectedCheckpointStatuses\nactual:\n$actualCheckpointStatuses") expectedCheckpointVersion match { @@ -154,22 +156,20 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { }.getOrElse((Seq.empty, Seq.empty)) val logSegment = snapshotManager.getLogSegmentForVersion( - createMockFSListFromEngine(listFromProvider(deltas ++ checkpointFiles)("/"), + createMockFSListFromEngine( + listFromProvider(deltas ++ checkpointFiles)("/"), new MockSidecarParquetHandler(expectedSidecars), new MockSidecarJsonHandler(expectedSidecars)), - versionToLoad - ) + versionToLoad) val expectedDeltas = deltaFileStatuses( deltaVersions.filter { v => v > expectedCheckpointVersion.getOrElse(-1L) && v <= versionToLoad.orElse(Long.MaxValue) - } - ) + }) val expectedCheckpoints = expectedCheckpointVersion.map { v => if (expectedV2Checkpoint.nonEmpty) { expectedV2Checkpoint - } - else if (checkpointVersions.toSet.contains(v)) { + } else if (checkpointVersions.toSet.contains(v)) { singularCheckpointFileStatuses(Seq(v)) } else { multiCheckpointFileStatuses(Seq(v), numParts) @@ -182,8 +182,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { expectedDeltas = expectedDeltas, expectedCheckpoints = expectedCheckpoints, expectedCheckpointVersion = expectedCheckpointVersion, - expectedLastCommitTimestamp = versionToLoad.orElse(deltaVersions.max) * 10 - ) + expectedLastCommitTimestamp = versionToLoad.orElse(deltaVersions.max) * 10) } } @@ -195,8 +194,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { deltaVersions, checkpointVersions = Seq.empty, multiCheckpointVersions = Seq.empty, - versionToLoad = versionToLoad - ) + versionToLoad = versionToLoad) } /** @@ -216,8 +214,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { checkpointVersions = checkpointVersions, multiCheckpointVersions = Seq.empty, startCheckpoint = startCheckpoint, - versionToLoad = versionToLoad - ) + versionToLoad = versionToLoad) // test with multi-part checkpoint numParts=5 testWithCheckpoints( @@ -226,8 +223,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { multiCheckpointVersions = checkpointVersions, numParts = 5, startCheckpoint = startCheckpoint, - versionToLoad = versionToLoad - ) + versionToLoad = versionToLoad) // test with multi-part checkpoint numParts=1 testWithCheckpoints( @@ -236,8 +232,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { multiCheckpointVersions = checkpointVersions, numParts = 1, startCheckpoint = startCheckpoint, - versionToLoad = versionToLoad - ) + versionToLoad = versionToLoad) } /** @@ -251,8 +246,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { val e = intercept[T] { snapshotManager.getLogSegmentForVersion( createMockFSAndJsonEngineForLastCheckpoint(files, lastCheckpointVersion), - versionToLoad - ) + versionToLoad) } assert(e.getMessage.contains(expectedErrorMessageContains)) } @@ -273,101 +267,84 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("getLogSegmentForVersion: 000.json..010.json + checkpoint(10)") { testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 10L), - checkpointVersions = Seq(10) - ) + checkpointVersions = Seq(10)) testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 10L), checkpointVersions = Seq(10), - startCheckpoint = Optional.of(10) - ) + startCheckpoint = Optional.of(10)) testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 10L), checkpointVersions = Seq(10), - versionToLoad = Optional.of(10) - ) + versionToLoad = Optional.of(10)) testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 10L), checkpointVersions = Seq(10), startCheckpoint = Optional.of(10), - versionToLoad = Optional.of(10) - ) + versionToLoad = Optional.of(10)) testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 10L), checkpointVersions = Seq(10), - versionToLoad = Optional.of(6) - ) + versionToLoad = Optional.of(6)) testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 10L), checkpointVersions = Seq(10), startCheckpoint = Optional.of(10), - versionToLoad = Optional.of(6) - ) + versionToLoad = Optional.of(6)) } test("getLogSegmentForVersion: 000.json...20.json + checkpoint(10) + checkpoint(20)") { testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 20L), - checkpointVersions = Seq(10, 20) - ) + checkpointVersions = Seq(10, 20)) testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 20L), checkpointVersions = Seq(10, 20), - startCheckpoint = Optional.of(20) - ) + startCheckpoint = Optional.of(20)) // _last_checkpoint hasn't been updated yet testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 20L), checkpointVersions = Seq(10, 20), - startCheckpoint = Optional.of(10) - ) + startCheckpoint = Optional.of(10)) testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 20L), checkpointVersions = Seq(10, 20), - versionToLoad = Optional.of(15) - ) + versionToLoad = Optional.of(15)) testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 20L), checkpointVersions = Seq(10, 20), startCheckpoint = Optional.of(10), - versionToLoad = Optional.of(15) - ) + versionToLoad = Optional.of(15)) testWithSingularAndMultipartCheckpoint( deltaVersions = (0L to 20L), checkpointVersions = Seq(10, 20), startCheckpoint = Optional.of(20), - versionToLoad = Optional.of(15) - ) + versionToLoad = Optional.of(15)) } test("getLogSegmentForVersion: outdated _last_checkpoint that does not exist") { testWithSingularAndMultipartCheckpoint( deltaVersions = (20L until 25L), checkpointVersions = Seq(20), - startCheckpoint = Optional.of(10) - ) + startCheckpoint = Optional.of(10)) testWithSingularAndMultipartCheckpoint( deltaVersions = (20L until 25L), checkpointVersions = Seq(20), startCheckpoint = Optional.of(10), - versionToLoad = Optional.of(20) - ) + versionToLoad = Optional.of(20)) } test("getLogSegmentForVersion: 20.json...25.json + checkpoint(20)") { testWithSingularAndMultipartCheckpoint( deltaVersions = (20L to 25L), - checkpointVersions = Seq(20) - ) + checkpointVersions = Seq(20)) testWithSingularAndMultipartCheckpoint( deltaVersions = (20L to 25L), checkpointVersions = Seq(20), - startCheckpoint = Optional.of(20) - ) + startCheckpoint = Optional.of(20)) testWithSingularAndMultipartCheckpoint( deltaVersions = (20L to 25L), checkpointVersions = Seq(20), - versionToLoad = Optional.of(23) - ) + versionToLoad = Optional.of(23)) } test("getLogSegmentForVersion: empty delta log") { @@ -388,14 +365,12 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { testExpectedError[TableNotFoundException]( files, expectedErrorMessageContains = - "No delta files found in the directory: /fake/path/to/table/_delta_log" - ) + "No delta files found in the directory: /fake/path/to/table/_delta_log") testExpectedError[TableNotFoundException]( files, versionToLoad = Optional.of(5), expectedErrorMessageContains = - "No delta files found in the directory: /fake/path/to/table/_delta_log" - ) + "No delta files found in the directory: /fake/path/to/table/_delta_log") } test("getLogSegmentForVersion: versionToLoad higher than possible") { @@ -403,14 +378,12 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { files = deltaFileStatuses(Seq(0L)), versionToLoad = Optional.of(15), expectedErrorMessageContains = - "Cannot load table version 15 as it does not exist. The latest available version is 0" - ) + "Cannot load table version 15 as it does not exist. The latest available version is 0") testExpectedError[RuntimeException]( files = deltaFileStatuses((10L until 13L)) ++ singularCheckpointFileStatuses(Seq(10L)), versionToLoad = Optional.of(15), expectedErrorMessageContains = - "Cannot load table version 15 as it does not exist. The latest available version is 12" - ) + "Cannot load table version 15 as it does not exist. The latest available version is 12") } test("getLogSegmentForVersion: start listing from _last_checkpoint when it is provided") { @@ -425,7 +398,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { if (filePath < FileNames.listingPrefix(logPath, lastCheckpointVersion)) { throw new RuntimeException( s"Listing from before the checkpoint version referenced by _last_checkpoint. " + - s"Last checkpoint version: $lastCheckpointVersion. Listing from: $filePath") + s"Last checkpoint version: $lastCheckpointVersion. Listing from: $filePath") } listFromProvider(files)(filePath) } @@ -433,9 +406,9 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { val logSegment = snapshotManager.getLogSegmentForVersion( mockEngine( jsonHandler = new MockReadLastCheckpointFileJsonHandler( - lastCheckpointFileStatus.getPath, lastCheckpointVersion), - fileSystemClient = new MockListFromFileSystemClient(listFrom) - ), + lastCheckpointFileStatus.getPath, + lastCheckpointVersion), + fileSystemClient = new MockListFromFileSystemClient(listFrom)), Optional.empty() /* versionToLoad */ ) @@ -445,8 +418,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { expectedDeltas = deltaFileStatuses(21L until 25L), expectedCheckpoints = singularCheckpointFileStatuses(Seq(20L)), expectedCheckpointVersion = Some(20), - expectedLastCommitTimestamp = 240L - ) + expectedLastCommitTimestamp = 240L) } } @@ -455,23 +427,20 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { (0L to 50L), Seq(10, 30, 50), Seq(20, 40), - numParts = 5 - ) + numParts = 5) testWithCheckpoints( (0L to 50L), Seq(10, 30, 50), Seq(20, 40), numParts = 5, - startCheckpoint = Optional.of(40) - ) + startCheckpoint = Optional.of(40)) } test("getLogSegmentForVersion: versionToLoad not constructable from history") { testExpectedError[RuntimeException]( deltaFileStatuses(20L until 25L) ++ singularCheckpointFileStatuses(Seq(20L)), versionToLoad = Optional.of(15), - expectedErrorMessageContains = "Cannot load table version 15" - ) + expectedErrorMessageContains = "Cannot load table version 15") } /* ------------------- V2 CHECKPOINT TESTS ------------------ */ @@ -585,8 +554,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { files = singularCheckpointFileStatuses(Seq(10L)), lastCheckpointVersion.map(Long.box), versionToLoad.map(Long.box), - expectedErrorMessageContains = "Missing delta file for version 10" - ) + expectedErrorMessageContains = "Missing delta file for version 10") } } } @@ -601,8 +569,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { fileList, lastCheckpointVersion.map(Long.box), versionToLoad.map(Long.box), - expectedErrorMessageContains = "versions are not contiguous: ([11, 13])" - ) + expectedErrorMessageContains = "versions are not contiguous: ([11, 13])") } } } @@ -615,8 +582,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { fileList, lastCheckpointVersion.map(Long.box), versionToLoad.map(Long.box), - expectedErrorMessageContains = "Missing delta file for version 10" - ) + expectedErrorMessageContains = "Missing delta file for version 10") } } } @@ -637,17 +603,14 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("getLogSegmentForVersion: corrupted log missing json files / no way to construct history") { testExpectedError[InvalidTableException]( deltaFileStatuses(1L until 10L), - expectedErrorMessageContains = "Cannot compute snapshot. Missing delta file version 0." - ) + expectedErrorMessageContains = "Cannot compute snapshot. Missing delta file version 0.") testExpectedError[InvalidTableException]( deltaFileStatuses(15L until 25L) ++ singularCheckpointFileStatuses(Seq(20L)), versionToLoad = Optional.of(17), - expectedErrorMessageContains = "Cannot compute snapshot. Missing delta file version 0." - ) + expectedErrorMessageContains = "Cannot compute snapshot. Missing delta file version 0.") testExpectedError[InvalidTableException]( deltaFileStatuses((0L until 5L) ++ (6L until 9L)), - expectedErrorMessageContains = "are not contiguous" - ) + expectedErrorMessageContains = "are not contiguous") // corrupt incomplete multi-part checkpoint val corruptedCheckpointStatuses = FileNames.checkpointFileWithParts(logPath, 10, 5).asScala .map(p => FileStatus.of(p.toString, 10, 10)) @@ -655,26 +618,22 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { val deltas = deltaFileStatuses(10L to 13L) testExpectedError[InvalidTableException]( corruptedCheckpointStatuses ++ deltas, - expectedErrorMessageContains = "Cannot compute snapshot. Missing delta file version 0." - ) + expectedErrorMessageContains = "Cannot compute snapshot. Missing delta file version 0.") } test("getLogSegmentForVersion: corrupt log but reading outside corrupted range") { testNoCheckpoint( deltaVersions = (0L until 5L) ++ (6L until 9L), - versionToLoad = Optional.of(4) - ) + versionToLoad = Optional.of(4)) testWithSingularAndMultipartCheckpoint( deltaVersions = 15L until 25L, checkpointVersions = Seq(20), - versionToLoad = Optional.of(22) - ) + versionToLoad = Optional.of(22)) testWithSingularAndMultipartCheckpoint( deltaVersions = 15L until 25L, checkpointVersions = Seq(20), startCheckpoint = Optional.of(20), - versionToLoad = Optional.of(22) - ) + versionToLoad = Optional.of(22)) } test("getLogSegmentForVersion: corrupt _last_checkpoint (is after existing versions)") { @@ -683,8 +642,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { testWithSingularAndMultipartCheckpoint( (0L until 25L), Seq(10L, 20L), - startCheckpoint = Optional.of(30) - ) + startCheckpoint = Optional.of(30)) } test("getLogSegmentForVersion: corrupt _last_checkpoint refers to in range version " + @@ -693,8 +651,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { testExpectedError[RuntimeException]( deltaFileStatuses(0L until 25L) ++ singularCheckpointFileStatuses(Seq(10L)), lastCheckpointVersion = Optional.of(20), - expectedErrorMessageContains = "Missing checkpoint at version 20" - ) + expectedErrorMessageContains = "Missing checkpoint at version 20") // _last_checkpoint refers to incomplete multi-part checkpoint at version 20 that is missing val corruptedCheckpointStatuses = FileNames.checkpointFileWithParts(logPath, 20, 5).asScala .map(p => FileStatus.of(p.toString, 10, 10)) @@ -703,8 +660,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { files = corruptedCheckpointStatuses ++ deltaFileStatuses(10L to 20L) ++ singularCheckpointFileStatuses(Seq(10L)), lastCheckpointVersion = Optional.of(20), - expectedErrorMessageContains = "Missing checkpoint at version 20" - ) + expectedErrorMessageContains = "Missing checkpoint at version 20") } test("getLogSegmentForVersion: corrupted incomplete multi-part checkpoint with no" + @@ -713,8 +669,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { /* (corruptedCheckpointVersion, validCheckpointVersions, deltaVersions, lastCheckpointV) */ (20, Seq(10), (10L to 20L), Optional.empty()), (20, Seq(10), (10L to 20L), Optional.of(10)), - (10, Seq.empty, (0L to 10L), Optional.empty()) - ) + (10, Seq.empty, (0L to 10L), Optional.empty())) cases.foreach { case (corruptedVersion, validVersions, deltaVersions, lastCheckpointVersion) => val corruptedCheckpoint = FileNames.checkpointFileWithParts(logPath, corruptedVersion, 5) .asScala @@ -725,8 +680,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { val allFiles = deltas ++ corruptedCheckpoint ++ checkpoints val logSegment = snapshotManager.getLogSegmentForVersion( createMockFSAndJsonEngineForLastCheckpoint(allFiles, lastCheckpointVersion), - Optional.empty() - ) + Optional.empty()) val checkpointVersion = validVersions.sorted.lastOption checkLogSegment( logSegment, @@ -735,8 +689,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { deltaVersions.filter(_ > checkpointVersion.getOrElse(-1L))), expectedCheckpoints = checkpoints, expectedCheckpointVersion = checkpointVersion, - expectedLastCommitTimestamp = deltaVersions.max*10 - ) + expectedLastCommitTimestamp = deltaVersions.max * 10) } } @@ -744,8 +697,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { val exMsg = intercept[InvalidTableException] { snapshotManager.getLogSegmentForVersion( createMockFSAndJsonEngineForLastCheckpoint(Seq.empty, Optional.of(1)), - Optional.empty() - ) + Optional.empty()) }.getMessage assert(exMsg.contains("Missing checkpoint at version 1")) @@ -754,7 +706,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { trait SidecarIteratorProvider extends VectorTestUtils { def singletonSidecarIterator(sidecars: Seq[FileStatus]) - : CloseableIterator[ColumnarBatch] = Utils.singletonCloseableIterator( + : CloseableIterator[ColumnarBatch] = Utils.singletonCloseableIterator( new ColumnarBatch { override def getSchema: StructType = SidecarFile.READ_SCHEMA @@ -772,7 +724,7 @@ trait SidecarIteratorProvider extends VectorTestUtils { } class MockSidecarParquetHandler(sidecars: Seq[FileStatus]) - extends BaseMockParquetHandler with SidecarIteratorProvider { + extends BaseMockParquetHandler with SidecarIteratorProvider { override def readParquetFiles( fileIter: CloseableIterator[FileStatus], physicalSchema: StructType, @@ -781,7 +733,7 @@ class MockSidecarParquetHandler(sidecars: Seq[FileStatus]) } class MockSidecarJsonHandler(sidecars: Seq[FileStatus]) - extends BaseMockJsonHandler + extends BaseMockJsonHandler with SidecarIteratorProvider { override def readJsonFiles( fileIter: CloseableIterator[FileStatus], @@ -793,7 +745,7 @@ class MockSidecarJsonHandler(sidecars: Seq[FileStatus]) class MockReadLastCheckpointFileJsonHandler( lastCheckpointPath: String, lastCheckpointVersion: Long) - extends BaseMockJsonHandler with VectorTestUtils { + extends BaseMockJsonHandler with VectorTestUtils { override def readJsonFiles( fileIter: CloseableIterator[FileStatus], physicalSchema: StructType, @@ -814,7 +766,6 @@ class MockReadLastCheckpointFileJsonHandler( } override def getSize: Int = 1 - } - ) + }) } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableConfigSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableConfigSuite.scala index 9587395a64b..da187841b58 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableConfigSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableConfigSuite.scala @@ -15,10 +15,11 @@ */ package io.delta.kernel.internal +import scala.collection.JavaConverters._ + import io.delta.kernel.exceptions.KernelException -import org.scalatest.funsuite.AnyFunSuite -import scala.collection.JavaConverters._ +import org.scalatest.funsuite.AnyFunSuite class TableConfigSuite extends AnyFunSuite { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableImplSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableImplSuite.scala index 0c20e2a4f08..fc770c63002 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableImplSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableImplSuite.scala @@ -15,19 +15,20 @@ */ package io.delta.kernel.internal -import io.delta.kernel.test.{MockFileSystemClientUtils, MockListFromResolvePathFileSystemClient} -import io.delta.kernel.utils.FileStatus import io.delta.kernel.Table import io.delta.kernel.exceptions.KernelException +import io.delta.kernel.test.{MockFileSystemClientUtils, MockListFromResolvePathFileSystemClient} +import io.delta.kernel.utils.FileStatus + import org.scalatest.funsuite.AnyFunSuite class TableImplSuite extends AnyFunSuite with MockFileSystemClientUtils { def checkGetVersionBeforeOrAtTimestamp( - fileList: Seq[FileStatus], - timestamp: Long, - expectedVersion: Option[Long] = None, - expectedErrorMessageContains: Option[String] = None): Unit = { + fileList: Seq[FileStatus], + timestamp: Long, + expectedVersion: Option[Long] = None, + expectedErrorMessageContains: Option[String] = None): Unit = { // Check our inputs are as expected assert(expectedVersion.isEmpty || expectedErrorMessageContains.isEmpty) assert(expectedVersion.nonEmpty || expectedErrorMessageContains.nonEmpty) @@ -47,10 +48,10 @@ class TableImplSuite extends AnyFunSuite with MockFileSystemClientUtils { } def checkGetVersionAtOrAfterTimestamp( - fileList: Seq[FileStatus], - timestamp: Long, - expectedVersion: Option[Long] = None, - expectedErrorMessageContains: Option[String] = None): Unit = { + fileList: Seq[FileStatus], + timestamp: Long, + expectedVersion: Option[Long] = None, + expectedErrorMessageContains: Option[String] = None): Unit = { // Check our inputs are as expected assert(expectedVersion.isEmpty || expectedErrorMessageContains.isEmpty) assert(expectedVersion.nonEmpty || expectedErrorMessageContains.nonEmpty) @@ -71,8 +72,11 @@ class TableImplSuite extends AnyFunSuite with MockFileSystemClientUtils { test("getVersionBeforeOrAtTimestamp: basic case from 0") { val deltaFiles = deltaFileStatuses(Seq(0L, 1L)) - checkGetVersionBeforeOrAtTimestamp(deltaFiles, -1, - expectedErrorMessageContains = Some("is before the earliest available version 0")) // before 0 + checkGetVersionBeforeOrAtTimestamp( + deltaFiles, + -1, + expectedErrorMessageContains = Some("is before the earliest available version 0") + ) // before 0 checkGetVersionBeforeOrAtTimestamp(deltaFiles, 0, expectedVersion = Some(0)) // at 0 checkGetVersionBeforeOrAtTimestamp(deltaFiles, 5, expectedVersion = Some(0)) // btw 0, 1 checkGetVersionBeforeOrAtTimestamp(deltaFiles, 10, expectedVersion = Some(1)) // at 1 @@ -85,8 +89,11 @@ class TableImplSuite extends AnyFunSuite with MockFileSystemClientUtils { checkGetVersionAtOrAfterTimestamp(deltaFiles, 0, expectedVersion = Some(0)) // at 0 checkGetVersionAtOrAfterTimestamp(deltaFiles, 5, expectedVersion = Some(1)) // btw 0, 1 checkGetVersionAtOrAfterTimestamp(deltaFiles, 10, expectedVersion = Some(1)) // at 1 - checkGetVersionAtOrAfterTimestamp(deltaFiles, 11, - expectedErrorMessageContains = Some("is after the latest available version 1")) // after 1 + checkGetVersionAtOrAfterTimestamp( + deltaFiles, + 11, + expectedErrorMessageContains = Some("is after the latest available version 1") + ) // after 1 } test("getVersionBeforeOrAtTimestamp: w/ checkpoint + w/o checkpoint") { @@ -94,7 +101,9 @@ class TableImplSuite extends AnyFunSuite with MockFileSystemClientUtils { deltaFileStatuses(Seq(10L, 11L, 12L)) ++ singularCheckpointFileStatuses(Seq(10L)), deltaFileStatuses(Seq(10L, 11L, 12L)) // checks that does not need to be recreatable ).foreach { deltaFiles => - checkGetVersionBeforeOrAtTimestamp(deltaFiles, 99, // before 10 + checkGetVersionBeforeOrAtTimestamp( + deltaFiles, + 99, // before 10 expectedErrorMessageContains = Some("is before the earliest available version 10")) checkGetVersionBeforeOrAtTimestamp(deltaFiles, 100, expectedVersion = Some(10)) // at 10 checkGetVersionBeforeOrAtTimestamp(deltaFiles, 105, expectedVersion = Some(10)) // btw 10, 11 @@ -116,8 +125,11 @@ class TableImplSuite extends AnyFunSuite with MockFileSystemClientUtils { checkGetVersionAtOrAfterTimestamp(deltaFiles, 110, expectedVersion = Some(11)) // at 11 checkGetVersionAtOrAfterTimestamp(deltaFiles, 115, expectedVersion = Some(12)) // btw 11, 12 checkGetVersionAtOrAfterTimestamp(deltaFiles, 120, expectedVersion = Some(12)) // at 12 - checkGetVersionAtOrAfterTimestamp(deltaFiles, 125, - expectedErrorMessageContains = Some("is after the latest available version 12")) // after 12 + checkGetVersionAtOrAfterTimestamp( + deltaFiles, + 125, + expectedErrorMessageContains = Some("is after the latest available version 12") + ) // after 12 } } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/AddFileSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/AddFileSuite.scala index dfcb756ab98..8a0740f3d9b 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/AddFileSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/AddFileSuite.scala @@ -15,7 +15,8 @@ */ package io.delta.kernel.internal.actions -import org.scalatest.funsuite.AnyFunSuite +import java.lang.{Boolean => JBoolean, Long => JLong} +import java.util.Optional import scala.collection.JavaConverters._ @@ -24,8 +25,7 @@ import io.delta.kernel.internal.util.VectorUtils import io.delta.kernel.internal.util.VectorUtils.stringStringMapValue import io.delta.kernel.utils.DataFileStatistics.deserializeFromJson -import java.util.Optional -import java.lang.{Boolean => JBoolean, Long => JLong} +import org.scalatest.funsuite.AnyFunSuite class AddFileSuite extends AnyFunSuite { @@ -42,8 +42,7 @@ class AddFileSuite extends AnyFunSuite { tags: Option[Map[String, String]] = Option.empty, baseRowId: Option[Long] = Option.empty, defaultRowCommitVersion: Option[Long] = Option.empty, - stats: Option[String] = Option.empty - ): Row = { + stats: Option[String] = Option.empty): Row = { def toJavaOptional[T](option: Option[T]): Optional[T] = option match { case Some(value) => Optional.of(value) case None => Optional.empty() @@ -59,8 +58,7 @@ class AddFileSuite extends AnyFunSuite { toJavaOptional(tags.map(_.asJava).map(stringStringMapValue)), toJavaOptional(baseRowId.asInstanceOf[Option[JLong]]), toJavaOptional(defaultRowCommitVersion.asInstanceOf[Option[JLong]]), - deserializeFromJson(stats.getOrElse("")) - ) + deserializeFromJson(stats.getOrElse(""))) } test("getters can read AddFile's fields from the backing row") { @@ -74,8 +72,7 @@ class AddFileSuite extends AnyFunSuite { tags = Option(Map("tag1" -> "value1")), baseRowId = Option(30L), defaultRowCommitVersion = Option(40L), - stats = Option("{\"numRecords\":100}") - ) + stats = Option("{\"numRecords\":100}")) val addFile = new AddFile(addFileRow) assert(addFile.getPath === "test/path") @@ -108,8 +105,7 @@ class AddFileSuite extends AnyFunSuite { generateTestAddFileRow( path = "test/path", baseRowId = Option(0L), - defaultRowCommitVersion = Option(0L) - ) + defaultRowCommitVersion = Option(0L)) var addFileAction = new AddFile(baseAddFileRow) (1L until 10L).foreach { i => @@ -133,8 +129,7 @@ class AddFileSuite extends AnyFunSuite { tags = Option(Map("tag1" -> "value1")), baseRowId = Option(12345L), defaultRowCommitVersion = Option(67890L), - stats = Option("{\"numRecords\":10000}") - ) + stats = Option("{\"numRecords\":10000}")) val addFile = new AddFile(addFileRow) val expectedString = "AddFile{" + "path='test/path', " + @@ -156,8 +151,7 @@ class AddFileSuite extends AnyFunSuite { size = 100L, partitionValues = Map("a" -> "1"), baseRowId = Option(12345L), - stats = Option("{\"numRecords\":100}") - ) + stats = Option("{\"numRecords\":100}")) // Create an identical AddFile val addFileRow2 = generateTestAddFileRow( @@ -165,8 +159,7 @@ class AddFileSuite extends AnyFunSuite { size = 100L, partitionValues = Map("a" -> "1"), baseRowId = Option(12345L), - stats = Option("{\"numRecords\":100}") - ) + stats = Option("{\"numRecords\":100}")) // Create a AddFile with different path val addFileRowDiffPath = generateTestAddFileRow( @@ -174,8 +167,7 @@ class AddFileSuite extends AnyFunSuite { size = 100L, partitionValues = Map("a" -> "1"), baseRowId = Option(12345L), - stats = Option("{\"numRecords\":100}") - ) + stats = Option("{\"numRecords\":100}")) // Create a AddFile with different partition values, which is handled specially in equals() val addFileRowDiffPartition = generateTestAddFileRow( @@ -183,8 +175,7 @@ class AddFileSuite extends AnyFunSuite { size = 100L, partitionValues = Map("x" -> "0"), baseRowId = Option(12345L), - stats = Option("{\"numRecords\":100}") - ) + stats = Option("{\"numRecords\":100}")) val addFile1 = new AddFile(addFileRow1) val addFile2 = new AddFile(addFileRow2) @@ -210,16 +201,14 @@ class AddFileSuite extends AnyFunSuite { size = 100L, partitionValues = Map("a" -> "1"), baseRowId = Option(12345L), - stats = Option("{\"numRecords\":100}") - ) + stats = Option("{\"numRecords\":100}")) val addFileRow2 = generateTestAddFileRow( path = "test/path", size = 100L, partitionValues = Map("a" -> "1"), baseRowId = Option(12345L), - stats = Option("{\"numRecords\":100}") - ) + stats = Option("{\"numRecords\":100}")) val addFile1 = new AddFile(addFileRow1) val addFile2 = new AddFile(addFileRow2) diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/ProtocolSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/ProtocolSuite.scala new file mode 100644 index 00000000000..3c9768f091b --- /dev/null +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/ProtocolSuite.scala @@ -0,0 +1,372 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.actions + +import scala.collection.JavaConverters._ + +import io.delta.kernel.internal.tablefeatures.TableFeatures + +import org.scalatest.funsuite.AnyFunSuite + +class ProtocolSuite extends AnyFunSuite { + ///////////////////////////////////////////////////////////////////////////////////////////////// + // Tests for TableFeature related methods on Protocol // + ///////////////////////////////////////////////////////////////////////////////////////////////// + + // Invalid protocol versions/features throw validation errors + Seq( + // Test format: + // minReaderVersion, minWriterVersion, readerFeatures, writerFeatures, expectedErrorMsg + (0, 1, Set(), Set(), "minReaderVersion should be at least 1"), + (1, 0, Set(), Set(), "minWriterVersion should be at least 1"), + ( + // writer version doesn't support writer features + 1, + 2, + Set("columnMapping"), + Set(), + "Reader features are not supported for the reader version: 1"), + ( + // writer version doesn't support writer features + 1, + 2, + Set(), + Set("columnMapping"), + "Writer features are not supported for the writer version: 2"), + // you can't have reader version with feature support, but not the writer version + (3, 5, Set(), Set(), "writer version doesn't support writer features: 5"), + // columnMapping feature is not supported for reader version 1 + (1, 5, Set(), Set(), "Reader version 1 does not support readerWriter feature columnMapping"), + ( + // readerWriter feature columnMapping is missing from the readerFeatures set + 3, + 7, + Set(), + Set("columnMapping"), + "ReaderWriter feature columnMapping is not present in readerFeatures"), + // minReaderVersion doesn't support readerWriter feature columnMapping requirement + ( + 1, + 7, + Set(), + Set("columnMapping"), + "Reader version 1 does not support readerWriter feature columnMapping")).foreach { + case ( + readerVersion, + writerVersion, + readerFeatures: Set[String], + writerFeatures: Set[String], + expectedError) => + test(s"Invalid protocol versions " + + s"($readerVersion, $writerVersion, $readerFeatures, $writerFeatures)") { + val protocol = + new Protocol(readerVersion, writerVersion, readerFeatures.asJava, writerFeatures.asJava) + val e = intercept[IllegalArgumentException] { + protocol.validate() + } + assert(e.getMessage === expectedError) + } + } + + // Tests for getImplicitlySupportedFeatures, getExplicitlySupportedFeatures and + // getImplicitlyAndExplicitlySupportedFeatures + Seq( + // Test format: + // (minReaderVersion, minWriterVersion, expected features) + (1, 1, Set()), + (1, 2, Set("appendOnly", "invariants")), + (1, 3, Set("appendOnly", "invariants", "checkConstraints")), + ( + 1, + 4, + Set("appendOnly", "invariants", "checkConstraints", "changeDataFeed", "generatedColumns")), + ( + 2, + 5, + Set( + "appendOnly", + "invariants", + "checkConstraints", + "changeDataFeed", + "generatedColumns", + "columnMapping")), + ( + 2, + 6, + Set( + "appendOnly", + "invariants", + "checkConstraints", + "changeDataFeed", + "generatedColumns", + "columnMapping", + "identityColumns"))).foreach { + case (minReaderVersion, minWriterVersion, expectedFeatures) => + test(s"getImplicitlySupportedFeatures with minReaderVersion $minReaderVersion and " + + s"minWriterVersion $minWriterVersion") { + val protocol = new Protocol(minReaderVersion, minWriterVersion) + assert( + protocol.getImplicitlySupportedFeatures.asScala.map(_.featureName()) === expectedFeatures) + + assert( + protocol.getImplicitlyAndExplicitlySupportedFeatures.asScala.map(_.featureName()) === + expectedFeatures) + + assert( + protocol.getExplicitlySupportedFeatures.asScala.map(_.featureName()) === Set()) + } + } + + Seq( + // Test format: readerFeatures, writerFeatures, expected set + (Set(), Set(), Set()), + (Set(), Set("rowTracking"), Set("rowTracking")), + (Set(), Set("checkConstraints", "rowTracking"), Set("checkConstraints", "rowTracking")), + ( + Set("columnMapping"), + Set("columnMapping", "domainMetadata"), + Set("columnMapping", "domainMetadata"))).foreach { + case ( + readerFeatures: Set[String], + writerFeatures: Set[String], + expectedFeatureSet: Set[String]) => + test(s"getExplicitlySupportedFeatures $readerFeatures $writerFeatures") { + val protocol = new Protocol(3, 7, readerFeatures.asJava, writerFeatures.asJava) + assert( + protocol.getExplicitlySupportedFeatures.asScala.map(_.featureName()) === + expectedFeatureSet) + + assert( + protocol.getImplicitlyAndExplicitlySupportedFeatures.asScala.map(_.featureName()) === + expectedFeatureSet) + + assert(protocol.getImplicitlySupportedFeatures.asScala.map(_.featureName()) === Set()) + } + } + + // Tests for `normalized + Seq( + // Test format: input, expected output out of the `normalized` + + // If the protocol has no table features, then the normalized shouldn't change + (1, 1, Set[String](), Set[String]()) -> (1, 1, Set[String](), Set[String]()), + (1, 2, Set[String](), Set[String]()) -> (1, 2, Set[String](), Set[String]()), + (2, 5, Set[String](), Set[String]()) -> (2, 5, Set[String](), Set[String]()), + + // If the protocol has table features, then the normalized may or + // may not have the table features + (3, 7, Set[String](), Set("appendOnly", "invariants")) -> + (1, 2, Set[String](), Set[String]()), + (3, 7, Set[String](), Set("appendOnly", "invariants", "checkConstraints")) -> + (1, 3, Set[String](), Set[String]()), + ( + 3, + 7, + Set[String](), + Set("appendOnly", "invariants", "checkConstraints", "changeDataFeed", "generatedColumns")) -> + (1, 4, Set[String](), Set[String]()), + ( + 3, + 7, + Set("columnMapping"), + Set( + "appendOnly", + "invariants", + "checkConstraints", + "changeDataFeed", + "generatedColumns", + "columnMapping")) -> + (2, 5, Set[String](), Set[String]()), + + // reader version is downgraded + // can't downgrade the writer version, because version 2 (appendOnly) also has support for + // invariants which is not supported in the writer features in the input + (1, 7, Set[String](), Set("appendOnly")) -> (1, 7, Set[String](), Set[String]("appendOnly")), + (3, 7, Set("columnMapping"), Set("columnMapping")) -> + (2, 7, Set[String](), Set("columnMapping")), + (3, 7, Set("columnMapping"), Set("columnMapping", "domainMetadata")) -> + (2, 7, Set[String](), Set("columnMapping", "domainMetadata"))).foreach { + case ( + (readerVersion, writerVersion, readerFeatures, writerFeatures), + ( + expReaderVersion, + expWriterVersion, + expReaderFeatures, + expWriterFeatures)) => + test(s"normalized $readerVersion $writerVersion $readerFeatures $writerFeatures") { + val protocol = + new Protocol(readerVersion, writerVersion, readerFeatures.asJava, writerFeatures.asJava) + val normalized = protocol.normalized() + assert(normalized.getMinReaderVersion === expReaderVersion) + assert(normalized.getMinWriterVersion === expWriterVersion) + assert(normalized.getReaderFeatures.asScala === expReaderFeatures) + assert(normalized.getWriterFeatures.asScala === expWriterFeatures) + } + } + + // Tests for `denormalized` + Seq( + // Test format: input, expected output out of the `denormalized` + (1, 1, Set[String](), Set[String]()) -> (1, 7, Set[String](), Set[String]()), + (1, 2, Set[String](), Set[String]()) -> (1, 7, Set[String](), Set("appendOnly", "invariants")), + (2, 5, Set[String](), Set[String]()) -> ( + 2, + 7, + Set[String](), + Set( + "appendOnly", + "invariants", + "checkConstraints", + "changeDataFeed", + "generatedColumns", + "columnMapping")), + + // invalid protocol versions (2, 3) + (2, 3, Set[String](), Set[String]()) -> ( + 1, + 7, + Set[String](), + Set("appendOnly", "invariants", "checkConstraints")), + + // shouldn't change the protocol already has the table feature set support + (3, 7, Set[String](), Set("appendOnly", "invariants")) -> + (3, 7, Set[String](), Set("appendOnly", "invariants")), + (3, 7, Set[String](), Set("appendOnly", "invariants", "checkConstraints")) -> + (3, 7, Set[String](), Set("appendOnly", "invariants", "checkConstraints"))).foreach { + case ( + (readerVersion, writerVersion, readerFeatures, writerFeatures), + ( + expReaderVersion, + expWriterVersion, + expReaderFeatures, + expWriterFeatures)) => + test(s"denormalized $readerVersion $writerVersion $readerFeatures $writerFeatures") { + val protocol = + new Protocol(readerVersion, writerVersion, readerFeatures.asJava, writerFeatures.asJava) + val denormalized = protocol.denormalized() + assert(denormalized.getMinReaderVersion === expReaderVersion) + assert(denormalized.getMinWriterVersion === expWriterVersion) + assert(denormalized.getReaderFeatures.asScala === expReaderFeatures) + assert(denormalized.getWriterFeatures.asScala === expWriterFeatures) + } + } + + // Tests for `withFeature` and `normalized` + Seq( + // can't downgrade the writer version, because version 2 (appendOnly) also has support for + // invariants which is not supported in the writer features in the input + Set("appendOnly") -> (1, 7, Set[String](), Set("appendOnly")), + Set("invariants") -> (1, 7, Set[String](), Set[String]("invariants")), + Set("appendOnly", "invariants") -> (1, 2, Set[String](), Set[String]()), + Set("checkConstraints") -> (1, 7, Set[String](), Set("checkConstraints")), + Set("changeDataFeed") -> (1, 7, Set[String](), Set("changeDataFeed")), + Set("appendOnly", "invariants", "checkConstraints") -> (1, 3, Set[String](), Set[String]()), + Set("generatedColumns") -> (1, 7, Set[String](), Set("generatedColumns")), + Set("columnMapping") -> (2, 7, Set(), Set("columnMapping")), + Set("identityColumns") -> (1, 7, Set[String](), Set[String]("identityColumns")), + + // expect the dependency features also to be supported + Set("icebergCompatV2") -> + (2, 7, Set[String](), Set[String]("icebergCompatV2", "columnMapping")), + Set("rowTracking") -> ( + 1, + 7, + Set[String](), + Set[String]("rowTracking", "domainMetadata"))).foreach { + case (features, (expReaderVersion, expWriterVersion, expReaderFeatures, expWriterFeatures)) => + test(s"withFeature $features") { + val protocol = new Protocol(3, 7) + val updated = protocol + .withFeatures(features.map(TableFeatures.getTableFeature).asJava) + .normalized() + assert(updated.getMinReaderVersion === expReaderVersion) + assert(updated.getMinWriterVersion === expWriterVersion) + assert(updated.getReaderFeatures.asScala === expReaderFeatures) + assert(updated.getWriterFeatures.asScala === expWriterFeatures) + } + } + + test("withFeature - can't add a feature at the current version") { + val protocol = new Protocol(1, 2) + val e = intercept[UnsupportedOperationException] { + protocol.withFeatures(Set(TableFeatures.getTableFeature("columnMapping")).asJava) + } + assert(e.getMessage === "TableFeature requires higher reader protocol version") + } + + // Tests for `merge` (also tests denormalized and normalized) + Seq( + // Test format: (protocol1, protocol2) -> expected merged protocol + ( + (1, 1, Set[String](), Set[String]()), + (1, 2, Set[String](), Set[String]())) -> + (1, 2, Set[String](), Set[String]()), + ((1, 2, Set[String](), Set[String]()), (1, 3, Set[String](), Set[String]())) -> + (1, 3, Set[String](), Set[String]()), + ((1, 4, Set[String](), Set[String]()), (2, 5, Set[String](), Set[String]())) -> + (2, 5, Set[String](), Set[String]()), + ((1, 4, Set[String](), Set[String]()), (2, 6, Set[String](), Set[String]())) -> + (2, 6, Set[String](), Set[String]()), + ((1, 2, Set[String](), Set[String]()), (1, 7, Set[String](), Set("invariants"))) -> + (1, 2, Set[String](), Set[String]()), + ((1, 2, Set[String](), Set[String]()), (3, 7, Set("columnMapping"), Set("columnMapping"))) -> + (2, 7, Set[String](), Set("columnMapping", "invariants", "appendOnly")), + ( + (1, 2, Set[String](), Set[String]()), + (3, 7, Set("columnMapping"), Set("columnMapping", "domainMetadata"))) -> + (2, 7, Set[String](), Set("domainMetadata", "columnMapping", "invariants", "appendOnly")), + ( + (2, 5, Set[String](), Set[String]()), + (3, 7, Set("v2Checkpoint"), Set("v2Checkpoint", "domainMetadata"))) -> + ( + 3, + 7, + Set("columnMapping", "v2Checkpoint"), + Set( + "domainMetadata", + "columnMapping", + "v2Checkpoint", + "invariants", + "appendOnly", + "checkConstraints", + "changeDataFeed", + "generatedColumns"))).foreach({ + case ( + ( + (readerVersion1, writerVersion1, readerFeatures1, writerFeatures1), + (readerVersion2, writerVersion2, readerFeatures2, writerFeatures2)), + (expReaderVersion, expWriterVersion, expReaderFeatures, expWriterFeatures)) => + test(s"merge $readerVersion1 $writerVersion1 $readerFeatures1 $writerFeatures1 " + + s"$readerVersion2 $writerVersion2 $readerFeatures2 $writerFeatures2") { + val protocol1 = + new Protocol( + readerVersion1, + writerVersion1, + readerFeatures1.asJava, + writerFeatures1.asJava) + val protocol2 = new Protocol( + readerVersion2, + writerVersion2, + readerFeatures2.asJava, + writerFeatures2.asJava) + val merged = protocol1.merge(protocol2) + assert(merged.getMinReaderVersion === expReaderVersion) + assert(merged.getMinWriterVersion === expWriterVersion) + assert(merged.getReaderFeatures.asScala === expReaderFeatures) + assert(merged.getWriterFeatures.asScala === expWriterFeatures) + } + }) +} diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointInstanceSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointInstanceSuite.scala index 739bf87eec9..b59bcab6914 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointInstanceSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointInstanceSuite.scala @@ -15,12 +15,14 @@ */ package io.delta.kernel.internal.checkpoints -import io.delta.kernel.internal.fs.Path -import org.scalatest.funsuite.AnyFunSuite - import java.util.Optional + import scala.collection.JavaConverters._ +import io.delta.kernel.internal.fs.Path + +import org.scalatest.funsuite.AnyFunSuite + class CheckpointInstanceSuite extends AnyFunSuite { private val FAKE_DELTA_LOG_PATH = new Path("/path/to/delta/log") @@ -28,11 +30,11 @@ class CheckpointInstanceSuite extends AnyFunSuite { test("checkpoint instance comparisons") { val ci1_single_1 = new CheckpointInstance(1, Optional.empty()) val ci1_withparts_2 = new CheckpointInstance(1, Optional.of(2)) - val ci1_v2_1 = new CheckpointInstance("01.checkpoint.abc.parquet" ) + val ci1_v2_1 = new CheckpointInstance("01.checkpoint.abc.parquet") val ci2_single_1 = new CheckpointInstance(2, Optional.empty()) val ci2_withparts_4 = new CheckpointInstance(2, Optional.of(4)) - val ci2_v2_1 = new CheckpointInstance("02.checkpoint.abc.parquet" ) + val ci2_v2_1 = new CheckpointInstance("02.checkpoint.abc.parquet") val ci2_v2_2 = new CheckpointInstance("02.checkpoint.def.parquet") val ci3_single_1 = new CheckpointInstance(3, Optional.empty()) @@ -54,11 +56,14 @@ class CheckpointInstanceSuite extends AnyFunSuite { // Everything is less than CheckpointInstance.MAX_VALUE Seq( - ci1_single_1, ci1_withparts_2, - ci2_single_1, ci2_withparts_4, - ci3_single_1, ci3_withparts_2, - ci1_v2_1, ci2_v2_1 - ).foreach(ci => assert(ci.compareTo(CheckpointInstance.MAX_VALUE) < 0)) + ci1_single_1, + ci1_withparts_2, + ci2_single_1, + ci2_withparts_4, + ci3_single_1, + ci3_withparts_2, + ci1_v2_1, + ci2_v2_1).foreach(ci => assert(ci.compareTo(CheckpointInstance.MAX_VALUE) < 0)) } test("checkpoint instance equality") { @@ -100,7 +105,8 @@ class CheckpointInstanceSuite extends AnyFunSuite { // multi-part checkpoint val multipartCheckpoint = new CheckpointInstance( - new Path(FAKE_DELTA_LOG_PATH, + new Path( + FAKE_DELTA_LOG_PATH, "00000000000000000010.checkpoint.0000000002.0000000003.parquet").toString) assert(multipartCheckpoint.version == 10) assert(multipartCheckpoint.numParts.isPresent() && multipartCheckpoint.numParts.get() == 3) @@ -109,7 +115,8 @@ class CheckpointInstanceSuite extends AnyFunSuite { // V2 checkpoint val v2Checkpoint = new CheckpointInstance( - new Path(FAKE_DELTA_LOG_PATH, + new Path( + FAKE_DELTA_LOG_PATH, "00000000000000000010.checkpoint.abcda-bacbac.parquet").toString) assert(v2Checkpoint.version == 10) assert(!v2Checkpoint.numParts.isPresent()) @@ -119,13 +126,11 @@ class CheckpointInstanceSuite extends AnyFunSuite { // invalid checkpoints intercept[RuntimeException] { new CheckpointInstance( - new Path(FAKE_DELTA_LOG_PATH, - "00000000000000000010.checkpoint.000000.a.parquet").toString) + new Path(FAKE_DELTA_LOG_PATH, "00000000000000000010.checkpoint.000000.a.parquet").toString) } intercept[RuntimeException] { new CheckpointInstance( - new Path(FAKE_DELTA_LOG_PATH, - "00000000000000000010.parquet").toString) + new Path(FAKE_DELTA_LOG_PATH, "00000000000000000010.parquet").toString) } } @@ -133,20 +138,19 @@ class CheckpointInstanceSuite extends AnyFunSuite { // classic checkpoint val classicCheckpoint0 = new CheckpointInstance(0) assert(classicCheckpoint0.getCorrespondingFiles(FAKE_DELTA_LOG_PATH).equals( - Seq(new Path(FAKE_DELTA_LOG_PATH, "00000000000000000000.checkpoint.parquet")).asJava - )) + Seq(new Path(FAKE_DELTA_LOG_PATH, "00000000000000000000.checkpoint.parquet")).asJava)) val classicCheckpoint10 = new CheckpointInstance(10) assert(classicCheckpoint10.getCorrespondingFiles(FAKE_DELTA_LOG_PATH).equals( - Seq(new Path(FAKE_DELTA_LOG_PATH, "00000000000000000010.checkpoint.parquet")).asJava - )) + Seq(new Path(FAKE_DELTA_LOG_PATH, "00000000000000000010.checkpoint.parquet")).asJava)) // multi-part checkpoint val multipartCheckpoint = new CheckpointInstance(10, Optional.of(3)) val expectedResult = Seq( "00000000000000000010.checkpoint.0000000001.0000000003.parquet", "00000000000000000010.checkpoint.0000000002.0000000003.parquet", - "00000000000000000010.checkpoint.0000000003.0000000003.parquet" - ).map(new Path(FAKE_DELTA_LOG_PATH, _)) + "00000000000000000010.checkpoint.0000000003.0000000003.parquet").map(new Path( + FAKE_DELTA_LOG_PATH, + _)) assert(multipartCheckpoint.getCorrespondingFiles(FAKE_DELTA_LOG_PATH).equals( expectedResult.asJava)) } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointerSuite.scala index 38c8e56c0cb..59c5ab7322b 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointerSuite.scala @@ -15,7 +15,12 @@ */ package io.delta.kernel.internal.checkpoints -import io.delta.kernel.data.{ColumnVector, ColumnarBatch} +import java.io.{FileNotFoundException, IOException} +import java.util.Optional + +import scala.util.control.NonFatal + +import io.delta.kernel.data.{ColumnarBatch, ColumnVector} import io.delta.kernel.exceptions.KernelEngineException import io.delta.kernel.expressions.Predicate import io.delta.kernel.internal.checkpoints.Checkpointer.findLastCompleteCheckpointBeforeHelper @@ -25,11 +30,8 @@ import io.delta.kernel.internal.util.Utils import io.delta.kernel.test.{BaseMockJsonHandler, MockFileSystemClientUtils, VectorTestUtils} import io.delta.kernel.types.StructType import io.delta.kernel.utils.{CloseableIterator, FileStatus} -import org.scalatest.funsuite.AnyFunSuite -import java.io.{FileNotFoundException, IOException} -import java.util.Optional -import scala.util.control.NonFatal +import org.scalatest.funsuite.AnyFunSuite class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { import CheckpointerSuite._ @@ -91,7 +93,7 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("findLastCompleteCheckpointBefore - no checkpoints") { val files = deltaFileStatuses(Seq.range(0, 25)) - Seq((0, 0), (10, 10), (20, 20), (27, 25 /* no delta log files after version 24 */)).foreach { + Seq((0, 0), (10, 10), (20, 20), (27, 25 /* no delta log files after version 24 */ )).foreach { case (beforeVersion, expNumFilesListed) => assertNoLastCheckpoint(files, beforeVersion, expNumFilesListed) } @@ -142,7 +144,7 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { // Listing size is 1000 delta versions (i.e list _delta_log/0001000* to _delta_log/0001999*) val versionsListed = Math.min(beforeVersion, 1000) val expNumFilesListed = - versionsListed /* delta files */ + + versionsListed /* delta files */ + (versionsListed / 10) * 20 /* checkpoints */ assertLastCheckpoint(files, beforeVersion, expCheckpointVersion, expNumFilesListed) } @@ -167,7 +169,7 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { val expNumFilesListed = numListCalls - 1 /* last file scanned that fails the search and stops */ + versionsListed /* delta files */ + - 50 /* one multi-part checkpoint */ + 50 /* one multi-part checkpoint */ assertLastCheckpoint(files, beforeVersion, expCheckpointVersion, expNumFilesListed) } } @@ -177,7 +179,10 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { val files = deltaFileStatuses(Seq.range(0, 25)) ++ singularCheckpointFileStatuses(Seq(10)) ++ Seq(FileStatus.of( - checkpointFileSingular(logPath, 20).toString, 0, 0)) // zero-sized CP + checkpointFileSingular(logPath, 20).toString, + 0, + 0 + )) // zero-sized CP Seq((0, 0), (4, 4), (9, 9), (10, 10)).foreach { case (beforeVersion, expNumFilesListed) => @@ -284,7 +289,6 @@ class MockLastCheckpointMetadataFileReader(maxFailures: Int) extends BaseMockJso } } catch { case NonFatal(e) => throw new KernelEngineException("Failed to read last checkpoint", e); - } - ) + }) } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala new file mode 100644 index 00000000000..1e73f99de19 --- /dev/null +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala @@ -0,0 +1,197 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.checksum + +import java.util +import java.util.{Collections, Optional} + +import io.delta.kernel.data.Row +import io.delta.kernel.internal.actions.{Format, Metadata, Protocol} +import io.delta.kernel.internal.checksum.CRCInfo.CRC_FILE_SCHEMA +import io.delta.kernel.internal.data.GenericRow +import io.delta.kernel.internal.fs.Path +import io.delta.kernel.internal.util.VectorUtils +import io.delta.kernel.internal.util.VectorUtils.{stringArrayValue, stringStringMapValue} +import io.delta.kernel.test.{BaseMockJsonHandler, MockEngineUtils} +import io.delta.kernel.types.StructType +import io.delta.kernel.utils.CloseableIterator + +import org.scalatest.funsuite.AnyFunSuite + +/** + * Test suite for ChecksumWriter functionality. + */ +class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { + + private val FAKE_DELTA_LOG_PATH = new Path("/path/to/delta/log") + + // Schema field indices in crc file + private val TABLE_SIZE_BYTES_IDX = CRC_FILE_SCHEMA.indexOf("tableSizeBytes") + private val NUM_FILES_IDX = CRC_FILE_SCHEMA.indexOf("numFiles") + private val NUM_METADATA_IDX = CRC_FILE_SCHEMA.indexOf("numMetadata") + private val NUM_PROTOCOL_IDX = CRC_FILE_SCHEMA.indexOf("numProtocol") + private val TXN_ID_IDX = CRC_FILE_SCHEMA.indexOf("txnId") + private val METADATA_IDX = CRC_FILE_SCHEMA.indexOf("metadata") + private val PROTOCOL_IDX = CRC_FILE_SCHEMA.indexOf("protocol") + + test("write checksum") { + val jsonHandler = new MockCheckSumFileJsonWriter() + val checksumWriter = new ChecksumWriter(FAKE_DELTA_LOG_PATH) + val protocol = createTestProtocol() + val metadata = createTestMetadata() + + def testChecksumWrite(txn: Optional[String]): Unit = { + val version = 1L + val tableSizeBytes = 100L + val numFiles = 1L + + checksumWriter.writeCheckSum( + mockEngine(jsonHandler = jsonHandler), + new CRCInfo(version, metadata, protocol, tableSizeBytes, numFiles, txn)) + + verifyChecksumFile(jsonHandler, version) + verifyChecksumContent(jsonHandler.capturedCrcRow.get, tableSizeBytes, numFiles, txn) + verifyMetadataAndProtocol(jsonHandler.capturedCrcRow.get, metadata, protocol) + } + + // Test with and without transaction ID + testChecksumWrite(Optional.of("txn")) + testChecksumWrite(Optional.empty()) + } + + private def verifyChecksumFile(jsonHandler: MockCheckSumFileJsonWriter, version: Long): Unit = { + assert(jsonHandler.checksumFilePath == s"$FAKE_DELTA_LOG_PATH/${"%020d".format(version)}.crc") + assert(jsonHandler.capturedCrcRow.isDefined) + assert(jsonHandler.capturedCrcRow.get.getSchema == CRC_FILE_SCHEMA) + } + + private def verifyChecksumContent( + actualCheckSumRow: Row, + expectedTableSizeBytes: Long, + expectedNumFiles: Long, + expectedTxnId: Optional[String]): Unit = { + assert(!actualCheckSumRow.isNullAt(TABLE_SIZE_BYTES_IDX) && actualCheckSumRow.getLong( + TABLE_SIZE_BYTES_IDX) == expectedTableSizeBytes) + assert(!actualCheckSumRow.isNullAt( + NUM_FILES_IDX) && actualCheckSumRow.getLong(NUM_FILES_IDX) == expectedNumFiles) + assert(!actualCheckSumRow.isNullAt( + NUM_METADATA_IDX) && actualCheckSumRow.getLong(NUM_METADATA_IDX) == 1L) + assert(!actualCheckSumRow.isNullAt( + NUM_PROTOCOL_IDX) && actualCheckSumRow.getLong(NUM_PROTOCOL_IDX) == 1L) + + if (expectedTxnId.isPresent) { + assert(actualCheckSumRow.getString(TXN_ID_IDX) == expectedTxnId.get()) + } else { + assert(actualCheckSumRow.isNullAt(TXN_ID_IDX)) + } + } + + private def verifyMetadataAndProtocol( + actualRow: Row, + expectedMetadata: Metadata, + expectedProtocol: Protocol): Unit = { + checkMetadata(expectedMetadata, actualRow.getStruct(METADATA_IDX)) + checkProtocol(expectedProtocol, actualRow.getStruct(PROTOCOL_IDX)) + } + + // TODO: implement compare in Metadata and remove this method + private def checkMetadata(expectedMetadata: Metadata, actualMetadataRow: Row): Unit = { + assert(actualMetadataRow.getSchema == Metadata.FULL_SCHEMA) + + def getOptionalString(field: String): Optional[String] = + Optional.ofNullable(actualMetadataRow.getString(Metadata.FULL_SCHEMA.indexOf(field))) + + assert( + actualMetadataRow.getString(Metadata.FULL_SCHEMA.indexOf("id")) == expectedMetadata.getId) + assert(getOptionalString("name") == expectedMetadata.getName) + assert(getOptionalString("description") == expectedMetadata.getDescription) + + val formatRow = actualMetadataRow.getStruct(Metadata.FULL_SCHEMA.indexOf("format")) + assert( + formatRow + .getString( + Format.FULL_SCHEMA.indexOf("provider")) == expectedMetadata.getFormat.getProvider) + + assert( + actualMetadataRow + .getString( + Metadata.FULL_SCHEMA.indexOf("schemaString")) == expectedMetadata.getSchemaString) + assert( + actualMetadataRow + .getArray(Metadata.FULL_SCHEMA.indexOf("partitionColumns")) + == expectedMetadata.getPartitionColumns) + assert( + Optional + .ofNullable(actualMetadataRow.getLong(Metadata.FULL_SCHEMA.indexOf("createdTime"))) + == expectedMetadata.getCreatedTime) + assert( + VectorUtils + .toJavaMap(actualMetadataRow.getMap(Metadata.FULL_SCHEMA.indexOf("configuration"))) + == expectedMetadata.getConfiguration) + } + + // TODO: implement compare in Protocol and remove this method + private def checkProtocol(expectedProtocol: Protocol, actualProtocolRow: Row): Unit = { + assert(actualProtocolRow.getSchema == Protocol.FULL_SCHEMA) + assert( + expectedProtocol.getMinReaderVersion == actualProtocolRow + .getInt(Protocol.FULL_SCHEMA.indexOf("minReaderVersion"))) + assert( + expectedProtocol.getMinWriterVersion == actualProtocolRow + .getInt(Protocol.FULL_SCHEMA.indexOf("minWriterVersion"))) + } + + private def createTestMetadata(): Metadata = { + new Metadata( + "id", + Optional.of("name"), + Optional.of("description"), + new Format("parquet", Collections.emptyMap()), + "schemaString", + new StructType(), + stringArrayValue(util.Arrays.asList("c3")), + Optional.of(123), + stringStringMapValue(new util.HashMap[String, String]() { + put("delta.appendOnly", "true") + })) + } + + private def createTestProtocol(): Protocol = { + new Protocol( + /* minReaderVersion= */ 1, + /* minWriterVersion= */ 2, + Collections.emptySet(), + Collections.emptySet()) + } +} + +/** + * Mock implementation of JsonHandler for testing checksum file writing. + */ +class MockCheckSumFileJsonWriter extends BaseMockJsonHandler { + var capturedCrcRow: Option[Row] = None + var checksumFilePath: String = "" + + override def writeJsonFileAtomically( + filePath: String, + data: CloseableIterator[Row], + overwrite: Boolean): Unit = { + checksumFilePath = filePath + assert(data.hasNext, "Expected data iterator to contain exactly one row") + capturedCrcRow = Some(data.next()) + assert(!data.hasNext, "Expected data iterator to contain exactly one row") + } +} diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metadatadomain/JsonMetadataDomainSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metadatadomain/JsonMetadataDomainSuite.scala index 4b837daded3..716b3c54ae4 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metadatadomain/JsonMetadataDomainSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metadatadomain/JsonMetadataDomainSuite.scala @@ -15,10 +15,11 @@ */ package io.delta.kernel.internal.metadatadomain +import java.util.Optional + import io.delta.kernel.internal.rowtracking.RowTrackingMetadataDomain -import org.scalatest.funsuite.AnyFunSuite -import java.util.Optional +import org.scalatest.funsuite.AnyFunSuite class JsonMetadataDomainSuite extends AnyFunSuite { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metrics/MetricsReportSerializerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metrics/MetricsReportSerializerSuite.scala index 5d6b57c18f3..11c0b31b2b0 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metrics/MetricsReportSerializerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metrics/MetricsReportSerializerSuite.scala @@ -20,6 +20,7 @@ import java.util.{Optional, UUID} import io.delta.kernel.expressions.{Column, Literal, Predicate} import io.delta.kernel.metrics.{ScanReport, SnapshotReport, TransactionReport} import io.delta.kernel.types.{IntegerType, StructType} + import org.scalatest.funsuite.AnyFunSuite class MetricsReportSerializerSuite extends AnyFunSuite { @@ -68,8 +69,7 @@ class MetricsReportSerializerSuite extends AnyFunSuite { val snapshotReport1 = SnapshotReportImpl.forError( snapshotContext1, - exception - ) + exception) // Manually check expected JSON val expectedJson = @@ -147,8 +147,7 @@ class MetricsReportSerializerSuite extends AnyFunSuite { Optional.of(2), /* committedVersion */ transactionMetrics1, snapshotReport1, - Optional.of(exception) - ) + Optional.of(exception)) // Manually check expected JSON val expectedJson = @@ -192,7 +191,6 @@ class MetricsReportSerializerSuite extends AnyFunSuite { testTransactionReport(transactionReport2) } - private def testScanReport(scanReport: ScanReport): Unit = { val exception: Optional[String] = scanReport.getException().map(_.toString) val filter: Optional[String] = scanReport.getFilter.map(_.toString) @@ -254,8 +252,7 @@ class MetricsReportSerializerSuite extends AnyFunSuite { Optional.empty(), true, scanMetrics, - Optional.of(exception) - ) + Optional.of(exception)) // Manually check expected JSON val expectedJson = @@ -300,8 +297,7 @@ class MetricsReportSerializerSuite extends AnyFunSuite { Optional.empty(), false, // isFullyConsumed new ScanMetrics(), - Optional.empty() - ) + Optional.empty()) testScanReport(scanReport2) } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/replay/LogReplaySuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/replay/LogReplaySuite.scala index bb9934b733b..798ff5f9c86 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/replay/LogReplaySuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/replay/LogReplaySuite.scala @@ -16,9 +16,11 @@ package io.delta.kernel.internal.replay import scala.collection.JavaConverters._ + import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.replay.LogReplayUtils.assertLogFilesBelongToTable import io.delta.kernel.utils.FileStatus + import org.scalatest.funsuite.AnyFunSuite class TestLogReplay extends AnyFunSuite { @@ -30,8 +32,7 @@ class TestLogReplay extends AnyFunSuite { FileStatus.of("s3://bucket/logPath/deltafile1", 0L, 0L), FileStatus.of("s3://bucket/logPath/deltafile2", 0L, 0L), FileStatus.of("s3://bucket/logPath/checkpointfile1", 0L, 0L), - FileStatus.of("s3://bucket/logPath/checkpointfile2", 0L, 0L) - ).asJava + FileStatus.of("s3://bucket/logPath/checkpointfile2", 0L, 0L)).asJava assertLogFilesBelongToTable(tablePath, logFiles) } @@ -41,8 +42,7 @@ class TestLogReplay extends AnyFunSuite { FileStatus.of("s3://bucket/logPath/deltafile1", 0L, 0L), FileStatus.of("s3://bucket/invalidLogPath/deltafile2", 0L, 0L), FileStatus.of("s3://bucket/logPath/checkpointfile1", 0L, 0L), - FileStatus.of("s3://bucket/invalidLogPath/checkpointfile2", 0L, 0L) - ).asJava + FileStatus.of("s3://bucket/invalidLogPath/checkpointfile2", 0L, 0L)).asJava // Test that files with incorrect log paths trigger the assertion val ex = intercept[RuntimeException] { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/LogSegmentSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/LogSegmentSuite.scala index 739ee67f2a0..1a1ccc99436 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/LogSegmentSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/LogSegmentSuite.scala @@ -22,6 +22,7 @@ import scala.collection.JavaConverters._ import io.delta.kernel.test.MockFileSystemClientUtils import io.delta.kernel.utils.FileStatus + import org.scalatest.funsuite.AnyFunSuite class LogSegmentSuite extends AnyFunSuite with MockFileSystemClientUtils { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/MetadataCleanupSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/MetadataCleanupSuite.scala index 04afb67bd68..d00cdb9e8f8 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/MetadataCleanupSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/MetadataCleanupSuite.scala @@ -19,6 +19,7 @@ import io.delta.kernel.internal.snapshot.MetadataCleanup.cleanupExpiredLogs import io.delta.kernel.internal.util.ManualClock import io.delta.kernel.test.{MockFileSystemClientUtils, MockListFromDeleteFileSystemClient} import io.delta.kernel.utils.FileStatus + import org.scalatest.funsuite.AnyFunSuite /** @@ -52,8 +53,7 @@ class MetadataCleanupSuite extends AnyFunSuite with MockFileSystemClientUtils { DeletedFileList(), // expected deleted files - none of them should be deleted 200, // current time 0 // retention period - ) - ).foreach { + )).foreach { case (testName, expectedDeletedFiles, currentTime, retentionPeriod) => // _deltalog directory contents - contains only delta files val logFiles = deltaFileStatuses(Seq(0, 1, 2, 3, 4, 5, 6)) @@ -118,10 +118,13 @@ class MetadataCleanupSuite extends AnyFunSuite with MockFileSystemClientUtils { Seq(), // expDeletedDeltaVersions 200, // current time 200 // retention period - ) - ).foreach { - case (testName, expDeletedDeltaVersions, expDeletedCheckpointVersions, - currentTime, retentionPeriod) => + )).foreach { + case ( + testName, + expDeletedDeltaVersions, + expDeletedCheckpointVersions, + currentTime, + retentionPeriod) => val expectedDeletedFiles = DeletedFileList( deltaVersions = expDeletedDeltaVersions, @@ -139,8 +142,7 @@ class MetadataCleanupSuite extends AnyFunSuite with MockFileSystemClientUtils { case "v2" => expDeletedCheckpointVersions case "hybrid" => expDeletedCheckpointVersions.filter(_ == 9) case _ => Seq.empty - } - ) + }) test(s"metadataCleanup: $checkpointType: $testName: $currentTime, $retentionPeriod") { cleanupAndVerify(logFiles, expectedDeletedFiles.fileList(), currentTime, retentionPeriod) @@ -164,18 +166,13 @@ class MetadataCleanupSuite extends AnyFunSuite with MockFileSystemClientUtils { 30, // retention period DeletedFileList( deltaVersions = Seq(25, 26, 27, 28), - multipartCheckpointVersions = Seq(25) - ) - ), + multipartCheckpointVersions = Seq(25))), ( 330, // current time 10, // retention period DeletedFileList( deltaVersions = Seq(25, 26, 27, 28), - multipartCheckpointVersions = Seq(25) - ) - ) - ).foreach { + multipartCheckpointVersions = Seq(25)))).foreach { case (currentTime, retentionPeriod, expectedDeletedFiles) => cleanupAndVerify(logFiles, expectedDeletedFiles.fileList(), currentTime, retentionPeriod) } @@ -222,17 +219,15 @@ class MetadataCleanupSuite extends AnyFunSuite with MockFileSystemClientUtils { Seq(3L, 6L), // expDeletedCheckpointVersions, 130, // current time 20 // retention period - ) - ).foreach { - case (expDeletedDeltaVersions, expDeletedCheckpointVersions, - currentTime, retentionPeriod) => + )).foreach { + case (expDeletedDeltaVersions, expDeletedCheckpointVersions, currentTime, retentionPeriod) => val expectedDeletedFiles = (deltaFileStatuses(expDeletedDeltaVersions) ++ expDeletedCheckpointVersions.flatMap { - case v@3 => multiCheckpointFileStatuses(Seq(v), multiPartCheckpointPartsSize) - .filterNot(_.getPath.contains(s"%010d.%010d".format(2, 4))) - case v@6 => multiCheckpointFileStatuses(Seq(v), multiPartCheckpointPartsSize) - case v@9 => v2CPFileStatuses(Seq(v)) + case v @ 3 => multiCheckpointFileStatuses(Seq(v), multiPartCheckpointPartsSize) + .filterNot(_.getPath.contains(s"%010d.%010d".format(2, 4))) + case v @ 6 => multiCheckpointFileStatuses(Seq(v), multiPartCheckpointPartsSize) + case v @ 9 => v2CPFileStatuses(Seq(v)) }).map(_.getPath) cleanupAndVerify(logFiles, expectedDeletedFiles, currentTime, retentionPeriod) @@ -258,8 +253,7 @@ class MetadataCleanupSuite extends AnyFunSuite with MockFileSystemClientUtils { mockEngine(fsClient), new ManualClock(currentTimeMillis), logPath, - retentionPeriodMillis - ) + retentionPeriodMillis) assert(resultDeletedCount === expectedDeletedFiles.size) assert(fsClient.getDeleteCalls.toSet === expectedDeletedFiles.toSet) @@ -272,17 +266,16 @@ object MetadataCleanupSuite extends MockFileSystemClientUtils { /** Case class containing the list of expected files in the deleted metadata log file list */ case class DeletedFileList( - deltaVersions: Seq[Long] = Seq.empty, - classicCheckpointVersions: Seq[Long] = Seq.empty, - multipartCheckpointVersions: Seq[Long] = Seq.empty, - v2CheckpointVersions: Seq[Long] = Seq.empty) { + deltaVersions: Seq[Long] = Seq.empty, + classicCheckpointVersions: Seq[Long] = Seq.empty, + multipartCheckpointVersions: Seq[Long] = Seq.empty, + v2CheckpointVersions: Seq[Long] = Seq.empty) { def fileList(): Seq[String] = { (deltaFileStatuses(deltaVersions) ++ singularCheckpointFileStatuses(classicCheckpointVersions) ++ multiCheckpointFileStatuses(multipartCheckpointVersions, multiPartCheckpointPartsSize) ++ - v2CPFileStatuses(v2CheckpointVersions) - ).sortBy(_.getPath).map(_.getPath) + v2CPFileStatuses(v2CheckpointVersions)).sortBy(_.getPath).map(_.getPath) } } @@ -298,11 +291,11 @@ object MetadataCleanupSuite extends MockFileSystemClientUtils { v2CheckpointFileStatuses( versions.map(v => (v, true, 20)), // to (version, useUUID, numSidecars) - "parquet" - ).map(_._1) - .map(f => FileStatus.of( - uuidPattern.replaceAllIn(f.getPath, standardUUID), - f.getSize, - f.getModificationTime)) + "parquet").map(_._1) + .map(f => + FileStatus.of( + uuidPattern.replaceAllIn(f.getPath, standardUUID), + f.getSize, + f.getModificationTime)) } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala index 81d870d15c1..364a395fdc3 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala @@ -15,19 +15,22 @@ */ package io.delta.kernel.internal.tablefeatures +import java.util +import java.util.{Collections, Optional} +import java.util.Collections.{emptySet, singleton} +import java.util.stream.Collectors.toList + +import scala.collection.JavaConverters._ + import io.delta.kernel.data.{ArrayValue, ColumnVector, MapValue} import io.delta.kernel.exceptions.KernelException import io.delta.kernel.internal.actions.{Format, Metadata, Protocol} -import io.delta.kernel.internal.tablefeatures.TableFeatures.{TABLE_FEATURES, validateWriteSupportedTable} +import io.delta.kernel.internal.tablefeatures.TableFeatures.{validateKernelCanReadTheTable, validateKernelCanWriteToTable, DOMAIN_METADATA_W_FEATURE, TABLE_FEATURES} import io.delta.kernel.internal.util.InternalUtils.singletonStringColumnVector -import io.delta.kernel.internal.util.VectorUtils.buildColumnVector +import io.delta.kernel.internal.util.VectorUtils.stringVector import io.delta.kernel.types._ -import org.scalatest.funsuite.AnyFunSuite -import java.util.stream.Collectors -import java.util.stream.Collectors.toList -import java.util.{Collections, Optional} -import scala.collection.JavaConverters._ +import org.scalatest.funsuite.AnyFunSuite /** * Suite that tests Kernel throws error when it receives a unsupported protocol and metadata @@ -37,16 +40,37 @@ class TableFeaturesSuite extends AnyFunSuite { ///////////////////////////////////////////////////////////////////////////////////////////////// // Tests for [[TableFeature]] implementations // ///////////////////////////////////////////////////////////////////////////////////////////////// - val readerWriterFeatures = Seq("columnMapping", "deletionVectors", "timestampNtz", - "typeWidening", "typeWidening-preview", "v2Checkpoint", "vacuumProtocolCheck", - "variantType", "variantType-preview") - - val writerOnlyFeatures = Seq("appendOnly", "invariants", "checkConstraints", - "generatedColumns", "changeDataFeed", "identityColumns", - "rowTracking", "domainMetadata", "icebergCompatV2", "inCommitTimestamp") - - val legacyFeatures = Seq("appendOnly", "invariants", "checkConstraints", - "generatedColumns", "changeDataFeed", "identityColumns", "columnMapping") + val readerWriterFeatures = Seq( + "columnMapping", + "deletionVectors", + "timestampNtz", + "typeWidening", + "typeWidening-preview", + "v2Checkpoint", + "vacuumProtocolCheck", + "variantType", + "variantType-preview") + + val writerOnlyFeatures = Seq( + "appendOnly", + "invariants", + "checkConstraints", + "generatedColumns", + "changeDataFeed", + "identityColumns", + "rowTracking", + "domainMetadata", + "icebergCompatV2", + "inCommitTimestamp") + + val legacyFeatures = Seq( + "appendOnly", + "invariants", + "checkConstraints", + "generatedColumns", + "changeDataFeed", + "identityColumns", + "columnMapping") test("basic properties checks") { @@ -73,50 +97,67 @@ class TableFeaturesSuite extends AnyFunSuite { TableFeatures.TABLE_FEATURES.size() == readerWriterFeatures.size + writerOnlyFeatures.size) } - val testProtocol = new Protocol(1, 2, Collections.emptyList(), Collections.emptyList()) + val testProtocol = new Protocol(1, 2, emptySet(), emptySet()) Seq( // Test feature, metadata, expected result ("appendOnly", testMetadata(tblProps = Map("delta.appendOnly" -> "true")), true), ("appendOnly", testMetadata(tblProps = Map("delta.appendOnly" -> "false")), false), - ("invariants", testMetadata(includeInvaraint = true), true), - ("invariants", testMetadata(includeInvaraint = false), false), + ("invariants", testMetadata(includeInvariant = true), true), + ("invariants", testMetadata(includeInvariant = false), false), ("checkConstraints", testMetadata(tblProps = Map("delta.constraints.a" -> "a = b")), true), ("checkConstraints", testMetadata(), false), ("generatedColumns", testMetadata(includeGeneratedColumn = true), true), ("generatedColumns", testMetadata(includeGeneratedColumn = false), false), - ("changeDataFeed", - testMetadata(tblProps = Map("delta.enableChangeDataFeed" -> "true")), true), - ("changeDataFeed", - testMetadata(tblProps = Map("delta.enableChangeDataFeed" -> "false")), false), + ("changeDataFeed", testMetadata(tblProps = Map("delta.enableChangeDataFeed" -> "true")), true), + ( + "changeDataFeed", + testMetadata(tblProps = Map("delta.enableChangeDataFeed" -> "false")), + false), ("identityColumns", testMetadata(includeIdentityColumn = true), true), ("identityColumns", testMetadata(includeIdentityColumn = false), false), ("columnMapping", testMetadata(tblProps = Map("delta.columnMapping.mode" -> "id")), true), ("columnMapping", testMetadata(tblProps = Map("delta.columnMapping.mode" -> "none")), false), - ("typeWidening-preview", - testMetadata(tblProps = Map("delta.enableTypeWidening" -> "true")), true), - ("typeWidening-preview", - testMetadata(tblProps = Map("delta.enableTypeWidening" -> "false")), false), + ( + "typeWidening-preview", + testMetadata(tblProps = Map("delta.enableTypeWidening" -> "true")), + true), + ( + "typeWidening-preview", + testMetadata(tblProps = Map("delta.enableTypeWidening" -> "false")), + false), ("typeWidening", testMetadata(tblProps = Map("delta.enableTypeWidening" -> "true")), true), ("typeWidening", testMetadata(tblProps = Map("delta.enableTypeWidening" -> "false")), false), - ("rowTracking", testMetadata(tblProps = Map("delta.enableRowTracking" -> "true")), true), + // Disable this until we have support to enable row tracking through metadata + // ("rowTracking", testMetadata(tblProps = Map("delta.enableRowTracking" -> "true")), true), ("rowTracking", testMetadata(tblProps = Map("delta.enableRowTracking" -> "false")), false), - ("deletionVectors", - testMetadata(tblProps = Map("delta.enableDeletionVectors" -> "true")), true), - ("deletionVectors", - testMetadata(tblProps = Map("delta.enableDeletionVectors" -> "false")), false), + ( + "deletionVectors", + testMetadata(tblProps = Map("delta.enableDeletionVectors" -> "true")), + true), + ( + "deletionVectors", + testMetadata(tblProps = Map("delta.enableDeletionVectors" -> "false")), + false), ("timestampNtz", testMetadata(includeTimestampNtzTypeCol = true), true), ("timestampNtz", testMetadata(includeTimestampNtzTypeCol = false), false), ("v2Checkpoint", testMetadata(tblProps = Map("delta.checkpointPolicy" -> "v2")), true), ("v2Checkpoint", testMetadata(tblProps = Map("delta.checkpointPolicy" -> "classic")), false), - ("icebergCompatV2", - testMetadata(tblProps = Map("delta.enableIcebergCompatV2" -> "true")), true), - ("icebergCompatV2", - testMetadata(tblProps = Map("delta.enableIcebergCompatV2" -> "false")), false), - ("inCommitTimestamp", - testMetadata(tblProps = Map("delta.enableInCommitTimestamps" -> "true")), true), - ("inCommitTimestamp", - testMetadata(tblProps = Map("delta.enableInCommitTimestamps" -> "false")), false) - ).foreach({ case (feature, metadata, expected) => + ( + "icebergCompatV2", + testMetadata(tblProps = Map("delta.enableIcebergCompatV2" -> "true")), + true), + ( + "icebergCompatV2", + testMetadata(tblProps = Map("delta.enableIcebergCompatV2" -> "false")), + false), + ( + "inCommitTimestamp", + testMetadata(tblProps = Map("delta.enableInCommitTimestamps" -> "true")), + true), + ( + "inCommitTimestamp", + testMetadata(tblProps = Map("delta.enableInCommitTimestamps" -> "false")), + false)).foreach({ case (feature, metadata, expected) => test(s"metadataRequiresFeatureToBeEnabled - $feature - $metadata") { val tableFeature = TableFeatures.getTableFeature(feature) assert(tableFeature.isInstanceOf[FeatureAutoEnabledByMetadata]) @@ -132,15 +173,33 @@ class TableFeaturesSuite extends AnyFunSuite { } } + test("row tracking enable throguh metadata property is not supported") { + val tableFeature = TableFeatures.getTableFeature("rowTracking") + val ex = intercept[UnsupportedOperationException] { + tableFeature.asInstanceOf[FeatureAutoEnabledByMetadata] + .metadataRequiresFeatureToBeEnabled( + testProtocol, + testMetadata(tblProps = Map("delta.enableRowTracking" -> "true"))) + } + assert(ex.getMessage.contains("Feature `rowTracking` is not yet supported in Kernel.")) + } + test("hasKernelReadSupport expected to be true") { val results = TABLE_FEATURES.stream() .filter(_.isReaderWriterFeature) .filter(_.hasKernelReadSupport()) .collect(toList()).asScala - val expected = Seq("columnMapping", "v2Checkpoint", "variantType", - "variantType-preview", "typeWidening", "typeWidening-preview", "deletionVectors", - "timestampNtz", "vacuumProtocolCheck") + val expected = Seq( + "columnMapping", + "v2Checkpoint", + "variantType", + "variantType-preview", + "typeWidening", + "typeWidening-preview", + "deletionVectors", + "timestampNtz", + "vacuumProtocolCheck") assert(results.map(_.featureName()).toSet == expected.toSet) } @@ -150,136 +209,703 @@ class TableFeaturesSuite extends AnyFunSuite { .filter(_.hasKernelWriteSupport(testMetadata())) .collect(toList()).asScala - // checkConstraints, generatedColumns, identityColumns, invariants are writable - // because the metadata has not been set the info that these features are enabled - val expected = Seq("columnMapping", "v2Checkpoint", "deletionVectors", - "vacuumProtocolCheck", "rowTracking", "domainMetadata", "icebergCompatV2", - "inCommitTimestamp", "appendOnly", "invariants", - "checkConstraints", "generatedColumns", "identityColumns" - ) + // checkConstraints, generatedColumns, identityColumns, invariants and changeDataFeed + // are writable because the metadata has not been set the info that + // these features are enabled + val expected = Seq( + "columnMapping", + "v2Checkpoint", + "deletionVectors", + "vacuumProtocolCheck", + "rowTracking", + "domainMetadata", + "icebergCompatV2", + "inCommitTimestamp", + "appendOnly", + "invariants", + "checkConstraints", + "generatedColumns", + "changeDataFeed", + "timestampNtz", + "identityColumns") assert(results.map(_.featureName()).toSet == expected.toSet) } Seq( // Test format: feature, metadata, expected value - ("invariants", testMetadata(includeInvaraint = true), false), + ("invariants", testMetadata(includeInvariant = true), false), ("checkConstraints", testMetadata(tblProps = Map("delta.constraints.a" -> "a = b")), false), ("generatedColumns", testMetadata(includeGeneratedColumn = true), false), - ("identityColumns", testMetadata(includeIdentityColumn = true), false) - ).foreach({ case (feature, metadata, expected) => - test(s"hasKernelWriteSupport - $feature has metadata") { - val tableFeature = TableFeatures.getTableFeature(feature) - assert(tableFeature.hasKernelWriteSupport(metadata) == expected) - } + ("identityColumns", testMetadata(includeIdentityColumn = true), false)).foreach({ + case (feature, metadata, expected) => + test(s"hasKernelWriteSupport - $feature has metadata") { + val tableFeature = TableFeatures.getTableFeature(feature) + assert(tableFeature.hasKernelWriteSupport(metadata) == expected) + } }) ///////////////////////////////////////////////////////////////////////////////////////////////// - // Legacy tests (will be modified or deleted in subsequent PRs) // + // Tests for validateKernelCanReadTheTable and validateKernelCanWriteToTable // ///////////////////////////////////////////////////////////////////////////////////////////////// - test("validate write supported: protocol 1") { - checkSupported(createTestProtocol(minWriterVersion = 1)) - } - test("validateWriteSupported: protocol 2") { - checkSupported(createTestProtocol(minWriterVersion = 2)) - } - - test("validateWriteSupported: protocol 2 with appendOnly") { - checkSupported( - createTestProtocol(minWriterVersion = 2), - metadata = testMetadata(tblProps = Map("delta.appendOnly" -> "true"))) + // Reads: All legacy protocols should be readable by Kernel + Seq( + // Test format: protocol (minReaderVersion, minWriterVersion) + (1, 1), + (1, 2), + (1, 3), + (1, 4), + (2, 5), + (2, 6)).foreach { + case (minReaderVersion, minWriterVersion) => + test(s"validateKernelCanReadTheTable: protocol ($minReaderVersion, $minWriterVersion)") { + val protocol = new Protocol(minReaderVersion, minWriterVersion) + validateKernelCanReadTheTable(protocol, "/test/table") + } } - test("validateWriteSupported: protocol 2 with invariants") { - checkUnsupported( - createTestProtocol(minWriterVersion = 2), - metadata = testMetadata(includeInvaraint = true)) + // Reads: Supported table features represented as readerFeatures in the protocol + Seq( + "variantType", + "variantType-preview", + "deletionVectors", + "typeWidening", + "typeWidening-preview", + "timestampNtz", + "v2Checkpoint", + "vacuumProtocolCheck", + "columnMapping").foreach { feature => + test(s"validateKernelCanReadTheTable: protocol 3 with $feature") { + val protocol = new Protocol(3, 1, singleton(feature), Set().asJava) + validateKernelCanReadTheTable(protocol, "/test/table") + } } - test("validateWriteSupported: protocol 2, with appendOnly and invariants") { - checkUnsupported( - createTestProtocol(minWriterVersion = 2), - metadata = testMetadata(includeInvaraint = true)) + // Read is supported when all table readerWriter features are supported by the Kernel, + // but the table has writeOnly table feature unknown to Kernel + test("validateKernelCanReadTheTable: with writeOnly feature unknown to Kernel") { + + // legacy reader protocol version + val protocol1 = new Protocol(1, 7, emptySet(), singleton("unknownFeature")) + validateKernelCanReadTheTable(protocol1, "/test/table") + + // table feature supported reader version + val protocol2 = new Protocol( + 3, + 7, + Set("columnMapping", "timestampNtz").asJava, + Set("columnMapping", "timestampNtz", "unknownFeature").asJava) + validateKernelCanReadTheTable(protocol2, "/test/table") } - Seq(3, 4, 5, 6).foreach { minWriterVersion => - test(s"validateWriteSupported: protocol $minWriterVersion") { - checkUnsupported(createTestProtocol(minWriterVersion = minWriterVersion)) + test("validateKernelCanReadTheTable: unknown readerWriter feature to Kernel") { + val protocol = new Protocol(3, 7, singleton("unknownFeature"), singleton("unknownFeature")) + val ex = intercept[KernelException] { + validateKernelCanReadTheTable(protocol, "/test/table") } + assert(ex.getMessage.contains( + "requires feature \"unknownFeature\" which is unsupported by this version of Delta Kernel")) } - test("validateWriteSupported: protocol 7 with no additional writer features") { - checkSupported(createTestProtocol(minWriterVersion = 7)) - } - - Seq("appendOnly", "inCommitTimestamp", "columnMapping", "typeWidening-preview", "typeWidening", - "domainMetadata", "rowTracking").foreach { supportedWriterFeature => - test(s"validateWriteSupported: protocol 7 with $supportedWriterFeature") { - val protocol = if (supportedWriterFeature == "rowTracking") { - createTestProtocol(minWriterVersion = 7, supportedWriterFeature, "domainMetadata") - } else { - createTestProtocol(minWriterVersion = 7, supportedWriterFeature) - } - checkSupported(protocol) + test("validateKernelCanReadTheTable: reader version > 3") { + val protocol = new Protocol(4, 7, emptySet(), singleton("unknownFeature")) + val ex = intercept[KernelException] { + validateKernelCanReadTheTable(protocol, "/test/table") } + assert(ex.getMessage.contains( + "requires reader version 4 which is unsupported by this version of Delta Kernel")) } - Seq("checkConstraints", "generatedColumns", "allowColumnDefaults", "changeDataFeed", - "identityColumns", "deletionVectors", "timestampNtz", "v2Checkpoint", "icebergCompatV1", - "icebergCompatV2", "clustering", "vacuumProtocolCheck").foreach { unsupportedWriterFeature => - test(s"validateWriteSupported: protocol 7 with $unsupportedWriterFeature") { - checkUnsupported(createTestProtocol(minWriterVersion = 7, unsupportedWriterFeature)) - } + // Writes + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 8", // beyond the table feature writer version + new Protocol(3, 8)) + + checkWriteUnsupported( + // beyond the table feature reader/writer version + "validateKernelCanWriteToTable: protocol 4, 8", + new Protocol(4, 8)) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 1", + new Protocol(1, 1), + testMetadata()) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 2", + new Protocol(1, 2), + testMetadata()) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 2 with appendOnly", + new Protocol(1, 2), + testMetadata(tblProps = Map("delta.appendOnly" -> "true"))) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 2 with invariants", + new Protocol(1, 2), + testMetadata(includeInvariant = true)) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 2, with appendOnly and invariants", + new Protocol(1, 2), + testMetadata(includeInvariant = true, tblProps = Map("delta.appendOnly" -> "true"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 3", + new Protocol(1, 3)) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 3 with checkConstraints", + new Protocol(1, 3), + testMetadata(tblProps = Map("delta.constraints.a" -> "a = b"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 4", + new Protocol(1, 4)) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 4 with generatedColumns", + new Protocol(1, 4), + testMetadata(includeGeneratedColumn = true)) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 4 with changeDataFeed", + new Protocol(1, 4), + testMetadata(tblProps = Map("delta.enableChangeDataFeed" -> "true"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 5 with columnMapping", + new Protocol(2, 5), + testMetadata(tblProps = Map("delta.columnMapping.mode" -> "id"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 6", + new Protocol(2, 6), + testMetadata(tblProps = Map("delta.columnMapping.mode" -> "none"))) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 6 with identityColumns", + new Protocol(2, 6), + testMetadata(includeIdentityColumn = true)) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with appendOnly supported", + new Protocol(1, 7, Set().asJava, singleton("appendOnly")), + testMetadata(tblProps = Map("delta.appendOnly" -> "true"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with invariants, " + + "schema doesn't contain invariants", + new Protocol(1, 7, Set().asJava, singleton("invariants")), + testMetadata(includeInvariant = false)) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 7 with invariants, " + + "schema contains invariants", + new Protocol(1, 7, Set().asJava, singleton("invariants")), + testMetadata(includeInvariant = true)) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with checkConstraints, " + + "metadata doesn't contains any constraints", + new Protocol(1, 7, Set().asJava, singleton("checkConstraints")), + testMetadata()) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 7 with checkConstraints, " + + "metadata contains constraints", + new Protocol(1, 7, Set().asJava, singleton("checkConstraints")), + testMetadata(tblProps = Map("delta.constraints.a" -> "a = b"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with generatedColumns, " + + "metadata doesn't contains any generated columns", + new Protocol(1, 7, Set().asJava, singleton("generatedColumns")), + testMetadata()) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 7 with generatedColumns, " + + "metadata contains generated columns", + new Protocol(1, 7, Set().asJava, singleton("generatedColumns")), + testMetadata(includeGeneratedColumn = true)) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with changeDataFeed, " + + "metadata doesn't contains changeDataFeed", + new Protocol(1, 7, Set().asJava, singleton("changeDataFeed")), + testMetadata()) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 7 with changeDataFeed, " + + "metadata contains changeDataFeed", + new Protocol(1, 7, Set().asJava, singleton("changeDataFeed")), + testMetadata(tblProps = Map("delta.enableChangeDataFeed" -> "true"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with columnMapping, " + + "metadata doesn't contains columnMapping", + new Protocol(2, 7, Set().asJava, singleton("columnMapping")), + testMetadata()) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with columnMapping, " + + "metadata contains columnMapping", + new Protocol(2, 7, Set().asJava, singleton("columnMapping")), + testMetadata(tblProps = Map("delta.columnMapping.mode" -> "id"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with identityColumns, " + + "schema doesn't contains identity columns", + new Protocol(2, 7, Set().asJava, singleton("identityColumns")), + testMetadata()) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 7 with identityColumns, " + + "schema contains identity columns", + new Protocol(2, 7, Set().asJava, singleton("identityColumns")), + testMetadata(includeIdentityColumn = true)) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with deletionVectors, " + + "metadata doesn't contains deletionVectors", + new Protocol(2, 7, Set().asJava, singleton("deletionVectors")), + testMetadata()) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with deletionVectors, " + + "metadata contains deletionVectors", + new Protocol(2, 7, Set().asJava, singleton("deletionVectors")), + testMetadata(tblProps = Map("delta.enableDeletionVectors" -> "true"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with timestampNtz, " + + "schema doesn't contains timestampNtz", + new Protocol(3, 7, singleton("timestampNtz"), singleton("timestampNtz")), + testMetadata()) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with timestampNtz, " + + "schema contains timestampNtz", + new Protocol(3, 7, singleton("timestampNtz"), singleton("timestampNtz")), + testMetadata(includeTimestampNtzTypeCol = true)) + + Seq("typeWidening", "typeWidening-preview").foreach { feature => + checkWriteUnsupported( + s"validateKernelCanWriteToTable: protocol 7 with $feature, " + + s"metadata doesn't contains $feature", + new Protocol(3, 7, singleton(feature), singleton(feature)), + testMetadata()) + + checkWriteUnsupported( + s"validateKernelCanWriteToTable: protocol 7 with $feature, " + + s"metadata contains $feature", + new Protocol(3, 7, singleton(feature), singleton(feature)), + testMetadata(tblProps = Map("delta.enableTypeWidening" -> "true"))) } - test("validateWriteSupported: protocol 7 with invariants, schema doesn't contain invariants") { - checkSupported( - createTestProtocol(minWriterVersion = 7, "invariants") - ) + Seq("variantType", "variantType-preview").foreach { feature => + checkWriteUnsupported( + s"validateKernelCanWriteToTable: protocol 7 with $feature, " + + s"metadata doesn't contains $feature", + new Protocol(3, 7, singleton(feature), singleton(feature)), + testMetadata()) + + checkWriteUnsupported( + s"validateKernelCanWriteToTable: protocol 7 with $feature, " + + s"metadata contains $feature", + new Protocol(3, 7, singleton(feature), singleton(feature)), + testMetadata(includeInvariant = true)) } - test("validateWriteSupported: protocol 7 with invariants, schema contains invariants") { - checkUnsupported( - createTestProtocol(minWriterVersion = 7, "invariants"), - metadata = testMetadata(includeInvaraint = true) - ) + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with vacuumProtocolCheck, " + + "metadata doesn't contains vacuumProtocolCheck", + new Protocol(3, 7, singleton("vacuumProtocolCheck"), singleton("vacuumProtocolCheck")), + testMetadata()) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with domainMetadata", + new Protocol(3, 7, emptySet(), singleton("domainMetadata")), + testMetadata()) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with rowTracking", + new Protocol(3, 7, emptySet(), singleton("rowTracking")), + testMetadata()) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with inCommitTimestamp", + new Protocol(3, 7, emptySet(), singleton("inCommitTimestamp")), + testMetadata()) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with icebergCompatV2", + new Protocol(3, 7, emptySet(), singleton("icebergCompatV2")), + testMetadata(tblProps = Map("delta.enableIcebergCompatV2" -> "true"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with v2Checkpoint, " + + "metadata enables v2Checkpoint", + new Protocol(3, 7, singleton("v2Checkpoint"), singleton("v2Checkpoint")), + testMetadata(tblProps = Map("delta.checkpointPolicy" -> "v2"))) + + checkWriteSupported( + "validateKernelCanWriteToTable: protocol 7 with multiple features supported", + new Protocol( + 3, + 7, + Set("v2Checkpoint", "columnMapping").asJava, + Set("v2Checkpoint", "columnMapping", "rowTracking", "domainMetadata").asJava), + testMetadata(tblProps = Map( + "delta.checkpointPolicy" -> "v2", + "delta.columnMapping.mode" -> "id", + "delta.enableRowTracking" -> "true"))) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 7 with multiple features supported, " + + "with one of the features not supported by Kernel for writing", + new Protocol( + 3, + 7, + Set("v2Checkpoint", "columnMapping", "invariants").asJava, + Set("v2Checkpoint", "columnMapping", "invariants").asJava), + testMetadata( + includeInvariant = true, // unsupported feature + tblProps = Map( + "delta.checkpointPolicy" -> "v2", + "delta.enableRowTracking" -> "true"))) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 7 with unknown writerOnly feature", + new Protocol(1, 7, emptySet(), singleton("unknownWriterOnlyFeature")), + testMetadata()) + + checkWriteUnsupported( + "validateKernelCanWriteToTable: protocol 7 with unknown readerWriter feature", + new Protocol( + 3, + 7, + singleton("unknownWriterOnlyFeature"), + singleton("unknownWriterOnlyFeature")), + testMetadata()) + + ///////////////////////////////////////////////////////////////////////////////////////////////// + // Tests for autoUpgradeProtocolBasedOnMetadata // + ///////////////////////////////////////////////////////////////////////////////////////////////// + Seq( + // Test format: + // new metadata, + // current protocol, + // expected protocol, + // expected new features added + ( + testMetadata(tblProps = Map("delta.appendOnly" -> "true")), + new Protocol(1, 1), + new Protocol(1, 7, emptySet(), set("appendOnly")), + set("appendOnly")), + ( + testMetadata(includeInvariant = true), + new Protocol(1, 1), + new Protocol(1, 7, emptySet(), set("invariants")), + set("invariants")), + ( + testMetadata(includeInvariant = true, tblProps = Map("delta.appendOnly" -> "true")), + new Protocol(1, 1), + new Protocol(1, 2), // (1, 2) covers both appendOnly and invariants + Set("invariants", "appendOnly").asJava), + ( + testMetadata(tblProps = Map("delta.constraints.a" -> "a = b")), + new Protocol(1, 1), + new Protocol(1, 7, emptySet(), set("checkConstraints")), + set("checkConstraints")), + ( + testMetadata( + includeInvariant = true, + tblProps = Map("delta.appendOnly" -> "true", "delta.constraints.a" -> "a = b")), + new Protocol(1, 1), + new Protocol(1, 3), + set("appendOnly", "checkConstraints", "invariants")), + ( + testMetadata(tblProps = Map("delta.constraints.a" -> "a = b")), + new Protocol(1, 2), + new Protocol(1, 3), // (1, 3) covers all: appendOnly, invariants, checkConstraints + set("checkConstraints")), + ( + testMetadata(tblProps = Map("delta.enableChangeDataFeed" -> "true")), + new Protocol(1, 1), + new Protocol(1, 7, emptySet(), set("changeDataFeed")), + set("changeDataFeed")), + ( + testMetadata(includeGeneratedColumn = true), + new Protocol(1, 1), + new Protocol(1, 7, emptySet(), set("generatedColumns")), + set("generatedColumns")), + ( + testMetadata( + includeGeneratedColumn = true, + tblProps = Map("delta.enableChangeDataFeed" -> "true")), + new Protocol(1, 1), + new Protocol(1, 7, emptySet(), set("generatedColumns", "changeDataFeed")), + set("generatedColumns", "changeDataFeed")), + ( + testMetadata( + includeGeneratedColumn = true, + tblProps = Map("delta.enableChangeDataFeed" -> "true")), + new Protocol(1, 2), + new Protocol( + 1, + 7, + set(), + set("generatedColumns", "changeDataFeed", "appendOnly", "invariants")), + set("generatedColumns", "changeDataFeed")), + ( + testMetadata( + includeGeneratedColumn = true, + tblProps = Map("delta.enableChangeDataFeed" -> "true")), + new Protocol(1, 3), + new Protocol(1, 4), // 4 - implicitly supports all features + set("generatedColumns", "changeDataFeed")), + ( + testMetadata(tblProps = Map("delta.columnMapping.mode" -> "name")), + new Protocol(1, 1), + new Protocol( + 2, + 7, + set(), + set("columnMapping")), + set("columnMapping")), + ( + testMetadata(tblProps = Map("delta.columnMapping.mode" -> "name")), + new Protocol(1, 2), + new Protocol( + 2, + 7, + set(), + set("columnMapping", "appendOnly", "invariants")), + set("columnMapping")), + ( + testMetadata(tblProps = Map("delta.columnMapping.mode" -> "name")), + new Protocol(1, 3), + new Protocol( + 2, + 7, + set(), + set("columnMapping", "appendOnly", "invariants", "checkConstraints")), + set("columnMapping")), + ( + testMetadata(tblProps = Map("delta.columnMapping.mode" -> "name")), + new Protocol(1, 4), + new Protocol(2, 5), // implicitly supports all features + set("columnMapping")), + ( + testMetadata(includeIdentityColumn = true), + new Protocol(1, 1), + new Protocol( + 1, + 7, + set(), + set("identityColumns")), + set("identityColumns")), + ( + testMetadata(includeIdentityColumn = true), + new Protocol(1, 2), + new Protocol( + 1, + 7, + set(), + set("identityColumns", "appendOnly", "invariants")), + set("identityColumns")), + ( + testMetadata(includeIdentityColumn = true), + new Protocol(1, 3), + new Protocol( + 1, + 7, + set(), + set("identityColumns", "appendOnly", "invariants", "checkConstraints")), + set("identityColumns")), + ( + testMetadata(includeIdentityColumn = true), + new Protocol(2, 5), + new Protocol(2, 6), // implicitly supports all features + set("identityColumns")), + ( + testMetadata(includeTimestampNtzTypeCol = true), + new Protocol(1, 1), + new Protocol( + 3, + 7, + set("timestampNtz"), + set("timestampNtz")), + set("timestampNtz")), + ( + testMetadata(includeTimestampNtzTypeCol = true), + new Protocol(1, 2), + new Protocol( + 3, + 7, + set("timestampNtz"), + set("timestampNtz", "appendOnly", "invariants")), + set("timestampNtz")), + ( + testMetadata(tblProps = Map("delta.enableIcebergCompatV2" -> "true")), + new Protocol(1, 2), + new Protocol( + 2, + 7, + set(), + set("columnMapping", "appendOnly", "invariants", "icebergCompatV2")), + set("icebergCompatV2", "columnMapping")), + ( + testMetadata(tblProps = + Map("delta.enableIcebergCompatV2" -> "true", "delta.enableDeletionVectors" -> "true")), + new Protocol(2, 5), + new Protocol( + 3, + 7, + set("columnMapping", "deletionVectors"), + set( + "columnMapping", + "appendOnly", + "invariants", + "icebergCompatV2", + "checkConstraints", + "deletionVectors", + "generatedColumns", + "changeDataFeed")), + set("icebergCompatV2", "deletionVectors")), + ( + testMetadata(tblProps = + Map("delta.enableIcebergCompatV2" -> "true")), + new Protocol(3, 7, set("columnMapping", "deletionVectors"), set("columnMapping")), + new Protocol( + 3, + 7, + set("columnMapping", "deletionVectors"), + set("columnMapping", "icebergCompatV2", "deletionVectors")), + set("icebergCompatV2"))).foreach { + case (newMetadata, currentProtocol, expectedProtocol, expectedNewFeatures) => + test(s"autoUpgradeProtocolBasedOnMetadata:" + + s"$currentProtocol -> $expectedProtocol, $expectedNewFeatures") { + + // try with domainMetadata disabled + val newProtocolAndNewFeaturesEnabled = + TableFeatures.autoUpgradeProtocolBasedOnMetadata( + newMetadata, + /* needDomainMetadataSupport = */ false, + currentProtocol) + assert(newProtocolAndNewFeaturesEnabled.isPresent, "expected protocol upgrade") + + val newProtocol = newProtocolAndNewFeaturesEnabled.get()._1 + val newFeaturesEnabled = newProtocolAndNewFeaturesEnabled.get()._2 + + assert(newProtocol == expectedProtocol) + assert(newFeaturesEnabled.asScala.map(_.featureName()).toSet === + expectedNewFeatures.asScala) + + // try with domainMetadata enabled + val newProtocolAndNewFeaturesEnabledWithDM = + TableFeatures.autoUpgradeProtocolBasedOnMetadata( + newMetadata, + /* needDomainMetadataSupport = */ true, + currentProtocol) + + assert(newProtocolAndNewFeaturesEnabledWithDM.isPresent, "expected protocol upgrade") + + val newProtocolWithDM = newProtocolAndNewFeaturesEnabledWithDM.get()._1 + val newFeaturesEnabledWithDM = newProtocolAndNewFeaturesEnabledWithDM.get()._2 + + // reader version should be same as expected protocol as the domain metadata + // is a writerOnly feature + assert(newProtocolWithDM.getMinReaderVersion == expectedProtocol.getMinReaderVersion) + // should be 7 as domainMetadata is enabled + assert(newProtocolWithDM.getMinWriterVersion === 7) + assert(newFeaturesEnabledWithDM.asScala.map(_.featureName()).toSet === + expectedNewFeatures.asScala ++ Set("domainMetadata")) + assert(newProtocolWithDM.getImplicitlyAndExplicitlySupportedFeatures.asScala === + expectedProtocol.getImplicitlyAndExplicitlySupportedFeatures.asScala + ++ Set(DOMAIN_METADATA_W_FEATURE)) + } } - def checkSupported( - protocol: Protocol, - metadata: Metadata = testMetadata()): Unit = { - validateWriteSupportedTable(protocol, metadata, "/test/table") + // No-op upgrade + Seq( + // Test format: new metadata, current protocol + (testMetadata(), new Protocol(1, 1)), + ( + // try to enable the writer that is already supported on a protocol + // that is of legacy protocol + testMetadata(tblProps = Map("delta.appendOnly" -> "true")), + new Protocol(1, 7, emptySet(), set("appendOnly"))), + ( + // try to enable the writer that is already supported on a protocol + // that is of legacy protocol + testMetadata(tblProps = Map("delta.appendOnly" -> "true", "delta.constraints.a" -> "a = b")), + new Protocol(1, 3)), + ( + // try to enable the reader writer feature that is already supported on a protocol + // that is of legacy protocol + testMetadata(tblProps = Map("delta.columnMapping.mode" -> "name")), + new Protocol(2, 5)), + ( + // try to enable the feature that is already supported on a protocol + // that is of partial (writer only) table feature support + testMetadata(tblProps = Map("delta.enableIcebergCompatV2" -> "true")), + new Protocol(2, 7, set(), set("columnMapping", "icebergCompatV2"))), + ( + // try to enable the feature that is already supported on a protocol + // that is of table feature support + testMetadata(tblProps = Map("delta.enableIcebergCompatV2" -> "true")), + new Protocol( + 3, + 7, + set("columnMapping", "deletionVectors"), + set("columnMapping", "deletionVectors", "icebergCompatV2")))).foreach { + case (newMetadata, currentProtocol) => + test(s"autoUpgradeProtocolBasedOnMetadata: no-op upgrade: $currentProtocol") { + val newProtocolAndNewFeaturesEnabled = + TableFeatures.autoUpgradeProtocolBasedOnMetadata( + newMetadata, + /* needDomainMetadataSupport = */ false, + currentProtocol) + assert(!newProtocolAndNewFeaturesEnabled.isPresent, "expected no-op upgrade") + } } - def checkUnsupported( - protocol: Protocol, - metadata: Metadata = testMetadata()): Unit = { - intercept[KernelException] { - validateWriteSupportedTable(protocol, metadata, "/test/table") + ///////////////////////////////////////////////////////////////////////////////////////////////// + // Test utility methods. // + ///////////////////////////////////////////////////////////////////////////////////////////////// + def checkWriteSupported( + testDesc: String, + protocol: Protocol, + metadata: Metadata = testMetadata()): Unit = { + test(testDesc) { + validateKernelCanWriteToTable(protocol, metadata, "/test/table") } } - def createTestProtocol(minWriterVersion: Int, writerFeatures: String*): Protocol = { - new Protocol( - // minReaderVersion - it doesn't matter as the read fails anyway before the writer check - 0, - minWriterVersion, - // reader features - it doesn't matter as the read fails anyway before the writer check - Collections.emptyList(), - writerFeatures.toSeq.asJava - ) + def checkWriteUnsupported( + testDesc: String, + protocol: Protocol, + metadata: Metadata = testMetadata()): Unit = { + test(testDesc) { + intercept[KernelException] { + validateKernelCanWriteToTable(protocol, metadata, "/test/table") + } + } } def testMetadata( - includeInvaraint: Boolean = false, + includeInvariant: Boolean = false, includeTimestampNtzTypeCol: Boolean = false, includeVariantTypeCol: Boolean = false, includeGeneratedColumn: Boolean = false, includeIdentityColumn: Boolean = false, tblProps: Map[String, String] = Map.empty): Metadata = { val testSchema = createTestSchema( - includeInvaraint, + includeInvariant, includeTimestampNtzTypeCol, includeVariantTypeCol, includeGeneratedColumn, @@ -299,13 +925,9 @@ class TableFeaturesSuite extends AnyFunSuite { Optional.empty(), new MapValue() { // conf override def getSize = tblProps.size - - override def getKeys: ColumnVector = - buildColumnVector(tblProps.toSeq.map(_._1).asJava, StringType.STRING) - override def getValues: ColumnVector = - buildColumnVector(tblProps.toSeq.map(_._2).asJava, StringType.STRING) - } - ) + override def getKeys: ColumnVector = stringVector(tblProps.toSeq.map(_._1).asJava) + override def getValues: ColumnVector = stringVector(tblProps.toSeq.map(_._2).asJava) + }) } def createTestSchema( @@ -352,4 +974,8 @@ class TableFeaturesSuite extends AnyFunSuite { structType } + + private def set(elements: String*): java.util.Set[String] = { + Set(elements: _*).asJava + } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/types/DataTypeJsonSerDeSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/types/DataTypeJsonSerDeSuite.scala index 91de2961fc2..83c22c42550 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/types/DataTypeJsonSerDeSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/types/DataTypeJsonSerDeSuite.scala @@ -15,15 +15,17 @@ */ package io.delta.kernel.internal.types -import com.fasterxml.jackson.databind.ObjectMapper -import io.delta.kernel.types._ -import StructField.COLLATIONS_METADATA_KEY -import org.scalatest.funsuite.AnyFunSuite - import java.util.HashMap + import scala.reflect.ClassTag import scala.util.control.Breaks.break +import io.delta.kernel.types._ + +import StructField.COLLATIONS_METADATA_KEY +import com.fasterxml.jackson.databind.ObjectMapper +import org.scalatest.funsuite.AnyFunSuite + class DataTypeJsonSerDeSuite extends AnyFunSuite { import DataTypeJsonSerDeSuite._ @@ -31,9 +33,10 @@ class DataTypeJsonSerDeSuite extends AnyFunSuite { private val objectMapper = new ObjectMapper() private def parse(json: String): DataType = { - DataTypeJsonSerDe.parseDataType(objectMapper.readTree(json), + DataTypeJsonSerDe.parseDataType( + objectMapper.readTree(json), "", /* fieldPath */ - new FieldMetadata.Builder().build() /* collation field metadata */) + new FieldMetadata.Builder().build() /* collation field metadata */ ) } private def serialize(dataType: DataType): String = { @@ -49,8 +52,8 @@ class DataTypeJsonSerDeSuite extends AnyFunSuite { assert(parse(serializedJson) === dataType) } - private def checkError[T <: Throwable](json: String, expectedErrorContains: String) - (implicit classTag: ClassTag[T]): Unit = { + private def checkError[T <: Throwable](json: String, expectedErrorContains: String)(implicit + classTag: ClassTag[T]): Unit = { val e = intercept[T] { parse(json) } @@ -73,8 +76,7 @@ class DataTypeJsonSerDeSuite extends AnyFunSuite { ("\"timestamp\"", TimestampType.TIMESTAMP), ("\"decimal\"", DecimalType.USER_DEFAULT), ("\"decimal(10, 5)\"", new DecimalType(10, 5)), - ("\"variant\"", VariantType.VARIANT) - ).foreach { case (json, dataType) => + ("\"variant\"", VariantType.VARIANT)).foreach { case (json, dataType) => test("serialize/deserialize: " + dataType) { testRoundTrip(json, dataType) } @@ -122,8 +124,7 @@ class DataTypeJsonSerDeSuite extends AnyFunSuite { for ((col2Json, col2Type) <- SAMPLE_JSON_TO_TYPES) { val fieldsJson = Seq( structFieldJson("col1", col1Json, false), - structFieldJson("col2", col2Json, true, Some("{ \"int\" : 0 }")) - ) + structFieldJson("col2", col2Json, true, Some("{ \"int\" : 0 }"))) val json = structTypeJson(fieldsJson) val expectedType = new StructType() @@ -146,7 +147,7 @@ class DataTypeJsonSerDeSuite extends AnyFunSuite { test("serialize/deserialize: types with collated strings") { SAMPLE_JSON_TO_TYPES_WITH_COLLATION .foreach { - case(json, dataType) => + case (json, dataType) => testRoundTrip(json, dataType) } } @@ -154,7 +155,10 @@ class DataTypeJsonSerDeSuite extends AnyFunSuite { test("serialize/deserialize: parsed and original struct" + " type differing just in StringType collation") { val json = structTypeJson(Seq( - structFieldJson("a1", "\"string\"", true, + structFieldJson( + "a1", + "\"string\"", + true, metadataJson = Some( s"""{"$COLLATIONS_METADATA_KEY" : | {"a1" : "SPARK.UTF8_LCASE"}}""".stripMargin)))) @@ -166,8 +170,7 @@ class DataTypeJsonSerDeSuite extends AnyFunSuite { test("serialize/deserialize: special characters for column name") { val json = structTypeJson(Seq( - structFieldJson("@_! *c", "\"string\"", true) - )) + structFieldJson("@_! *c", "\"string\"", true))) val expectedType = new StructType() .add("@_! *c", StringType.STRING, true) @@ -188,8 +191,7 @@ class DataTypeJsonSerDeSuite extends AnyFunSuite { test("serialize/deserialize: parsing FieldMetadata") { def testFieldMetadata(fieldMetadataJson: String, expectedFieldMetadata: FieldMetadata): Unit = { val json = structTypeJson(Seq( - structFieldJson("testCol", "\"string\"", true, Some(fieldMetadataJson)) - )) + structFieldJson("testCol", "\"string\"", true, Some(fieldMetadataJson)))) val dataType = new StructType().add("testCol", StringType.STRING, true, expectedFieldMetadata) @@ -229,7 +231,8 @@ class DataTypeJsonSerDeSuite extends AnyFunSuite { .putDoubleArray("double_arr", Array(1.0, 2.0, 3.0)) .putBooleanArray("boolean_arr", Array(true)) .putStringArray("string_arr", Array("one", "two")) - .putFieldMetadataArray("metadata_arr", + .putFieldMetadataArray( + "metadata_arr", Array( FieldMetadata.builder().putLong("one", 1).putBoolean("two", true).build(), FieldMetadata.empty())) @@ -247,15 +250,13 @@ class DataTypeJsonSerDeSuite extends AnyFunSuite { | "two" : "val2" |} |""".stripMargin, - "Could not parse the following JSON as a valid Delta data type" - ) + "Could not parse the following JSON as a valid Delta data type") } test("parseDataType: not a valid JSON node (not a string or object)") { checkError[IllegalArgumentException]( "0", - "Could not parse the following JSON as a valid Delta data type" - ) + "Could not parse the following JSON as a valid Delta data type") } } @@ -266,18 +267,18 @@ object DataTypeJsonSerDeSuite { ("\"integer\"", IntegerType.INTEGER), ("\"variant\"", VariantType.VARIANT), (arrayTypeJson("\"string\"", true), new ArrayType(StringType.STRING, true)), - (mapTypeJson("\"integer\"", "\"string\"", true), + ( + mapTypeJson("\"integer\"", "\"string\"", true), new MapType(IntegerType.INTEGER, StringType.STRING, true)), - (structTypeJson(Seq( - structFieldJson("col1", "\"string\"", true), - structFieldJson("col2", "\"string\"", false, Some("{ \"int\" : 0 }")), - structFieldJson("col3", "\"variant\"", false))), + ( + structTypeJson(Seq( + structFieldJson("col1", "\"string\"", true), + structFieldJson("col2", "\"string\"", false, Some("{ \"int\" : 0 }")), + structFieldJson("col3", "\"variant\"", false))), new StructType() .add("col1", StringType.STRING, true) .add("col2", StringType.STRING, false, FieldMetadata.builder().putLong("int", 0).build()) - .add("col3", VariantType.VARIANT, false) - ) - ) + .add("col3", VariantType.VARIANT, false))) val SAMPLE_COMPLEX_JSON_TO_TYPES = Seq( ( @@ -295,8 +296,7 @@ object DataTypeJsonSerDeSuite { structFieldJson("c11", "\"string\"", true), structFieldJson("c12", "\"timestamp_ntz\"", false), structFieldJson("c13", "\"timestamp\"", false), - structFieldJson("c14", "\"variant\"", false) - )), + structFieldJson("c14", "\"variant\"", false))), new StructType() .add("c1", BinaryType.BINARY, true) .add("c2", BooleanType.BOOLEAN, false) @@ -311,171 +311,285 @@ object DataTypeJsonSerDeSuite { .add("c11", StringType.STRING, true) .add("c12", TimestampNTZType.TIMESTAMP_NTZ, false) .add("c13", TimestampType.TIMESTAMP, false) - .add("c14", VariantType.VARIANT, false) - ), + .add("c14", VariantType.VARIANT, false)), ( structTypeJson(Seq( structFieldJson("a1", "\"string\"", true), - structFieldJson("a2", structTypeJson(Seq( - structFieldJson("b1", mapTypeJson( - arrayTypeJson( - arrayTypeJson( - "\"string\"", true), true), - structTypeJson(Seq( - structFieldJson("c1", "\"string\"", false), - structFieldJson("c2", "\"string\"", true) - )), true), true), - structFieldJson("b2", "\"long\"", true))), true), - structFieldJson("a3", + structFieldJson( + "a2", + structTypeJson(Seq( + structFieldJson( + "b1", + mapTypeJson( + arrayTypeJson( + arrayTypeJson( + "\"string\"", + true), + true), + structTypeJson(Seq( + structFieldJson("c1", "\"string\"", false), + structFieldJson("c2", "\"string\"", true))), + true), + true), + structFieldJson("b2", "\"long\"", true))), + true), + structFieldJson( + "a3", arrayTypeJson( mapTypeJson( "\"string\"", structTypeJson(Seq( - structFieldJson("b1", "\"date\"", false) - )), false), false), true) - )), + structFieldJson("b1", "\"date\"", false))), + false), + false), + true))), new StructType() .add("a1", StringType.STRING, true) - .add("a2", new StructType() - .add("b1", new MapType( - new ArrayType( - new ArrayType(StringType.STRING, true), true), - new StructType() - .add("c1", StringType.STRING, false) - .add("c2", StringType.STRING, true), true)) - .add("b2", LongType.LONG), true) - .add("a3", new ArrayType( - new MapType( - StringType.STRING, - new StructType() - .add("b1", DateType.DATE, false), false), false), true) - ) - ) + .add( + "a2", + new StructType() + .add( + "b1", + new MapType( + new ArrayType( + new ArrayType(StringType.STRING, true), + true), + new StructType() + .add("c1", StringType.STRING, false) + .add("c2", StringType.STRING, true), + true)) + .add("b2", LongType.LONG), + true) + .add( + "a3", + new ArrayType( + new MapType( + StringType.STRING, + new StructType() + .add("b1", DateType.DATE, false), + false), + false), + true))) val SAMPLE_JSON_TO_TYPES_WITH_COLLATION = Seq( ( structTypeJson(Seq( - structFieldJson("a1", "\"string\"", true, + structFieldJson( + "a1", + "\"string\"", + true, metadataJson = Some(s"""{"$COLLATIONS_METADATA_KEY" : {"a1" : "ICU.UNICODE"}}""")), structFieldJson("a2", "\"integer\"", false), - structFieldJson("a3", "\"string\"", false, + structFieldJson( + "a3", + "\"string\"", + false, metadataJson = Some(s"""{"$COLLATIONS_METADATA_KEY" : {"a3" : "SPARK.UTF8_LCASE"}}""")), structFieldJson("a4", "\"string\"", true))), new StructType() .add("a1", new StringType("ICU.UNICODE"), true) .add("a2", IntegerType.INTEGER, false) .add("a3", new StringType("SPARK.UTF8_LCASE"), false) - .add("a4", StringType.STRING, true) - ), + .add("a4", StringType.STRING, true)), ( structTypeJson(Seq( - structFieldJson("a1", structTypeJson(Seq( - structFieldJson("b1", "\"string\"", true, - metadataJson = Some( - s"""{"$COLLATIONS_METADATA_KEY" - | : {"b1" : "ICU.UNICODE"}}""".stripMargin)))), true), - structFieldJson("a2", structTypeJson(Seq( - structFieldJson("b1", arrayTypeJson("\"string\"", false), true, - metadataJson = Some( - s"""{"$COLLATIONS_METADATA_KEY" + structFieldJson( + "a1", + structTypeJson(Seq( + structFieldJson( + "b1", + "\"string\"", + true, + metadataJson = Some( + s"""{"$COLLATIONS_METADATA_KEY" + | : {"b1" : "ICU.UNICODE"}}""".stripMargin)))), + true), + structFieldJson( + "a2", + structTypeJson(Seq( + structFieldJson( + "b1", + arrayTypeJson("\"string\"", false), + true, + metadataJson = Some( + s"""{"$COLLATIONS_METADATA_KEY" | : {"b1.element" : "SPARK.UTF8_LCASE"}}""".stripMargin)), - structFieldJson("b2", mapTypeJson("\"string\"", "\"string\"", true), false, - metadataJson = Some( - s"""{"$COLLATIONS_METADATA_KEY" + structFieldJson( + "b2", + mapTypeJson("\"string\"", "\"string\"", true), + false, + metadataJson = Some( + s"""{"$COLLATIONS_METADATA_KEY" | : {"b2.key" : "ICU.UNICODE_CI", | "b2.value" : "SPARK.UTF8_LCASE"}}""".stripMargin)), - structFieldJson("b3", arrayTypeJson("\"string\"", false), true), - structFieldJson("b4", mapTypeJson("\"string\"", "\"string\"", false), false))), true), - structFieldJson("a3", structTypeJson(Seq( - structFieldJson("b1", "\"string\"", false), - structFieldJson("b2", arrayTypeJson("\"integer\"", false), true))), false, + structFieldJson("b3", arrayTypeJson("\"string\"", false), true), + structFieldJson("b4", mapTypeJson("\"string\"", "\"string\"", false), false))), + true), + structFieldJson( + "a3", + structTypeJson(Seq( + structFieldJson("b1", "\"string\"", false), + structFieldJson("b2", arrayTypeJson("\"integer\"", false), true))), + false, metadataJson = Some( s"""{"$COLLATIONS_METADATA_KEY" | : {"b1" : "SPARK.UTF8_LCASE"}}""".stripMargin)))), new StructType() - .add("a1", new StructType() - .add("b1", new StringType("ICU.UNICODE")), true) - .add("a2", new StructType() - .add("b1", new ArrayType(new StringType("SPARK.UTF8_LCASE"), false)) - .add("b2", new MapType( - new StringType("ICU.UNICODE_CI"), new StringType("SPARK.UTF8_LCASE"), true), false) - .add("b3", new ArrayType(StringType.STRING, false)) - .add("b4", new MapType( - StringType.STRING, StringType.STRING, false), false), true) - .add("a3", new StructType() - .add("b1", StringType.STRING, false) - .add("b2", new ArrayType(IntegerType.INTEGER, false), true), false) - ), + .add( + "a1", + new StructType() + .add("b1", new StringType("ICU.UNICODE")), + true) + .add( + "a2", + new StructType() + .add("b1", new ArrayType(new StringType("SPARK.UTF8_LCASE"), false)) + .add( + "b2", + new MapType( + new StringType("ICU.UNICODE_CI"), + new StringType("SPARK.UTF8_LCASE"), + true), + false) + .add("b3", new ArrayType(StringType.STRING, false)) + .add( + "b4", + new MapType( + StringType.STRING, + StringType.STRING, + false), + false), + true) + .add( + "a3", + new StructType() + .add("b1", StringType.STRING, false) + .add("b2", new ArrayType(IntegerType.INTEGER, false), true), + false)), ( structTypeJson(Seq( structFieldJson("a1", "\"string\"", true), - structFieldJson("a2", structTypeJson(Seq( - structFieldJson("b1", mapTypeJson( - arrayTypeJson(arrayTypeJson("\"string\"", true), true), - structTypeJson(Seq( - structFieldJson("c1", "\"string\"", false, - metadataJson = Some( - s"""{"$COLLATIONS_METADATA_KEY" + structFieldJson( + "a2", + structTypeJson(Seq( + structFieldJson( + "b1", + mapTypeJson( + arrayTypeJson(arrayTypeJson("\"string\"", true), true), + structTypeJson(Seq( + structFieldJson( + "c1", + "\"string\"", + false, + metadataJson = Some( + s"""{"$COLLATIONS_METADATA_KEY" | : {"c1" : "SPARK.UTF8_LCASE"}}""".stripMargin)), - structFieldJson("c2", "\"string\"", true, - metadataJson = Some( - s"""{"$COLLATIONS_METADATA_KEY" + structFieldJson( + "c2", + "\"string\"", + true, + metadataJson = Some( + s"""{"$COLLATIONS_METADATA_KEY" | : {\"c2\" : \"ICU.UNICODE\"}}""".stripMargin)), - structFieldJson("c3", "\"string\"", true))), true), true, - metadataJson = Some( - s"""{"$COLLATIONS_METADATA_KEY" + structFieldJson("c3", "\"string\"", true))), + true), + true, + metadataJson = Some( + s"""{"$COLLATIONS_METADATA_KEY" | : {"b1.key.element.element" : "SPARK.UTF8_LCASE"}}""".stripMargin)), - structFieldJson("b2", "\"long\"", true))), true), - structFieldJson("a3", arrayTypeJson( - mapTypeJson( - "\"string\"", - structTypeJson(Seq( - structFieldJson("b1", "\"string\"", false, - metadataJson = Some( - s"""{"$COLLATIONS_METADATA_KEY" + structFieldJson("b2", "\"long\"", true))), + true), + structFieldJson( + "a3", + arrayTypeJson( + mapTypeJson( + "\"string\"", + structTypeJson(Seq( + structFieldJson( + "b1", + "\"string\"", + false, + metadataJson = Some( + s"""{"$COLLATIONS_METADATA_KEY" | : {"b1" : "SPARK.UTF8_LCASE"}}""".stripMargin)))), - false), false), true, + false), + false), + true, metadataJson = Some( s"""{"$COLLATIONS_METADATA_KEY" | : {"a3.element.key" : "ICU.UNICODE_CI"}}""".stripMargin)), - structFieldJson("a4", arrayTypeJson( - structTypeJson(Seq( - structFieldJson("b1", "\"string\"", false, - metadataJson = Some( - s"""{"$COLLATIONS_METADATA_KEY" - | : {"b1" : "SPARK.UTF8_LCASE"}}""".stripMargin)))), false), false), - structFieldJson("a5", mapTypeJson( - structTypeJson(Seq( - structFieldJson("b1", "\"string\"", false, metadataJson = Some( - s"""{"$COLLATIONS_METADATA_KEY" + structFieldJson( + "a4", + arrayTypeJson( + structTypeJson(Seq( + structFieldJson( + "b1", + "\"string\"", + false, + metadataJson = Some( + s"""{"$COLLATIONS_METADATA_KEY" + | : {"b1" : "SPARK.UTF8_LCASE"}}""".stripMargin)))), + false), + false), + structFieldJson( + "a5", + mapTypeJson( + structTypeJson(Seq( + structFieldJson( + "b1", + "\"string\"", + false, + metadataJson = Some( + s"""{"$COLLATIONS_METADATA_KEY" | : {"b1" : "SPARK.UTF8_LCASE"}}""".stripMargin)))), - "\"string\"", false), false))), + "\"string\"", + false), + false))), new StructType() .add("a1", StringType.STRING, true) - .add("a2", new StructType() - .add("b1", new MapType( - new ArrayType( - new ArrayType( - new StringType("SPARK.UTF8_LCASE"), true), true), + .add( + "a2", + new StructType() + .add( + "b1", + new MapType( + new ArrayType( + new ArrayType( + new StringType("SPARK.UTF8_LCASE"), + true), + true), + new StructType() + .add("c1", new StringType("SPARK.UTF8_LCASE"), false) + .add("c2", new StringType("ICU.UNICODE"), true) + .add("c3", StringType.STRING), + true)) + .add("b2", LongType.LONG), + true) + .add( + "a3", + new ArrayType( + new MapType( + new StringType("ICU.UNICODE_CI"), + new StructType() + .add("b1", new StringType("SPARK.UTF8_LCASE"), false), + false), + false), + true) + .add( + "a4", + new ArrayType( new StructType() - .add("c1", new StringType("SPARK.UTF8_LCASE"), false) - .add("c2", new StringType("ICU.UNICODE"), true) - .add("c3", StringType.STRING), true)) - .add("b2", LongType.LONG), true) - .add("a3", new ArrayType( + .add("b1", new StringType("SPARK.UTF8_LCASE"), false), + false), + false) + .add( + "a5", new MapType( - new StringType("ICU.UNICODE_CI"), new StructType() - .add("b1", new StringType("SPARK.UTF8_LCASE"), false), false), false), true) - .add("a4", new ArrayType( - new StructType() - .add("b1", new StringType("SPARK.UTF8_LCASE"), false), false), false) - .add("a5", new MapType( - new StructType() - .add("b1", new StringType("SPARK.UTF8_LCASE"), false), - StringType.STRING, false), false) - ) - ) + .add("b1", new StringType("SPARK.UTF8_LCASE"), false), + StringType.STRING, + false), + false))) def arrayTypeJson(elementJson: String, containsNull: Boolean): String = { s""" @@ -499,10 +613,10 @@ object DataTypeJsonSerDeSuite { } def structFieldJson( - name: String, - typeJson: String, - nullable: Boolean, - metadataJson: Option[String] = None): String = { + name: String, + typeJson: String, + nullable: Boolean, + metadataJson: Option[String] = None): String = { metadataJson match { case Some(metadata) => s""" diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/ColumnMappingSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/ColumnMappingSuite.scala index d552df918bd..2ff45506ac6 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/ColumnMappingSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/ColumnMappingSuite.scala @@ -15,18 +15,19 @@ */ package io.delta.kernel.internal.util +import java.util +import java.util.{Collections, Optional, UUID} + import io.delta.kernel.internal.TableConfig import io.delta.kernel.internal.actions.{Format, Metadata} -import io.delta.kernel.internal.util.ColumnMapping.ColumnMappingMode._ import io.delta.kernel.internal.util.ColumnMapping._ +import io.delta.kernel.internal.util.ColumnMapping.ColumnMappingMode._ import io.delta.kernel.types.{ArrayType, FieldMetadata, IntegerType, MapType, StringType, StructField, StructType} + import org.assertj.core.api.Assertions.{assertThat, assertThatNoException, assertThatThrownBy} import org.assertj.core.util.Maps import org.scalatest.funsuite.AnyFunSuite -import java.util -import java.util.{Collections, Optional, UUID} - class ColumnMappingSuite extends AnyFunSuite { test("column mapping is only enabled on known mapping modes") { assertThat(ColumnMapping.isColumnMappingModeEnabled(null)).isFalse @@ -38,7 +39,9 @@ class ColumnMappingSuite extends AnyFunSuite { test("column mapping change with empty config") { assertThatNoException.isThrownBy(() => ColumnMapping.verifyColumnMappingChange( - new util.HashMap(), new util.HashMap(), true /* isNewTable */)) + new util.HashMap(), + new util.HashMap(), + true /* isNewTable */ )) } test("column mapping mode change is allowed") { @@ -73,21 +76,24 @@ class ColumnMappingSuite extends AnyFunSuite { assertThatThrownBy(() => ColumnMapping.verifyColumnMappingChange( Maps.newHashMap(COLUMN_MAPPING_MODE_KEY, NAME.toString), - Maps.newHashMap(COLUMN_MAPPING_MODE_KEY, ID.toString), isNewTable)) + Maps.newHashMap(COLUMN_MAPPING_MODE_KEY, ID.toString), + isNewTable)) .isInstanceOf(classOf[IllegalArgumentException]) .hasMessage("Changing column mapping mode from 'name' to 'id' is not supported") assertThatThrownBy(() => ColumnMapping.verifyColumnMappingChange( Maps.newHashMap(COLUMN_MAPPING_MODE_KEY, ID.toString), - Maps.newHashMap(COLUMN_MAPPING_MODE_KEY, NAME.toString), isNewTable)) + Maps.newHashMap(COLUMN_MAPPING_MODE_KEY, NAME.toString), + isNewTable)) .isInstanceOf(classOf[IllegalArgumentException]) .hasMessage("Changing column mapping mode from 'id' to 'name' is not supported") assertThatThrownBy(() => ColumnMapping.verifyColumnMappingChange( new util.HashMap(), - Maps.newHashMap(COLUMN_MAPPING_MODE_KEY, ID.toString), isNewTable)) + Maps.newHashMap(COLUMN_MAPPING_MODE_KEY, ID.toString), + isNewTable)) .isInstanceOf(classOf[IllegalArgumentException]) .hasMessage("Changing column mapping mode from 'none' to 'id' is not supported") } @@ -111,10 +117,12 @@ class ColumnMappingSuite extends AnyFunSuite { // nested columns are currently not supported assertThat(ColumnMapping.findMaxColumnId( new StructType().add("a", StringType.STRING, createMetadataWithFieldId(14)) - .add("b", + .add( + "b", new StructType() .add("d", IntegerType.INTEGER, true) - .add("e", IntegerType.INTEGER, true), createMetadataWithFieldId(15)) + .add("e", IntegerType.INTEGER, true), + createMetadataWithFieldId(15)) .add("c", IntegerType.INTEGER, createMetadataWithFieldId(7)))) .isEqualTo(15) } @@ -152,7 +160,8 @@ class ColumnMappingSuite extends AnyFunSuite { val nestedMeta = FieldMetadata.builder() .putLong(COLUMN_MAPPING_ID_KEY, 2) - .putFieldMetadata(COLUMN_MAPPING_NESTED_IDS_KEY, + .putFieldMetadata( + COLUMN_MAPPING_NESTED_IDS_KEY, FieldMetadata.builder().putLong("b.element", 6).build()) .build() @@ -171,7 +180,8 @@ class ColumnMappingSuite extends AnyFunSuite { val nestedMeta = FieldMetadata.builder() .putLong(COLUMN_MAPPING_ID_KEY, 2) - .putFieldMetadata(COLUMN_MAPPING_NESTED_IDS_KEY, + .putFieldMetadata( + COLUMN_MAPPING_NESTED_IDS_KEY, FieldMetadata.builder() .putLong("b.key", 11) .putLong("b.value", 12).build()) @@ -179,8 +189,13 @@ class ColumnMappingSuite extends AnyFunSuite { val schema = new StructType() .add("a", StringType.STRING, createMetadataWithFieldId(1)) - .add("b", new MapType(IntegerType.INTEGER, - new StructField("d", nestedStruct, false).getDataType, false), nestedMeta) + .add( + "b", + new MapType( + IntegerType.INTEGER, + new StructField("d", nestedStruct, false).getDataType, + false), + nestedMeta) .add("c", IntegerType.INTEGER, createMetadataWithFieldId(3)) assertThat(ColumnMapping.findMaxColumnId(schema)).isEqualTo(12) @@ -225,7 +240,9 @@ class ColumnMappingSuite extends AnyFunSuite { test("assigning id and physical name preserves field metadata") { val schema = new StructType() - .add("a", StringType.STRING, + .add( + "a", + StringType.STRING, FieldMetadata.builder.putString("key1", "val1").putString("key2", "val2").build) val metadata = ColumnMapping.updateColumnMappingMetadata(createMetadata(schema), ID, true) @@ -235,7 +252,8 @@ class ColumnMappingSuite extends AnyFunSuite { .containsEntry("key1", "val1") .containsEntry("key2", "val2") .containsEntry(ColumnMapping.COLUMN_MAPPING_ID_KEY, (1L).asInstanceOf[AnyRef]) - .hasEntrySatisfying(ColumnMapping.COLUMN_MAPPING_PHYSICAL_NAME_KEY, + .hasEntrySatisfying( + ColumnMapping.COLUMN_MAPPING_PHYSICAL_NAME_KEY, (k: AnyRef) => assertThat(k).asString.startsWith("col-")) } @@ -244,7 +262,8 @@ class ColumnMappingSuite extends AnyFunSuite { val schema: StructType = new StructType() .add("a", StringType.STRING) - .add("b", + .add( + "b", new StructType() .add("d", IntegerType.INTEGER) .add("e", IntegerType.INTEGER)) @@ -254,7 +273,9 @@ class ColumnMappingSuite extends AnyFunSuite { ColumnMapping.updateColumnMappingMetadata( withIcebergCompatV2Enabled(createMetadata(schema)), ID, - true /** isNewTable */) + true + /** isNewTable */ + ) assertColumnMapping(metadata.getSchema.get("a"), 1L) assertColumnMapping(metadata.getSchema.get("b"), 2L) @@ -278,7 +299,9 @@ class ColumnMappingSuite extends AnyFunSuite { ColumnMapping.updateColumnMappingMetadata( withIcebergCompatV2Enabled(createMetadata(schema)), ID, - true /** isNewTable */) + true + /** isNewTable */ + ) assertColumnMapping(metadata.getSchema.get("a"), 1L) assertColumnMapping(metadata.getSchema.get("b"), 2L) @@ -310,7 +333,9 @@ class ColumnMappingSuite extends AnyFunSuite { ColumnMapping.updateColumnMappingMetadata( withIcebergCompatV2Enabled(createMetadata(schema)), ID, - false /** isNewTable */) + false + /** isNewTable */ + ) assertColumnMapping(metadata.getSchema.get("a"), 1L, "a") assertColumnMapping(metadata.getSchema.get("b"), 2L, "b") @@ -337,7 +362,9 @@ class ColumnMappingSuite extends AnyFunSuite { ColumnMapping.updateColumnMappingMetadata( withIcebergCompatV2Enabled(createMetadata(schema)), ID, - true /** isNewTable */) + true + /** isNewTable */ + ) assertColumnMapping(metadata.getSchema.get("a"), 1L) assertColumnMapping(metadata.getSchema.get("b"), 2L) @@ -374,7 +401,9 @@ class ColumnMappingSuite extends AnyFunSuite { ColumnMapping.updateColumnMappingMetadata( withIcebergCompatV2Enabled(createMetadata(schema)), ID, - false /** isNewTable */) + false + /** isNewTable */ + ) assertColumnMapping(metadata.getSchema.get("a"), 1L, "a") assertColumnMapping(metadata.getSchema.get("b"), 2L, "b") @@ -396,25 +425,31 @@ class ColumnMappingSuite extends AnyFunSuite { val schema: StructType = new StructType() .add("a", StringType.STRING) - .add("b", new MapType( - IntegerType.INTEGER, - new StructType() - .add("d", IntegerType.INTEGER) - .add("e", IntegerType.INTEGER) - .add("f", new ArrayType( - new StructType() - .add("g", IntegerType.INTEGER) - .add("h", IntegerType.INTEGER), + .add( + "b", + new MapType( + IntegerType.INTEGER, + new StructType() + .add("d", IntegerType.INTEGER) + .add("e", IntegerType.INTEGER) + .add( + "f", + new ArrayType( + new StructType() + .add("g", IntegerType.INTEGER) + .add("h", IntegerType.INTEGER), + false), false), - false), - false)) + false)) .add("c", IntegerType.INTEGER) val metadata: Metadata = ColumnMapping.updateColumnMappingMetadata( withIcebergCompatV2Enabled(createMetadata(schema)), ID, - true /** isNewTable */) + true + /** isNewTable */ + ) assertColumnMapping(metadata.getSchema.get("a"), 1L) assertColumnMapping(metadata.getSchema.get("b"), 2L) @@ -478,9 +513,9 @@ class ColumnMappingSuite extends AnyFunSuite { VectorUtils.stringStringMapValue(Collections.emptyMap())) private def assertColumnMapping( - field: StructField, - expId: Long, - expPhysicalName: String = "UUID"): Unit = { + field: StructField, + expId: Long, + expPhysicalName: String = "UUID"): Unit = { assertThat(field.getMetadata.getEntries) .containsEntry(ColumnMapping.COLUMN_MAPPING_ID_KEY, expId.asInstanceOf[AnyRef]) .hasEntrySatisfying( diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/DataSkippingUtilsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/DataSkippingUtilsSuite.scala index c1c86d4e8ef..27d2895f7f2 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/DataSkippingUtilsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/DataSkippingUtilsSuite.scala @@ -19,8 +19,9 @@ import scala.collection.JavaConverters._ import io.delta.kernel.expressions.Column import io.delta.kernel.internal.skipping.DataSkippingUtils -import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types.{DataType, StructField, StructType} +import io.delta.kernel.types.IntegerType.INTEGER + import org.scalatest.funsuite.AnyFunSuite class DataSkippingUtilsSuite extends AnyFunSuite { @@ -41,7 +42,7 @@ class DataSkippingUtilsSuite extends AnyFunSuite { } else { fields1.zip(fields2).forall { case (field1: StructField, field2: StructField) => field1.getName == field2.getName && - compareDataTypeUnordered(field1.getDataType, field2.getDataType) + compareDataTypeUnordered(field1.getDataType, field2.getDataType) } } case _ => @@ -49,9 +50,12 @@ class DataSkippingUtilsSuite extends AnyFunSuite { } def checkPruneStatsSchema( - inputSchema: StructType, referencedCols: Set[Column], expectedSchema: StructType): Unit = { + inputSchema: StructType, + referencedCols: Set[Column], + expectedSchema: StructType): Unit = { val prunedSchema = DataSkippingUtils.pruneStatsSchema(inputSchema, referencedCols.asJava) - assert(compareDataTypeUnordered(expectedSchema, prunedSchema), + assert( + compareDataTypeUnordered(expectedSchema, prunedSchema), s"expected=$expectedSchema\nfound=$prunedSchema") } @@ -61,8 +65,7 @@ class DataSkippingUtilsSuite extends AnyFunSuite { new StructType() .add("col1", INTEGER) .add("col2", INTEGER), - true - ) + true) val testSchema = new StructType() .add(nestedField) .add("top_level_col", INTEGER) @@ -70,34 +73,29 @@ class DataSkippingUtilsSuite extends AnyFunSuite { checkPruneStatsSchema( testSchema, Set(col("top_level_col"), nestedCol("nested.col1"), nestedCol("nested.col2")), - testSchema - ) + testSchema) // top level column pruned checkPruneStatsSchema( testSchema, Set(nestedCol("nested.col1"), nestedCol("nested.col2")), - new StructType().add(nestedField) - ) + new StructType().add(nestedField)) // nested column only one field pruned checkPruneStatsSchema( testSchema, Set(nestedCol("top_level_col"), nestedCol("nested.col1")), new StructType() .add("nested", new StructType().add("col1", INTEGER)) - .add("top_level_col", INTEGER) - ) + .add("top_level_col", INTEGER)) // nested column completely pruned checkPruneStatsSchema( testSchema, Set(nestedCol("top_level_col")), - new StructType().add("top_level_col", INTEGER) - ) + new StructType().add("top_level_col", INTEGER)) // prune all columns checkPruneStatsSchema( testSchema, Set(), - new StructType() - ) + new StructType()) } test("pruneStatsSchema - 3 levels of nesting") { @@ -110,7 +108,8 @@ class DataSkippingUtilsSuite extends AnyFunSuite { | |--level_2_col: int */ val testSchema = new StructType() - .add("level1", + .add( + "level1", new StructType() .add( "level2", @@ -118,10 +117,8 @@ class DataSkippingUtilsSuite extends AnyFunSuite { .add( "level3", new StructType().add("level_4_col", INTEGER)) - .add("level_3_col", INTEGER) - ) - .add("level_2_col", INTEGER) - ) + .add("level_3_col", INTEGER)) + .add("level_2_col", INTEGER)) // prune only 4th level col checkPruneStatsSchema( testSchema, @@ -131,40 +128,35 @@ class DataSkippingUtilsSuite extends AnyFunSuite { "level1", new StructType() .add("level2", new StructType().add("level_3_col", INTEGER)) - .add("level_2_col", INTEGER)) - ) + .add("level_2_col", INTEGER))) // prune only 3rd level column checkPruneStatsSchema( testSchema, Set(nestedCol("level1.level2.level3.level_4_col"), nestedCol("level1.level_2_col")), new StructType() - .add("level1", + .add( + "level1", new StructType() .add( "level2", new StructType() .add( "level3", - new StructType().add("level_4_col", INTEGER)) - ) - .add("level_2_col", INTEGER) - ) - ) + new StructType().add("level_4_col", INTEGER))) + .add("level_2_col", INTEGER))) // prune 4th and 3rd level column checkPruneStatsSchema( testSchema, Set(nestedCol("level1.level_2_col")), new StructType() - .add("level1", + .add( + "level1", new StructType() - .add("level_2_col", INTEGER) - ) - ) + .add("level_2_col", INTEGER))) // prune all columns checkPruneStatsSchema( testSchema, Set(), - new StructType() - ) + new StructType()) } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/FileNamesSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/FileNamesSuite.scala index dd33865e78b..7fdf690754b 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/FileNamesSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/FileNamesSuite.scala @@ -15,11 +15,12 @@ */ package io.delta.kernel.internal.util +import scala.collection.JavaConverters._ + import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.util.FileNames._ -import org.scalatest.funsuite.AnyFunSuite -import scala.collection.JavaConverters._ +import org.scalatest.funsuite.AnyFunSuite class FileNamesSuite extends AnyFunSuite { @@ -74,22 +75,22 @@ class FileNamesSuite extends AnyFunSuite { test("checkpointFileSingular") { assert( checkpointFileSingular(new Path("/a"), 1234).toString == - "/a/00000000000000001234.checkpoint.parquet") + "/a/00000000000000001234.checkpoint.parquet") } test("topLevelV2CheckpointFile") { assert( topLevelV2CheckpointFile(new Path("/a"), 1234, "7d17ac10", "json").toString == - "/a/00000000000000001234.checkpoint.7d17ac10.json") + "/a/00000000000000001234.checkpoint.7d17ac10.json") assert( topLevelV2CheckpointFile(new Path("/a"), 1234, "7d17ac10", "parquet").toString == - "/a/00000000000000001234.checkpoint.7d17ac10.parquet") + "/a/00000000000000001234.checkpoint.7d17ac10.parquet") } test("v2CheckpointSidecarFile") { assert( v2CheckpointSidecarFile(new Path("/a"), "7d17ac10").toString == - "/a/_sidecars/7d17ac10.parquet") + "/a/_sidecars/7d17ac10.parquet") } test("checkpointFileWithParts") { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/IntervalParserUtilsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/IntervalParserUtilsSuite.scala index f431aaf4650..446344942e4 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/IntervalParserUtilsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/IntervalParserUtilsSuite.scala @@ -17,6 +17,7 @@ package io.delta.kernel.internal.util import io.delta.kernel.internal.util.DateTimeConstants._ import io.delta.kernel.internal.util.IntervalParserUtils.parseIntervalAsMicros + import org.scalatest.funsuite.AnyFunSuite /** @@ -40,8 +41,7 @@ class IntervalParserUtilsSuite extends AnyFunSuite { "foo", "foo 1 day", "month 3", - "year 3" - ).foreach { input => + "year 3").foreach { input => checkFromInvalidString(input, "Error parsing") } } @@ -65,8 +65,7 @@ class IntervalParserUtilsSuite extends AnyFunSuite { "interval - 1 day + 3 Microseconds" -> micros(-1, 3), " interval 123 weeks -1 day " + "23 hours -22 minutes 1 second -123 millisecond 567 microseconds " -> - micros(860, 81480877567L) - ).foreach { case (input, expected) => + micros(860, 81480877567L)).foreach { case (input, expected) => checkFromString(input, expected) } } @@ -138,7 +137,9 @@ class IntervalParserUtilsSuite extends AnyFunSuite { } private def failFuncWithInvalidInput( - input: String, errorMsg: String, converter: String => Long): Unit = { + input: String, + errorMsg: String, + converter: String => Long): Unit = { withClue(s"Expected to throw an exception for the invalid input: $input") { val e = intercept[IllegalArgumentException](converter(input)) assert(e.getMessage.contains(errorMsg)) diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/JsonUtilsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/JsonUtilsSuite.scala index 30492196057..0fbb1bf6dc9 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/JsonUtilsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/JsonUtilsSuite.scala @@ -16,10 +16,11 @@ package io.delta.kernel.internal.util +import scala.collection.JavaConverters._ + import io.delta.kernel.exceptions.KernelException -import org.scalatest.funsuite.AnyFunSuite -import scala.collection.JavaConverters._ +import org.scalatest.funsuite.AnyFunSuite class JsonUtilsSuite extends AnyFunSuite { test("Parse Map[String, String] JSON - positive case") { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/PartitionUtilsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/PartitionUtilsSuite.scala index dd44c0a6bf8..0e428948301 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/PartitionUtilsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/PartitionUtilsSuite.scala @@ -15,14 +15,16 @@ */ package io.delta.kernel.internal.util -import io.delta.kernel.expressions.Literal._ +import java.util + +import scala.collection.JavaConverters._ + import io.delta.kernel.expressions._ +import io.delta.kernel.expressions.Literal._ import io.delta.kernel.internal.util.PartitionUtils._ import io.delta.kernel.types._ -import org.scalatest.funsuite.AnyFunSuite -import java.util -import scala.collection.JavaConverters._ +import org.scalatest.funsuite.AnyFunSuite class PartitionUtilsSuite extends AnyFunSuite { // Table schema @@ -31,9 +33,11 @@ class PartitionUtilsSuite extends AnyFunSuite { val tableSchema = new StructType() .add("data1", IntegerType.INTEGER) .add("data2", StringType.STRING) - .add("data3", new StructType() - .add("data31", BooleanType.BOOLEAN) - .add("data32", LongType.LONG)) + .add( + "data3", + new StructType() + .add("data31", BooleanType.BOOLEAN) + .add("data32", LongType.LONG)) .add("part1", IntegerType.INTEGER) .add("part2", DateType.DATE) .add("part3", StringType.STRING) @@ -55,12 +59,14 @@ class PartitionUtilsSuite extends AnyFunSuite { predicate("=", col("data1"), ofInt(12)) -> ("ALWAYS_TRUE()", "(column(`data1`) = 12)"), // multiple predicates on data columns joined with AND - predicate("AND", + predicate( + "AND", predicate("=", col("data1"), ofInt(12)), predicate(">=", col("data2"), ofString("sss"))) -> ("ALWAYS_TRUE()", "((column(`data1`) = 12) AND (column(`data2`) >= sss))"), // multiple predicates on data columns joined with OR - predicate("OR", + predicate( + "OR", predicate("<=", col("data2"), ofString("sss")), predicate("=", col("data3", "data31"), ofBoolean(true))) -> ("ALWAYS_TRUE()", "((column(`data2`) <= sss) OR (column(`data3`.`data31`) = true))"), @@ -68,81 +74,90 @@ class PartitionUtilsSuite extends AnyFunSuite { predicate("=", col("part1"), ofInt(12)) -> ("(column(`part1`) = 12)", "ALWAYS_TRUE()"), // multiple predicates on partition columns joined with AND - predicate("AND", + predicate( + "AND", predicate("=", col("part1"), ofInt(12)), predicate(">=", col("part3"), ofString("sss"))) -> ("((column(`part1`) = 12) AND (column(`part3`) >= sss))", "ALWAYS_TRUE()"), // multiple predicates on partition columns joined with OR - predicate("OR", + predicate( + "OR", predicate("<=", col("part3"), ofString("sss")), predicate("=", col("part1"), ofInt(2781))) -> ("((column(`part3`) <= sss) OR (column(`part1`) = 2781))", "ALWAYS_TRUE()"), // predicates (each on data and partition column) joined with AND - predicate("AND", + predicate( + "AND", predicate("=", col("data1"), ofInt(12)), predicate(">=", col("part3"), ofString("sss"))) -> ("(column(`part3`) >= sss)", "(column(`data1`) = 12)"), // predicates (each on data and partition column) joined with OR - predicate("OR", + predicate( + "OR", predicate("=", col("data1"), ofInt(12)), predicate(">=", col("part3"), ofString("sss"))) -> ("ALWAYS_TRUE()", "((column(`data1`) = 12) OR (column(`part3`) >= sss))"), // predicates (multiple on data and partition columns) joined with AND - predicate("AND", - predicate("AND", + predicate( + "AND", + predicate( + "AND", predicate("=", col("data1"), ofInt(12)), predicate(">=", col("data2"), ofString("sss"))), - predicate("AND", + predicate( + "AND", predicate("=", col("part1"), ofInt(12)), predicate(">=", col("part3"), ofString("sss")))) -> ( "((column(`part1`) = 12) AND (column(`part3`) >= sss))", - "((column(`data1`) = 12) AND (column(`data2`) >= sss))" - ), + "((column(`data1`) = 12) AND (column(`data2`) >= sss))"), // predicates (multiple on data and partition columns joined with OR) joined with AND - predicate("AND", - predicate("OR", + predicate( + "AND", + predicate( + "OR", predicate("=", col("data1"), ofInt(12)), predicate(">=", col("data2"), ofString("sss"))), - predicate("OR", + predicate( + "OR", predicate("=", col("part1"), ofInt(12)), predicate(">=", col("part3"), ofString("sss")))) -> ( "((column(`part1`) = 12) OR (column(`part3`) >= sss))", - "((column(`data1`) = 12) OR (column(`data2`) >= sss))" - ), + "((column(`data1`) = 12) OR (column(`data2`) >= sss))"), // predicates (multiple on data and partition columns joined with OR) joined with OR - predicate("OR", - predicate("OR", + predicate( + "OR", + predicate( + "OR", predicate("=", col("data1"), ofInt(12)), predicate(">=", col("data2"), ofString("sss"))), - predicate("OR", + predicate( + "OR", predicate("=", col("part1"), ofInt(12)), predicate(">=", col("part3"), ofString("sss")))) -> ( "ALWAYS_TRUE()", "(((column(`data1`) = 12) OR (column(`data2`) >= sss)) OR " + - "((column(`part1`) = 12) OR (column(`part3`) >= sss)))" - ), + "((column(`part1`) = 12) OR (column(`part3`) >= sss)))"), // predicates (data and partitions compared in the same expression) - predicate("AND", + predicate( + "AND", predicate("=", col("data1"), col("part1")), predicate(">=", col("part3"), ofString("sss"))) -> ( "(column(`part3`) >= sss)", - "(column(`data1`) = column(`part1`))" - ), + "(column(`data1`) = column(`part1`))"), // predicate only on data column but reverse order of literal and column predicate("=", ofInt(12), col("data1")) -> - ("ALWAYS_TRUE()", "(12 = column(`data1`))") - ) + ("ALWAYS_TRUE()", "(12 = column(`data1`))")) partitionTestCases.foreach { case (predicate, (partitionPredicate, dataPredicate)) => @@ -163,10 +178,10 @@ class PartitionUtilsSuite extends AnyFunSuite { "(partition_value(ELEMENT_AT(column(`add`.`partitionValues`), part2), date) = 12)", // exp predicate for checkpoint reader pushdown - "(column(`add`.`partitionValues_parsed`.`part2`) = 12)" - ), + "(column(`add`.`partitionValues_parsed`.`part2`) = 12)"), // multiple predicates on partition columns joined with AND - predicate("AND", + predicate( + "AND", predicate("=", col("part1"), ofInt(12)), predicate(">=", col("part3"), ofString("sss"))) -> ( @@ -178,10 +193,10 @@ class PartitionUtilsSuite extends AnyFunSuite { // exp predicate for checkpoint reader pushdown """((column(`add`.`partitionValues_parsed`.`part1`) = 12) AND |(column(`add`.`partitionValues_parsed`.`part3`) >= sss))""" - .stripMargin.replaceAll("\n", " ") - ), + .stripMargin.replaceAll("\n", " ")), // multiple predicates on partition columns joined with OR - predicate("OR", + predicate( + "OR", predicate("<=", col("part3"), ofString("sss")), predicate("=", col("part1"), ofInt(2781))) -> ( @@ -193,9 +208,7 @@ class PartitionUtilsSuite extends AnyFunSuite { // exp predicate for checkpoint reader pushdown """((column(`add`.`partitionValues_parsed`.`part3`) <= sss) OR |(column(`add`.`partitionValues_parsed`.`part1`) = 2781))""" - .stripMargin.replaceAll("\n", " ") - ) - ) + .stripMargin.replaceAll("\n", " "))) rewriteTestCases.foreach { case (predicate, (expPartitionPruningPredicate, expCheckpointReaderPushdownPredicate)) => test(s"rewrite partition predicate on scan file schema: $predicate") { @@ -236,34 +249,35 @@ class PartitionUtilsSuite extends AnyFunSuite { ofNull(new DecimalType(15, 7)) -> (null, nullFileName), ofBinary("binary_val".getBytes) -> ("binary_val", "binary_val"), ofNull(BinaryType.BINARY) -> (null, nullFileName), - ofDate(4234) -> ("1981-08-05", "1981-08-05"), + ofDate(4234) -> ("1981-08-05", "1981-08-05"), ofNull(DateType.DATE) -> (null, nullFileName), ofTimestamp(2342342342232L) -> ("1970-01-28 02:39:02.342232", "1970-01-28 02%3A39%3A02.342232"), ofNull(TimestampType.TIMESTAMP) -> (null, nullFileName), ofTimestampNtz(-2342342342L) -> ("1969-12-31 23:20:58.657658", "1969-12-31 23%3A20%3A58.657658"), - ofNull(TimestampNTZType.TIMESTAMP_NTZ) -> (null, nullFileName) - ).foreach { case (literal, (expSerializedValue, expFileName)) => - test(s"serialize partition value literal as string: ${literal.getDataType}($literal)") { - val result = serializePartitionValue(literal) - assert(result === expSerializedValue) - } + ofNull(TimestampNTZType.TIMESTAMP_NTZ) -> (null, nullFileName)).foreach { + case (literal, (expSerializedValue, expFileName)) => + test(s"serialize partition value literal as string: ${literal.getDataType}($literal)") { + val result = serializePartitionValue(literal) + assert(result === expSerializedValue) + } - test(s"construct partition data output directory: ${literal.getDataType}($literal)") { - val result = getTargetDirectory( - "/tmp/root", - Seq("part1").asJava, - Map("part1" -> literal).asJava) - assert(result === s"/tmp/root/part1=$expFileName") - } + test(s"construct partition data output directory: ${literal.getDataType}($literal)") { + val result = getTargetDirectory( + "/tmp/root", + Seq("part1").asJava, + Map("part1" -> literal).asJava) + assert(result === s"/tmp/root/part1=$expFileName") + } } test("construct partition data output directory with multiple partition columns") { val result = getTargetDirectory( "/tmp/root", Seq("part1", "part2", "part3").asJava, - Map("part1" -> ofInt(12), + Map( + "part1" -> ofInt(12), "part3" -> ofTimestamp(234234234L), "part2" -> ofString("sss")).asJava) assert(result === "/tmp/root/part1=12/part2=sss/part3=1970-01-01 00%3A03%3A54.234234") @@ -277,4 +291,3 @@ class PartitionUtilsSuite extends AnyFunSuite { new Predicate(name, children.asJava) } } - diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/SchemaUtilsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/SchemaUtilsSuite.scala index 0db8d7591ca..05e8ca09a9b 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/SchemaUtilsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/SchemaUtilsSuite.scala @@ -15,16 +15,18 @@ */ package io.delta.kernel.internal.util +import java.util.Locale + +import scala.collection.JavaConverters._ + import io.delta.kernel.exceptions.KernelException import io.delta.kernel.internal.util.SchemaUtils.{filterRecursively, validateSchema} +import io.delta.kernel.types.{ArrayType, MapType, StringType, StructField, StructType} import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types.LongType.LONG import io.delta.kernel.types.TimestampType.TIMESTAMP -import io.delta.kernel.types.{ArrayType, MapType, StringType, StructField, StructType} -import org.scalatest.funsuite.AnyFunSuite -import java.util.Locale -import scala.collection.JavaConverters._ +import org.scalatest.funsuite.AnyFunSuite class SchemaUtilsSuite extends AnyFunSuite { private def expectFailure(shouldContain: String*)(f: => Unit): Unit = { @@ -32,7 +34,8 @@ class SchemaUtilsSuite extends AnyFunSuite { f } val msg = e.getMessage.toLowerCase(Locale.ROOT) - assert(shouldContain.map(_.toLowerCase(Locale.ROOT)).forall(msg.contains), + assert( + shouldContain.map(_.toLowerCase(Locale.ROOT)).forall(msg.contains), s"Error message '$msg' didn't contain: $shouldContain") } @@ -46,7 +49,7 @@ class SchemaUtilsSuite extends AnyFunSuite { .add("b", INTEGER) .add("dupColName", StringType.STRING) expectFailure("dupColName") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } @@ -56,99 +59,110 @@ class SchemaUtilsSuite extends AnyFunSuite { .add("b", INTEGER) .add("dupCOLNAME", StringType.STRING) expectFailure("dupColName") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } test("duplicate column name for nested column + non-nested column") { val schema = new StructType() - .add("dupColName", new StructType() - .add("a", INTEGER) - .add("b", INTEGER)) + .add( + "dupColName", + new StructType() + .add("a", INTEGER) + .add("b", INTEGER)) .add("dupColName", INTEGER) expectFailure("dupColName") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } test("duplicate column name for nested column + non-nested column - case sensitivity") { val schema = new StructType() - .add("dupColName", new StructType() - .add("a", INTEGER) - .add("b", INTEGER)) + .add( + "dupColName", + new StructType() + .add("a", INTEGER) + .add("b", INTEGER)) .add("dupCOLNAME", INTEGER) expectFailure("dupCOLNAME") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } test("duplicate column name in nested level") { val schema = new StructType() - .add("top", new StructType() - .add("dupColName", INTEGER) - .add("b", INTEGER) - .add("dupColName", StringType.STRING) - ) + .add( + "top", + new StructType() + .add("dupColName", INTEGER) + .add("b", INTEGER) + .add("dupColName", StringType.STRING)) expectFailure("top.dupColName") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } test("duplicate column name in nested level - case sensitivity") { val schema = new StructType() - .add("top", new StructType() - .add("dupColName", INTEGER) - .add("b", INTEGER) - .add("dupCOLNAME", StringType.STRING) - ) + .add( + "top", + new StructType() + .add("dupColName", INTEGER) + .add("b", INTEGER) + .add("dupCOLNAME", StringType.STRING)) expectFailure("top.dupColName") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } test("duplicate column name in double nested level") { val schema = new StructType() - .add("top", new StructType() - .add("b", new StructType() - .add("dupColName", StringType.STRING) - .add("c", INTEGER) - .add("dupColName", StringType.STRING)) - .add("d", INTEGER) - ) + .add( + "top", + new StructType() + .add( + "b", + new StructType() + .add("dupColName", StringType.STRING) + .add("c", INTEGER) + .add("dupColName", StringType.STRING)) + .add("d", INTEGER)) expectFailure("top.b.dupColName") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } test("duplicate column name in double nested array") { val schema = new StructType() - .add("top", new StructType() - .add("b", new ArrayType( - new ArrayType(new StructType() - .add("dupColName", StringType.STRING) - .add("c", INTEGER) - .add("dupColName", StringType.STRING), - true), - true)) - .add("d", INTEGER) - ) + .add( + "top", + new StructType() + .add( + "b", + new ArrayType( + new ArrayType( + new StructType() + .add("dupColName", StringType.STRING) + .add("c", INTEGER) + .add("dupColName", StringType.STRING), + true), + true)) + .add("d", INTEGER)) expectFailure("top.b.element.element.dupColName") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } test("only duplicate columns are listed in the error message") { val schema = new StructType() - .add("top", - new StructType().add("a", INTEGER).add("b", INTEGER).add("c", INTEGER) - ).add("top", - new StructType().add("b", INTEGER).add("c", INTEGER).add("d", INTEGER) - ).add("bottom", - new StructType().add("b", INTEGER).add("c", INTEGER).add("d", INTEGER) - ) + .add("top", new StructType().add("a", INTEGER).add("b", INTEGER).add("c", INTEGER)).add( + "top", + new StructType().add("b", INTEGER).add("c", INTEGER).add("d", INTEGER)).add( + "bottom", + new StructType().add("b", INTEGER).add("c", INTEGER).add("d", INTEGER)) val e = intercept[KernelException] { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } assert(e.getMessage.contains("Schema contains duplicate columns: top, top.b, top.c")) } @@ -159,88 +173,107 @@ class SchemaUtilsSuite extends AnyFunSuite { .add("d", StringType.STRING) expectFailure("top.b.key.dupColName") { val schema = new StructType() - .add("top", new StructType() - .add("b", new MapType(keyType.add("dupColName", StringType.STRING), keyType, true)) - ) - validateSchema(schema, false /* isColumnMappingEnabled */) + .add( + "top", + new StructType() + .add("b", new MapType(keyType.add("dupColName", StringType.STRING), keyType, true))) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } expectFailure("top.b.value.dupColName") { val schema = new StructType() - .add("top", new StructType() - .add("b", new MapType(keyType, keyType.add("dupColName", StringType.STRING), true)) - ) - validateSchema(schema, false /* isColumnMappingEnabled */) + .add( + "top", + new StructType() + .add("b", new MapType(keyType, keyType.add("dupColName", StringType.STRING), true))) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } // This is okay val schema = new StructType() - .add("top", new StructType() - .add("b", new MapType(keyType, keyType, true)) - ) - validateSchema(schema, false /* isColumnMappingEnabled */) + .add( + "top", + new StructType() + .add("b", new MapType(keyType, keyType, true))) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } test("duplicate column name in nested array") { val schema = new StructType() - .add("top", new ArrayType(new StructType() - .add("dupColName", INTEGER) - .add("b", INTEGER) - .add("dupColName", StringType.STRING), true) - ) + .add( + "top", + new ArrayType( + new StructType() + .add("dupColName", INTEGER) + .add("b", INTEGER) + .add("dupColName", StringType.STRING), + true)) expectFailure("top.element.dupColName") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } test("duplicate column name in nested array - case sensitivity") { val schema = new StructType() - .add("top", new ArrayType(new StructType() - .add("dupColName", INTEGER) - .add("b", INTEGER) - .add("dupCOLNAME", StringType.STRING), true) - ) + .add( + "top", + new ArrayType( + new StructType() + .add("dupColName", INTEGER) + .add("b", INTEGER) + .add("dupCOLNAME", StringType.STRING), + true)) expectFailure("top.element.dupColName") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } test("non duplicate column because of back tick") { val schema = new StructType() - .add("top", new StructType() - .add("a", INTEGER) - .add("b", INTEGER)) + .add( + "top", + new StructType() + .add("a", INTEGER) + .add("b", INTEGER)) .add("top.a", INTEGER) - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } test("non duplicate column because of back tick - nested") { val schema = new StructType() - .add("first", new StructType() - .add("top", new StructType() - .add("a", INTEGER) - .add("b", INTEGER)) - .add("top.a", INTEGER)) - validateSchema(schema, false /* isColumnMappingEnabled */) + .add( + "first", + new StructType() + .add( + "top", + new StructType() + .add("a", INTEGER) + .add("b", INTEGER)) + .add("top.a", INTEGER)) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } test("duplicate column with back ticks - nested") { val schema = new StructType() - .add("first", new StructType() - .add("top.a", StringType.STRING) - .add("b", INTEGER) - .add("top.a", INTEGER)) + .add( + "first", + new StructType() + .add("top.a", StringType.STRING) + .add("b", INTEGER) + .add("top.a", INTEGER)) expectFailure("first.`top.a`") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } test("duplicate column with back ticks - nested and case sensitivity") { val schema = new StructType() - .add("first", new StructType() - .add("TOP.a", StringType.STRING) - .add("b", INTEGER) - .add("top.a", INTEGER)) + .add( + "first", + new StructType() + .add("TOP.a", StringType.STRING) + .add("b", INTEGER) + .add("top.a", INTEGER)) expectFailure("first.`top.a`") { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } } @@ -255,12 +288,12 @@ class SchemaUtilsSuite extends AnyFunSuite { Seq(s"a${char}b", s"${char}ab", s"ab${char}", char.toString).foreach { name => val schema = new StructType().add(name, INTEGER) val e = intercept[KernelException] { - validateSchema(schema, false /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) } if (char != '\n') { // with column mapping disabled this should be a valid name - validateSchema(schema, true /* isColumnMappingEnabled */) + validateSchema(schema, true /* isColumnMappingEnabled */ ) } assert(e.getMessage.contains("contains one of the unsupported")) @@ -271,8 +304,8 @@ class SchemaUtilsSuite extends AnyFunSuite { // no issues here Seq(s"a${char}b", s"${char}ab", s"ab${char}", char.toString).foreach { name => val schema = new StructType().add(name, INTEGER); - validateSchema(schema, false /* isColumnMappingEnabled */) - validateSchema(schema, true /* isColumnMappingEnabled */) + validateSchema(schema, false /* isColumnMappingEnabled */ ) + validateSchema(schema, true /* isColumnMappingEnabled */ ) } } } @@ -284,83 +317,111 @@ class SchemaUtilsSuite extends AnyFunSuite { .add("a", INTEGER) .add("b", INTEGER) .add("c", LONG) - .add("s", new StructType() - .add("a", TIMESTAMP) - .add("e", INTEGER) - .add("f", LONG) - .add("g", new StructType() - .add("a", INTEGER) - .add("b", TIMESTAMP) - .add("c", LONG) - ).add("h", new MapType( - new StructType().add("a", TIMESTAMP), - new StructType().add("b", INTEGER), - true) - ).add("i", new ArrayType( - new StructType().add("b", TIMESTAMP), - true) - ) - ).add("d", new MapType( - new StructType().add("b", TIMESTAMP), - new StructType().add("a", INTEGER), - true) - ).add("e", new ArrayType( + .add( + "s", new StructType() - .add("f", TIMESTAMP) - .add("b", INTEGER), - true) - ) + .add("a", TIMESTAMP) + .add("e", INTEGER) + .add("f", LONG) + .add( + "g", + new StructType() + .add("a", INTEGER) + .add("b", TIMESTAMP) + .add("c", LONG)).add( + "h", + new MapType( + new StructType().add("a", TIMESTAMP), + new StructType().add("b", INTEGER), + true)).add( + "i", + new ArrayType( + new StructType().add("b", TIMESTAMP), + true))).add( + "d", + new MapType( + new StructType().add("b", TIMESTAMP), + new StructType().add("a", INTEGER), + true)).add( + "e", + new ArrayType( + new StructType() + .add("f", TIMESTAMP) + .add("b", INTEGER), + true)) val flattenedTestSchema = { SchemaUtils.filterRecursively( testSchema, /* visitListMapTypes = */ true, /* stopOnFirstMatch = */ false, - (v1: StructField) => true - ).asScala.map(f => f._1.asScala.mkString(".") -> f._2).toMap + (v1: StructField) => true).asScala.map(f => f._1.asScala.mkString(".") -> f._2).toMap } Seq( // Format: (testPrefix, visitListMapTypes, stopOnFirstMatch, filter, expectedColumns) - ("Filter by name 'b', stop on first match", - true, true, (field: StructField) => field.getName == "b", Seq("b")), - ("Filter by name 'b', visit all matches", - false, false, (field: StructField) => field.getName == "b", + ( + "Filter by name 'b', stop on first match", + true, + true, + (field: StructField) => field.getName == "b", + Seq("b")), + ( + "Filter by name 'b', visit all matches", + false, + false, + (field: StructField) => field.getName == "b", Seq("b", "s.g.b")), - ("Filter by name 'b', visit all matches including nested structures", - true, false, (field: StructField) => field.getName == "b", + ( + "Filter by name 'b', visit all matches including nested structures", + true, + false, + (field: StructField) => field.getName == "b", Seq( "b", "s.g.b", "s.h.value.b", "s.i.element.b", "d.key.b", - "e.element.b" - )), - ("Filter by TIMESTAMP type, stop on first match", - false, true, (field: StructField) => field.getDataType == TIMESTAMP, + "e.element.b")), + ( + "Filter by TIMESTAMP type, stop on first match", + false, + true, + (field: StructField) => field.getDataType == TIMESTAMP, Seq("s.a")), - ("Filter by TIMESTAMP type, visit all matches including nested structures", - true, false, (field: StructField) => field.getDataType == TIMESTAMP, + ( + "Filter by TIMESTAMP type, visit all matches including nested structures", + true, + false, + (field: StructField) => field.getDataType == TIMESTAMP, Seq( "s.a", "s.g.b", "s.h.key.a", "s.i.element.b", "d.key.b", - "e.element.f" - )), - ("Filter by TIMESTAMP type and name 'f', visit all matches", true, false, + "e.element.f")), + ( + "Filter by TIMESTAMP type and name 'f', visit all matches", + true, + false, (field: StructField) => field.getDataType == TIMESTAMP && field.getName == "f", Seq("e.element.f")), - ("Filter by non-existent field name 'z'", - true, false, (field: StructField) => field.getName == "z", Seq()) - ).foreach { + ( + "Filter by non-existent field name 'z'", + true, + false, + (field: StructField) => field.getName == "z", + Seq())).foreach { case (testDescription, visitListMapTypes, stopOnFirstMatch, filter, expectedColumns) => test(s"filterRecursively - $testDescription | " + s"visitListMapTypes=$visitListMapTypes, stopOnFirstMatch=$stopOnFirstMatch") { val results = - filterRecursively(testSchema, visitListMapTypes, stopOnFirstMatch, - (v1: StructField) => filter(v1)) + filterRecursively( + testSchema, + visitListMapTypes, + stopOnFirstMatch, + (v1: StructField) => filter(v1)) // convert to map of column path concatenated with '.' and the StructField .asScala.map(f => (f._1.asScala.mkString("."), f._2)).toMap diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/metrics/MetricsUtilsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/metrics/MetricsUtilsSuite.scala index e299dd911b0..186cebbc6be 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/metrics/MetricsUtilsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/metrics/MetricsUtilsSuite.scala @@ -19,6 +19,7 @@ import java.util.concurrent.Callable import java.util.function.Supplier import io.delta.kernel.internal.metrics.{Counter, Timer} + import org.scalatest.funsuite.AnyFunSuite class MetricsUtilsSuite extends AnyFunSuite { @@ -30,7 +31,7 @@ class MetricsUtilsSuite extends AnyFunSuite { val NANOSECONDS_PER_MILLISECOND = 1000000 def millisToNanos(millis: Long): Long = { - millis*NANOSECONDS_PER_MILLISECOND + millis * NANOSECONDS_PER_MILLISECOND } /** @@ -53,8 +54,7 @@ class MetricsUtilsSuite extends AnyFunSuite { val upperLimitDuration = Math.max( // we pad by 1 due to rounding of nanoseconds to milliseconds for system time System.currentTimeMillis() - startTime + 1, - amtMillis - ) + amtMillis) // check count assert(timer.count == initialCount + 1) @@ -82,10 +82,11 @@ class MetricsUtilsSuite extends AnyFunSuite { // Using Timer.time(supplier) def supplier(amount: Long): Supplier[Long] = { - () => { - Thread.sleep(amount) - amount - } + () => + { + Thread.sleep(amount) + amount + } } testTimer((amount, timer) => { timer.time(supplier(amount)) @@ -93,10 +94,11 @@ class MetricsUtilsSuite extends AnyFunSuite { // Using Timer.timeCallable def callable(amount: Long): Callable[Long] = { - () => { - Thread.sleep(amount) - amount - } + () => + { + Thread.sleep(amount) + amount + } } testTimer((amount, timer) => { timer.timeCallable(callable(amount)) @@ -123,10 +125,11 @@ class MetricsUtilsSuite extends AnyFunSuite { // Using Timer.time(supplier) def supplier(amount: Long): Supplier[Long] = { - () => { - Thread.sleep(amount) - throw new RuntimeException() - } + () => + { + Thread.sleep(amount) + throw new RuntimeException() + } } testTimer((amount, timer) => { catchException(() => timer.time(supplier(amount))) @@ -134,10 +137,11 @@ class MetricsUtilsSuite extends AnyFunSuite { // Using Timer.timeCallable def callable(amount: Long): Callable[Long] = { - () => { - Thread.sleep(amount) - throw new RuntimeException() - } + () => + { + Thread.sleep(amount) + throw new RuntimeException() + } } testTimer((amount, timer) => { catchException(() => timer.timeCallable(callable(amount))) @@ -145,10 +149,11 @@ class MetricsUtilsSuite extends AnyFunSuite { // Using Timer.time(runnable) def runnable(amount: Long): Runnable = { - () => { - Thread.sleep(amount) - throw new RuntimeException() - } + () => + { + Thread.sleep(amount) + throw new RuntimeException() + } } testTimer((amount, timer) => { catchException(() => timer.time(runnable(amount))) diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala index be6dcb4b192..427226fb867 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala @@ -15,16 +15,16 @@ */ package io.delta.kernel.test +import java.io.ByteArrayInputStream +import java.util +import java.util.Optional + +import io.delta.kernel.data.{ColumnarBatch, ColumnVector, FilteredColumnarBatch, Row} import io.delta.kernel.engine._ -import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, Row} import io.delta.kernel.expressions.{Column, Expression, ExpressionEvaluator, Predicate, PredicateEvaluator} import io.delta.kernel.types.{DataType, StructType} import io.delta.kernel.utils.{CloseableIterator, DataFileStatus, FileStatus} -import java.io.ByteArrayInputStream -import java.util -import java.util.Optional - /** * Contains broiler plate code for mocking [[Engine]] and its sub-interfaces. * @@ -44,15 +44,16 @@ import java.util.Optional * }}} */ trait MockEngineUtils { + /** * Create a mock Engine with the given components. If a component is not provided, it will * throw an exception when accessed. */ def mockEngine( - fileSystemClient: FileSystemClient = null, - jsonHandler: JsonHandler = null, - parquetHandler: ParquetHandler = null, - expressionHandler: ExpressionHandler = null): Engine = { + fileSystemClient: FileSystemClient = null, + jsonHandler: JsonHandler = null, + parquetHandler: ParquetHandler = null, + expressionHandler: ExpressionHandler = null): Engine = { new Engine() { override def getExpressionHandler: ExpressionHandler = Option(expressionHandler).getOrElse( diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala index a40e55093ae..98e93362abe 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala @@ -17,16 +17,15 @@ package io.delta.kernel.test import java.util.{Optional, UUID} +import scala.collection.JavaConverters._ + import io.delta.kernel.engine._ +import io.delta.kernel.internal.MockReadLastCheckpointFileJsonHandler import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.util.FileNames import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.utils.{CloseableIterator, FileStatus} -import scala.collection.JavaConverters._ - -import io.delta.kernel.internal.MockReadLastCheckpointFileJsonHandler - /** * This is an extension to [[BaseMockFileSystemClient]] containing specific mock implementations * [[FileSystemClient]] which are shared across multiple test suite. @@ -41,25 +40,24 @@ trait MockFileSystemClientUtils extends MockEngineUtils { /** Delta file statuses where the timestamp = 10*version */ def deltaFileStatuses(deltaVersions: Seq[Long]): Seq[FileStatus] = { assert(deltaVersions.size == deltaVersions.toSet.size) - deltaVersions.map(v => FileStatus.of(FileNames.deltaFile(logPath, v), v, v*10)) + deltaVersions.map(v => FileStatus.of(FileNames.deltaFile(logPath, v), v, v * 10)) } /** Checkpoint file statuses where the timestamp = 10*version */ def singularCheckpointFileStatuses(checkpointVersions: Seq[Long]): Seq[FileStatus] = { assert(checkpointVersions.size == checkpointVersions.toSet.size) checkpointVersions.map(v => - FileStatus.of(FileNames.checkpointFileSingular(logPath, v).toString, v, v*10) - ) + FileStatus.of(FileNames.checkpointFileSingular(logPath, v).toString, v, v * 10)) } /** Checkpoint file statuses where the timestamp = 10*version */ def multiCheckpointFileStatuses( - checkpointVersions: Seq[Long], numParts: Int): Seq[FileStatus] = { + checkpointVersions: Seq[Long], + numParts: Int): Seq[FileStatus] = { assert(checkpointVersions.size == checkpointVersions.toSet.size) checkpointVersions.flatMap(v => FileNames.checkpointFileWithParts(logPath, v, numParts).asScala - .map(p => FileStatus.of(p.toString, v, v*10)) - ) + .map(p => FileStatus.of(p.toString, v, v * 10))) } /** @@ -74,15 +72,22 @@ trait MockFileSystemClientUtils extends MockEngineUtils { fileType: String): Seq[(FileStatus, Seq[FileStatus])] = { checkpointVersions.map { case (v, useUUID, numSidecars) => val topLevelFile = if (useUUID) { - FileStatus.of(FileNames.topLevelV2CheckpointFile( - logPath, v, UUID.randomUUID().toString, fileType).toString, v, v * 10) + FileStatus.of( + FileNames.topLevelV2CheckpointFile( + logPath, + v, + UUID.randomUUID().toString, + fileType).toString, + v, + v * 10) } else { FileStatus.of(FileNames.checkpointFileSingular(logPath, v).toString, v, v * 10) } val sidecars = (0 until numSidecars).map { _ => FileStatus.of( FileNames.v2CheckpointSidecarFile(logPath, UUID.randomUUID().toString).toString, - v, v * 10) + v, + v * 10) } (topLevelFile, sidecars) } @@ -103,25 +108,25 @@ trait MockFileSystemClientUtils extends MockEngineUtils { contents: Seq[FileStatus], parquetHandler: ParquetHandler, jsonHandler: JsonHandler): Engine = { - mockEngine(fileSystemClient = - new MockListFromFileSystemClient(listFromProvider(contents)), + mockEngine( + fileSystemClient = + new MockListFromFileSystemClient(listFromProvider(contents)), parquetHandler = parquetHandler, jsonHandler = jsonHandler) } def createMockFSAndJsonEngineForLastCheckpoint( - contents: Seq[FileStatus], lastCheckpointVersion: Optional[java.lang.Long]): Engine = { + contents: Seq[FileStatus], + lastCheckpointVersion: Optional[java.lang.Long]): Engine = { mockEngine( fileSystemClient = new MockListFromFileSystemClient(listFromProvider(contents)), jsonHandler = if (lastCheckpointVersion.isPresent) { new MockReadLastCheckpointFileJsonHandler( s"$logPath/_last_checkpoint", - lastCheckpointVersion.get() - ) + lastCheckpointVersion.get()) } else { null - } - ) + }) } /** @@ -166,7 +171,7 @@ class MockListFromFileSystemClient(listFromProvider: String => Seq[FileStatus]) * It also maintains metrics on number of times `listFrom` is called and arguments for each call. */ class MockListFromResolvePathFileSystemClient(listFromProvider: String => Seq[FileStatus]) - extends BaseMockFileSystemClient { + extends BaseMockFileSystemClient { private var listFromCalls: Seq[String] = Seq.empty override def listFrom(filePath: String): CloseableIterator[FileStatus] = { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/VectorTestUtils.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/VectorTestUtils.scala index 7396522909d..0b29f86ac74 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/VectorTestUtils.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/VectorTestUtils.scala @@ -15,13 +15,14 @@ */ package io.delta.kernel.test +import java.lang.{Boolean => BooleanJ, Double => DoubleJ, Float => FloatJ} + +import scala.collection.JavaConverters._ + import io.delta.kernel.data.{ColumnVector, MapValue} import io.delta.kernel.internal.util.VectorUtils import io.delta.kernel.types._ -import java.lang.{Boolean => BooleanJ, Double => DoubleJ, Float => FloatJ} -import scala.collection.JavaConverters._ - trait VectorTestUtils { protected def booleanVector(values: Seq[BooleanJ]): ColumnVector = { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/types/CollationIdentifierSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/types/CollationIdentifierSuite.scala index 2e4f8c29947..22b3f713804 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/types/CollationIdentifierSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/types/CollationIdentifierSuite.scala @@ -15,10 +15,10 @@ */ package io.delta.kernel.types -import org.scalatest.funsuite.AnyFunSuite - import java.util.Optional +import org.scalatest.funsuite.AnyFunSuite + class CollationIdentifierSuite extends AnyFunSuite { val PROVIDER_SPARK = "SPARK" val PROVIDER_ICU = "ICU" @@ -29,18 +29,14 @@ class CollationIdentifierSuite extends AnyFunSuite { Seq( ( s"$PROVIDER_SPARK.$DEFAULT_COLLATION_NAME", - DEFAULT_COLLATION_IDENTIFIER - ), + DEFAULT_COLLATION_IDENTIFIER), ( s"$PROVIDER_ICU.sr_Cyrl_SRB", - CollationIdentifier.fromString(s"$PROVIDER_ICU.sr_Cyrl_SRB") - ), + CollationIdentifier.fromString(s"$PROVIDER_ICU.sr_Cyrl_SRB")), ( s"$PROVIDER_ICU.sr_Cyrl_SRB.75.1", - CollationIdentifier.fromString(s"$PROVIDER_ICU.sr_Cyrl_SRB.75.1") - ) - ).foreach { - case(stringIdentifier, collationIdentifier) => + CollationIdentifier.fromString(s"$PROVIDER_ICU.sr_Cyrl_SRB.75.1"))).foreach { + case (stringIdentifier, collationIdentifier) => assert(CollationIdentifier.fromString(stringIdentifier).equals(collationIdentifier)) } } @@ -48,8 +44,7 @@ class CollationIdentifierSuite extends AnyFunSuite { test("check fromString with invalid string") { Seq( PROVIDER_SPARK, - s"${PROVIDER_SPARK}_sr_Cyrl_SRB" - ).foreach { + s"${PROVIDER_SPARK}_sr_Cyrl_SRB").foreach { stringIdentifier => val e = intercept[IllegalArgumentException] { val collationIdentifier = CollationIdentifier.fromString(stringIdentifier) @@ -62,18 +57,14 @@ class CollationIdentifierSuite extends AnyFunSuite { Seq( ( DEFAULT_COLLATION_IDENTIFIER, - s"$PROVIDER_SPARK.$DEFAULT_COLLATION_NAME" - ), + s"$PROVIDER_SPARK.$DEFAULT_COLLATION_NAME"), ( CollationIdentifier.fromString(s"$PROVIDER_ICU.sr_Cyrl_SRB"), - s"$PROVIDER_ICU.SR_CYRL_SRB" - ), + s"$PROVIDER_ICU.SR_CYRL_SRB"), ( CollationIdentifier.fromString(s"$PROVIDER_ICU.sr_Cyrl_SRB.75.1"), - s"$PROVIDER_ICU.SR_CYRL_SRB" - ) - ).foreach { - case(collationIdentifier, toStringWithoutVersion) => + s"$PROVIDER_ICU.SR_CYRL_SRB")).foreach { + case (collationIdentifier, toStringWithoutVersion) => assert(collationIdentifier.toStringWithoutVersion == toStringWithoutVersion) } } @@ -82,18 +73,14 @@ class CollationIdentifierSuite extends AnyFunSuite { Seq( ( DEFAULT_COLLATION_IDENTIFIER, - s"$PROVIDER_SPARK.$DEFAULT_COLLATION_NAME" - ), + s"$PROVIDER_SPARK.$DEFAULT_COLLATION_NAME"), ( CollationIdentifier.fromString(s"$PROVIDER_ICU.sr_Cyrl_SRB"), - s"$PROVIDER_ICU.SR_CYRL_SRB" - ), + s"$PROVIDER_ICU.SR_CYRL_SRB"), ( CollationIdentifier.fromString(s"$PROVIDER_ICU.sr_Cyrl_SRB.75.1"), - s"$PROVIDER_ICU.SR_CYRL_SRB.75.1" - ) - ).foreach { - case(collationIdentifier, toString) => + s"$PROVIDER_ICU.SR_CYRL_SRB.75.1")).foreach { + case (collationIdentifier, toString) => assert(collationIdentifier.toString == toString) } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/types/FieldMetadataSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/types/FieldMetadataSuite.scala index 2b28bc83b80..9353194930c 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/types/FieldMetadataSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/types/FieldMetadataSuite.scala @@ -33,10 +33,10 @@ class FieldMetadataSuite extends AnyFunSuite { } test("retrieving key with wrong type throws exception") { - val longs : Seq[java.lang.Long] = Seq(1L, 2L, 3L) - val doubles : Seq[java.lang.Double] = Seq(1.0, 2.0, 3.0) - val booleans : Seq[java.lang.Boolean] = Seq(true, false, true) - val strings : Seq[java.lang.String] = Seq("a", "b", "c") + val longs: Seq[java.lang.Long] = Seq(1L, 2L, 3L) + val doubles: Seq[java.lang.Double] = Seq(1.0, 2.0, 3.0) + val booleans: Seq[java.lang.Boolean] = Seq(true, false, true) + val strings: Seq[java.lang.String] = Seq("a", "b", "c") val innerMeta = FieldMetadata.builder().putBoolean("key", true).build() val meta = FieldMetadata.builder() .putLong("longKey", 23L) @@ -83,10 +83,10 @@ class FieldMetadataSuite extends AnyFunSuite { } test("retrieving key with correct type returns value") { - val longs : Seq[java.lang.Long] = Seq(1L, 2L, 3L) - val doubles : Seq[java.lang.Double] = Seq(1.0, 2.0, 3.0) - val booleans : Seq[java.lang.Boolean] = Seq(true, false, true) - val strings : Seq[java.lang.String] = Seq("a", "b", "c") + val longs: Seq[java.lang.Long] = Seq(1L, 2L, 3L) + val doubles: Seq[java.lang.Double] = Seq(1.0, 2.0, 3.0) + val booleans: Seq[java.lang.Boolean] = Seq(true, false, true) + val strings: Seq[java.lang.String] = Seq("a", "b", "c") val innerMeta = FieldMetadata.builder().putBoolean("key", true).build() val meta = FieldMetadata.builder() .putLong("longKey", 23L) diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/types/StringTypeSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/types/StringTypeSuite.scala index d6acfa47e93..4fccf22e7d4 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/types/StringTypeSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/types/StringTypeSuite.scala @@ -24,34 +24,27 @@ class StringTypeSuite extends AnyFunSuite { ( StringType.STRING, StringType.STRING, - true - ), + true), ( StringType.STRING, new StringType("sPark.UTF8_bINary"), - true - ), + true), ( StringType.STRING, new StringType("SPARK.UTF8_LCASE"), - false - ), + false), ( new StringType("ICU.UNICODE"), new StringType("SPARK.UTF8_LCASE"), - false - ), + false), ( new StringType("ICU.UNICODE"), new StringType("ICU.UNICODE_CI"), - false - ), + false), ( new StringType("ICU.UNICODE_CI"), new StringType("icU.uniCODe_Ci"), - true - ) - ).foreach { + true)).foreach { case (st1, st2, expResult) => assert(st1.equals(st2) == expResult) } diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/data/DefaultJsonRow.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/data/DefaultJsonRow.java index c30e014990a..ca3457d46bd 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/data/DefaultJsonRow.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/data/DefaultJsonRow.java @@ -308,10 +308,23 @@ public ColumnVector getElements() { List values = new ArrayList<>(jsonValue.size()); final Iterator> iter = jsonValue.fields(); + boolean isValueOfStringType = mapType.getValueType() instanceof StringType; while (iter.hasNext()) { Map.Entry entry = iter.next(); String keyParsed = entry.getKey(); - Object valueParsed = decodeElement(entry.getValue(), mapType.getValueType()); + + Object valueParsed = null; + if (isValueOfStringType) { + // Special handling for value which is of type string. Delta tables generated by + // Delta-Spark ended up having serializing values as their original type and not + // as string in the Delta commit files. + // Ex. {"key": true} instead of {"key": "true"} + if (!entry.getValue().isNull()) { + valueParsed = entry.getValue().asText(); + } + } else { + valueParsed = decodeElement(entry.getValue(), mapType.getValueType()); + } if (valueParsed == null && !mapType.isValueContainsNull()) { throw new RuntimeException( "Map type expects no nulls in values, but " + "received `null` as value"); diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala index 959acb9238f..f32a89a95b0 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala @@ -22,20 +22,21 @@ import scala.collection.JavaConverters._ import io.delta.kernel.defaults.engine.DefaultEngine import io.delta.kernel.defaults.utils.{ExpressionTestUtils, TestRow, TestUtils} import io.delta.kernel.expressions.Literal -import io.delta.kernel.internal.checkpoints.CheckpointInstance import io.delta.kernel.internal.{InternalScanFileUtils, SnapshotImpl} +import io.delta.kernel.internal.checkpoints.CheckpointInstance import io.delta.tables.DeltaTable -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.scalatest.funsuite.AnyFunSuite -import org.apache.spark.sql.Row import org.apache.spark.sql.delta.{DeltaLog, Snapshot} import org.apache.spark.sql.delta.actions.{AddFile, Metadata, Protocol} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.DeltaTestImplicits.OptimisticTxnTestHelper import org.apache.spark.sql.delta.util.FileNames + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.Row import org.apache.spark.sql.types.{BooleanType, IntegerType, LongType, MapType, StringType, StructType} +import org.scalatest.funsuite.AnyFunSuite class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils { private final val supportedFileFormats = Seq("json", "parquet") @@ -88,14 +89,13 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe // interval of 2, this will be the most recent even version. val expectedV2CkptToRead = ckptVersionExpected.getOrElse(snapshotFromSpark.version - (snapshotFromSpark.version % 2)) - assert(snapshotImpl.getLogSegment.getCheckpoints.asScala.map( - f => FileNames.checkpointVersion(new Path(f.getPath))) + assert(snapshotImpl.getLogSegment.getCheckpoints.asScala.map(f => + FileNames.checkpointVersion(new Path(f.getPath))) .contains(expectedV2CkptToRead)) - assert(snapshotImpl.getLogSegment.getCheckpoints.asScala.map( - f => new CheckpointInstance(f.getPath).format == CheckpointInstance.CheckpointFormat.V2) + assert(snapshotImpl.getLogSegment.getCheckpoints.asScala.map(f => + new CheckpointInstance(f.getPath).format == CheckpointInstance.CheckpointFormat.V2) .contains(expectV2CheckpointFormat)) - // Validate AddFiles from sidecars found against Spark connector. val scan = snapshot.getScanBuilder().build() val foundFiles = @@ -115,7 +115,8 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe val tbl = "tbl" withTable(tbl) { // Create table. - withSQLConf(DeltaSQLConf.CHECKPOINT_V2_TOP_LEVEL_FILE_FORMAT.key -> format, + withSQLConf( + DeltaSQLConf.CHECKPOINT_V2_TOP_LEVEL_FILE_FORMAT.key -> format, "spark.databricks.delta.clusteredTable.enableClusteringTablePreview" -> "true") { createSourceTable(tbl, path.toString, "CLUSTER") @@ -153,7 +154,8 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe val tbl = "tbl" withTable(tbl) { // Create table. - withSQLConf(DeltaSQLConf.CHECKPOINT_V2_TOP_LEVEL_FILE_FORMAT.key -> format, + withSQLConf( + DeltaSQLConf.CHECKPOINT_V2_TOP_LEVEL_FILE_FORMAT.key -> format, DeltaSQLConf.DELTA_CHECKPOINT_PART_SIZE.key -> "1", // Ensure 1 action per checkpoint. "spark.databricks.delta.clusteredTable.enableClusteringTablePreview" -> "true") { createSourceTable(tbl, path.toString, "CLUSTER") @@ -163,8 +165,7 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe validateSnapshot(path.toString, DeltaLog.forTable(spark, path.toString).update()) checkTable( path = path.toString, - expectedAnswer = (1 to 6).map(i => TestRow(i, (i - 1 + 'a').toChar.toString)) - ) + expectedAnswer = (1 to 6).map(i => TestRow(i, (i - 1 + 'a').toChar.toString))) // Remove some files from the table, then add a new one. spark.sql(s"DELETE FROM $tbl WHERE a=1 OR a=2") @@ -174,8 +175,7 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe validateSnapshot(path.toString, DeltaLog.forTable(spark, path.toString).update()) checkTable( path = path.toString, - expectedAnswer = (3 to 8).map(i => TestRow(i, (i - 1 + 'a').toChar.toString)) - ) + expectedAnswer = (3 to 8).map(i => TestRow(i, (i - 1 + 'a').toChar.toString))) } } } @@ -187,7 +187,8 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe val tbl = "tbl" withTable(tbl) { // Create table. - withSQLConf(DeltaSQLConf.CHECKPOINT_V2_TOP_LEVEL_FILE_FORMAT.key -> format, + withSQLConf( + DeltaSQLConf.CHECKPOINT_V2_TOP_LEVEL_FILE_FORMAT.key -> format, DeltaSQLConf.DELTA_CHECKPOINT_PART_SIZE.key -> "1", // Ensure 1 action per checkpoint. "spark.databricks.delta.clusteredTable.enableClusteringTablePreview" -> "true") { createSourceTable(tbl, path.toString, "CLUSTER") @@ -205,15 +206,20 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe .filter(f => !f.getName.endsWith(".crc")).head.toURI).toUri // Create new empty sidecar file, then move it to the sidecar filepath. Delete the sidecar // checksum file to prevent corruption check. - spark.createDataFrame(spark.sparkContext.parallelize(Seq.empty[Row]), - new StructType() - .add("add", new StructType() + spark.createDataFrame( + spark.sparkContext.parallelize(Seq.empty[Row]), + new StructType() + .add( + "add", + new StructType() .add("path", StringType) .add("partitionValues", MapType(StringType, StringType)) .add("size", LongType) .add("modificationTime", LongType) .add("dataChange", BooleanType)) - .add("remove", new StructType() + .add( + "remove", + new StructType() .add("path", StringType))) .coalesce(1) .write.mode("append").parquet(tmpPath.toString) @@ -240,12 +246,13 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe val log = DeltaLog.forTable(spark, new Path(path.toString)) new File(log.logPath.toUri).mkdirs() - val metadata = Metadata("testId", schemaString = "{\"type\":\"struct\",\"fields\":[" + - "{\"name\":\"a\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}") + val metadata = Metadata( + "testId", + schemaString = "{\"type\":\"struct\",\"fields\":[" + + "{\"name\":\"a\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}") val supportedFeatures = Set("v2Checkpoint", "appendOnly", "invariants") val protocol = Protocol(3, 7, Some(Set("v2Checkpoint")), Some(supportedFeatures)) - val add = AddFile(new Path("addfile").toUri.toString, Map.empty, 100L, - 10L, dataChange = true) + val add = AddFile(new Path("addfile").toUri.toString, Map.empty, 100L, 10L, dataChange = true) log.startTransaction().commitManually(Seq(metadata, add): _*) log.upgradeProtocol(None, log.update(), protocol) @@ -262,8 +269,10 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe // Rename to UUID. val ckptPath = new Path(new File(log.logPath.toUri).listFiles().filter(f => FileNames.isCheckpointFile(new Path(f.getPath))).head.toURI) - new File(ckptPath.toUri).renameTo(new File(new Path(ckptPath.getParent, ckptPath.getName - .replace("checkpoint.parquet", "checkpoint.abc-def.parquet")).toUri)) + new File(ckptPath.toUri).renameTo(new File(new Path( + ckptPath.getParent, + ckptPath.getName + .replace("checkpoint.parquet", "checkpoint.abc-def.parquet")).toUri)) // Validate snapshot. validateSnapshot(path.toString, snapshotFromSpark, ckptVersionExpected = Some(1)) @@ -275,7 +284,8 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe val tbl = "tbl" withTable(tbl) { // Create checkpoint with sidecars. - withSQLConf(DeltaSQLConf.CHECKPOINT_V2_TOP_LEVEL_FILE_FORMAT.key -> "parquet", + withSQLConf( + DeltaSQLConf.CHECKPOINT_V2_TOP_LEVEL_FILE_FORMAT.key -> "parquet", "spark.databricks.delta.clusteredTable.enableClusteringTablePreview" -> "true") { spark.conf.set(DeltaSQLConf.CHECKPOINT_V2_TOP_LEVEL_FILE_FORMAT.key, "parquet") createSourceTable(tbl, path.toString, "CLUSTER") @@ -296,8 +306,7 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe validateSnapshot(path.toString, snapshotFromSpark, expectV2CheckpointFormat = false) checkTable( path = path.toString, - expectedAnswer = (1 to 6).map(i => TestRow(i, (i - 1 + 'a').toChar.toString)) - ) + expectedAnswer = (1 to 6).map(i => TestRow(i, (i - 1 + 'a').toChar.toString))) } } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala new file mode 100644 index 00000000000..754a1572b00 --- /dev/null +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala @@ -0,0 +1,173 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.defaults + +import java.io.File +import java.nio.file.Files +import java.util + +import scala.collection.immutable.Seq +import scala.jdk.CollectionConverters.setAsJavaSetConverter + +import io.delta.kernel.{Operation, Table} +import io.delta.kernel.data.Row +import io.delta.kernel.defaults.utils.TestUtils +import io.delta.kernel.engine.Engine +import io.delta.kernel.internal.DeltaLogActionUtils.DeltaAction +import io.delta.kernel.internal.TableImpl +import io.delta.kernel.internal.actions.{AddFile, SingleAction} +import io.delta.kernel.internal.checksum.{ChecksumReader, CRCInfo} +import io.delta.kernel.internal.fs.Path +import io.delta.kernel.internal.util.FileNames +import io.delta.kernel.internal.util.Utils.toCloseableIterator +import io.delta.kernel.types.LongType.LONG +import io.delta.kernel.types.StructType +import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} + +import org.apache.spark.sql.functions.col + +/** + * Test suite to verify checksum file correctness by comparing + * Delta Spark and Delta Kernel generated checksum files. + * This suite ensures that both implementations generate consistent checksums + * for various table operations. + */ +class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUtils { + + private val PARTITION_COLUMN = "part" + + test("create table, insert data and verify checksum") { + withTempDirAndEngine { (tablePath, engine) => + val sparkTablePath = tablePath + "spark" + val kernelTablePath = tablePath + "kernel" + + createTxn( + engine, + kernelTablePath, + isNewTable = true, + schema = new StructType().add("id", LONG), + partCols = Seq.empty).commit(engine, emptyIterable()) + .getPostCommitHooks + .forEach(hook => hook.threadSafeInvoke(engine)) + spark.sql(s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` (id LONG) USING DELTA") + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + + (1 to 10).foreach { version => + spark.range(0, version).write.format("delta").mode("append").save(sparkTablePath) + commitSparkChangeToKernel(kernelTablePath, engine, sparkTablePath, version) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, version) + } + } + } + + test("create partitioned table, insert and verify checksum") { + withTempDirAndEngine { (tablePath, engine) => + val sparkTablePath = tablePath + "spark" + val kernelTablePath = tablePath + "kernel" + + createTxn( + engine, + kernelTablePath, + isNewTable = true, + schema = new StructType().add("id", LONG).add(PARTITION_COLUMN, LONG), + partCols = Seq(PARTITION_COLUMN)).commit(engine, emptyIterable()) + .getPostCommitHooks + .forEach(hook => hook.threadSafeInvoke(engine)) + spark.sql( + s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` " + + s"(id LONG, part LONG) USING DELTA PARTITIONED BY (part)") + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + + (1 to 10).foreach { version => + spark.range(0, version).withColumn(PARTITION_COLUMN, col("id") % 2) + .write.format("delta").mode("append").save(sparkTablePath) + commitSparkChangeToKernel(kernelTablePath, engine, sparkTablePath, version) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, version) + } + } + } + + private def assertChecksumEquals( + engine: Engine, + sparkTablePath: String, + kernelTablePath: String, + version: Long): Unit = { + val sparkCrcPath = buildCrcPath(sparkTablePath, version) + val kernelCrcPath = buildCrcPath(kernelTablePath, version) + + assert( + Files.exists(sparkCrcPath) && Files.exists(kernelCrcPath), + s"CRC files not found for version $version") + + val sparkCrc = readCrcInfo(engine, sparkTablePath, version) + val kernelCrc = readCrcInfo(engine, kernelTablePath, version) + + assertCrcInfoEquals(sparkCrc, kernelCrc) + } + + private def readCrcInfo(engine: Engine, path: String, version: Long): CRCInfo = { + ChecksumReader + .getCRCInfo(engine, new Path(s"$path/_delta_log"), version, version) + .orElseThrow(() => new IllegalStateException(s"CRC info not found for version $version")) + } + + private def buildCrcPath(basePath: String, version: Long): java.nio.file.Path = { + new File(FileNames.checksumFile(new Path(f"$basePath/_delta_log"), version).toString).toPath + } + + // TODO: Add equals/hashCode to metadata, protocol then CRCInfo. + private def assertCrcInfoEquals(crc1: CRCInfo, crc2: CRCInfo): Unit = { + assert(crc1.getVersion === crc2.getVersion) + assert(crc1.getNumFiles === crc2.getNumFiles) + assert(crc1.getTableSizeBytes === crc2.getTableSizeBytes) + assert(crc1.getMetadata.getSchema === crc2.getMetadata.getSchema) + assert(crc1.getMetadata.getPartitionColNames === crc2.getMetadata.getPartitionColNames) + } + + // TODO docs + private def commitSparkChangeToKernel( + path: String, + engine: Engine, + sparkTablePath: String, + versionToConvert: Long): Unit = { + + val txn = Table.forPath(engine, path) + .createTransactionBuilder(engine, "test-engine", Operation.WRITE) + .build(engine) + + val tableChange = Table.forPath(engine, sparkTablePath).asInstanceOf[TableImpl].getChanges( + engine, + versionToConvert, + versionToConvert, + // TODO include REMOVE action as well once we support it + Set(DeltaAction.ADD).asJava) + + val addFilesRows = new util.ArrayList[Row]() + tableChange.forEach(batch => + batch.getRows.forEach(row => { + val addIndex = row.getSchema.indexOf("add") + if (!row.isNullAt(addIndex)) { + addFilesRows.add( + SingleAction.createAddFileSingleAction(new AddFile(row.getStruct(addIndex)).toRow)) + } + })) + + txn + .commit(engine, inMemoryIterable(toCloseableIterator(addFilesRows.iterator()))) + .getPostCommitHooks + .forEach(_.threadSafeInvoke(engine)) + } +} diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala index a27a225edd7..54482c6093e 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala @@ -15,24 +15,25 @@ */ package io.delta.kernel.defaults +import java.io.File + import io.delta.golden.GoldenTableUtils.goldenTablePath -import io.delta.kernel.defaults.utils.{TestRow, TestUtils} import io.delta.kernel.Table -import org.apache.commons.io.FileUtils -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import io.delta.kernel.defaults.engine.DefaultEngine +import io.delta.kernel.defaults.utils.{TestRow, TestUtils} +import io.delta.kernel.engine.Engine +import io.delta.kernel.exceptions.{CheckpointAlreadyExistsException, TableNotFoundException} +import org.apache.spark.sql.delta.{DeltaLog, VersionNotFoundException} import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate import org.apache.spark.sql.delta.actions.{AddFile, Metadata, RemoveFile} import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.{DeltaLog, VersionNotFoundException} + +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.spark.sql.types.{IntegerType, StructType} import org.scalatest.funsuite.AnyFunSuite -import java.io.File - -import io.delta.kernel.defaults.engine.DefaultEngine -import io.delta.kernel.engine.Engine -import io.delta.kernel.exceptions.{CheckpointAlreadyExistsException, TableNotFoundException} /** * Test suite for `io.delta.kernel.Table.checkpoint(engine, version)` @@ -70,7 +71,6 @@ class CreateCheckpointSuite extends DeltaTableWriteSuiteBase { } } - Seq(true, false).foreach { includeRemoves => Seq( // Create a checkpoint using Spark (either classic or multi-part checkpoint) @@ -152,7 +152,8 @@ class CreateCheckpointSuite extends DeltaTableWriteSuiteBase { | 'delta.minReaderVersion' = '1', | 'delta.minWriterVersion' = '2' |) - |""".stripMargin) // makes the latest table version 16 + |""".stripMargin + ) // makes the latest table version 16 // before creating checkpoint, read and save the expected results using Spark val expResults = readUsingSpark(tablePath) @@ -252,7 +253,15 @@ class CreateCheckpointSuite extends DeltaTableWriteSuiteBase { // version 0 addFiles( - "file1", "file2", "file3", "file4", "file5", "file6", "file7", "file8", "file9") + "file1", + "file2", + "file3", + "file4", + "file5", + "file6", + "file7", + "file8", + "file9") val now = System.currentTimeMillis() removeFile("file8", deletionTimestamp = 1) // set delete time very old @@ -263,7 +272,15 @@ class CreateCheckpointSuite extends DeltaTableWriteSuiteBase { // add few more files - version 5 addFiles( - "file10", "file11", "file12", "file13", "file14", "file15", "file16", "file17", "file18") + "file10", + "file11", + "file12", + "file13", + "file14", + "file15", + "file16", + "file17", + "file18") // delete some files again removeFile("file3", deletionTimestamp = now - millisPerDays(9)) @@ -345,13 +362,32 @@ class CreateCheckpointSuite extends DeltaTableWriteSuiteBase { test("try create a checkpoint on a unsupported table feature table") { withTempDirAndEngine { (tablePath, tc) => - copyTable("dv-with-columnmapping", tablePath) + spark.sql(s"CREATE TABLE delta.`$tablePath` (name STRING, age INT) USING delta " + + "TBLPROPERTIES ('delta.constraints.checks' = 'name IS NOT NULL')") + + for (_ <- 0 to 3) { + spark.sql(s"INSERT INTO delta.`$tablePath` VALUES ('John Doe', 30), ('Bob Johnson', 35)") + } val ex2 = intercept[Exception] { - kernelCheckpoint(tc, tablePath, checkpointVersion = 5) + kernelCheckpoint(tc, tablePath, checkpointVersion = 4) } - assert(ex2.getMessage.contains("Unsupported Delta writer feature") && - ex2.getMessage.contains("writer table feature \"deletionVectors\"")) + assert(ex2.getMessage.contains("requires writer table feature \"[checkConstraints]\" " + + "which is unsupported by this version of Delta Kernel")) + } + } + + test("create a checkpoint on a table with deletion vectors") { + withTempDirAndEngine { (tablePath, tc) => + copyTable("dv-with-columnmapping", tablePath) + + // before creating checkpoint, read and save the expected results using Spark + val expResults = readUsingSpark(tablePath) + assert(expResults.size === 35) + + val checkpointVersion = 15 + kernelCheckpoint(tc, tablePath, checkpointVersion) + verifyResults(tablePath, expResults, checkpointVersion) } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeletionVectorSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeletionVectorSuite.scala index fb99caf38d8..e0f38b2d0ca 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeletionVectorSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeletionVectorSuite.scala @@ -16,8 +16,8 @@ package io.delta.kernel.defaults import io.delta.golden.GoldenTableUtils.goldenTablePath - import io.delta.kernel.defaults.utils.{TestRow, TestUtils} + import org.apache.hadoop.conf.Configuration import org.scalatest.funsuite.AnyFunSuite @@ -26,15 +26,13 @@ class DeletionVectorSuite extends AnyFunSuite with TestUtils { test("end-to-end usage: reading a table with dv") { checkTable( path = getTestResourceFilePath("basic-dv-no-checkpoint"), - expectedAnswer = (2L until 10L).map(TestRow(_)) - ) + expectedAnswer = (2L until 10L).map(TestRow(_))) } test("end-to-end usage: reading a table with dv with checkpoint") { checkTable( path = getTestResourceFilePath("basic-dv-with-checkpoint"), - expectedAnswer = (0L until 500L).filter(_ % 11 != 0).map(TestRow(_)) - ) + expectedAnswer = (0L until 500L).filter(_ % 11 != 0).map(TestRow(_))) } test("end-to-end usage: reading partitioned dv table with checkpoint") { @@ -42,28 +40,26 @@ class DeletionVectorSuite extends AnyFunSuite with TestUtils { // Set the batch size small enough so there will be multiple batches conf.setInt("delta.kernel.default.parquet.reader.batch-size", 2) - val expectedResult = (0 until 50).map(x => (x%10, x, s"foo${x % 5}")) - .filter{ case (_, col1, _) => + val expectedResult = (0 until 50).map(x => (x % 10, x, s"foo${x % 5}")) + .filter { case (_, col1, _) => !(col1 % 2 == 0 && col1 < 30) } checkTable( path = goldenTablePath("dv-partitioned-with-checkpoint"), expectedAnswer = expectedResult.map(TestRow.fromTuple(_)), - engine = defaultEngine - ) + engine = defaultEngine) } test( "end-to-end usage: reading partitioned dv table with checkpoint with columnMappingMode=name") { - val expectedResult = (0 until 50).map(x => (x%10, x, s"foo${x % 5}")) - .filter{ case (_, col1, _) => + val expectedResult = (0 until 50).map(x => (x % 10, x, s"foo${x % 5}")) + .filter { case (_, col1, _) => !(col1 % 2 == 0 && col1 < 30) } checkTable( path = goldenTablePath("dv-with-columnmapping"), - expectedAnswer = expectedResult.map(TestRow.fromTuple(_)) - ) + expectedAnswer = expectedResult.map(TestRow.fromTuple(_))) } // TODO detect corrupted DV checksum diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableFeaturesSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableFeaturesSuite.scala new file mode 100644 index 00000000000..25ec6cf3d4b --- /dev/null +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableFeaturesSuite.scala @@ -0,0 +1,193 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.defaults + +import scala.collection.immutable.Seq + +import io.delta.kernel.engine.Engine +import io.delta.kernel.expressions.Literal + +import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.delta.actions.Protocol + +/** + * Integration test suite for Delta table features. + */ +class DeltaTableFeaturesSuite extends DeltaTableWriteSuiteBase { + + /////////////////////////////////////////////////////////////////////////// + // Tests for deletionVector, v2Checkpoint table features + /////////////////////////////////////////////////////////////////////////// + Seq( + // Test format: feature (readerWriter type), table property to enable the feature + // For each feature, we test the following scenarios: + // 1. able to write to an existing Delta table with the feature supported + // 2. create a table with the feature supported and append data + // 3. update an existing table with the feature supported + ("deletionVectors", "delta.enableDeletionVectors", "true"), + ("v2Checkpoint", "delta.checkpointPolicy", "v2")).foreach { + case (feature, tblProp, propValue) => + test(s"able to write to an existing Delta table with $feature supported") { + withTempDirAndEngine { (tablePath, engine) => + // Create a table with the feature supported + spark.sql(s"CREATE TABLE delta.`$tablePath` (id INTEGER) USING delta " + + s"TBLPROPERTIES ('$tblProp' = '$propValue')") + + checkReaderWriterFeaturesSupported(tablePath, feature) + + // Write data to the table using Kernel + val testData = Seq(Map.empty[String, Literal] -> dataBatches1) + appendData( + engine, + tablePath, + isNewTable = false, + testSchema, + partCols = Seq.empty, + testData) + + // Check the data using Kernel and Delta-Spark readers + verifyWrittenContent(tablePath, testSchema, dataBatches1.flatMap(_.toTestRows)) + } + } + + test(s"create a table with $feature supported") { + withTempDirAndEngine { (tablePath, engine) => + val testData = Seq(Map.empty[String, Literal] -> dataBatches1) + + // create a table with the feature supported and append testData + appendData( + engine, + tablePath, + isNewTable = true, + testSchema, + partCols = Seq.empty, + testData, + tableProperties = Map(tblProp -> propValue)) + + checkReaderWriterFeaturesSupported(tablePath, feature) + + // insert more data + appendData( + engine, + tablePath, + isNewTable = false, + testSchema, + partCols = Seq.empty, + testData) + + // Check the data using Kernel and Delta-Spark readers + verifyWrittenContent( + tablePath, + testSchema, + dataBatches1.flatMap(_.toTestRows) ++ dataBatches1.flatMap(_.toTestRows)) + } + } + + test(s"update an existing table with $feature support") { + withTempDirAndEngine { (tablePath, engine) => + val testData = Seq(Map.empty[String, Literal] -> dataBatches1) + + // create a table without the table feature supported + appendData( + engine, + tablePath, + isNewTable = true, + testSchema, + partCols = Seq.empty, + testData) + + checkNoReaderWriterFeaturesSupported(tablePath, feature) + + // insert more data and enable the feature + appendData( + engine, + tablePath, + isNewTable = false, + testSchema, + partCols = Seq.empty, + testData, + tableProperties = Map(tblProp -> propValue)) + + checkReaderWriterFeaturesSupported(tablePath, feature) + + // Check the data using Kernel and Delta-Spark readers + verifyWrittenContent( + tablePath, + testSchema, + dataBatches1.flatMap(_.toTestRows) ++ dataBatches1.flatMap(_.toTestRows)) + } + } + } + + /////////////////////////////////////////////////////////////////////////// + // Helper methods + /////////////////////////////////////////////////////////////////////////// + def checkWriterFeaturesSupported( + tblPath: String, + expWriterOnlyFeatures: String*): Unit = { + val protocol = getLatestProtocol(tblPath) + val missingFeatures = + expWriterOnlyFeatures.toSet -- protocol.writerFeatures.getOrElse(Set.empty) + + assert( + missingFeatures.isEmpty, + s"The following expected writer features are not supported: " + + s"${missingFeatures.mkString(", ")}") + } + + def checkNoWriterFeaturesSupported(tblPath: String, notExpWriterOnlyFeatures: String*): Unit = { + val protocol = getLatestProtocol(tblPath) + assert(protocol.writerFeatures.getOrElse(Set.empty) + .intersect(notExpWriterOnlyFeatures.toSet).isEmpty) + } + + def checkReaderWriterFeaturesSupported( + tblPath: String, + expectedReaderWriterFeatures: String*): Unit = { + + val protocol = getLatestProtocol(tblPath) + + val missingInWriterSet = + expectedReaderWriterFeatures.toSet -- protocol.writerFeatures.getOrElse(Set.empty) + assert( + missingInWriterSet.isEmpty, + s"The following expected readerWriter features are not supported in writerFeatures set: " + + s"${missingInWriterSet.mkString(", ")}") + + val missingInReaderSet = + expectedReaderWriterFeatures.toSet -- protocol.readerFeatures.getOrElse(Set.empty) + assert( + missingInReaderSet.isEmpty, + s"The following expected readerWriter features are not supported in readerFeatures set: " + + s"${missingInReaderSet.mkString(", ")}") + } + + def checkNoReaderWriterFeaturesSupported( + tblPath: String, + notExpReaderWriterFeatures: String*): Unit = { + val protocol = getLatestProtocol(tblPath) + assert(protocol.readerFeatures.getOrElse(Set.empty) + .intersect(notExpReaderWriterFeatures.toSet).isEmpty) + assert(protocol.writerFeatures.getOrElse(Set.empty) + .intersect(notExpReaderWriterFeatures.toSet).isEmpty) + } + + def getLatestProtocol(tblPath: String): Protocol = { + val deltaLog = DeltaLog.forTable(spark, tblPath) + deltaLog.update() + deltaLog.snapshot.protocol + } +} diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala index 55b8aa99724..162d303cc90 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala @@ -15,27 +15,30 @@ */ package io.delta.kernel.defaults +import java.io.File +import java.math.BigDecimal +import java.sql.Date +import java.time.Instant + +import scala.collection.JavaConverters._ + import io.delta.golden.GoldenTableUtils.goldenTablePath -import io.delta.kernel.exceptions.{InvalidTableException, KernelException, TableNotFoundException} +import io.delta.kernel.Table import io.delta.kernel.defaults.utils.{TestRow, TestUtils} +import io.delta.kernel.exceptions.{InvalidTableException, KernelException, TableNotFoundException} import io.delta.kernel.internal.TableImpl import io.delta.kernel.internal.fs.Path -import io.delta.kernel.internal.util.InternalUtils.daysSinceEpoch import io.delta.kernel.internal.util.{DateTimeConstants, FileNames} +import io.delta.kernel.internal.util.InternalUtils.daysSinceEpoch import io.delta.kernel.types.{LongType, StructType} -import io.delta.kernel.Table -import org.apache.hadoop.shaded.org.apache.commons.io.FileUtils + import org.apache.spark.sql.delta.{DeltaLog, DeltaOperations} import org.apache.spark.sql.delta.actions.{AddFile, Metadata} + +import org.apache.hadoop.shaded.org.apache.commons.io.FileUtils import org.apache.spark.sql.functions.col import org.scalatest.funsuite.AnyFunSuite -import java.io.File -import java.math.BigDecimal -import java.sql.Date -import java.time.Instant -import scala.collection.JavaConverters._ - class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { ////////////////////////////////////////////////////////////////////////////////// @@ -52,7 +55,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { 2 | 2021-10-01 08:09:20 | 2000-01-01 09:00:00 3 | 1969-01-01 00:00:00 | 1969-01-01 00:00:00 4 | null | null - */ + */ def row0: TestRow = TestRow( 0, @@ -81,20 +84,18 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { def row4: TestRow = TestRow( 4, null, - null - ) + null) def utcTableExpectedResult: Seq[TestRow] = Seq(row0, row1, row2, row3, row4) def testTimestampTable( - goldenTableName: String, - timeZone: String, - expectedResult: Seq[TestRow]): Unit = { + goldenTableName: String, + timeZone: String, + expectedResult: Seq[TestRow]): Unit = { withTimeZone(timeZone) { checkTable( path = goldenTablePath(goldenTableName), - expectedAnswer = expectedResult - ) + expectedAnswer = expectedResult) } } @@ -120,8 +121,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { null } else { values(2).asInstanceOf[Long] + DateTimeConstants.MICROS_PER_HOUR * 8 - } - ) + }) } for (timeZone <- Seq("UTC", "Iceland", "PST", "America/Los_Angeles")) { @@ -154,8 +154,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { TestRow(5, null, 1373043660123456L), TestRow(6, 1637202600123456L, null), TestRow(7, 1373043660123456L, null), - TestRow(8, null, null) - ) + TestRow(8, null, null)) Seq("", "-name-mode", "-id-mode").foreach { cmMode => test(s"end-to-end: read table with timestamp_ntz columns (including partition): $cmMode") { @@ -175,16 +174,17 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { ("234.00000", "1.00", "2.00000", "3.0000000000"), ("2342222.23454", "111.11", "22222.22222", "3333333333.3333333333"), ("0.00004", "0.00", "0.00000", "0E-10"), - ("-2342342.23423", "-999.99", "-99999.99999", "-9999999999.9999999999") - ).map { tup => - (new BigDecimal(tup._1), new BigDecimal(tup._2), new BigDecimal(tup._3), + ("-2342342.23423", "-999.99", "-99999.99999", "-9999999999.9999999999")).map { tup => + ( + new BigDecimal(tup._1), + new BigDecimal(tup._2), + new BigDecimal(tup._3), new BigDecimal(tup._4)) } checkTable( path = goldenTablePath(tablePath), - expectedAnswer = expectedResult.map(TestRow.fromTuple(_)) - ) + expectedAnswer = expectedResult.map(TestRow.fromTuple(_))) } } @@ -196,8 +196,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { checkTable( path = goldenTablePath("decimal-various-scale-precision"), - expectedAnswer = expResults - ) + expectedAnswer = expResults) } ////////////////////////////////////////////////////////////////////////////////// @@ -273,8 +272,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { test("end to end: multi-part checkpoint") { checkTable( path = goldenTablePath("multi-part-checkpoint"), - expectedAnswer = (Seq(0L) ++ (0L until 30L)).map(TestRow(_)) - ) + expectedAnswer = (Seq(0L) ++ (0L until 30L)).map(TestRow(_))) } test("read partitioned table") { @@ -303,8 +301,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { new BigDecimal(i), Seq(TestRow(i), TestRow(i), TestRow(i)), TestRow(i.toString, i.toString, TestRow(i, i.toLong)), - i.toString - ) + i.toString) } ++ (TestRow( null, null, @@ -319,14 +316,12 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { null, Seq(TestRow(2), TestRow(2), TestRow(2)), TestRow("2", "2", TestRow(2, 2L)), - "2" - ) :: Nil) + "2") :: Nil) checkTable( path = path, expectedAnswer = expectedAnswer, - readCols = readCols - ) + readCols = readCols) } test("table with complex array types") { @@ -338,20 +333,16 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { Seq(Seq(Seq(i, i, i), Seq(i, i, i)), Seq(Seq(i, i, i), Seq(i, i, i))), Seq( Seq(Seq(Seq(i, i, i), Seq(i, i, i)), Seq(Seq(i, i, i), Seq(i, i, i))), - Seq(Seq(Seq(i, i, i), Seq(i, i, i)), Seq(Seq(i, i, i), Seq(i, i, i))) - ), + Seq(Seq(Seq(i, i, i), Seq(i, i, i)), Seq(Seq(i, i, i), Seq(i, i, i)))), Seq( Map[String, Long](i.toString -> i.toLong), - Map[String, Long](i.toString -> i.toLong) - ), - Seq(TestRow(i), TestRow(i), TestRow(i)) - ) + Map[String, Long](i.toString -> i.toLong)), + Seq(TestRow(i), TestRow(i), TestRow(i))) } checkTable( path = path, - expectedAnswer = expectedAnswer - ) + expectedAnswer = expectedAnswer) } Seq("name", "id").foreach { columnMappingMode => @@ -379,8 +370,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { Seq(TestRow(i), TestRow(i)), // array_of_structs TestRow( // struct_of_arrays_maps_of_structs Seq(i, i + 1), - Map(Seq(i, i + 1) -> TestRow(i + 2)) - ), + Map(Seq(i, i + 1) -> TestRow(i + 2))), Map( i -> (i + 1).longValue(), (i + 2) -> (i + 3).longValue() @@ -396,15 +386,12 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { }, Map( // map_of_maps i.toLong -> Map(i -> i), - (i + 1).toLong -> Map(i + 2 -> i) - ) - ) + (i + 1).toLong -> Map(i + 2 -> i))) } ++ Seq(TestRow(Seq.fill(22)(null): _*)) // all nulls row, 22 columns checkTable( path = path, - expectedAnswer = expectedAnswer - ) + expectedAnswer = expectedAnswer) } } @@ -434,8 +421,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { checkTable( path = path, expectedAnswer = expectedAnswer, - readCols = Seq("ByteType", "decimal", "nested_struct", "array_of_prims", "map_of_prims") - ) + readCols = Seq("ByteType", "decimal", "nested_struct", "array_of_prims", "map_of_prims")) } } @@ -449,13 +435,21 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { val expectedAnswer = Seq( TestRow( - 1L, 2L, 3.4.toFloat.toDouble, 5.0, 6.0, 7.0, timestampToMicros("2024-09-09T00:00:00Z") - ), + 1L, + 2L, + 3.4.toFloat.toDouble, + 5.0, + 6.0, + 7.0, + timestampToMicros("2024-09-09T00:00:00Z")), TestRow( - Long.MaxValue, Long.MaxValue, 1.234567890123, 1.234567890123, 1.234567890123, - 1.234567890123, timestampToMicros("2024-09-09T12:34:56.123456Z") - ) - ) + Long.MaxValue, + Long.MaxValue, + 1.234567890123, + 1.234567890123, + 1.234567890123, + 1.234567890123, + timestampToMicros("2024-09-09T12:34:56.123456Z"))) checkTable( path = path, expectedAnswer = expectedAnswer, @@ -466,8 +460,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { "byte_double", "short_double", "int_double", - "date_timestamp_ntz") - ) + "date_timestamp_ntz")) } test("table with type widening to decimal types") { @@ -479,17 +472,14 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { BigDecimal.valueOf(10L, 1), BigDecimal.valueOf(20L, 1), BigDecimal.valueOf(30L, 1), - BigDecimal.valueOf(40L, 1) - ), + BigDecimal.valueOf(40L, 1)), TestRow( BigDecimal.valueOf(1234567890123456L, 2), BigDecimal.valueOf(1234567890123456L, 5), BigDecimal.valueOf(1234L, 1), BigDecimal.valueOf(123456L, 1), BigDecimal.valueOf(12345678901L, 1), - BigDecimal.valueOf(1234567890123456789L, 1) - ) - ) + BigDecimal.valueOf(1234567890123456789L, 1))) checkTable( path = path, expectedAnswer = expectedAnswer, @@ -499,9 +489,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { "byte_decimal", "short_decimal", "int_decimal", - "long_decimal" - ) - ) + "long_decimal")) } test("table with type widening to nested types") { @@ -509,14 +497,13 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { val expectedAnswer = Seq( TestRow(TestRow(1L), Map(2L -> 3L), Seq(4L, 5L)), TestRow( - TestRow(Long.MaxValue), Map(Long.MaxValue -> Long.MaxValue), - Seq(Long.MaxValue, Long.MinValue)) - ) + TestRow(Long.MaxValue), + Map(Long.MaxValue -> Long.MaxValue), + Seq(Long.MaxValue, Long.MinValue))) checkTable( path = path, expectedAnswer = expectedAnswer, - readCols = Seq("struct", "map", "array") - ) + readCols = Seq("struct", "map", "array")) } test("table with complex map types") { @@ -530,58 +517,58 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { Map(i.toShort -> (i % 2 == 0)), Map(i.toFloat -> i.toDouble), Map(i.toString -> new BigDecimal(i)), - Map(i -> Seq(TestRow(i), TestRow(i), TestRow(i))) - ) + Map(i -> Seq(TestRow(i), TestRow(i), TestRow(i)))) } checkTable( path = path, - expectedAnswer = expectedAnswer - ) + expectedAnswer = expectedAnswer) } test("table with array of primitives") { val expectedAnswer = (0 until 10).map { i => TestRow( - Seq(i), Seq(i.toLong), Seq(i.toByte), Seq(i.toShort), - Seq(i % 2 == 0), Seq(i.toFloat), Seq(i.toDouble), Seq(i.toString), - Seq(Array(i.toByte, i.toByte)), Seq(new BigDecimal(i)) - ) + Seq(i), + Seq(i.toLong), + Seq(i.toByte), + Seq(i.toShort), + Seq(i % 2 == 0), + Seq(i.toFloat), + Seq(i.toDouble), + Seq(i.toString), + Seq(Array(i.toByte, i.toByte)), + Seq(new BigDecimal(i))) } checkTable( path = goldenTablePath("data-reader-array-primitives"), - expectedAnswer = expectedAnswer - ) + expectedAnswer = expectedAnswer) } test("table primitives") { val expectedAnswer = (0 to 10).map { case 10 => TestRow(null, null, null, null, null, null, null, null, null, null) case i => TestRow( - i, - i.toLong, - i.toByte, - i.toShort, - i % 2 == 0, - i.toFloat, - i.toDouble, - i.toString, - Array[Byte](i.toByte, i.toByte), - new BigDecimal(i) - ) + i, + i.toLong, + i.toByte, + i.toShort, + i % 2 == 0, + i.toFloat, + i.toDouble, + i.toString, + Array[Byte](i.toByte, i.toByte), + new BigDecimal(i)) } checkTable( path = goldenTablePath("data-reader-primitives"), - expectedAnswer = expectedAnswer - ) + expectedAnswer = expectedAnswer) } test("table with checkpoint") { checkTable( path = getTestResourceFilePath("basic-with-checkpoint"), - expectedAnswer = (0 until 150).map(i => TestRow(i.toLong)) - ) + expectedAnswer = (0 until 150).map(i => TestRow(i.toLong))) } test(s"table with spaces in the table path") { @@ -591,8 +578,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { s"SELECT * FROM delta.`${getTestResourceFilePath("basic-with-checkpoint")}`") checkTable( path = target, - expectedAnswer = (0 until 150).map(i => TestRow(i.toLong)) - ) + expectedAnswer = (0 until 150).map(i => TestRow(i.toLong))) } } @@ -600,23 +586,21 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { val expectedAnswer = (0 to 10).map { case 10 => TestRow(null, null, null, null, null, null, null, null, null, null) case i => TestRow( - i, - i.toLong, - i.toByte, - i.toShort, - i % 2 == 0, - i.toFloat, - i.toDouble, - i.toString, - Array[Byte](i.toByte, i.toByte), - new BigDecimal(i) - ) + i, + i.toLong, + i.toByte, + i.toShort, + i % 2 == 0, + i.toFloat, + i.toDouble, + i.toString, + Array[Byte](i.toByte, i.toByte), + new BigDecimal(i)) } checkTable( path = getTestResourceFilePath("data-reader-primitives-column-mapping-name"), - expectedAnswer = expectedAnswer - ) + expectedAnswer = expectedAnswer) } test("partitioned table with column mapping") { @@ -629,14 +613,12 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { "as_int", "as_double", // data fields - "value" - ) + "value") checkTable( path = getTestResourceFilePath("data-reader-partition-values-column-mapping-name"), readCols = readCols, - expectedAnswer = expectedAnswer - ) + expectedAnswer = expectedAnswer) } test("simple end to end with vacuum protocol check feature") { @@ -652,22 +634,19 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { } checkTable( path = goldenTablePath("data-reader-nested-struct"), - expectedAnswer = expectedAnswer - ) + expectedAnswer = expectedAnswer) } test("table with empty parquet files") { checkTable( path = goldenTablePath("125-iterator-bug"), - expectedAnswer = (1 to 5).map(TestRow(_)) - ) + expectedAnswer = (1 to 5).map(TestRow(_))) } test("handle corrupted '_last_checkpoint' file") { checkTable( path = goldenTablePath("corrupted-last-checkpoint-kernel"), - expectedAnswer = (0L until 100L).map(TestRow(_)) - ) + expectedAnswer = (0L until 100L).map(TestRow(_))) } test("error - version not contiguous") { @@ -720,7 +699,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { withTempDir { tempDir => val path = tempDir.getCanonicalPath (0 to 10).foreach { i => - spark.range(i*10, i*10 + 10).write + spark.range(i * 10, i * 10 + 10).write .format("delta") .mode("append") .save(path) @@ -730,22 +709,19 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { path = path, expectedAnswer = (0L to 99L).map(TestRow(_)), version = Some(9), - expectedVersion = Some(9) - ) + expectedVersion = Some(9)) // Read a JSON version checkTable( path = path, expectedAnswer = (0L to 89L).map(TestRow(_)), version = Some(8), - expectedVersion = Some(8) - ) + expectedVersion = Some(8)) // Read the current version checkTable( path = path, expectedAnswer = (0L to 109L).map(TestRow(_)), version = Some(10), - expectedVersion = Some(10) - ) + expectedVersion = Some(10)) // Cannot read a version that does not exist val e = intercept[RuntimeException] { Table.forPath(defaultEngine, path) @@ -761,13 +737,14 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { val tablePath = tempDir.getCanonicalPath // Write versions [0, 10] (inclusive) including a checkpoint (0 to 10).foreach { i => - spark.range(i*10, i*10 + 10).write + spark.range(i * 10, i * 10 + 10).write .format("delta") .mode("append") .save(tablePath) } val log = org.apache.spark.sql.delta.DeltaLog.forTable( - spark, new org.apache.hadoop.fs.Path(tablePath)) + spark, + new org.apache.hadoop.fs.Path(tablePath)) val deltaCommitFileProvider = org.apache.spark.sql.delta.util.DeltaCommitFileProvider( log.unsafeVolatileSnapshot) // Delete the log files for versions 0-9, truncating the table history to version 10 @@ -792,15 +769,13 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { path = tablePath, expectedAnswer = (0L to 109L).map(TestRow(_)), version = Some(10), - expectedVersion = Some(10) - ) + expectedVersion = Some(10)) // Can read version 11 checkTable( path = tablePath, expectedAnswer = (0L until 50L).map(TestRow(_)), version = Some(11), - expectedVersion = Some(11) - ) + expectedVersion = Some(11)) } } @@ -815,8 +790,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { expectedAnswer = (0L until 10L).map(TestRow(_)), expectedSchema = new StructType().add("id", LongType.LONG), version = Some(0), - expectedVersion = Some(0) - ) + expectedVersion = Some(0)) } } @@ -839,8 +813,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { .add("id", LongType.LONG) .add("part5", LongType.LONG), version = Some(0), - expectedVersion = Some(0) - ) + expectedVersion = Some(0)) } } @@ -850,7 +823,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { private def generateCommits(path: String, commits: Long*): Unit = { commits.zipWithIndex.foreach { case (ts, i) => - spark.range(i*10, i*10 + 10).write.format("delta").mode("append").save(path) + spark.range(i * 10, i * 10 + 10).write.format("delta").mode("append").save(path) val file = new File(FileNames.deltaFile(new Path(path, "_delta_log"), i)) file.setLastModified(ts) } @@ -860,43 +833,41 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { withTempDir { tempDir => val start = 1540415658000L val minuteInMilliseconds = 60000L - generateCommits(tempDir.getCanonicalPath, start, start + 20 * minuteInMilliseconds, + generateCommits( + tempDir.getCanonicalPath, + start, + start + 20 * minuteInMilliseconds, start + 40 * minuteInMilliseconds) // Exact timestamp for version 0 checkTable( path = tempDir.getCanonicalPath, expectedAnswer = (0L until 10L).map(TestRow(_)), timestamp = Some(start), - expectedVersion = Some(0) - ) + expectedVersion = Some(0)) // Timestamp between version 0 and 1 should load version 0 checkTable( path = tempDir.getCanonicalPath, expectedAnswer = (0L until 10L).map(TestRow(_)), timestamp = Some(start + 10 * minuteInMilliseconds), - expectedVersion = Some(0) - ) + expectedVersion = Some(0)) // Exact timestamp for version 1 checkTable( path = tempDir.getCanonicalPath, expectedAnswer = (0L until 20L).map(TestRow(_)), timestamp = Some(start + 20 * minuteInMilliseconds), - expectedVersion = Some(1) - ) + expectedVersion = Some(1)) // Exact timestamp for the last version checkTable( path = tempDir.getCanonicalPath, expectedAnswer = (0L until 30L).map(TestRow(_)), timestamp = Some(start + 40 * minuteInMilliseconds), - expectedVersion = Some(2) - ) + expectedVersion = Some(2)) // Timestamp after last commit fails val e1 = intercept[RuntimeException] { checkTable( path = tempDir.getCanonicalPath, expectedAnswer = Seq(), - timestamp = Some(start + 50 * minuteInMilliseconds) - ) + timestamp = Some(start + 50 * minuteInMilliseconds)) } assert(e1.getMessage.contains( s"The provided timestamp ${start + 50 * minuteInMilliseconds} ms " + @@ -906,8 +877,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { checkTable( path = tempDir.getCanonicalPath, expectedAnswer = Seq(), - timestamp = Some(start - 1L) - ) + timestamp = Some(start - 1L)) } assert(e2.getMessage.contains( s"The provided timestamp ${start - 1L} ms (2018-10-24T21:14:17.999Z) is before " + @@ -970,7 +940,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { (0 to 2).foreach { i => val files = AddFile(i.toString, Map.empty, 1, 1, true) :: Nil val metadata = if (i == 0) Metadata() :: Nil else Nil - log.startTransaction().commit( metadata ++ files, DeltaOperations.ManualUpdate) + log.startTransaction().commit(metadata ++ files, DeltaOperations.ManualUpdate) } // Setup part 2 of 2: edit lastModified times @@ -1069,7 +1039,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { assert(tableImpl.getVersionAtOrAfterTimestamp(defaultEngine, nowEpochMs + i * 1000) == i) assert( - tableImpl.getVersionBeforeOrAtTimestamp(defaultEngine, nowEpochMs + i * 1000 + 1)== i) + tableImpl.getVersionBeforeOrAtTimestamp(defaultEngine, nowEpochMs + i * 1000 + 1) == i) if (i == 35) { tableImpl.getVersionAtOrAfterTimestamp(defaultEngine, nowEpochMs + i * 1000 + 1) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala index 6f24cca7aa1..eabca9523b8 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala @@ -15,50 +15,45 @@ */ package io.delta.kernel.defaults -import com.fasterxml.jackson.databind.ObjectMapper +import java.io.File +import java.nio.file.{Files, Paths} +import java.util.Optional + +import scala.collection.JavaConverters._ +import scala.collection.immutable.{ListMap, Seq} + import io.delta.golden.GoldenTableUtils.goldenTablePath +import io.delta.kernel.{Meta, Operation, Table, Transaction, TransactionBuilder, TransactionCommitResult} +import io.delta.kernel.Operation.CREATE_TABLE +import io.delta.kernel.data.{ColumnarBatch, ColumnVector, FilteredColumnarBatch, Row} import io.delta.kernel.defaults.engine.DefaultEngine +import io.delta.kernel.defaults.internal.data.DefaultColumnarBatch import io.delta.kernel.defaults.utils.{TestRow, TestUtils} import io.delta.kernel.engine.Engine -import io.delta.kernel.internal.actions.{Metadata, Protocol, SingleAction} -import io.delta.kernel.internal.fs.{Path => DeltaPath} -import io.delta.kernel.internal.util.FileNames -import io.delta.kernel.internal.util.Utils.singletonCloseableIterator -import io.delta.kernel.internal.{SnapshotImpl, TableConfig, TableImpl} -import io.delta.kernel.utils.FileStatus -import io.delta.kernel.{ - Meta, - Operation, - Table, - Transaction, - TransactionBuilder, - TransactionCommitResult -} -import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, Row} -import io.delta.kernel.defaults.internal.data.DefaultColumnarBatch import io.delta.kernel.expressions.Literal import io.delta.kernel.expressions.Literal.ofInt +import io.delta.kernel.hook.PostCommitHook.PostCommitHookType +import io.delta.kernel.internal.{SnapshotImpl, TableConfig, TableImpl} +import io.delta.kernel.internal.actions.{Metadata, Protocol, SingleAction} +import io.delta.kernel.internal.fs.{Path => DeltaPath} import io.delta.kernel.internal.util.Clock +import io.delta.kernel.internal.util.FileNames import io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames +import io.delta.kernel.internal.util.Utils.singletonCloseableIterator import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types.StructType +import io.delta.kernel.utils.{CloseableIterable, CloseableIterator, FileStatus} import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} -import io.delta.kernel.utils.CloseableIterator -import io.delta.kernel.Operation.CREATE_TABLE -import io.delta.kernel.hook.PostCommitHook.PostCommitHookType + +import org.apache.spark.sql.delta.VersionNotFoundException + +import com.fasterxml.jackson.databind.ObjectMapper import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.sql.delta.VersionNotFoundException import org.scalatest.funsuite.AnyFunSuite -import java.io.File -import java.nio.file.{Files, Paths} -import java.util.Optional -import scala.collection.JavaConverters._ -import scala.collection.immutable.{ListMap, Seq} - /** * Common utility methods for write test suites. */ @@ -95,18 +90,23 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { val filePath = f"$tablePath/_delta_log/_last_checkpoint" val source = scala.io.Source.fromFile(filePath) - val result = try source.getLines().mkString(",") finally source.close() + val result = + try source.getLines().mkString(",") + finally source.close() assert(result === s"""{"version":$checkpointAt,"size":$expSize}""") } - /** Helper method to remove the delta files before the given version, to make sure the read is + /** + * Helper method to remove the delta files before the given version, to make sure the read is * using a checkpoint as base for state reconstruction. */ def deleteDeltaFilesBefore(tablePath: String, beforeVersion: Long): Unit = { Seq.range(0, beforeVersion).foreach { version => val filePath = new Path(f"$tablePath/_delta_log/$version%020d.json") - new Path(tablePath).getFileSystem(new Configuration()).delete(filePath, false /* recursive */) + new Path(tablePath).getFileSystem(new Configuration()).delete( + filePath, + false /* recursive */ ) } // try to query a version < beforeVersion @@ -143,18 +143,16 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def checkpointIfReady( - engine: Engine, - tablePath: String, - result: TransactionCommitResult, - expSize: Long): Unit = { - result.getPostCommitHooks.forEach( - hook => { - if (hook.getType == PostCommitHookType.CHECKPOINT) { - hook.threadSafeInvoke(engine) - verifyLastCheckpointMetadata(tablePath, checkpointAt = result.getVersion, expSize) - } + engine: Engine, + tablePath: String, + result: TransactionCommitResult, + expSize: Long): Unit = { + result.getPostCommitHooks.forEach(hook => { + if (hook.getType == PostCommitHookType.CHECKPOINT) { + hook.threadSafeInvoke(engine) + verifyLastCheckpointMetadata(tablePath, checkpointAt = result.getVersion, expSize) } - ) + }) } /** @@ -162,9 +160,10 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { * ordinal if it is not null and the consumer returns a value, otherwise return null. */ def readCommitFile( - engine: Engine, - tablePath: String, - version: Long, consumer: Row => Option[Any]): Option[Any] = { + engine: Engine, + tablePath: String, + version: Long, + consumer: Row => Option[Any]): Option[Any] = { val table = Table.forPath(engine, tablePath) val logPath = new DeltaPath(table.getPath(engine), "_delta_log") val file = FileStatus.of(FileNames.deltaFile(logPath, version), 0, 0) @@ -191,15 +190,21 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { * null, otherwise return null. */ def getMetadataActionFromCommit( - engine: Engine, table: Table, version: Long): Option[Row] = { - readCommitFile(engine, table.getPath(engine), version, (row) => { - val ord = row.getSchema.indexOf("metaData") - if (!row.isNullAt(ord)) { - Option(row.getStruct(ord)) - } else { - Option.empty - } - }).map{ case metadata: Row => Some(metadata)}.getOrElse(Option.empty) + engine: Engine, + table: Table, + version: Long): Option[Row] = { + readCommitFile( + engine, + table.getPath(engine), + version, + (row) => { + val ord = row.getSchema.indexOf("metaData") + if (!row.isNullAt(ord)) { + Option(row.getStruct(ord)) + } else { + Option.empty + } + }).map { case metadata: Row => Some(metadata) }.getOrElse(Option.empty) } /** @@ -207,22 +212,26 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { * null, otherwise return null. */ def getProtocolActionFromCommit(engine: Engine, table: Table, version: Long): Option[Row] = { - readCommitFile(engine, table.getPath(engine), version, (row) => { - val ord = row.getSchema.indexOf("protocol") - if (!row.isNullAt(ord)) { - Some(row.getStruct(ord)) - } else { - Option.empty - } - }).map{ case protocol: Row => Some(protocol)}.getOrElse(Option.empty) + readCommitFile( + engine, + table.getPath(engine), + version, + (row) => { + val ord = row.getSchema.indexOf("protocol") + if (!row.isNullAt(ord)) { + Some(row.getStruct(ord)) + } else { + Option.empty + } + }).map { case protocol: Row => Some(protocol) }.getOrElse(Option.empty) } def generateData( - schema: StructType, - partitionCols: Seq[String], - partitionValues: Map[String, Literal], - batchSize: Int, - numBatches: Int): Seq[FilteredColumnarBatch] = { + schema: StructType, + partitionCols: Seq[String], + partitionValues: Map[String, Literal], + batchSize: Int, + numBatches: Int): Seq[FilteredColumnarBatch] = { val partitionValuesSchemaCase = casePreservingPartitionColNames(partitionCols.asJava, partitionValues.asJava) @@ -252,10 +261,10 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def stageData( - state: Row, - partitionValues: Map[String, Literal], - data: Seq[FilteredColumnarBatch]) - : CloseableIterator[Row] = { + state: Row, + partitionValues: Map[String, Literal], + data: Seq[FilteredColumnarBatch]) + : CloseableIterator[Row] = { val physicalDataIter = Transaction.transformLogicalData( defaultEngine, state, @@ -275,13 +284,13 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def createTxn( - engine: Engine = defaultEngine, - tablePath: String, - isNewTable: Boolean = false, - schema: StructType = null, - partCols: Seq[String] = null, - tableProperties: Map[String, String] = null, - clock: Clock = () => System.currentTimeMillis): Transaction = { + engine: Engine = defaultEngine, + tablePath: String, + isNewTable: Boolean = false, + schema: StructType = null, + partCols: Seq[String] = null, + tableProperties: Map[String, String] = null, + clock: Clock = () => System.currentTimeMillis): Transaction = { var txnBuilder = createWriteTxnBuilder( TableImpl.forPath(engine, tablePath, clock)) @@ -299,9 +308,9 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def commitAppendData( - engine: Engine = defaultEngine, - txn: Transaction, - data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])]): TransactionCommitResult = { + engine: Engine = defaultEngine, + txn: Transaction, + data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])]): TransactionCommitResult = { val txnState = txn.getTransactionState(engine) @@ -310,27 +319,27 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } val combineActions = inMemoryIterable(actions.reduceLeft(_ combine _)) - txn.commit(engine, combineActions) + commitTransaction(txn, engine, combineActions) } def appendData( - engine: Engine = defaultEngine, - tablePath: String, - isNewTable: Boolean = false, - schema: StructType = null, - partCols: Seq[String] = null, - data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])], - clock: Clock = () => System.currentTimeMillis, - tableProperties: Map[String, String] = null): TransactionCommitResult = { + engine: Engine = defaultEngine, + tablePath: String, + isNewTable: Boolean = false, + schema: StructType = null, + partCols: Seq[String] = null, + data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])], + clock: Clock = () => System.currentTimeMillis, + tableProperties: Map[String, String] = null): TransactionCommitResult = { val txn = createTxn(engine, tablePath, isNewTable, schema, partCols, tableProperties, clock) commitAppendData(engine, txn, data) } def assertMetadataProp( - snapshot: SnapshotImpl, - key: TableConfig[_ <: Any], - expectedValue: Any): Unit = { + snapshot: SnapshotImpl, + key: TableConfig[_ <: Any], + expectedValue: Any): Unit = { assert(key.fromMetadata(snapshot.getMetadata) == expectedValue) } @@ -347,25 +356,34 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def setTablePropAndVerify( - engine: Engine, - tablePath: String, - isNewTable: Boolean = true, - key: TableConfig[_ <: Any], value: String, expectedValue: Any, - clock: Clock = () => System.currentTimeMillis): Unit = { + engine: Engine, + tablePath: String, + isNewTable: Boolean = true, + key: TableConfig[_ <: Any], + value: String, + expectedValue: Any, + clock: Clock = () => System.currentTimeMillis): Unit = { val table = Table.forPath(engine, tablePath) createTxn( engine, tablePath, - isNewTable, testSchema, Seq.empty, tableProperties = Map(key.getKey -> value), clock) + isNewTable, + testSchema, + Seq.empty, + tableProperties = Map(key.getKey -> value), + clock) .commit(engine, emptyIterable()) val snapshot = table.getLatestSnapshot(engine).asInstanceOf[SnapshotImpl] assertMetadataProp(snapshot, key, expectedValue) } - def verifyWrittenContent(path: String, expSchema: StructType, expData: Seq[TestRow]): Unit = { + protected def verifyWrittenContent( + path: String, + expSchema: StructType, + expData: Seq[TestRow]): Unit = { val actSchema = tableSchema(path) assert(actSchema === expSchema) @@ -382,11 +400,11 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def verifyCommitInfo( - tablePath: String, - version: Long, - partitionCols: Seq[String] = Seq.empty, - isBlindAppend: Boolean = true, - operation: Operation = CREATE_TABLE): Unit = { + tablePath: String, + version: Long, + partitionCols: Seq[String] = Seq.empty, + isBlindAppend: Boolean = true, + operation: Operation = CREATE_TABLE): Unit = { val row = spark.sql(s"DESCRIBE HISTORY delta.`$tablePath`") .filter(s"version = $version") .select( @@ -407,16 +425,20 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def verifyCommitResult( - result: TransactionCommitResult, - expVersion: Long, - expIsReadyForCheckpoint: Boolean): Unit = { + result: TransactionCommitResult, + expVersion: Long, + expIsReadyForCheckpoint: Boolean): Unit = { assert(result.getVersion === expVersion) assertCheckpointReadiness(result, expIsReadyForCheckpoint) } + // TODO: Change this to use the table metadata and protocol and + // not rely on DESCRIBE which adds some properties based on the protocol. def verifyTableProperties( - tablePath: String, - expProperties: ListMap[String, Any], minReaderVersion: Int, minWriterVersion: Int): Unit = { + tablePath: String, + expProperties: ListMap[String, Any], + minReaderVersion: Int, + minWriterVersion: Int): Unit = { val resultProperties = spark.sql(s"DESCRIBE EXTENDED delta.`$tablePath`") .filter("col_name = 'Table Properties'") .select("data_type") @@ -440,9 +462,17 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { assert( txnResult.getPostCommitHooks .stream() - .anyMatch( - hook => hook.getType == PostCommitHookType.CHECKPOINT - ) === isReadyForCheckpoint - ) + .anyMatch(hook => hook.getType == PostCommitHookType.CHECKPOINT) === isReadyForCheckpoint) + } + + /** + * Commit transaction, all child suites should use this instead of txn.commit + * directly and could override it for specific test cases (e.g. commit and write CRC). + */ + protected def commitTransaction( + txn: Transaction, + engine: Engine, + dataActions: CloseableIterable[Row]): TransactionCommitResult = { + txn.commit(engine, dataActions) } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteWithCrcSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteWithCrcSuite.scala new file mode 100644 index 00000000000..53750cbdbbb --- /dev/null +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteWithCrcSuite.scala @@ -0,0 +1,67 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.defaults +import scala.collection.immutable.Seq +import scala.language.implicitConversions + +import io.delta.kernel.{Transaction, TransactionCommitResult} +import io.delta.kernel.data.Row +import io.delta.kernel.defaults.utils.TestRow +import io.delta.kernel.engine.Engine +import io.delta.kernel.hook.PostCommitHook.PostCommitHookType +import io.delta.kernel.internal.checksum.ChecksumReader +import io.delta.kernel.internal.fs.Path +import io.delta.kernel.types.StructType +import io.delta.kernel.utils.CloseableIterable + +/** + * Test suite that run all tests in DeltaTableWritesSuite with CRC file written + * after each delta commit. This test suite will verify that the written CRC files are valid. + */ +class DeltaTableWriteWithCrcSuite extends DeltaTableWritesSuite { + + override def commitTransaction( + txn: Transaction, + engine: Engine, + dataActions: CloseableIterable[Row]): TransactionCommitResult = { + val result = txn.commit(engine, dataActions) + result.getPostCommitHooks + .stream() + .filter(hook => hook.getType == PostCommitHookType.CHECKSUM_SIMPLE) + .forEach(hook => hook.threadSafeInvoke(engine)) + result + } + + override def verifyWrittenContent( + path: String, + expSchema: StructType, + expData: Seq[TestRow]): Unit = { + super.verifyWrittenContent(path, expSchema, expData) + verifyChecksumValid(path) + } + + /** Ensure checksum is readable by CRC reader. */ + def verifyChecksumValid( + tablePath: String): Unit = { + val checksumVersion = latestSnapshot(tablePath, defaultEngine).getVersion + val crcInfo = ChecksumReader.getCRCInfo( + defaultEngine, + new Path(f"$tablePath/_delta_log/"), + checksumVersion, + checksumVersion) + assert(crcInfo.isPresent) + } +} diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala index 22f3d07fcd0..3d04d95e1a6 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala @@ -15,9 +15,14 @@ */ package io.delta.kernel.defaults +import java.util.{Locale, Optional} + +import scala.collection.JavaConverters._ +import scala.collection.immutable.Seq + import io.delta.golden.GoldenTableUtils.goldenTablePath -import io.delta.kernel.Operation.{CREATE_TABLE, WRITE} import io.delta.kernel._ +import io.delta.kernel.Operation.{CREATE_TABLE, WRITE} import io.delta.kernel.data.{ColumnarBatch, FilteredColumnarBatch, Row} import io.delta.kernel.defaults.internal.parquet.ParquetSuiteBase import io.delta.kernel.defaults.utils.TestRow @@ -26,24 +31,21 @@ import io.delta.kernel.exceptions._ import io.delta.kernel.expressions.Literal import io.delta.kernel.expressions.Literal._ import io.delta.kernel.hook.PostCommitHook.PostCommitHookType -import io.delta.kernel.internal.checkpoints.CheckpointerSuite.selectSingleElement -import io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames import io.delta.kernel.internal.{SnapshotImpl, TableConfig} +import io.delta.kernel.internal.checkpoints.CheckpointerSuite.selectSingleElement import io.delta.kernel.internal.util.ColumnMapping +import io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames +import io.delta.kernel.types._ import io.delta.kernel.types.DateType.DATE import io.delta.kernel.types.DoubleType.DOUBLE import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types.StringType.STRING import io.delta.kernel.types.TimestampNTZType.TIMESTAMP_NTZ import io.delta.kernel.types.TimestampType.TIMESTAMP -import io.delta.kernel.types._ -import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} import io.delta.kernel.utils.CloseableIterable +import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} -import java.util.{Locale, Optional} -import scala.collection.JavaConverters._ -import scala.collection.immutable.Seq - +/** Transaction commit in this suite IS REQUIRED TO use commitTransaction than .commit */ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { /////////////////////////////////////////////////////////////////////////// @@ -82,10 +84,10 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) val ex = intercept[KernelException] { txnBuilder - .withSchema(engine, new StructType().add("ts_ntz", TIMESTAMP_NTZ)) + .withSchema(engine, new StructType().add("variant_type", VariantType.VARIANT)) .build(engine) } - assert(ex.getMessage.contains("Kernel doesn't support writing data of type: timestamp_ntz")) + assert(ex.getMessage.contains("Kernel doesn't support writing data of type: variant")) } } @@ -95,7 +97,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) val txn = txnBuilder.withSchema(engine, testSchema).build(engine) - txn.commit(engine, emptyIterable()) + commitTransaction(txn, engine, emptyIterable()) { val ex = intercept[TableAlreadyExistsException] { @@ -129,7 +131,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa assert(txn.getSchema(engine) === testSchema) assert(txn.getPartitionColumns(engine) === Seq.empty.asJava) - val txnResult = txn.commit(engine, emptyIterable()) + val txnResult = commitTransaction(txn, engine, emptyIterable()) assert(txnResult.getVersion === 0) assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) @@ -172,8 +174,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa appendData( engine, tablePath, - data = Seq(Map.empty[String, Literal] -> dataBatches1) - ) + data = Seq(Map.empty[String, Literal] -> dataBatches1)) val ver1Snapshot = table.getLatestSnapshot(engine).asInstanceOf[SnapshotImpl] assertMetadataProp(ver1Snapshot, TableConfig.CHECKPOINT_INTERVAL, 2) } @@ -194,8 +195,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa appendData( engine, tablePath, - data = Seq(Map.empty[String, Literal] -> dataBatches1) - ) + data = Seq(Map.empty[String, Literal] -> dataBatches1)) val ver1Snapshot = table.getLatestSnapshot(engine).asInstanceOf[SnapshotImpl] assertMetadataProp(ver1Snapshot, TableConfig.CHECKPOINT_INTERVAL, 10) @@ -225,8 +225,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa appendData( engine, tablePath, - data = Seq(Map.empty[String, Literal] -> dataBatches1) - ) + data = Seq(Map.empty[String, Literal] -> dataBatches1)) val ver1Snapshot = table.getLatestSnapshot(engine).asInstanceOf[SnapshotImpl] assertMetadataProp(ver1Snapshot, TableConfig.CHECKPOINT_INTERVAL, 10) @@ -348,7 +347,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa assert(txn.getSchema(engine) === schema) // Expect the partition column name is exactly same as the one in the schema assert(txn.getPartitionColumns(engine) === Seq("Part1", "part2").asJava) - val txnResult = txn.commit(engine, emptyIterable()) + val txnResult = commitTransaction(txn, engine, emptyIterable()) assert(txnResult.getVersion === 0) assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) @@ -358,21 +357,25 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } } - test("create table with all supported types") { - withTempDirAndEngine { (tablePath, engine) => - val parquetAllTypes = goldenTablePath("parquet-all-types") - val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) + Seq(true, false).foreach { includeTimestampNtz => + test(s"create table with all supported types - timestamp_ntz included=$includeTimestampNtz") { + withTempDirAndEngine { (tablePath, engine) => + val parquetAllTypes = goldenTablePath("parquet-all-types") + val goldenTableSchema = tableSchema(parquetAllTypes) + val schema = if (includeTimestampNtz) goldenTableSchema + else removeTimestampNtzTypeColumns(goldenTableSchema) - val table = Table.forPath(engine, tablePath) - val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) - val txn = txnBuilder.withSchema(engine, schema).build(engine) - val txnResult = txn.commit(engine, emptyIterable()) + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + val txn = txnBuilder.withSchema(engine, schema).build(engine) + val txnResult = commitTransaction(txn, engine, emptyIterable()) - assert(txnResult.getVersion === 0) - assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) + assert(txnResult.getVersion === 0) + assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) - verifyCommitInfo(tablePath, version = 0) - verifyWrittenContent(tablePath, schema, Seq.empty) + verifyCommitInfo(tablePath, version = 0) + verifyWrittenContent(tablePath, schema, Seq.empty) + } } } @@ -387,8 +390,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa isNewTable = true, testSchema, partCols = Seq.empty, - data = Seq(Map.empty[String, Literal] -> (dataBatches1 ++ dataBatches2)) - ) + data = Seq(Map.empty[String, Literal] -> (dataBatches1 ++ dataBatches2))) val expectedAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap(_.toTestRows) @@ -406,8 +408,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa isNewTable = true, testSchema, partCols = Seq.empty, - data = Seq(Map.empty[String, Literal] -> dataBatches1) - ) + data = Seq(Map.empty[String, Literal] -> dataBatches1)) verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) @@ -416,8 +417,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val commitResult1 = appendData( engine, tblPath, - data = Seq(Map.empty[String, Literal] -> dataBatches2) - ) + data = Seq(Map.empty[String, Literal] -> dataBatches2)) val expAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap(_.toTestRows) @@ -439,12 +439,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val stagedFiles = stageData(txnState, Map.empty, dataBatches1) val stagedActionsIterable = inMemoryIterable(stagedFiles) - val commitResult = txn.commit(engine, stagedActionsIterable) + val commitResult = commitTransaction(txn, engine, stagedActionsIterable) assert(commitResult.getVersion == 0) // try to commit the same transaction and expect failure val ex = intercept[IllegalStateException] { - txn.commit(engine, stagedActionsIterable) + commitTransaction(txn, engine, stagedActionsIterable) } assert(ex.getMessage.contains( "Transaction is already attempted to commit. Create a new transaction.")) @@ -461,9 +461,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa testPartitionColumns, Seq( Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1, - Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2 - ) - ) + Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2)) val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ dataPartitionBatches2.flatMap(_.toTestRows) @@ -486,8 +484,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa isNewTable = true, testPartitionSchema, testPartitionColumns, - data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) - ) + data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1)) val expData = dataPartitionBatches1.flatMap(_.toTestRows) @@ -499,8 +496,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val commitResult1 = appendData( engine, tblPath, - data = Seq(Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2) - ) + data = Seq(Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2)) val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ dataPartitionBatches2.flatMap(_.toTestRows) @@ -552,8 +548,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val dataPerVersion = Map( 0 -> Seq(v0Part0Values -> v0Part0Data, v0Part1Values -> v0Part1Data), - 1 -> Seq(v1Part0Values -> v1Part0Data, v1Part1Values -> v1Part1Data) - ) + 1 -> Seq(v1Part0Values -> v1Part0Data, v1Part1Values -> v1Part1Data)) val expV0Data = v0Part0Data.flatMap(_.toTestRows) ++ v0Part1Data.flatMap(_.toTestRows) val expV1Data = v1Part0Data.flatMap(_.toTestRows) ++ v1Part1Data.flatMap(_.toTestRows) @@ -571,7 +566,10 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa // partition cols are not written in the commit info for inserts val partitionBy = if (i == 0) expPartCols else null verifyCommitInfo(tblPath, version = i, partitionBy, operation = WRITE) - verifyWrittenContent(tblPath, schema, if (i == 0) expV0Data else expV0Data ++ expV1Data) + verifyWrittenContent( + tblPath, + schema, + if (i == 0) expV0Data else expV0Data ++ expV1Data) } } } @@ -631,128 +629,139 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } } - test("insert into table - all supported types data") { - withTempDirAndEngine { (tblPath, engine) => - val parquetAllTypes = goldenTablePath("parquet-all-types") - val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) + Seq(true, false).foreach { includeTimestampNtz => + test(s"insert into table - all supported types data - " + + s"timestamp_ntz included = $includeTimestampNtz") { + withTempDirAndEngine { (tblPath, engine) => + val parquetAllTypes = goldenTablePath("parquet-all-types") + val goldenTableSchema = tableSchema(parquetAllTypes) + val schema = if (includeTimestampNtz) goldenTableSchema + else removeTimestampNtzTypeColumns(goldenTableSchema) - val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] - val dataWithPartInfo = Seq(Map.empty[String, Literal] -> data) + val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] + val dataWithPartInfo = Seq(Map.empty[String, Literal] -> data) - appendData(engine, tblPath, isNewTable = true, schema, Seq.empty, dataWithPartInfo) - var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + appendData(engine, tblPath, isNewTable = true, schema, Seq.empty, dataWithPartInfo) + var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - val checkpointInterval = 4 - setCheckpointInterval(tblPath, checkpointInterval) + val checkpointInterval = 4 + setCheckpointInterval(tblPath, checkpointInterval) - for (i <- 2 until 5) { - // insert until a checkpoint is required - val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) + for (i <- 2 until 5) { + // insert until a checkpoint is required + val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) - expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - checkpointIfReady(engine, tblPath, commitResult, expSize = i /* one file per version */) + expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + checkpointIfReady(engine, tblPath, commitResult, expSize = i /* one file per version */ ) - verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) - verifyCommitInfo(tblPath, version = i, null, operation = WRITE) - verifyWrittenContent(tblPath, schema, expData) + verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) + verifyCommitInfo(tblPath, version = i, null, operation = WRITE) + verifyWrittenContent(tblPath, schema, expData) + } + assertCheckpointExists(tblPath, atVersion = checkpointInterval) } - assertCheckpointExists(tblPath, atVersion = checkpointInterval) } } - test("insert into partitioned table - all supported partition column types data") { - withTempDirAndEngine { (tblPath, engine) => - val parquetAllTypes = goldenTablePath("parquet-all-types") - val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) - val partCols = Seq( - "byteType", - "shortType", - "integerType", - "longType", - "floatType", - "doubleType", - "decimal", - "booleanType", - "stringType", - "binaryType", - "dateType", - "timestampType" - ) - val casePreservingPartCols = - casePreservingPartitionColNames(schema, partCols.asJava).asScala.to[Seq] - - // get the partition values from the data batch at the given rowId - def getPartitionValues(batch: ColumnarBatch, rowId: Int): Map[String, Literal] = { - casePreservingPartCols.map { partCol => - val colIndex = schema.indexOf(partCol) - val vector = batch.getColumnVector(colIndex) - - val literal = if (vector.isNullAt(rowId)) { - Literal.ofNull(vector.getDataType) - } else { - vector.getDataType match { - case _: ByteType => Literal.ofByte(vector.getByte(rowId)) - case _: ShortType => Literal.ofShort(vector.getShort(rowId)) - case _: IntegerType => Literal.ofInt(vector.getInt(rowId)) - case _: LongType => Literal.ofLong(vector.getLong(rowId)) - case _: FloatType => Literal.ofFloat(vector.getFloat(rowId)) - case _: DoubleType => Literal.ofDouble(vector.getDouble(rowId)) - case dt: DecimalType => - Literal.ofDecimal(vector.getDecimal(rowId), dt.getPrecision, dt.getScale) - case _: BooleanType => Literal.ofBoolean(vector.getBoolean(rowId)) - case _: StringType => Literal.ofString(vector.getString(rowId)) - case _: BinaryType => Literal.ofBinary(vector.getBinary(rowId)) - case _: DateType => Literal.ofDate(vector.getInt(rowId)) - case _: TimestampType => Literal.ofTimestamp(vector.getLong(rowId)) - case _ => - throw new IllegalArgumentException(s"Unsupported type: ${vector.getDataType}") + Seq(true, false).foreach { includeTimestampNtz => + test(s"insert into partitioned table - all supported partition column types data - " + + s"timestamp_ntz included = $includeTimestampNtz") { + withTempDirAndEngine { (tblPath, engine) => + val parquetAllTypes = goldenTablePath("parquet-all-types") + val goldenTableSchema = tableSchema(parquetAllTypes) + val schema = if (includeTimestampNtz) goldenTableSchema + else removeTimestampNtzTypeColumns(goldenTableSchema) + + val partCols = Seq( + "byteType", + "shortType", + "integerType", + "longType", + "floatType", + "doubleType", + "decimal", + "booleanType", + "stringType", + "binaryType", + "dateType", + "timestampType") ++ (if (includeTimestampNtz) Seq("timestampNtzType") else Seq.empty) + val casePreservingPartCols = + casePreservingPartitionColNames(schema, partCols.asJava).asScala.to[Seq] + + // get the partition values from the data batch at the given rowId + def getPartitionValues(batch: ColumnarBatch, rowId: Int): Map[String, Literal] = { + casePreservingPartCols.map { partCol => + val colIndex = schema.indexOf(partCol) + val vector = batch.getColumnVector(colIndex) + + val literal = if (vector.isNullAt(rowId)) { + Literal.ofNull(vector.getDataType) + } else { + vector.getDataType match { + case _: ByteType => Literal.ofByte(vector.getByte(rowId)) + case _: ShortType => Literal.ofShort(vector.getShort(rowId)) + case _: IntegerType => Literal.ofInt(vector.getInt(rowId)) + case _: LongType => Literal.ofLong(vector.getLong(rowId)) + case _: FloatType => Literal.ofFloat(vector.getFloat(rowId)) + case _: DoubleType => Literal.ofDouble(vector.getDouble(rowId)) + case dt: DecimalType => + Literal.ofDecimal(vector.getDecimal(rowId), dt.getPrecision, dt.getScale) + case _: BooleanType => Literal.ofBoolean(vector.getBoolean(rowId)) + case _: StringType => Literal.ofString(vector.getString(rowId)) + case _: BinaryType => Literal.ofBinary(vector.getBinary(rowId)) + case _: DateType => Literal.ofDate(vector.getInt(rowId)) + case _: TimestampType => Literal.ofTimestamp(vector.getLong(rowId)) + case _: TimestampNTZType => Literal.ofTimestampNtz(vector.getLong(rowId)) + case _ => + throw new IllegalArgumentException(s"Unsupported type: ${vector.getDataType}") + } } - } - (partCol, literal) - }.toMap - } + (partCol, literal) + }.toMap + } - val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] - - // From the above table read data, convert each row as a new batch with partition info - // Take the values of the partitionCols from the data and create a new batch with the - // selection vector to just select a single row. - var dataWithPartInfo = Seq.empty[(Map[String, Literal], Seq[FilteredColumnarBatch])] - - data.foreach { filteredBatch => - val batch = filteredBatch.getData - Seq.range(0, batch.getSize).foreach { rowId => - val partValues = getPartitionValues(batch, rowId) - val filteredBatch = new FilteredColumnarBatch( - batch, - Optional.of(selectSingleElement(batch.getSize, rowId))) - dataWithPartInfo = dataWithPartInfo :+ (partValues, Seq(filteredBatch)) + val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] + + // From the above table read data, convert each row as a new batch with partition info + // Take the values of the partitionCols from the data and create a new batch with the + // selection vector to just select a single row. + var dataWithPartInfo = Seq.empty[(Map[String, Literal], Seq[FilteredColumnarBatch])] + + data.foreach { filteredBatch => + val batch = filteredBatch.getData + Seq.range(0, batch.getSize).foreach { rowId => + val partValues = getPartitionValues(batch, rowId) + val filteredBatch = new FilteredColumnarBatch( + batch, + Optional.of(selectSingleElement(batch.getSize, rowId))) + dataWithPartInfo = dataWithPartInfo :+ (partValues, Seq(filteredBatch)) + } } - } - appendData(engine, tblPath, isNewTable = true, schema, partCols, dataWithPartInfo) - verifyCommitInfo(tblPath, version = 0, casePreservingPartCols, operation = WRITE) + appendData(engine, tblPath, isNewTable = true, schema, partCols, dataWithPartInfo) + verifyCommitInfo(tblPath, version = 0, casePreservingPartCols, operation = WRITE) - var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - val checkpointInterval = 2 - setCheckpointInterval(tblPath, checkpointInterval) // version 1 + val checkpointInterval = 2 + setCheckpointInterval(tblPath, checkpointInterval) // version 1 - for (i <- 2 until 4) { - // insert until a checkpoint is required - val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) + for (i <- 2 until 4) { + // insert until a checkpoint is required + val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) - expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - val fileCount = dataFileCount(tblPath) - checkpointIfReady(engine, tblPath, commitResult, expSize = fileCount) + val fileCount = dataFileCount(tblPath) + checkpointIfReady(engine, tblPath, commitResult, expSize = fileCount) - verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) - verifyCommitInfo(tblPath, version = i, partitionCols = null, operation = WRITE) - verifyWrittenContent(tblPath, schema, expData) - } + verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) + verifyCommitInfo(tblPath, version = i, partitionCols = null, operation = WRITE) + verifyWrittenContent(tblPath, schema, expData) + } - assertCheckpointExists(tblPath, atVersion = checkpointInterval) + assertCheckpointExists(tblPath, atVersion = checkpointInterval) + } } } @@ -770,7 +779,8 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa test("insert into table - missing partition column info") { withTempDirAndEngine { (tblPath, engine) => val ex = intercept[IllegalArgumentException] { - appendData(engine, + appendData( + engine, tblPath, isNewTable = true, testPartitionSchema, @@ -789,7 +799,8 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa // part2 type should be int, be giving a string value val data = Seq(Map("part1" -> ofInt(1), "part2" -> ofString("sdsd")) -> dataPartitionBatches1) - appendData(engine, + appendData( + engine, tblPath, isNewTable = true, testPartitionSchema, @@ -803,11 +814,13 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa test("insert into table - idempotent writes") { withTempDirAndEngine { (tblPath, engine) => + // TODO: re-enable when CRC_FULL post commit hook is added, txn2 requires CRC_FULL + assume(this.suiteName != ("DeltaTableWriteWithCrcSuite")) val data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) var expData = Seq.empty[TestRow] // as the data in inserted, update this. def prepTxnAndActions(newTbl: Boolean, appId: String, txnVer: Long) - : (Transaction, CloseableIterable[Row]) = { + : (Transaction, CloseableIterable[Row]) = { var txnBuilder = createWriteTxnBuilder(Table.forPath(engine, tblPath)) if (appId != null) txnBuilder = txnBuilder.withTransactionId(engine, appId, txnVer) @@ -826,9 +839,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa (txn, combinedActions) } - def commitAndVerify(newTbl: Boolean, txn: Transaction, - actions: CloseableIterable[Row], expTblVer: Long): Unit = { - val commitResult = txn.commit(engine, actions) + def commitAndVerify( + newTbl: Boolean, + txn: Transaction, + actions: CloseableIterable[Row], + expTblVer: Long): Unit = { + val commitResult = commitTransaction(txn, engine, actions) expData = expData ++ data.flatMap(_._2).flatMap(_.toTestRows) @@ -897,6 +913,8 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa test("conflicts - creating new table - table created by other txn after current txn start") { withTempDirAndEngine { (tablePath, engine) => + // TODO: re-enable when CRC_FULL post commit hook is added + assume(this.suiteName != ("DeltaTableWriteWithCrcSuite")) val losingTx = createTestTxn(engine, tablePath, Some(testSchema)) // don't commit losingTxn, instead create a new txn and commit it @@ -945,6 +963,8 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa // Different scenarios that have multiple winning txns and with a checkpoint in between. Seq(1, 5, 12).foreach { numWinningTxs => test(s"conflicts - concurrent data append ($numWinningTxs) after the losing txn has started") { + // TODO: re-enable when CRC_FULL post commit hook is added + assume(this.suiteName != ("DeltaTableWriteWithCrcSuite")) withTempDirAndEngine { (tablePath, engine) => val testData = Seq(Map.empty[String, Literal] -> dataBatches1) var expData = Seq.empty[TestRow] @@ -970,14 +990,16 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val txn1Result = txn1.commit(engine, actions) verifyCommitResult( - txn1Result, expVersion = numWinningTxs + 1, expIsReadyForCheckpoint = false) + txn1Result, + expVersion = numWinningTxs + 1, + expIsReadyForCheckpoint = false) verifyCommitInfo(tablePath = tablePath, version = 0, operation = WRITE) verifyWrittenContent(tablePath, testSchema, expData) } } } - def removeUnsupportedTypes(structType: StructType): StructType = { + def removeTimestampNtzTypeColumns(structType: StructType): StructType = { def process(dataType: DataType): Option[DataType] = dataType match { case a: ArrayType => val newElementType = process(a.getElementType) @@ -992,7 +1014,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } case _: TimestampNTZType => None // ignore case s: StructType => - val newType = removeUnsupportedTypes(s); + val newType = removeTimestampNtzTypeColumns(s); if (newType.length() > 0) { Some(newType) } else { @@ -1012,7 +1034,9 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } def createTestTxn( - engine: Engine, tablePath: String, schema: Option[StructType] = None): Transaction = { + engine: Engine, + tablePath: String, + schema: Option[StructType] = None): Transaction = { val table = Table.forPath(engine, tablePath) var txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) schema.foreach(s => txnBuilder = txnBuilder.withSchema(engine, s)) @@ -1022,7 +1046,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa test("create table with unsupported column mapping mode") { withTempDirAndEngine { (tablePath, engine) => val ex = intercept[InvalidConfigurationValueException] { - createTxn(engine, tablePath, isNewTable = true, testSchema, partCols = Seq.empty, + createTxn( + engine, + tablePath, + isNewTable = true, + testSchema, + partCols = Seq.empty, tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "invalid")) .commit(engine, emptyIterable()) } @@ -1033,7 +1062,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa test("create table with column mapping mode = none") { withTempDirAndEngine { (tablePath, engine) => - createTxn(engine, tablePath, isNewTable = true, testSchema, partCols = Seq.empty, + createTxn( + engine, + tablePath, + isNewTable = true, + testSchema, + partCols = Seq.empty, tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "none")) .commit(engine, emptyIterable()) @@ -1063,7 +1097,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa test("cannot update table with unsupported column mapping mode change") { withTempDirAndEngine { (tablePath, engine) => val table = Table.forPath(engine, tablePath) - createTxn(engine, tablePath, isNewTable = true, testSchema, partCols = Seq.empty, + createTxn( + engine, + tablePath, + isNewTable = true, + testSchema, + partCols = Seq.empty, tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "name")) .commit(engine, emptyIterable()) @@ -1086,7 +1125,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa .add("a", StringType.STRING, true) .add("b", IntegerType.INTEGER, true) - createTxn(engine, tablePath, isNewTable = true, schema, partCols = Seq.empty, + createTxn( + engine, + tablePath, + isNewTable = true, + schema, + partCols = Seq.empty, tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "id")) .commit(engine, emptyIterable()) @@ -1114,7 +1158,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa .add("a", StringType.STRING, true) .add("b", IntegerType.INTEGER, true) - createTxn(engine, tablePath, isNewTable = true, schema, partCols = Seq.empty, + createTxn( + engine, + tablePath, + isNewTable = true, + schema, + partCols = Seq.empty, tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "name")) .commit(engine, emptyIterable()) @@ -1161,7 +1210,6 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } } - test("unsupported protocol version with column mapping mode and no protocol update in metadata") { // TODO } @@ -1177,7 +1225,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa .add("a", StringType.STRING, true) .add("b", IntegerType.INTEGER, true) - createTxn(engine, tablePath, isNewTable = true, schema, partCols = Seq.empty, + createTxn( + engine, + tablePath, + isNewTable = true, + schema, + partCols = Seq.empty, tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "name")) .commit(engine, emptyIterable()) @@ -1194,7 +1247,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa .add("a", StringType.STRING, true) .add("b", IntegerType.INTEGER, true) - createTxn(engine, tablePath, isNewTable = true, schema, partCols = Seq.empty, + createTxn( + engine, + tablePath, + isNewTable = true, + schema, + partCols = Seq.empty, tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "id")) .commit(engine, emptyIterable()) @@ -1235,14 +1293,21 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val table = Table.forPath(engine, tablePath) val schema = new StructType() .add("a", StringType.STRING, true) - .add("b", + .add( + "b", new StructType() .add("d", IntegerType.INTEGER, true) .add("e", IntegerType.INTEGER, true)) .add("c", IntegerType.INTEGER, true) - createTxn(engine, tablePath, isNewTable = true, schema, partCols = Seq.empty, - tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "id", + createTxn( + engine, + tablePath, + isNewTable = true, + schema, + partCols = Seq.empty, + tableProperties = Map( + TableConfig.COLUMN_MAPPING_MODE.getKey -> "id", TableConfig.ICEBERG_COMPAT_V2_ENABLED.getKey -> "true")) .commit(engine, emptyIterable()) @@ -1257,9 +1322,9 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } private def assertColumnMapping( - field: StructField, - expId: Long, - expPhyName: String = "UUID"): Unit = { + field: StructField, + expId: Long, + expPhyName: String = "UUID"): Unit = { val meta = field.getMetadata assert(meta.get(ColumnMapping.COLUMN_MAPPING_ID_KEY) == expId) // For new tables the physical column name is a UUID. For existing tables, we diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala index 49c7b75f2ff..cf84816ed3d 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala @@ -15,34 +15,46 @@ */ package io.delta.kernel.defaults +import java.util.Collections + +import scala.collection.JavaConverters._ +import scala.collection.immutable.Seq + import io.delta.kernel._ import io.delta.kernel.defaults.internal.parquet.ParquetSuiteBase import io.delta.kernel.engine.Engine import io.delta.kernel.exceptions._ import io.delta.kernel.expressions.Literal -import io.delta.kernel.internal.{SnapshotImpl, TableImpl, TransactionBuilderImpl, TransactionImpl} +import io.delta.kernel.internal.{SnapshotImpl, TableConfig, TableImpl, TransactionBuilderImpl, TransactionImpl} import io.delta.kernel.internal.actions.{DomainMetadata, Protocol, SingleAction} -import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.internal.rowtracking.RowTrackingMetadataDomain +import io.delta.kernel.internal.tablefeatures.TableFeatures +import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} -import org.apache.hadoop.fs.Path + import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.delta.RowId.{RowTrackingMetadataDomain => SparkRowTrackingMetadataDomain} import org.apache.spark.sql.delta.actions.{DomainMetadata => SparkDomainMetadata} -import org.apache.spark.sql.delta.RowId.{ - RowTrackingMetadataDomain => SparkRowTrackingMetadataDomain -} import org.apache.spark.sql.delta.test.DeltaTestImplicits.OptimisticTxnTestHelper -import java.util.Collections -import scala.collection.JavaConverters._ -import scala.collection.immutable.Seq +import org.apache.hadoop.fs.Path class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { private def assertDomainMetadata( snapshot: SnapshotImpl, expectedValue: Map[String, DomainMetadata]): Unit = { + // Check using internal API assert(expectedValue === snapshot.getDomainMetadataMap.asScala) + // Verify public API + expectedValue.foreach { case (key, domainMetadata) => + snapshot.getDomainMetadata(key).toScala match { + case Some(config) => + assert(!domainMetadata.isRemoved && config == domainMetadata.getConfiguration) + case None => + assert(domainMetadata.isRemoved) + } + } } private def assertDomainMetadata( @@ -57,23 +69,35 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase private def createTxnWithDomainMetadatas( engine: Engine, tablePath: String, - domainMetadatas: Seq[DomainMetadata]): Transaction = { + domainMetadatas: Seq[DomainMetadata], + useInternalApi: Boolean = false): Transaction = { - val txnBuilder = createWriteTxnBuilder(TableImpl.forPath(engine, tablePath)) - .asInstanceOf[TransactionBuilderImpl] + var txnBuilder = createWriteTxnBuilder(TableImpl.forPath(engine, tablePath)) - val txn = txnBuilder.build(engine).asInstanceOf[TransactionImpl] - txn.addDomainMetadatas(domainMetadatas.asJava) - txn + if (useInternalApi) { + val txn = txnBuilder.build(engine).asInstanceOf[TransactionImpl] + txn.addDomainMetadatas(domainMetadatas.asJava) + txn + } else { + domainMetadatas.foreach { dm => + if (dm.isRemoved) { + txnBuilder = txnBuilder.withDomainMetadataRemoved(dm.getDomain()) + } else { + txnBuilder = txnBuilder.withDomainMetadata(dm.getDomain(), dm.getConfiguration()) + } + } + txnBuilder.build(engine) + } } private def commitDomainMetadataAndVerify( engine: Engine, tablePath: String, domainMetadatas: Seq[DomainMetadata], - expectedValue: Map[String, DomainMetadata]): Unit = { + expectedValue: Map[String, DomainMetadata], + useInternalApi: Boolean = false): Unit = { // Create the transaction with domain metadata and commit - val txn = createTxnWithDomainMetadatas(engine, tablePath, domainMetadatas) + val txn = createTxnWithDomainMetadatas(engine, tablePath, domainMetadatas, useInternalApi) txn.commit(engine, emptyIterable()) // Verify the final state includes the expected domain metadata @@ -81,12 +105,13 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase assertDomainMetadata(table, engine, expectedValue) } + // TODO we probably don't always need this since domain metadata is now automatically enabled private def setDomainMetadataSupport(engine: Engine, tablePath: String): Unit = { val protocol = new Protocol( 3, // minReaderVersion 7, // minWriterVersion - Collections.emptyList(), // readerFeatures - Seq("domainMetadata").asJava // writerFeatures + Collections.emptySet(), // readerFeatures + Set("domainMetadata").asJava // writerFeatures ) val protocolAction = SingleAction.createProtocolSingleAction(protocol.toRow) @@ -143,9 +168,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase } assert( ex.getMessage.contains( - "A concurrent writer added a domainMetadata action for the same domain" - ) - ) + "A concurrent writer added a domainMetadata action for the same domain")) } else { // We expect the commit of txn1 to succeed txn1.commit(engine, emptyIterable()) @@ -179,7 +202,8 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase .commit(engine, emptyIterable()) val dm1 = new DomainMetadata("domain1", "", false) - val txn1 = createTxnWithDomainMetadatas(engine, tablePath, List(dm1)) + // We use the internal API because our public API will automatically upgrade the protocol + val txn1 = createTxnWithDomainMetadatas(engine, tablePath, List(dm1), useInternalApi = true) // We expect the commit to fail because the table doesn't support domain metadata val e = intercept[KernelException] { @@ -189,9 +213,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase e.getMessage .contains( "Cannot commit DomainMetadata action(s) because the feature 'domainMetadata' " - + "is not supported on this table." - ) - ) + + "is not supported on this table.")) // Set writer version and writer feature to support domain metadata setDomainMetadataSupport(engine, tablePath) @@ -201,8 +223,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase engine, tablePath, domainMetadatas = Seq(dm1), - expectedValue = Map("domain1" -> dm1) - ) + expectedValue = Map("domain1" -> dm1)) } } @@ -214,16 +235,19 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase val dm2 = new DomainMetadata("domain2", "", false) val dm1_2 = new DomainMetadata("domain1", """{"key1":"10"}"""", false) - val txn = createTxnWithDomainMetadatas(engine, tablePath, List(dm1_1, dm2, dm1_2)) + // use internal API because public API overrides multiple domains with the same identifier + val txn = createTxnWithDomainMetadatas( + engine, + tablePath, + List(dm1_1, dm2, dm1_2), + useInternalApi = true) val e = intercept[IllegalArgumentException] { txn.commit(engine, emptyIterable()) } assert( e.getMessage.contains( - "Multiple actions detected for domain 'domain1' in single transaction" - ) - ) + "Multiple actions detected for domain 'domain1' in single transaction")) } } @@ -241,8 +265,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase Seq( (Seq(dm1), Map("domain1" -> dm1)), (Seq(dm2, dm3, dm1_2), Map("domain1" -> dm1_2, "domain2" -> dm2, "domain3" -> dm3)), - (Seq(dm3_2), Map("domain1" -> dm1_2, "domain2" -> dm2, "domain3" -> dm3_2)) - ).foreach { + (Seq(dm3_2), Map("domain1" -> dm1_2, "domain2" -> dm2, "domain3" -> dm3_2))).foreach { case (domainMetadatas, expectedValue) => commitDomainMetadataAndVerify(engine, tablePath, domainMetadatas, expectedValue) } @@ -260,8 +283,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase engine, tablePath, domainMetadatas = Seq(dm1, dm2), - expectedValue = Map("domain1" -> dm1, "domain2" -> dm2) - ) + expectedValue = Map("domain1" -> dm1, "domain2" -> dm2)) // Restart the table and verify the domain metadata val table2 = Table.forPath(engine, tablePath) @@ -278,14 +300,15 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase val dm2 = new DomainMetadata("domain2", "", false) val dm3 = new DomainMetadata("domain3", """{"key3":"3"}""", false) val dm1_2 = new DomainMetadata("domain1", """{"key1":"10"}""", false) - val dm3_2 = new DomainMetadata("domain3", """{"key3":"30"}""", true) + val dm3_2 = new DomainMetadata("domain3", """{"key3":"3"}""", true) Seq( (Seq(dm1), Map("domain1" -> dm1)), (Seq(dm2), Map("domain1" -> dm1, "domain2" -> dm2)), (Seq(dm3), Map("domain1" -> dm1, "domain2" -> dm2, "domain3" -> dm3)), - (Seq(dm1_2, dm3_2), Map("domain1" -> dm1_2, "domain2" -> dm2, "domain3" -> dm3_2)) - ).foreach { + ( + Seq(dm1_2, dm3_2), + Map("domain1" -> dm1_2, "domain2" -> dm2, "domain3" -> dm3_2))).foreach { case (domainMetadatas, expectedValue) => commitDomainMetadataAndVerify(engine, tablePath, domainMetadatas, expectedValue) } @@ -299,8 +322,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase assertDomainMetadata( table2, engine, - Map("domain1" -> dm1_2, "domain2" -> dm2, "domain3" -> dm3_2) - ) + Map("domain1" -> dm1_2, "domain2" -> dm2, "domain3" -> dm3_2)) } } @@ -326,14 +348,12 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase currentTxn1DomainMetadatas = Seq(dm1), winningTxn2DomainMetadatas = Seq.empty, winningTxn3DomainMetadatas = Seq.empty, - expectedConflict = false - ) + expectedConflict = false) } } test( - "Conflict resolution - three concurrent txns have DomainMetadata w/o conflicting domains" - ) { + "Conflict resolution - three concurrent txns have DomainMetadata w/o conflicting domains") { withTempDirAndEngine { (tablePath, engine) => /** * Txn1: include DomainMetadata action for "domain1". @@ -357,14 +377,12 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase currentTxn1DomainMetadatas = Seq(dm1), winningTxn2DomainMetadatas = Seq(dm2), winningTxn3DomainMetadatas = Seq(dm3), - expectedConflict = false - ) + expectedConflict = false) } } test( - "Conflict resolution - three concurrent txns have DomainMetadata w/ conflicting domains" - ) { + "Conflict resolution - three concurrent txns have DomainMetadata w/ conflicting domains") { withTempDirAndEngine { (tablePath, engine) => /** * Txn1: include DomainMetadata action for "domain1". @@ -388,14 +406,12 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase currentTxn1DomainMetadatas = Seq(dm1), winningTxn2DomainMetadatas = Seq(dm2), winningTxn3DomainMetadatas = Seq(dm3), - expectedConflict = true - ) + expectedConflict = true) } } test( - "Conflict resolution - three concurrent txns have DomainMetadata w/ conflict domains - 2" - ) { + "Conflict resolution - three concurrent txns have DomainMetadata w/ conflict domains - 2") { withTempDirAndEngine { (tablePath, engine) => /** * Txn1: include DomainMetadata action for "domain1". @@ -419,8 +435,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase currentTxn1DomainMetadatas = Seq(dm1), winningTxn2DomainMetadatas = Seq(dm2), winningTxn3DomainMetadatas = Seq(dm3), - expectedConflict = true - ) + expectedConflict = true) } } @@ -433,9 +448,8 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase spark.sql(s"CREATE TABLE $tbl (id LONG) USING delta LOCATION '$tablePath'") spark.sql( s"ALTER TABLE $tbl SET TBLPROPERTIES(" + - s"'delta.feature.domainMetadata' = 'enabled'," + - s"'delta.checkpointInterval' = '3')" - ) + s"'delta.feature.domainMetadata' = 'enabled'," + + s"'delta.checkpointInterval' = '3')") // Manually commit domain metadata actions. This will create 02.json val deltaLog = DeltaLog.forTable(spark, new Path(tablePath)) @@ -445,9 +459,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase List( SparkDomainMetadata("testDomain1", "{\"key1\":\"1\"}", removed = false), SparkDomainMetadata("testDomain2", "", removed = false), - SparkDomainMetadata("testDomain3", "", removed = false) - ): _* - ) + SparkDomainMetadata("testDomain3", "", removed = false)): _*) // This will create 03.json and 03.checkpoint spark.range(0, 2).write.format("delta").mode("append").save(tablePath) @@ -458,9 +470,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase .commitManually( List( SparkDomainMetadata("testDomain1", "{\"key1\":\"10\"}", removed = false), - SparkDomainMetadata("testDomain2", "", removed = true) - ): _* - ) + SparkDomainMetadata("testDomain2", "", removed = true)): _*) // Use Delta Kernel to read the table's domain metadata and verify the result. // We will need to read 1 checkpoint file and 1 log file to replay the table. @@ -476,8 +486,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase val snapshot = latestSnapshot(tablePath).asInstanceOf[SnapshotImpl] assertDomainMetadata( snapshot, - Map("testDomain1" -> dm1, "testDomain2" -> dm2, "testDomain3" -> dm3) - ) + Map("testDomain1" -> dm1, "testDomain2" -> dm2, "testDomain3" -> dm3)) } }) } @@ -497,14 +506,12 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase engine, tablePath, domainMetadatas = Seq(dm1, dm2, dm3), - expectedValue = Map("testDomain1" -> dm1, "testDomain2" -> dm2, "testDomain3" -> dm3) - ) + expectedValue = Map("testDomain1" -> dm1, "testDomain2" -> dm2, "testDomain3" -> dm3)) appendData( engine, tablePath, - data = Seq(Map.empty[String, Literal] -> dataBatches1) - ) + data = Seq(Map.empty[String, Literal] -> dataBatches1)) // Checkpoint the table so domain metadata is distributed to both checkpoint and log files val table = Table.forPath(engine, tablePath) @@ -518,8 +525,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase engine, tablePath, domainMetadatas = Seq(dm1_2, dm2_2), - expectedValue = Map("testDomain1" -> dm1_2, "testDomain2" -> dm2_2, "testDomain3" -> dm3) - ) + expectedValue = Map("testDomain1" -> dm1_2, "testDomain2" -> dm2_2, "testDomain3" -> dm3)) // Use Spark to read the table's domain metadata and verify the result val deltaLog = DeltaLog.forTable(spark, new Path(tablePath)) @@ -535,11 +541,8 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase "testDomain1" -> SparkDomainMetadata( "testDomain1", """{"key1":"10"}""", - removed = false - ), - "testDomain3" -> SparkDomainMetadata("testDomain3", "", removed = false) - ) - ) + removed = false), + "testDomain3" -> SparkDomainMetadata("testDomain3", "", removed = false))) } } } @@ -559,7 +562,8 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase engine, tablePath, domainMetadatas = Seq(dmAction), - expectedValue = Map(rowTrackingMetadataDomain.getDomainName -> dmAction) + expectedValue = Map(rowTrackingMetadataDomain.getDomainName -> dmAction), + useInternalApi = true // cannot commit system-controlled domains through public API ) // Read the RowTrackingMetadataDomain from the table and verify @@ -581,9 +585,8 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase spark.sql(s"CREATE TABLE $tbl (id LONG) USING delta LOCATION '$tablePath'") spark.sql( s"ALTER TABLE $tbl SET TBLPROPERTIES(" + - s"'delta.feature.domainMetadata' = 'enabled'," + - s"'delta.feature.rowTracking' = 'supported')" - ) + s"'delta.feature.domainMetadata' = 'enabled'," + + s"'delta.feature.rowTracking' = 'supported')") // Append 100 rows to the table, with fresh row IDs from 0 to 99 // The `delta.rowTracking.rowIdHighWaterMark` should be 99 @@ -612,7 +615,8 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase engine, tablePath, domainMetadatas = Seq(dmAction), - expectedValue = Map(dmAction.getDomain -> dmAction) + expectedValue = Map(dmAction.getDomain -> dmAction), + useInternalApi = true // cannot commit system-controlled domains through public API ) // Use Spark to read the table's row tracking metadata domain and verify the result @@ -623,4 +627,258 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase } } } + + test("basic txnBuilder.withDomainMetadata API tests") { + // withDomainMetadata is tested thoroughly elsewhere in this suite, here we just test API + // specific behaviors + + // Cannot set system-controlled domain metadata + Seq("delta.foo", "DELTA.foo").foreach { domain => + val e = intercept[IllegalArgumentException] { + createWriteTxnBuilder(Table.forPath(defaultEngine, "fake/path")) + .withDomainMetadata(domain, "misc config") + } + assert(e.getMessage.contains("Setting a system-controlled domain is not allowed")) + } + + // Setting the same domain more than once uses the latest pair + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + + val dm1_1 = new DomainMetadata("domain1", """{"key1":"1"}""", false) + val dm1_2 = new DomainMetadata("domain1", """{"key1":"10"}"""", false) + + commitDomainMetadataAndVerify(engine, tablePath, List(dm1_1, dm1_2), Map("domain1" -> dm1_2)) + } + } + + test("basic txnBuilder.withDomainMetadataRemoved API tests") { + // withDomainMetadataRemoved is tested thoroughly elsewhere in this suite, here we just test API + // specific behaviors + + // Cannot remove system-controlled domain metadata + Seq("delta.foo", "DELTA.foo").foreach { domain => + val e = intercept[IllegalArgumentException] { + createWriteTxnBuilder(Table.forPath(defaultEngine, "fake/path")) + .withDomainMetadataRemoved(domain) + } + + assert(e.getMessage.contains("Removing a system-controlled domain is not allowed")) + } + + // Can remove same domain more than once in same txn + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + + // Set up table with domain "domain1 + val dm1 = new DomainMetadata("domain1", """{"key1":"1"}""", false) + commitDomainMetadataAndVerify(engine, tablePath, List(dm1), Map("domain1" -> dm1)) + + val dm1_removed = dm1.removed() + commitDomainMetadataAndVerify( + engine, + tablePath, + List(dm1_removed, dm1_removed, dm1_removed), + Map("domain1" -> dm1_removed)) + } + } + + test("txnBuilder.withDomainMetadataRemoved removing a non-existent domain") { + // Remove domain that does not exist and has never existed + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + + intercept[DomainDoesNotExistException] { + createWriteTxnBuilder(Table.forPath(defaultEngine, tablePath)) + .withDomainMetadataRemoved("foo") + .build(defaultEngine) + } + } + + // Remove domain that exists as a tombstone + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + + // Set up table with domain "domain1" + val dm1 = new DomainMetadata("domain1", """{"key1":"1"}""", false) + commitDomainMetadataAndVerify(engine, tablePath, List(dm1), Map("domain1" -> dm1)) + + // Remove domain1 so it exists as a tombstone + val dm1_removed = dm1.removed() + commitDomainMetadataAndVerify( + engine, + tablePath, + List(dm1_removed), + Map("domain1" -> dm1_removed)) + + // Removing it again should fail since it doesn't exist + intercept[DomainDoesNotExistException] { + commitDomainMetadataAndVerify( + engine, + tablePath, + List(dm1_removed), + Map("domain1" -> dm1_removed)) + } + } + } + + test("Using add and remove with the same domain in the same txn") { + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + // We forbid adding + removing a domain with the same identifier in a transaction to avoid + // any ambiguous behavior + // For example, is the expected behavior + // a) we don't write any domain metadata, and it's a no-op (remove cancels out the add) + // b) we remove the previous domain from the read snapshot, and add the new one as the current + // domain metadata + + { + val e = intercept[IllegalArgumentException] { + createWriteTxnBuilder(Table.forPath(defaultEngine, tablePath)) + .withDomainMetadata("foo", "fake config") + .withDomainMetadataRemoved("foo") + } + assert(e.getMessage.contains("Cannot remove a domain that is added in this transaction")) + } + { + val e = intercept[IllegalArgumentException] { + createWriteTxnBuilder(Table.forPath(defaultEngine, tablePath)) + .withDomainMetadataRemoved("foo") + .withDomainMetadata("foo", "fake config") + } + assert(e.getMessage.contains("Cannot add a domain that is removed in this transaction")) + } + } + } + + test("basic snapshot.getDomainMetadataConfiguration API tests") { + // getDomainMetadataConfiguration is tested thoroughly elsewhere in this suite, here we just + // test the API directly to be safe + + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + + // Non-existent domain is not returned + assert(!latestSnapshot(tablePath).getDomainMetadata("foo").isPresent) + + // Commit domain foo + val fooDm = new DomainMetadata("foo", "foo!", false) + commitDomainMetadataAndVerify(engine, tablePath, List(fooDm), Map("foo" -> fooDm)) + assert( // Check here even though already verified in commitDomainMetadataAndVerify + latestSnapshot(tablePath).getDomainMetadata("foo").toScala.contains("foo!")) + + // Remove domain foo (so tombstone exists but should not be returned) + val fooDm_removed = fooDm.removed() + commitDomainMetadataAndVerify( + engine, + tablePath, + List(fooDm_removed), + Map("foo" -> fooDm_removed)) + // Already checked in commitDomainMetadataAndVerify but check again + assert(!latestSnapshot(tablePath).getDomainMetadata("foo").isPresent) + } + } + + /* --------------- Automatic upgrade table feature tests -------------- */ + + private def verifyDomainMetadataFeatureSupport( + engine: Engine, + tablePath: String, + isSupported: Boolean = true): Unit = { + val snapshotImpl = Table.forPath(engine, tablePath).getLatestSnapshot(engine) + .asInstanceOf[SnapshotImpl] + assert(TableFeatures.isDomainMetadataSupported(snapshotImpl.getProtocol) == isSupported) + } + + /** + * For the given tablePath, commit a transaction that sets a domain metadata using + * withDomainMetadata when building the transaction (which should automatically upgrade the table + * protocol). Verifies that the domain metadata table feature is unsupported before committing + * (for an existing table) and is supported after committing. + * + * @param isNewTable if true, sets a schema during transaction building to create a new table + * @param tableProperties if provided sets the table properties during transaction building + */ + private def verifyAutomaticUpgrade( + engine: Engine, + tablePath: String, + isNewTable: Boolean = false, + tableProperties: Option[Map[String, String]] = None): Unit = { + if (!isNewTable) { + // Verify it's not supported yet + verifyDomainMetadataFeatureSupport(engine, tablePath, isSupported = false) + } + // Commit to table using withDomainMetadata and upgrade protocol + var txnBuilder = createWriteTxnBuilder(Table.forPath(engine, tablePath)) + .withDomainMetadata("foo", "configuration string") + if (isNewTable) { + txnBuilder = txnBuilder.withSchema(engine, testSchema) + } + if (tableProperties.nonEmpty) { + txnBuilder = txnBuilder.withTableProperties(engine, tableProperties.get.asJava) + } + txnBuilder.build(engine).commit(engine, emptyIterable()) + // Check the feature is now supported + verifyDomainMetadataFeatureSupport(engine, tablePath) + } + + test("automatically enable DomainMetadata when using withDomainMetadata - new table") { + // New table using the withDomainMetadata API + withTempDirAndEngine { (tablePath, engine) => + verifyAutomaticUpgrade(engine, tablePath, isNewTable = true) + } + } + + test("automatically enable DomainMetadata when using withDomainMetadata - existing " + + "table with legacy protocol") { + withTempDirAndEngine { (tablePath, engine) => + // Create table with legacy protocol + createTxn(tablePath = tablePath, isNewTable = true, schema = testSchema, partCols = Seq()) + .commit(engine, emptyIterable()) + verifyAutomaticUpgrade(engine, tablePath) + } + } + + test("automatically enable DomainMetadata when using withDomainMetadata - existing " + + "table with TF protocol") { + withTempDirAndEngine { (tablePath, engine) => + // Create table with table feature protocol + createTxn( + tablePath = tablePath, + isNewTable = true, + schema = testSchema, + partCols = Seq(), + // Enable inCommitTimestamps to bump the protocol + tableProperties = Map(TableConfig.IN_COMMIT_TIMESTAMPS_ENABLED.getKey -> "true")) + .commit(engine, emptyIterable()) + verifyAutomaticUpgrade(engine, tablePath) + } + } + + test("automatically enable DomainMetadata when using withDomainMetadata - existing " + + "table, upgrade two TF in same txn") { + withTempDirAndEngine { (tablePath, engine) => + // Create table with legacy protocol + createTxn(tablePath = tablePath, isNewTable = true, schema = testSchema, partCols = Seq()) + .commit(engine, emptyIterable()) + verifyAutomaticUpgrade( + engine, + tablePath, + // Enable inCommitTimestamps as well + tableProperties = Some(Map(TableConfig.IN_COMMIT_TIMESTAMPS_ENABLED.getKey -> "true"))) + } + } + + test("removing a domain on a table without DomainMetadata support") { + withTempDirAndEngine { (tablePath, engine) => + // Create table with legacy protocol + createTxn(tablePath = tablePath, isNewTable = true, schema = testSchema, partCols = Seq()) + .commit(engine, emptyIterable()) + intercept[DomainDoesNotExistException] { + createWriteTxnBuilder(Table.forPath(engine, tablePath)) + .withDomainMetadataRemoved("foo") + .build(engine) + } + } + } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/InCommitTimestampSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/InCommitTimestampSuite.scala index ac7b302c2ce..4094acc884b 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/InCommitTimestampSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/InCommitTimestampSuite.scala @@ -15,28 +15,29 @@ */ package io.delta.kernel.defaults -import io.delta.kernel.Operation.{CREATE_TABLE, WRITE} +import java.util.{Locale, Optional} + +import scala.collection.JavaConverters._ +import scala.collection.immutable.{ListMap, Seq} +import scala.collection.mutable + import io.delta.kernel._ +import io.delta.kernel.Operation.{CREATE_TABLE, WRITE} import io.delta.kernel.engine.Engine import io.delta.kernel.exceptions.{InvalidTableException, ProtocolChangedException} import io.delta.kernel.expressions.Literal +import io.delta.kernel.internal.{DeltaHistoryManager, SnapshotImpl, TableImpl} +import io.delta.kernel.internal.TableConfig._ import io.delta.kernel.internal.actions.{CommitInfo, SingleAction} +import io.delta.kernel.internal.actions.SingleAction.createCommitInfoSingleAction import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.util.{FileNames, VectorUtils} -import io.delta.kernel.internal.{DeltaHistoryManager, SnapshotImpl, TableImpl} import io.delta.kernel.internal.util.ManualClock import io.delta.kernel.internal.util.Utils.singletonCloseableIterator -import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types._ +import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} - -import java.util.{Locale, Optional} -import scala.collection.JavaConverters._ -import scala.collection.immutable.{ListMap, Seq} -import scala.collection.mutable -import io.delta.kernel.internal.TableConfig._ import io.delta.kernel.utils.FileStatus -import io.delta.kernel.internal.actions.SingleAction.createCommitInfoSingleAction class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { @@ -54,7 +55,9 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { engine .getJsonHandler .writeJsonFileAtomically( - FileNames.deltaFile(logPath, version), rowsWithoutCommitInfo, true /* overwrite */) + FileNames.deltaFile(logPath, version), + rowsWithoutCommitInfo, + true /* overwrite */ ) } test("Enable ICT on commit 0") { @@ -127,8 +130,7 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { partCols = Seq.empty, data = Seq(Map.empty[String, Literal] -> dataBatches1), clock = clock, - tableProperties = Map(IN_COMMIT_TIMESTAMPS_ENABLED.getKey -> "true") - ) + tableProperties = Map(IN_COMMIT_TIMESTAMPS_ENABLED.getKey -> "true")) val ver1Snapshot = table.getLatestSnapshot(engine).asInstanceOf[SnapshotImpl] val ver1Timestamp = ver1Snapshot.getTimestamp(engine) @@ -139,8 +141,7 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { engine, tablePath, data = Seq(Map.empty[String, Literal] -> dataBatches2), - clock = clock - ) + clock = clock) val ver2Snapshot = table.getLatestSnapshot(engine).asInstanceOf[SnapshotImpl] val ver2Timestamp = ver2Snapshot.getTimestamp(engine) @@ -168,7 +169,9 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { assert(ex.getMessage.contains(String.format( "This table has the feature %s enabled which requires the presence of the " + "CommitInfo action in every commit. However, the CommitInfo action is " + - "missing from commit version %s.", "inCommitTimestamp", "0"))) + "missing from commit version %s.", + "inCommitTimestamp", + "0"))) } } @@ -178,7 +181,12 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { val table = Table.forPath(engine, tablePath) setTablePropAndVerify( - engine, tablePath, isNewTable = true, IN_COMMIT_TIMESTAMPS_ENABLED, "true", true) + engine, + tablePath, + isNewTable = true, + IN_COMMIT_TIMESTAMPS_ENABLED, + "true", + true) // Remove CommitInfo.inCommitTimestamp from the commit. val logPath = new Path(table.getPath(engine), "_delta_log") val file = FileStatus.of(FileNames.deltaFile(logPath, 0), 0, 0) @@ -206,7 +214,8 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { .getJsonHandler .writeJsonFileAtomically( FileNames.deltaFile(logPath, 0), - rowsWithoutCommitInfoInCommitTimestamp, true /* overwrite */) + rowsWithoutCommitInfoInCommitTimestamp, + true /* overwrite */ ) val ex = intercept[InvalidTableException] { table.getLatestSnapshot(engine).asInstanceOf[SnapshotImpl].getTimestamp(engine) @@ -214,7 +223,9 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { assert(ex.getMessage.contains(String.format( "This table has the feature %s enabled which requires the presence of " + "inCommitTimestamp in the CommitInfo action. However, this field has not " + - "been set in commit version %s.", "inCommitTimestamp", "0"))) + "been set in commit version %s.", + "inCommitTimestamp", + "0"))) } } @@ -389,13 +400,20 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { verifyCommitResult(commitResult, expVersion = 1, expIsReadyForCheckpoint = false) verifyCommitInfo(tablePath, version = 1, partitionCols = null, operation = WRITE) verifyWrittenContent(tablePath, testSchema, expData) - verifyTableProperties(tablePath, - ListMap(IN_COMMIT_TIMESTAMPS_ENABLED.getKey -> true, - "delta.feature.inCommitTimestamp" -> "supported", - IN_COMMIT_TIMESTAMP_ENABLEMENT_TIMESTAMP.getKey - -> getInCommitTimestamp(engine, table, version = 1).get, - IN_COMMIT_TIMESTAMP_ENABLEMENT_VERSION.getKey -> 1L), - 3, + verifyTableProperties( + tablePath, + ListMap( + // appendOnly, invariants implicitly supported as the protocol is upgraded from 2 to 7 + // These properties are not set in the table properties, but are generated by the + // Spark describe + IN_COMMIT_TIMESTAMPS_ENABLED.getKey -> true, + "delta.feature.appendOnly" -> "supported", + "delta.feature.inCommitTimestamp" -> "supported", + "delta.feature.invariants" -> "supported", + IN_COMMIT_TIMESTAMP_ENABLEMENT_TIMESTAMP.getKey + -> getInCommitTimestamp(engine, table, version = 1).get, + IN_COMMIT_TIMESTAMP_ENABLEMENT_VERSION.getKey -> 1L), + 1, 7) } } @@ -418,7 +436,12 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { withTempDirAndEngine { (tablePath, engine) => val table = TableImpl.forPath(engine, tablePath, () => System.currentTimeMillis) setTablePropAndVerify( - engine, tablePath, isNewTable = true, IN_COMMIT_TIMESTAMPS_ENABLED, "true", true) + engine, + tablePath, + isNewTable = true, + IN_COMMIT_TIMESTAMPS_ENABLED, + "true", + true) val startTime = System.currentTimeMillis() val clock = new ManualClock(startTime) @@ -427,15 +450,13 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { tablePath, schema = testSchema, partCols = Seq.empty, - clock = clock - ) + clock = clock) clock.setTime(startTime) appendData( engine, tablePath, data = Seq(Map.empty[String, Literal] -> dataBatches2), - clock = clock - ) + clock = clock) clock.setTime(startTime - 1000) commitAppendData(engine, txn1, Seq(Map.empty[String, Literal] -> dataBatches1)) assert( @@ -473,14 +494,14 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { engine, tablePath, data = Seq(Map.empty[String, Literal] -> dataBatches2), - clock = clock - ) + clock = clock) } commitAppendData(engine, txn1, Seq(Map.empty[String, Literal] -> dataBatches1)) val lastSnapshot = table.getSnapshotAsOfVersion( - engine, winningCommitCount).asInstanceOf[SnapshotImpl] + engine, + winningCommitCount).asInstanceOf[SnapshotImpl] val curSnapshot = table.getLatestSnapshot(engine).asInstanceOf[SnapshotImpl] val observedEnablementTimestamp = IN_COMMIT_TIMESTAMP_ENABLEMENT_TIMESTAMP.fromMetadata(curSnapshot.getMetadata) @@ -500,7 +521,12 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { withTempDirAndEngine { (tablePath, engine) => val table = TableImpl.forPath(engine, tablePath, () => System.currentTimeMillis) setTablePropAndVerify( - engine, tablePath, isNewTable = true, IN_COMMIT_TIMESTAMPS_ENABLED, "true", true) + engine, + tablePath, + isNewTable = true, + IN_COMMIT_TIMESTAMPS_ENABLED, + "true", + true) val startTime = System.currentTimeMillis() val clock = new ManualClock(startTime) @@ -509,21 +535,18 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { tablePath, schema = testSchema, partCols = Seq.empty, - clock = clock - ) + clock = clock) clock.setTime(startTime) appendData( engine, tablePath, data = Seq(Map.empty[String, Literal] -> dataBatches2), - clock = clock - ) + clock = clock) appendData( engine, tablePath, data = Seq(Map.empty[String, Literal] -> dataBatches2), - clock = clock - ) + clock = clock) // Remove CommitInfo from the commit. val logPath = new Path(table.getPath(engine), "_delta_log") @@ -536,7 +559,9 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { assert(ex.getMessage.contains(String.format( "This table has the feature %s enabled which requires the presence of the " + "CommitInfo action in every commit. However, the CommitInfo action is " + - "missing from commit version %s.", "inCommitTimestamp", "2"))) + "missing from commit version %s.", + "inCommitTimestamp", + "2"))) } } @@ -560,16 +585,14 @@ class InCommitTimestampSuite extends DeltaTableWriteSuiteBase { schema = testSchema, partCols = Seq.empty, tableProperties = Map(IN_COMMIT_TIMESTAMPS_ENABLED.getKey -> "true"), - clock = clock - ) + clock = clock) clock.setTime(startTime) appendData( engine, tablePath, data = Seq(Map.empty[String, Literal] -> dataBatches2), tableProperties = Map(IN_COMMIT_TIMESTAMPS_ENABLED.getKey -> "true"), - clock = clock - ) + clock = clock) clock.setTime(startTime - 1000) val ex = intercept[ProtocolChangedException] { commitAppendData(engine, txn1, Seq(Map.empty[String, Literal] -> dataBatches1)) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayEngineMetricsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayEngineMetricsSuite.scala index 4468011847e..f0b31d7c7da 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayEngineMetricsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayEngineMetricsSuite.scala @@ -16,6 +16,14 @@ package io.delta.kernel.defaults +import java.io.File +import java.nio.file.Files +import java.util +import java.util.Optional + +import scala.collection.convert.ImplicitConversions._ +import scala.collection.mutable.ArrayBuffer + import io.delta.kernel.Table import io.delta.kernel.data.ColumnarBatch import io.delta.kernel.defaults.engine.{DefaultEngine, DefaultJsonHandler, DefaultParquetHandler} @@ -28,19 +36,14 @@ import io.delta.kernel.internal.util.FileNames import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.types.StructType import io.delta.kernel.utils.{CloseableIterator, FileStatus} -import org.apache.hadoop.conf.Configuration + import org.apache.spark.sql.delta.{DeltaConfig, DeltaLog} import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.functions.{col, log} -import org.scalatest.funsuite.AnyFunSuite -import java.io.File -import java.nio.file.Files -import java.util -import java.util.Optional -import scala.collection.convert.ImplicitConversions._ -import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.conf.Configuration import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions.{col, log} +import org.scalatest.funsuite.AnyFunSuite /** * Suite to test the engine metrics while replaying logs for getting the table protocol and @@ -125,34 +128,35 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { val actualParquetVersionsRead = engine.getParquetHandler.getVersionsRead assert( - actualJsonVersionsRead === expJsonVersionsRead, s"Expected to read json versions " + - s"$expJsonVersionsRead but read $actualJsonVersionsRead" - ) + actualJsonVersionsRead === expJsonVersionsRead, + s"Expected to read json versions " + + s"$expJsonVersionsRead but read $actualJsonVersionsRead") assert( - actualParquetVersionsRead === expParquetVersionsRead, s"Expected to read parquet " + - s"versions $expParquetVersionsRead but read $actualParquetVersionsRead" - ) + actualParquetVersionsRead === expParquetVersionsRead, + s"Expected to read parquet " + + s"versions $expParquetVersionsRead but read $actualParquetVersionsRead") if (expParquetReadSetSizes != null) { val actualParquetReadSetSizes = engine.getParquetHandler.checkpointReadRequestSizes assert( - actualParquetReadSetSizes === expParquetReadSetSizes, s"Expected parquet read set sizes " + - s"$expParquetReadSetSizes but read $actualParquetReadSetSizes" - ) + actualParquetReadSetSizes === expParquetReadSetSizes, + s"Expected parquet read set sizes " + + s"$expParquetReadSetSizes but read $actualParquetReadSetSizes") } expLastCheckpointReadCalls.foreach { expCalls => val actualCalls = engine.getJsonHandler.getLastCheckpointMetadataReadCalls - assert(actualCalls === expCalls, + assert( + actualCalls === expCalls, s"Expected to read last checkpoint metadata $expCalls times but read $actualCalls times") } if (expChecksumReadSet != null) { val actualChecksumReadSet = engine.getJsonHandler.checksumsRead assert( - actualChecksumReadSet === expChecksumReadSet, s"Expected checksum read set " + - s"$expChecksumReadSet but read $actualChecksumReadSet" - ) + actualChecksumReadSet === expChecksumReadSet, + s"Expected checksum read set " + + s"$expChecksumReadSet but read $actualChecksumReadSet") } } @@ -178,8 +182,7 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { table.getLatestSnapshot(engine).getSchema(), engine, expJsonVersionsRead = 9L to 0L by -1L, - expParquetVersionsRead = Nil - ) + expParquetVersionsRead = Nil) } } @@ -246,7 +249,7 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { val table = Table.forPath(engine, path) - table.getLatestSnapshot(engine).getSchema() + table.getLatestSnapshot(engine).getSchema() // A hint is now saved at v14 @@ -287,7 +290,7 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { val table = Table.forPath(engine, path) - table.getLatestSnapshot(engine).getSchema() + table.getLatestSnapshot(engine).getSchema() // A hint is now saved at v3 @@ -374,8 +377,7 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { expParquetReadSetSizes = Seq(1, 1), // We try to read `_last_checkpoint` once. If it doesn't exist, we don't try reading // again. If it exists, we succeed reading in the first time - expLastCheckpointReadCalls = Some(1) - ) + expLastCheckpointReadCalls = Some(1)) } } } @@ -388,7 +390,7 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { test(s"checksum found at the read version: ${if (version == -1) "latest" else version}") { withTempDirAndMetricsEngine { (path, engine) => // Produce a test table with 0 to 11 .json, 0 to 11.crc, 10.checkpoint.parquet - buildTableWithCrc(path) + buildTableWithCrc(path) val table = Table.forPath(engine, path) loadPandMCheckMetrics( @@ -410,33 +412,29 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { withTempDirAndMetricsEngine { (path, engine) => // Produce a test table with 0 to 11 .json, 0 to 11.crc, 10.checkpoint.parquet buildTableWithCrc(path) - Seq(10L, 11L, 5L, 6L).foreach { version => + Seq(10L, 11L, 5L, 6L).foreach { version => assert( - Files.deleteIfExists( - new File( - FileNames.checksumFile(new Path(f"$path/_delta_log"), version).toString - ).toPath - ) - ) + Files.deleteIfExists( + new File( + FileNames.checksumFile(new Path(f"$path/_delta_log"), version).toString).toPath)) } loadPandMCheckMetrics( Table.forPath(engine, path) .getLatestSnapshot(engine).getSchema(), engine, - // 10.checkpoint found, so use it and combined with 11.crc - expJsonVersionsRead = Seq(11), - expParquetVersionsRead = Seq(10), - expParquetReadSetSizes = Seq(1), - expChecksumReadSet = Seq(11) - ) + // 10.checkpoint found, so use it and combined with 11.crc + expJsonVersionsRead = Seq(11), + expParquetVersionsRead = Seq(10), + expParquetReadSetSizes = Seq(1), + expChecksumReadSet = Seq(11)) - loadPandMCheckMetrics( - Table - .forPath(engine, path) - .getSnapshotAsOfVersion(engine, 6 /* versionId */ ) - .getSchema(), - engine, + loadPandMCheckMetrics( + Table + .forPath(engine, path) + .getSnapshotAsOfVersion(engine, 6 /* versionId */ ) + .getSchema(), + engine, // We find the checksum from crc at version 4, but still read commit files 5 and 6 // to find the P&M which could have been updated in version 5 and 6. expJsonVersionsRead = Seq(6, 5), @@ -446,11 +444,10 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { // last 100 crc files and read the latest one which is version 4 (as version 5 is deleted) expChecksumReadSet = Seq(6, 4)) - // now try to load version 3 and it should get P&M from checksum files only loadPandMCheckMetrics( Table.forPath(engine, path) - .getSnapshotAsOfVersion(engine, 3 /* versionId */).getSchema(), + .getSnapshotAsOfVersion(engine, 3 /* versionId */ ).getSchema(), engine, // We find the checksum from crc at version 3, so shouldn't read anything else expJsonVersionsRead = Nil, @@ -465,18 +462,17 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { // Produce a test table with 0 to 11 .json, 0 to 11.crc, 10.checkpoint.parquet buildTableWithCrc(path) (3 to 6).foreach { version => - assert( - Files.deleteIfExists( - new File(FileNames.checksumFile( - new Path(f"$path/_delta_log"), version).toString).toPath - ) - ) - } + assert( + Files.deleteIfExists( + new File(FileNames.checksumFile( + new Path(f"$path/_delta_log"), + version).toString).toPath)) + } - val table = Table.forPath(engine, path) + val table = Table.forPath(engine, path) loadPandMCheckMetrics( - table.getSnapshotAsOfVersion(engine, 4 /* versionId */).getSchema(), + table.getSnapshotAsOfVersion(engine, 4 /* versionId */ ).getSchema(), engine, // There are no checksum files for versions 4. Latest is at version 2. // We need to read the commit files 3 and 4 to get the P&M in addition the P&M from @@ -492,7 +488,7 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { // now try to load version 6 and we expect no checksums are read loadPandMCheckMetrics( - table.getSnapshotAsOfVersion(engine, 6 /* versionId */).getSchema(), + table.getSnapshotAsOfVersion(engine, 6 /* versionId */ ).getSchema(), engine, // We have snapshot P&M hint at version 4, and no checksum after 2 expJsonVersionsRead = Seq(6, 5), @@ -501,8 +497,7 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { // First we attempt to read at version 6, then we do a listing of last 100 crc files // bound by the snapshot hint which is at version 4 and we don't try to read checksums // beyond version 4 - expChecksumReadSet = Seq(6) - ) + expChecksumReadSet = Seq(6)) } } @@ -520,15 +515,12 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { expJsonVersionsRead = Nil, expParquetVersionsRead = Nil, expParquetReadSetSizes = Nil, - expChecksumReadSet = Nil - ) + expChecksumReadSet = Nil) } } - test( - "checksum not found at read version and checkpoint exists at read version => use checkpoint" - ) { + "checksum not found at read version and checkpoint exists at read version => use checkpoint") { withTempDirAndMetricsEngine { (path, engine) => // Produce a test table with 0 to 11 .json, 0 to 11.crc, 10.checkpoint.parquet buildTableWithCrc(path) @@ -536,18 +528,13 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { val logPath = f"$path/_delta_log" assert( Files.exists( - new File( - FileNames - .checkpointFileSingular(new Path(logPath), checkpointVersion) - .toString - ).toPath - ) - ) - assert( - Files.deleteIfExists( - new File(FileNames.checksumFile(new Path(logPath), checkpointVersion).toString).toPath - ) - ) + new File( + FileNames + .checkpointFileSingular(new Path(logPath), checkpointVersion) + .toString).toPath)) + assert( + Files.deleteIfExists( + new File(FileNames.checksumFile(new Path(logPath), checkpointVersion).toString).toPath)) val table = Table.forPath(engine, path) @@ -559,15 +546,13 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { expJsonVersionsRead = Nil, expParquetVersionsRead = Seq(10), expParquetReadSetSizes = Seq(1), - expChecksumReadSet = Seq(10) - ) + expChecksumReadSet = Seq(10)) } } test( "checksum missing read version and the previous version, " + - "checkpoint exists the read version the previous version => use checkpoint" - ) { + "checkpoint exists the read version the previous version => use checkpoint") { withTempDirAndMetricsEngine { (path, engine) => // Produce a test table with 0 to 11 .json, 0 to 11.crc, 10.checkpoint.parquet buildTableWithCrc(path) @@ -577,22 +562,16 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { Files .exists( new File( - FileNames.checkpointFileSingular(new Path(logPath), checkpointVersion).toString - ).toPath - ) - ) + FileNames.checkpointFileSingular( + new Path(logPath), + checkpointVersion).toString).toPath)) assert( Files.deleteIfExists( - new File(FileNames.checksumFile(new Path(logPath), checkpointVersion).toString).toPath - ) - ) + new File(FileNames.checksumFile(new Path(logPath), checkpointVersion).toString).toPath)) assert( Files.deleteIfExists( new File( - FileNames.checksumFile(new Path(logPath), checkpointVersion + 1).toString - ).toPath - ) - ) + FileNames.checksumFile(new Path(logPath), checkpointVersion + 1).toString).toPath)) val table = Table.forPath(engine, path) @@ -604,15 +583,13 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { expJsonVersionsRead = Seq(11), expParquetVersionsRead = Seq(10), expParquetReadSetSizes = Seq(1), - expChecksumReadSet = Seq(11) - ) + expChecksumReadSet = Seq(11)) } } test( "checksum missing read version, " + - "both checksum and checkpoint exist the read version the previous version => use checksum" - ) { + "both checksum and checkpoint exist the read version the previous version => use checksum") { withTempDirAndMetricsEngine { (path, engine) => // Produce a test table with 0 to 11 .json, 0 to 11.crc, 10.checkpoint.parquet buildTableWithCrc(path) @@ -623,17 +600,11 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { new File( FileNames .checkpointFileSingular(logPath, checkpointVersion) - .toString - ).toPath - ) - ) + .toString).toPath)) assert( Files.deleteIfExists( new File( - FileNames.checksumFile(logPath, checkpointVersion + 1).toString - ).toPath - ) - ) + FileNames.checksumFile(logPath, checkpointVersion + 1).toString).toPath)) val table = Table.forPath(engine, path) @@ -645,12 +616,10 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { expJsonVersionsRead = Seq(11), expParquetVersionsRead = Nil, expParquetReadSetSizes = Nil, - expChecksumReadSet = Seq(11, 10) - ) + expChecksumReadSet = Seq(11, 10)) } } - test("crc found at read version and checkpoint at read version => use checksum") { withTempDirAndMetricsEngine { (path, engine) => // Produce a test table with 0 to 11 .json, 0 to 11.crc, 10.checkpoint.parquet @@ -662,10 +631,7 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { new File( FileNames .checkpointFileSingular(logPath, checkpointVersion) - .toString - ).toPath - ) - ) + .toString).toPath)) val table = Table.forPath(engine, path) loadPandMCheckMetrics( @@ -674,8 +640,7 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { expJsonVersionsRead = Nil, expParquetVersionsRead = Nil, expParquetReadSetSizes = Nil, - expChecksumReadSet = Seq(10) - ) + expChecksumReadSet = Seq(10)) } } @@ -684,8 +649,7 @@ class LogReplayEngineMetricsSuite extends AnyFunSuite with TestUtils { withSQLConf(DeltaSQLConf.DELTA_WRITE_CHECKSUM_ENABLED.key -> "true") { spark.sql( s"CREATE TABLE delta.`$path` USING DELTA AS " + - s"SELECT 0L as id" - ) + s"SELECT 0L as id") for (_ <- 0 to 10) { appendCommit(path) } } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala index 58d9e064605..c4a30449ca3 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala @@ -16,32 +16,35 @@ package io.delta.kernel.defaults import java.io.File +import java.nio.file.Files import java.util.Optional import scala.collection.JavaConverters._ -import io.delta.golden.GoldenTableUtils.goldenTablePath -import org.scalatest.funsuite.AnyFunSuite -import org.apache.hadoop.conf.Configuration -import io.delta.kernel.types.{LongType, StructType} -import io.delta.kernel.internal.{InternalScanFileUtils, SnapshotImpl} -import io.delta.kernel.internal.data.ScanStateRow +import io.delta.golden.GoldenTableUtils.goldenTablePath +import io.delta.kernel.Table import io.delta.kernel.defaults.engine.DefaultEngine import io.delta.kernel.defaults.utils.{TestRow, TestUtils} -import io.delta.kernel.Table +import io.delta.kernel.internal.{InternalScanFileUtils, SnapshotImpl} +import io.delta.kernel.internal.data.ScanStateRow import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.util.FileNames +import io.delta.kernel.types.{LongType, StructType} + import org.apache.spark.sql.delta.sources.DeltaSQLConf -import java.nio.file.Files +import org.apache.hadoop.conf.Configuration +import org.scalatest.funsuite.AnyFunSuite class LogReplaySuite extends AnyFunSuite with TestUtils { - override lazy val defaultEngine = DefaultEngine.create(new Configuration() {{ - // Set the batch sizes to small so that we get to test the multiple batch scenarios. - set("delta.kernel.default.parquet.reader.batch-size", "2"); - set("delta.kernel.default.json.reader.batch-size", "2"); - }}) + override lazy val defaultEngine = DefaultEngine.create(new Configuration() { + { + // Set the batch sizes to small so that we get to test the multiple batch scenarios. + set("delta.kernel.default.parquet.reader.batch-size", "2"); + set("delta.kernel.default.json.reader.batch-size", "2"); + } + }) test("simple end to end with inserts and deletes and checkpoint") { val expectedValues = (0L until 5L) ++ (10L until 15L) ++ (20L until 25L) ++ @@ -50,8 +53,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { path = goldenTablePath("basic-with-inserts-deletes-checkpoint"), expectedAnswer = expectedValues.map(TestRow(_)), expectedSchema = new StructType().add("id", LongType.LONG), - expectedVersion = Some(13L) - ) + expectedVersion = Some(13L)) } test("simple end to end with inserts and updates") { @@ -59,8 +61,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { (50 until 100).map(x => (x, s"val=$x")) checkTable( path = goldenTablePath("basic-with-inserts-updates"), - expectedAnswer = expectedValues.map(TestRow.fromTuple) - ) + expectedAnswer = expectedValues.map(TestRow.fromTuple)) } test("simple end to end with inserts and merge") { @@ -68,24 +69,21 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { (50 until 100).map((_, "N/A")) ++ (100 until 150).map((_, "EXT")) checkTable( path = goldenTablePath("basic-with-inserts-merge"), - expectedAnswer = expectedValues.map(TestRow.fromTuple) - ) + expectedAnswer = expectedValues.map(TestRow.fromTuple)) } test("simple end to end with restore") { checkTable( path = goldenTablePath("basic-with-inserts-overwrite-restore"), expectedAnswer = (0L until 200L).map(TestRow(_)), - expectedVersion = Some(3) - ) + expectedVersion = Some(3)) } test("end to end only checkpoint files") { val expectedValues = (5L until 10L) ++ (0L until 20L) checkTable( path = goldenTablePath("only-checkpoint-files"), - expectedAnswer = expectedValues.map(TestRow(_)) - ) + expectedAnswer = expectedValues.map(TestRow(_))) } Seq("protocol", "metadata").foreach { action => @@ -101,7 +99,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { // TODO missing protocol should fail when missing from checkpoint // GoldenTable("deltalog-state-reconstruction-from-checkpoint-missing-protocol") // generation is broken and cannot be regenerated with a non-null schemaString until fixed - Seq("metadata" /* , "protocol" */).foreach { action => + Seq("metadata" /* , "protocol" */ ).foreach { action => test(s"missing $action should fail missing from checkpoint") { val path = goldenTablePath(s"deltalog-state-reconstruction-from-checkpoint-missing-$action") val e = intercept[IllegalStateException] { @@ -119,8 +117,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { // schema is updated assert(ScanStateRow.getLogicalSchema(scanStateRow) - .fieldNames().asScala.toSet == Set("col1", "col2") - ) + .fieldNames().asScala.toSet == Set("col1", "col2")) // check protocol version is upgraded val readerVersionOrd = scanStateRow.getSchema().indexOf("minReaderVersion") @@ -145,9 +142,9 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { } def verifySnapshotScanFiles( - tablePath: String, - expectedFiles: Array[File], - expectedVersion: Int): Unit = { + tablePath: String, + expectedFiles: Array[File], + expectedVersion: Int): Unit = { val snapshot = latestSnapshot(tablePath) assert(snapshot.getVersion() == expectedVersion) val scanFileRows = collectScanFileRows( @@ -221,8 +218,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test("DV cases with same path different DV keys") { val snapshot = latestSnapshot(goldenTablePath("log-replay-dv-key-cases")) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder().build() - ) + snapshot.getScanBuilder().build()) assert(scanFileRows.length == 1) // there should only be 1 add file val dv = InternalScanFileUtils.getDeletionVectorDescriptorFromRow(scanFileRows.head) assert(dv.getCardinality == 3) // dv cardinality should be 3 @@ -232,15 +228,13 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { withGoldenTable("log-replay-special-characters-a") { path => val snapshot = latestSnapshot(path) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder().build() - ) + snapshot.getScanBuilder().build()) assert(scanFileRows.isEmpty) } withGoldenTable("log-replay-special-characters-b") { path => val snapshot = latestSnapshot(path) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder().build() - ) + snapshot.getScanBuilder().build()) assert(scanFileRows.length == 1) val addFileStatus = InternalScanFileUtils.getAddFileStatus(scanFileRows.head) // get the relative path to compare @@ -272,8 +266,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { checkTable( path = goldenTablePath("data-reader-escaped-chars"), expectedAnswer = TestRow("foo1", "bar+%21") :: TestRow("foo2", "bar+%22") :: - TestRow("foo3", "bar+%23") :: Nil - ) + TestRow("foo3", "bar+%23") :: Nil) } test("delete and re-add same file in different transactions") { @@ -309,11 +302,9 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { val tablePath = tempFile.getAbsolutePath spark.sql( s"CREATE TABLE delta.`$tablePath` USING DELTA AS " + - s"SELECT 0L as id" - ) + s"SELECT 0L as id") spark.sql( - s"INSERT INTO delta.`$tablePath` SELECT 1L as id" - ) + s"INSERT INTO delta.`$tablePath` SELECT 1L as id") val table = Table.forPath(defaultEngine, tablePath) val snapshot = table.getLatestSnapshot(defaultEngine).asInstanceOf[SnapshotImpl] assert(snapshot.getCurrentCrcInfo.isPresent) @@ -329,11 +320,9 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { val tablePath = tempFile.getAbsolutePath spark.sql( s"CREATE TABLE delta.`$tablePath` USING DELTA AS " + - s"SELECT 0L as id" - ) + s"SELECT 0L as id") spark.sql( - s"INSERT INTO delta.`$tablePath` SELECT 1L as id" - ) + s"INSERT INTO delta.`$tablePath` SELECT 1L as id") deleteChecksumFileForTable(tablePath, versions = Seq(1)) val table = Table.forPath(defaultEngine, tablePath) @@ -347,11 +336,9 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { val tablePath = tempFile.getAbsolutePath spark.sql( s"CREATE TABLE delta.`$tablePath` USING DELTA AS " + - s"SELECT 0L as id" - ) + s"SELECT 0L as id") spark.sql( - s"INSERT INTO delta.`$tablePath` SELECT 1L as id" - ) + s"INSERT INTO delta.`$tablePath` SELECT 1L as id") deleteChecksumFileForTable(tablePath, versions = Seq(0, 1)) val table = Table.forPath(defaultEngine, tablePath) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/PartitionPruningSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/PartitionPruningSuite.scala index a3e07fac333..51f8a98dd52 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/PartitionPruningSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/PartitionPruningSuite.scala @@ -15,15 +15,16 @@ */ package io.delta.kernel.defaults +import java.math.{BigDecimal => BigDecimalJ} + import io.delta.golden.GoldenTableUtils.goldenTablePath import io.delta.kernel.defaults.utils.{ExpressionTestUtils, TestRow, TestUtils} -import io.delta.kernel.expressions.Literal._ import io.delta.kernel.expressions.{Column, Literal, Predicate} -import io.delta.kernel.types.TimestampNTZType.TIMESTAMP_NTZ +import io.delta.kernel.expressions.Literal._ import io.delta.kernel.types._ -import org.scalatest.funsuite.AnyFunSuite +import io.delta.kernel.types.TimestampNTZType.TIMESTAMP_NTZ -import java.math.{BigDecimal => BigDecimalJ} +import org.scalatest.funsuite.AnyFunSuite class PartitionPruningSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils { @@ -44,7 +45,7 @@ class PartitionPruningSuite extends AnyFunSuite with TestUtils with ExpressionTe col("as_float") -> (ofFloat(1), ofNull(FloatType.FLOAT)), col("as_double") -> (ofDouble(1), ofNull(DoubleType.DOUBLE)), // 2021-09-08 in days since epoch 18878 - col("as_date") -> (ofDate(18878 /* daysSinceEpochUTC */), ofNull(DateType.DATE)), + col("as_date") -> (ofDate(18878 /* daysSinceEpochUTC */ ), ofNull(DateType.DATE)), col("as_string") -> (ofString("1"), ofNull(StringType.STRING)), // 2021-09-08 11:11:11 in micros since epoch UTC col("as_timestamp") -> (ofTimestamp(1631099471000000L), ofNull(TimestampType.TIMESTAMP)), @@ -107,56 +108,45 @@ class PartitionPruningSuite extends AnyFunSuite with TestUtils with ExpressionTe // test case format: (test_name, predicate) -> (remainingPredicate, expectedResults) // expected results is for query selecting `as_date` (partition column) and `value` (data column) val combinationTestCases = Map( - ("partition pruning: with predicate on two different partition col combined with AND", + ( + "partition pruning: with predicate on two different partition col combined with AND", and( predicate(">=", col("as_float"), ofFloat(-200)), - predicate("=", col("as_date"), ofDate(18878 /* daysSinceEpochUTC */)) - ) - ) -> (null, Seq((18878, "0"), (18878, "1"))), - + predicate("=", col("as_date"), ofDate(18878 /* daysSinceEpochUTC */ )))) -> ( + null, + Seq((18878, "0"), (18878, "1"))), ( "partition pruning: with predicate on two different partition col combined with OR", or( predicate("=", col("as_float"), ofFloat(0)), - predicate("=", col("as_int"), ofInt(1))) - ) -> (null, Seq((18878, "0"), (18878, "1"))), - + predicate("=", col("as_int"), ofInt(1)))) -> (null, Seq((18878, "0"), (18878, "1"))), ( "partition pruning: with predicate on data and partition column mix with AND", and( predicate("=", col("as_value"), ofString("1")), // data col filter predicate("=", col("as_float"), ofFloat(0)) // partition col filter - ) - ) -> ( + )) -> ( predicate("=", col("as_value"), ofString("1")), - Seq((18878, "0")) - ), - + Seq((18878, "0"))), ( "partition pruning: with predicate on data and partition column mix with OR", or( predicate("=", col("as_value"), ofString("1")), // data col filter predicate("=", col("as_float"), ofFloat(0)) // partition col filter - ) - ) -> ( + )) -> ( or( predicate("=", col("as_value"), ofString("1")), // data col filter predicate("=", col("as_float"), ofFloat(0)) // partition col filter ), - Seq((18878, "0"), (18878, "1"), (null, "2")) - ), - + Seq((18878, "0"), (18878, "1"), (null, "2"))), ( "partition pruning: partition predicate prunes everything", and( predicate("=", col("as_value"), ofString("200")), // data col filter predicate("=", col("as_float"), ofFloat(234)) // partition col filter - ) - ) -> ( + )) -> ( predicate("=", col("as_value"), ofString("200")), - Seq() - ) - ) + Seq())) combinationTestCases.foreach { case ((testTag, predicate), (expRemainingFilter, expResults)) => @@ -241,17 +231,17 @@ class PartitionPruningSuite extends AnyFunSuite with TestUtils with ExpressionTe ( // Filter on just the data column // 1637202600123456L in epoch micros for '2021-11-18 02:30:00.123456' - predicate("OR", + predicate( + "OR", predicate("=", col("tsNtz"), ofTimestampNtz(1637202600123456L)), predicate("=", col("tsNtz"), ofTimestampNtz(1373043660123456L))), "", 9, // expected row count // expected remaining filter - predicate("OR", + predicate( + "OR", predicate("=", col("tsNtz"), ofTimestampNtz(1637202600123456L)), - predicate("=", col("tsNtz"), ofTimestampNtz(1373043660123456L))) - ) - ).foreach { + predicate("=", col("tsNtz"), ofTimestampNtz(1373043660123456L))))).foreach { case (kernelPredicate, sparkPredicate, expectedRowCount, expRemainingFilter) => test(s"partition pruning on timestamp_ntz columns: $cmMode ($kernelPredicate)") { val tablePath = goldenTablePath(s"data-reader-timestamp_ntz$cmMode") diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/PartitionUtilsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/PartitionUtilsSuite.scala index 17430dd1d66..cefa53ffd5f 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/PartitionUtilsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/PartitionUtilsSuite.scala @@ -20,6 +20,7 @@ import io.delta.kernel.Table import io.delta.kernel.defaults.utils.{ExpressionTestUtils, TestUtils} import io.delta.kernel.expressions.Literal.ofInt import io.delta.kernel.utils.PartitionUtils + import org.apache.spark.sql.functions.{col => sparkCol} import org.scalatest.funsuite.AnyFunSuite @@ -47,8 +48,7 @@ class PartitionUtilsSuite extends AnyFunSuite with TestUtils with ExpressionTest { val badPredicate = and( predicate("=", col("part1"), ofInt(0)), - predicate("=", col("col1"), ofInt(0)) - ) + predicate("=", col("col1"), ofInt(0))) val exMsg = intercept[IllegalArgumentException] { PartitionUtils.partitionExists(engine, snapshot, badPredicate) }.getMessage diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala index ba6d9d57049..af376889204 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala @@ -15,6 +15,12 @@ */ package io.delta.kernel.defaults +import java.util +import java.util.{Collections, Optional} + +import scala.collection.JavaConverters._ +import scala.collection.immutable.Seq + import io.delta.kernel.data.{FilteredColumnarBatch, Row} import io.delta.kernel.defaults.internal.parquet.ParquetSuiteBase import io.delta.kernel.engine.Engine @@ -22,21 +28,18 @@ import io.delta.kernel.exceptions.{KernelException, ProtocolChangedException} import io.delta.kernel.expressions.Literal import io.delta.kernel.internal.{InternalScanFileUtils, SnapshotImpl, TableImpl} import io.delta.kernel.internal.actions.{AddFile, Protocol, SingleAction} -import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.internal.rowtracking.RowTrackingMetadataDomain +import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.internal.util.VectorUtils import io.delta.kernel.internal.util.VectorUtils.stringStringMapValue -import io.delta.kernel.types.StructType import io.delta.kernel.types.LongType.LONG +import io.delta.kernel.types.StructType import io.delta.kernel.utils.CloseableIterable import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} -import org.apache.hadoop.fs.Path + import org.apache.spark.sql.delta.DeltaLog -import java.util -import java.util.{Collections, Optional} -import scala.collection.JavaConverters._ -import scala.collection.immutable.Seq +import org.apache.hadoop.fs.Path class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { private def prepareActionsForCommit(actions: Row*): CloseableIterable[Row] = { @@ -51,8 +54,8 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val protocol = new Protocol( 3, // minReaderVersion 7, // minWriterVersion - Collections.emptyList(), // readerFeatures - writerFeatures.asJava // writerFeatures + Collections.emptySet(), // readerFeatures + writerFeatures.toSet.asJava // writerFeatures ) val protocolAction = SingleAction.createProtocolSingleAction(protocol.toRow) val txn = createTxn(engine, tablePath, isNewTable = false, schema, Seq.empty) @@ -126,8 +129,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val commitVersion = appendData( engine, tablePath, - data = prepareDataForCommit(dataBatch1, dataBatch2, dataBatch3) - ).getVersion + data = prepareDataForCommit(dataBatch1, dataBatch2, dataBatch3)).getVersion verifyBaseRowIDs(engine, tablePath, Seq(0, 100, 300)) verifyDefaultRowCommitVersion(engine, tablePath, Seq.fill(3)(commitVersion)) @@ -143,8 +145,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val commitVersion1 = appendData( engine, tablePath, - data = Seq(dataBatch1).map(Map.empty[String, Literal] -> _) - ).getVersion + data = Seq(dataBatch1).map(Map.empty[String, Literal] -> _)).getVersion verifyBaseRowIDs(engine, tablePath, Seq(0)) verifyDefaultRowCommitVersion(engine, tablePath, Seq(commitVersion1)) @@ -154,8 +155,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val commitVersion2 = appendData( engine, tablePath, - data = prepareDataForCommit(dataBatch2) - ).getVersion + data = prepareDataForCommit(dataBatch2)).getVersion verifyBaseRowIDs(engine, tablePath, Seq(0, 100)) verifyDefaultRowCommitVersion(engine, tablePath, Seq(commitVersion1, commitVersion2)) @@ -174,14 +174,12 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val commitVersion1 = appendData( engine, tablePath, - data = prepareDataForCommit(dataBatch1) - ).getVersion + data = prepareDataForCommit(dataBatch1)).getVersion val commitVersion2 = appendData( engine, tablePath, - data = prepareDataForCommit(dataBatch2) - ).getVersion + data = prepareDataForCommit(dataBatch2)).getVersion // Checkpoint the table val table = TableImpl.forPath(engine, tablePath) @@ -191,15 +189,13 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val commitVersion3 = appendData( engine, tablePath, - data = prepareDataForCommit(dataBatch3) - ).getVersion + data = prepareDataForCommit(dataBatch3)).getVersion verifyBaseRowIDs(engine, tablePath, Seq(0, 100, 300)) verifyDefaultRowCommitVersion( engine, tablePath, - Seq(commitVersion1, commitVersion2, commitVersion3) - ) + Seq(commitVersion1, commitVersion2, commitVersion3)) verifyHighWatermark(engine, tablePath, 699) } } @@ -230,40 +226,11 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { assert( e.getMessage.contains( "Cannot write to a rowTracking-supported table without 'numRecords' statistics. " - + "Connectors are expected to populate the number of records statistics when " - + "writing to a Delta table with 'rowTracking' table feature supported." - ) - ) + + "Connectors are expected to populate the number of records statistics when " + + "writing to a Delta table with 'rowTracking' table feature supported.")) } } - test("Fail if row tracking is supported but domain metadata is not supported") { - withTempDirAndEngine((tablePath, engine) => { - createTxn(engine, tablePath, isNewTable = true, testSchema, Seq.empty) - .commit(engine, emptyIterable()) - - // Only 'rowTracking' is supported, not 'domainMetadata' - setWriterFeatureSupported(engine, tablePath, testSchema, Seq("rowTracking")) - - val dataBatch1 = generateData(testSchema, Seq.empty, Map.empty, 100, 1) - val e = intercept[KernelException] { - appendData( - engine, - tablePath, - data = prepareDataForCommit(dataBatch1) - ).getVersion - } - - assert( - e.getMessage - .contains( - "Feature 'rowTracking' is supported and depends on feature 'domainMetadata'," - + " but 'domainMetadata' is unsupported" - ) - ) - }) - } - test("Integration test - Write table with Kernel then write with Spark") { withTempDirAndEngine((tablePath, engine) => { val tbl = "tbl" @@ -310,8 +277,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { | 'delta.feature.domainMetadata' = 'enabled', | 'delta.feature.rowTracking' = 'supported' |) - |""".stripMargin - ) + |""".stripMargin) // Write to the table using delta-spark spark.range(0, 20).write.format("delta").mode("append").save(tablePath) // version 1 @@ -441,8 +407,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { tablePath, dataSizeTxn1 = 200, dataSizeTxn2 = 300, - dataSizeTxn3 = 400 - ) + dataSizeTxn3 = 400) }) } @@ -453,8 +418,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { tablePath, dataSizeTxn1 = 200, dataSizeTxn2 = 300, - dataSizeTxn3 = 0 - ) + dataSizeTxn3 = 0) }) withTempDirAndEngine((tablePath, engine) => { validateConflictResolution( @@ -462,8 +426,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { tablePath, dataSizeTxn1 = 200, dataSizeTxn2 = 0, - dataSizeTxn3 = 300 - ) + dataSizeTxn3 = 300) }) } @@ -474,8 +437,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { tablePath, dataSizeTxn1 = 200, dataSizeTxn2 = 0, - dataSizeTxn3 = 0 - ) + dataSizeTxn3 = 0) }) } @@ -486,15 +448,13 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { tablePath, dataSizeTxn1 = 0, dataSizeTxn2 = 200, - dataSizeTxn3 = 300 - ) + dataSizeTxn3 = 300) }) } test( "Conflict resolution - two concurrent txns were commited by delta-spark " + - "and both added new files" - ) { + "and both added new files") { withTempDirAndEngine((tablePath, engine) => { validateConflictResolution( engine, @@ -503,8 +463,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { dataSizeTxn2 = 300, useSparkTxn2 = true, dataSizeTxn3 = 400, - useSparkTxn3 = true - ) + useSparkTxn3 = true) }) } @@ -526,8 +485,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { commitAppendData( engine, txn1, - prepareDataForCommit(dataBatch1) - ) + prepareDataForCommit(dataBatch1)) } }) } @@ -548,8 +506,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { commitAppendData( engine, txn1, - prepareDataForCommit(dataBatch1) - ) + prepareDataForCommit(dataBatch1)) } }) } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala index 49992ce6ecc..7ea219c3f97 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala @@ -15,34 +15,37 @@ */ package io.delta.kernel.defaults +import java.math.{BigDecimal => JBigDecimal} +import java.sql.Date +import java.time.{Instant, OffsetDateTime} +import java.time.temporal.ChronoUnit +import java.util.Optional + +import scala.collection.JavaConverters._ + import io.delta.golden.GoldenTableUtils.goldenTablePath -import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, Row} +import io.delta.kernel.{Scan, Snapshot, Table} +import io.delta.kernel.data.{ColumnarBatch, ColumnVector, FilteredColumnarBatch, Row} import io.delta.kernel.defaults.engine.{DefaultEngine, DefaultJsonHandler, DefaultParquetHandler} import io.delta.kernel.defaults.utils.{ExpressionTestUtils, TestUtils} import io.delta.kernel.engine.{Engine, JsonHandler, ParquetHandler} -import io.delta.kernel.expressions.Literal._ import io.delta.kernel.expressions._ -import io.delta.kernel.internal.util.InternalUtils +import io.delta.kernel.expressions.Literal._ import io.delta.kernel.internal.{InternalScanFileUtils, ScanImpl} +import io.delta.kernel.internal.util.InternalUtils import io.delta.kernel.types._ import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types.StringType.STRING import io.delta.kernel.utils.{CloseableIterator, FileStatus} -import io.delta.kernel.{Scan, Snapshot, Table} + +import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog} + import org.apache.hadoop.conf.Configuration +import org.apache.spark.sql.{Row => SparkRow} import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog} import org.apache.spark.sql.types.{IntegerType => SparkIntegerType, StructField => SparkStructField, StructType => SparkStructType} -import org.apache.spark.sql.{Row => SparkRow} import org.scalatest.funsuite.AnyFunSuite -import java.math.{BigDecimal => JBigDecimal} -import java.sql.Date -import java.time.temporal.ChronoUnit -import java.time.{Instant, OffsetDateTime} -import java.util.Optional -import scala.collection.JavaConverters._ - class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with SQLHelper { import io.delta.kernel.defaults.ScanSuite._ @@ -52,7 +55,8 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with // scalastyle:on sparkimplicits private def getDataSkippingConfs( - indexedCols: Option[Int], deltaStatsColNamesOpt: Option[String]): Seq[(String, String)] = { + indexedCols: Option[Int], + deltaStatsColNamesOpt: Option[String]): Seq[(String, String)] = { val numIndexedColsConfOpt = indexedCols .map(DeltaConfigs.DATA_SKIPPING_NUM_INDEXED_COLS.defaultTablePropertyKey -> _.toString) val indexedColNamesConfOpt = deltaStatsColNamesOpt @@ -61,11 +65,11 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with } def writeDataSkippingTable( - tablePath: String, - data: String, - schema: SparkStructType, - indexedCols: Option[Int], - deltaStatsColNamesOpt: Option[String]): Unit = { + tablePath: String, + data: String, + schema: SparkStructType, + indexedCols: Option[Int], + deltaStatsColNamesOpt: Option[String]): Unit = { withSQLConf(getDataSkippingConfs(indexedCols, deltaStatsColNamesOpt): _*) { val jsonRecords = data.split("\n").toSeq val reader = spark.read @@ -105,9 +109,10 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with snapshot.getScanBuilder() .withFilter(predicate) .build()) - assert(scanFiles.isEmpty, s"Expected miss but got hit for $predicate\n" + - s"Returned scan files have stats: ${getScanFileStats(scanFiles)}" - ) + assert( + scanFiles.isEmpty, + s"Expected miss but got hit for $predicate\n" + + s"Returned scan files have stats: ${getScanFileStats(scanFiles)}") } } @@ -120,19 +125,20 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with filterToNumExpFiles.foreach { case (filter, numExpFiles) => val scanFiles = collectScanFileRows( snapshot.getScanBuilder().withFilter(filter).build()) - assert(scanFiles.length == numExpFiles, + assert( + scanFiles.length == numExpFiles, s"Expected $numExpFiles but found ${scanFiles.length} for $filter") } } def testSkipping( - testName: String, - data: String, - schema: SparkStructType = null, - hits: Seq[Predicate], - misses: Seq[Predicate], - indexedCols: Option[Int] = None, - deltaStatsColNamesOpt: Option[String] = None): Unit = { + testName: String, + data: String, + schema: SparkStructType = null, + hits: Seq[Predicate], + misses: Seq[Predicate], + indexedCols: Option[Int] = None, + deltaStatsColNamesOpt: Option[String] = None): Unit = { test(testName) { withTempDir { tempDir => writeDataSkippingTable( @@ -140,28 +146,30 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with data, schema, indexedCols, - deltaStatsColNamesOpt - ) + deltaStatsColNamesOpt) checkSkipping( tempDir.getCanonicalPath, hits, - misses - ) + misses) } } } /* Where timestampStr is in the format of "yyyy-MM-dd'T'HH:mm:ss.SSSXXX" */ - def getTimestampPredicate(expr: String, col: Column, - timestampStr: String, timeStampType: String): Predicate = { + def getTimestampPredicate( + expr: String, + col: Column, + timestampStr: String, + timeStampType: String): Predicate = { val time = OffsetDateTime.parse(timestampStr) - new Predicate(expr, col, + new Predicate( + expr, + col, if (timeStampType.equalsIgnoreCase("timestamp")) { ofTimestamp(ChronoUnit.MICROS.between(Instant.EPOCH, time)) } else { ofTimestampNtz(ChronoUnit.MICROS.between(Instant.EPOCH, time)) - } - ) + }) } ////////////////////////////////////////////////////////////////////////////////// @@ -208,8 +216,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with not(nullSafeEquals(ofInt(1), col("a"))), // NOT 1 <=> a nullSafeEquals(ofInt(2), col("a")), // 2 <=> a nullSafeEquals(col("a"), ofInt(2)) // a <=> 2 - ) - ) + )) testSkipping( "data skipping - nested, single 1", @@ -225,8 +232,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with equals(nestedCol("a.b"), ofInt(2)), // a.b = 2 greaterThan(nestedCol("a.b"), ofInt(1)), // a.b > 1 lessThan(nestedCol("a.b"), ofInt(1)) // a.b < 1 - ) - ) + )) testSkipping( "data skipping - double nested, single 1", @@ -242,8 +248,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with equals(nestedCol("a.b.c"), ofInt(2)), // a.b.c = 2 greaterThan(nestedCol("a.b.c"), ofInt(1)), // a.b.c > 1 lessThan(nestedCol("a.b.c"), ofInt(1)) // a.b.c < 1 - ) - ) + )) private def longString(str: String) = str * 1000 @@ -263,9 +268,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with ), misses = Seq( lessThan(col("a"), ofString("AA")), - greaterThan(col("a"), ofString("CD")) - ) - ) + greaterThan(col("a"), ofString("CD")))) testSkipping( "data skipping - long strings - long max", @@ -285,9 +288,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with ), misses = Seq( greaterThanOrEqual(col("a"), ofString("D")), - greaterThan(col("a"), ofString("CD")) - ) - ) + greaterThan(col("a"), ofString("CD")))) // Test:'starts with' Expression: like // Test:'starts with, nested' Expression: like @@ -301,20 +302,14 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with hits = Seq( new And( greaterThan(col("a"), ofInt(0)), - lessThan(col("a"), ofInt(3)) - ), + lessThan(col("a"), ofInt(3))), new And( lessThanOrEqual(col("a"), ofInt(1)), - greaterThan(col("a"), ofInt(-1)) - ) - ), + greaterThan(col("a"), ofInt(-1)))), misses = Seq( new And( lessThan(col("a"), ofInt(0)), - greaterThan(col("a"), ofInt(-2)) - ) - ) - ) + greaterThan(col("a"), ofInt(-2))))) testSkipping( "data skipping - and statements - two fields", @@ -325,25 +320,19 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with hits = Seq( new And( greaterThan(col("a"), ofInt(0)), - equals(col("b"), ofString("2017-09-01")) - ), + equals(col("b"), ofString("2017-09-01"))), new And( equals(col("a"), ofInt(2)), - greaterThanOrEqual(col("b"), ofString("2017-08-30")) - ), + greaterThanOrEqual(col("b"), ofString("2017-08-30"))), // note startsWith is not supported yet but these should still be hits once supported new And( // a >= 2 AND b like '2017-08-%' greaterThanOrEqual(col("a"), ofInt(2)), - startsWith(col("b"), ofString("2017-08-")) - ), + startsWith(col("b"), ofString("2017-08-"))), // MOVE BELOW EXPRESSION TO MISSES ONCE SUPPORTED BY DATA SKIPPING new And( // a > 0 AND b like '2016-%' greaterThan(col("a"), ofInt(0)), - startsWith(col("b"), ofString("2016-")) - ) - ), - misses = Seq() - ) + startsWith(col("b"), ofString("2016-")))), + misses = Seq()) private val aRem100 = new ScalarExpression("%", Seq(col("a"), ofInt(100)).asJava) private val bRem100 = new ScalarExpression("%", Seq(col("b"), ofInt(100)).asJava) @@ -356,15 +345,12 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with """, hits = Seq( // a % 100 < 10 AND b % 100 > 20 - new And(lessThan(aRem100, ofInt(10)), greaterThan(bRem100, ofInt(20))) - ), + new And(lessThan(aRem100, ofInt(10)), greaterThan(bRem100, ofInt(20)))), misses = Seq( // a < 10 AND b % 100 > 20 new And(lessThan(col("a"), ofInt(10)), greaterThan(bRem100, ofInt(20))), // a % 100 < 10 AND b > 20 - new And(lessThan(aRem100, ofInt(10)), greaterThan(col("b"), ofInt(20))) - ) - ) + new And(lessThan(aRem100, ofInt(10)), greaterThan(col("b"), ofInt(20))))) testSkipping( "data skipping - or statements - simple", @@ -376,13 +362,10 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with // a > 0 or a < -3 new Or(greaterThan(col("a"), ofInt(0)), lessThan(col("a"), ofInt(-3))), // a >= 2 or a < -1 - new Or(greaterThanOrEqual(col("a"), ofInt(2)), lessThan(col("a"), ofInt(-1))) - ), + new Or(greaterThanOrEqual(col("a"), ofInt(2)), lessThan(col("a"), ofInt(-1)))), misses = Seq( // a > 5 or a < -2 - new Or(greaterThan(col("a"), ofInt(5)), lessThan(col("a"), ofInt(-2))) - ) - ) + new Or(greaterThan(col("a"), ofInt(5)), lessThan(col("a"), ofInt(-2))))) testSkipping( "data skipping - or statements - two fields", @@ -393,29 +376,22 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with hits = Seq( new Or( lessThan(col("a"), ofInt(0)), - equals(col("b"), ofString("2017-09-01")) - ), + equals(col("b"), ofString("2017-09-01"))), new Or( equals(col("a"), ofInt(2)), - lessThan(col("b"), ofString("2017-08-30")) - ), + lessThan(col("b"), ofString("2017-08-30"))), // note startsWith is not supported yet but these should still be hits once supported new Or( // a < 2 or b like '2017-08-%' lessThan(col("a"), ofInt(2)), - startsWith(col("b"), ofString("2017-08-")) - ), + startsWith(col("b"), ofString("2017-08-"))), new Or( // a >= 2 or b like '2016-08-%' greaterThanOrEqual(col("a"), ofInt(2)), - startsWith(col("b"), ofString("2016-08-")) - ), + startsWith(col("b"), ofString("2016-08-"))), // MOVE BELOW EXPRESSION TO MISSES ONCE SUPPORTED BY DATA SKIPPING new Or( // a < 0 or b like '2016-%' lessThan(col("a"), ofInt(0)), - startsWith(col("b"), ofString("2016-")) - ) - ), - misses = Seq() - ) + startsWith(col("b"), ofString("2016-")))), + misses = Seq()) // One side of OR by itself isn't powerful enough to prune any files. testSkipping( @@ -428,13 +404,10 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with // a % 100 < 10 OR b > 20 new Or(lessThan(aRem100, ofInt(10)), greaterThan(col("b"), ofInt(20))), // a < 10 OR b % 100 > 20 - new Or(lessThan(col("a"), ofInt(10)), greaterThan(bRem100, ofInt(20))) - ), + new Or(lessThan(col("a"), ofInt(10)), greaterThan(bRem100, ofInt(20)))), misses = Seq( // a < 10 OR b > 20 - new Or(lessThan(col("a"), ofInt(10)), greaterThan(col("b"), ofInt(20))) - ) - ) + new Or(lessThan(col("a"), ofInt(10)), greaterThan(col("b"), ofInt(20))))) testSkipping( "data skipping - not statements - simple", @@ -443,17 +416,14 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with {"a": 2} """, hits = Seq( - not(lessThan(col("a"), ofInt(0))) - ), + not(lessThan(col("a"), ofInt(0)))), misses = Seq( not(greaterThan(col("a"), ofInt(0))), not(lessThan(col("a"), ofInt(3))), not(greaterThanOrEqual(col("a"), ofInt(1))), not(lessThanOrEqual(col("a"), ofInt(2))), not(not(lessThan(col("a"), ofInt(0)))), - not(not(equals(col("a"), ofInt(3)))) - ) - ) + not(not(equals(col("a"), ofInt(3)))))) // NOT(AND(a, b)) === OR(NOT(a), NOT(b)) ==> One side by itself cannot prune. testSkipping( @@ -466,31 +436,20 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with not( new And( greaterThanOrEqual(aRem100, ofInt(10)), - lessThanOrEqual(bRem100, ofInt(20)) - ) - ), + lessThanOrEqual(bRem100, ofInt(20)))), not( new And( greaterThanOrEqual(col("a"), ofInt(10)), - lessThanOrEqual(bRem100, ofInt(20)) - ) - ), + lessThanOrEqual(bRem100, ofInt(20)))), not( new And( greaterThanOrEqual(aRem100, ofInt(10)), - lessThanOrEqual(col("b"), ofInt(20)) - ) - ) - ), + lessThanOrEqual(col("b"), ofInt(20))))), misses = Seq( not( new And( greaterThanOrEqual(col("a"), ofInt(10)), - lessThanOrEqual(col("b"), ofInt(20)) - ) - ) - ) - ) + lessThanOrEqual(col("b"), ofInt(20)))))) // NOT(OR(a, b)) === AND(NOT(a), NOT(b)) => One side by itself is enough to prune. testSkipping( @@ -503,23 +462,17 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with // NOT(a < 1 OR b > 20), not(new Or(lessThan(col("a"), ofInt(1)), greaterThan(col("b"), ofInt(20)))), // NOT(a % 100 >= 1 OR b % 100 <= 20) - not(new Or(greaterThanOrEqual(aRem100, ofInt(1)), lessThanOrEqual(bRem100, ofInt(20)))) - ), + not(new Or(greaterThanOrEqual(aRem100, ofInt(1)), lessThanOrEqual(bRem100, ofInt(20))))), misses = Seq( // NOT(a >= 1 OR b <= 20) not( - new Or(greaterThanOrEqual(col("a"), ofInt(1)), lessThanOrEqual(col("b"), ofInt(20))) - ), + new Or(greaterThanOrEqual(col("a"), ofInt(1)), lessThanOrEqual(col("b"), ofInt(20)))), // NOT(a % 100 >= 1 OR b <= 20), not( - new Or(greaterThanOrEqual(aRem100, ofInt(1)), lessThanOrEqual(col("b"), ofInt(20))) - ), + new Or(greaterThanOrEqual(aRem100, ofInt(1)), lessThanOrEqual(col("b"), ofInt(20)))), // NOT(a >= 1 OR b % 100 <= 20) not( - new Or(greaterThanOrEqual(col("a"), ofInt(1)), lessThanOrEqual(bRem100, ofInt(20))) - ) - ) - ) + new Or(greaterThanOrEqual(col("a"), ofInt(1)), lessThanOrEqual(bRem100, ofInt(20)))))) // If a column does not have stats, it does not participate in data skipping, which disqualifies // that leg of whatever conjunct it was part of. @@ -534,21 +487,18 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with lessThan(col("b"), ofInt(10)), // b < 10: disqualified // note OR is not supported yet but these should still be hits once supported new Or( // a < 1 OR b < 10: a disqualified by b (same conjunct) - lessThan(col("a"), ofInt(1)), lessThan(col("b"), ofInt(10))), + lessThan(col("a"), ofInt(1)), + lessThan(col("b"), ofInt(10))), new Or( // a < 1 OR (a >= 1 AND b < 10): ==> a < 1 OR a >=1 ==> TRUE lessThan(col("a"), ofInt(1)), - new And(greaterThanOrEqual(col("a"), ofInt(1)), lessThan(col("b"), ofInt(10))) - ) - ), + new And(greaterThanOrEqual(col("a"), ofInt(1)), lessThan(col("b"), ofInt(10))))), misses = Seq( new And( // a < 1 AND b < 10: ==> a < 1 ==> FALSE - lessThan(col("a"), ofInt(1)), lessThan(col("b"), ofInt(10))), + lessThan(col("a"), ofInt(1)), + lessThan(col("b"), ofInt(10))), new Or( // a < 1 OR (a > 10 AND b < 10): ==> a < 1 OR a > 10 ==> FALSE lessThan(col("a"), ofInt(1)), - new And(greaterThan(col("a"), ofInt(10)), lessThan(col("b"), ofInt(10))) - ) - ) - ) + new And(greaterThan(col("a"), ofInt(10)), lessThan(col("b"), ofInt(10)))))) private def generateJsonData(numCols: Int): String = { val fields = (0 until numCols).map(i => s""""col${"%02d".format(i)}":$i""".stripMargin) @@ -562,12 +512,9 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with hits = Seq( equals(col("col00"), ofInt(0)), equals(col("col32"), ofInt(32)), - equals(col("col32"), ofInt(-1)) - ), + equals(col("col32"), ofInt(-1))), misses = Seq( - equals(col("col00"), ofInt(1)) - ) - ) + equals(col("col00"), ofInt(1)))) testSkipping( "data skipping - nested schema - # indexed column = 3", @@ -603,8 +550,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with lessThan(col("a"), ofInt(0)), // a < 0 lessThan(nestedCol("b.c.d"), ofInt(0)), // b.c.d < 0 lessThan(nestedCol("b.c.e"), ofInt(0)) // b.c.e < 0 - ) - ) + )) testSkipping( "data skipping - nested schema - # indexed column = 0", @@ -633,10 +579,8 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with lessThan(nestedCol("b.c.d"), ofInt(0)), lessThan(nestedCol("b.c.f.i"), ofInt(0)), lessThan(nestedCol("b.l"), ofInt(0)), - lessThan(col("m"), ofInt(0)) - ), - misses = Seq() - ) + lessThan(col("m"), ofInt(0))), + misses = Seq()) testSkipping( "data skipping - indexed column names - " + @@ -665,8 +609,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with // these all have missing stats lessThan(col("a"), ofInt(0)), lessThan(nestedCol("b.l"), ofInt(0)), - lessThan(col("m"), ofInt(0)) - ), + lessThan(col("m"), ofInt(0))), misses = Seq( lessThan(nestedCol("b.c.d"), ofInt(0)), lessThan(nestedCol("b.c.e"), ofInt(0)), @@ -674,9 +617,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with lessThan(nestedCol("b.c.f.h"), ofInt(0)), lessThan(nestedCol("b.c.f.i"), ofInt(0)), lessThan(nestedCol("b.c.j"), ofInt(0)), - lessThan(nestedCol("b.c.k"), ofInt(0)) - ) - ) + lessThan(nestedCol("b.c.k"), ofInt(0)))) testSkipping( "data skipping - indexed column names - index only a subset of leaf columns", @@ -707,15 +648,12 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with lessThan(nestedCol("b.c.f.g"), ofInt(0)), lessThan(nestedCol("b.c.f.i"), ofInt(0)), lessThan(nestedCol("b.c.j"), ofInt(0)), - lessThan(col("m"), ofInt(0)) - ), + lessThan(col("m"), ofInt(0))), misses = Seq( lessThan(nestedCol("b.c.e"), ofInt(0)), lessThan(nestedCol("b.c.f.h"), ofInt(0)), lessThan(nestedCol("b.c.k"), ofInt(0)), - lessThan(nestedCol("b.l"), ofInt(0)) - ) - ) + lessThan(nestedCol("b.l"), ofInt(0)))) testSkipping( "data skipping - boolean comparisons", @@ -726,10 +664,8 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with lessThanOrEqual(col("a"), ofBoolean(false)), equals(ofBoolean(true), col("a")), lessThan(ofBoolean(true), col("a")), - not(equals(col("a"), ofBoolean(false))) - ), - misses = Seq() - ) + not(equals(col("a"), ofBoolean(false)))), + misses = Seq()) // Data skipping by stats should still work even when the only data in file is null, in spite of // the NULL min/max stats that result -- this is different to having no stats at all. @@ -759,15 +695,12 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with lessThan(col("a"), ofInt(1)), greaterThan(col("a"), ofInt(1)), not(equals(col("a"), ofInt(1))), - notEquals(col("a"), ofInt(1)) - ), + notEquals(col("a"), ofInt(1))), misses = Seq( AlwaysFalse.ALWAYS_FALSE, nullSafeEquals(col("a"), ofInt(1)), not(nullSafeEquals(col("a"), ofNull(INTEGER))), - isNotNull(col("a")) - ) - ) + isNotNull(col("a")))) testSkipping( "data skipping - nulls - null + not-null in same file", @@ -795,15 +728,12 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with not(nullSafeEquals(col("a"), ofInt(1))), // MOVE BELOW EXPRESSIONS TO MISSES ONCE SUPPORTED BY DATA SKIPPING - notEquals(col("a"), ofInt(1)) - ), + notEquals(col("a"), ofInt(1))), misses = Seq( AlwaysFalse.ALWAYS_FALSE, lessThan(col("a"), ofInt(1)), greaterThan(col("a"), ofInt(1)), - not(equals(col("a"), ofInt(1))) - ) - ) + not(equals(col("a"), ofInt(1))))) Seq("TIMESTAMP", "TIMESTAMP_NTZ").foreach { dataType => test(s"data skipping - on $dataType type") { @@ -826,20 +756,29 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with getTimestampPredicate(">=", col("ts"), "2019-09-09T01:02:03.456789Z", dataType), getTimestampPredicate("<=", col("ts"), "2019-09-09T01:02:03.456789Z", dataType), getTimestampPredicate( - ">=", nestedCol("nested.ts"), "2019-09-09T01:02:03.456789Z", dataType), + ">=", + nestedCol("nested.ts"), + "2019-09-09T01:02:03.456789Z", + dataType), getTimestampPredicate( - "<=", nestedCol("nested.ts"), "2019-09-09T01:02:03.456789Z", dataType) - ), + "<=", + nestedCol("nested.ts"), + "2019-09-09T01:02:03.456789Z", + dataType)), misses = Seq( getTimestampPredicate("=", col("ts"), "2019-09-09T01:02:03.457001Z", dataType), getTimestampPredicate(">=", col("ts"), "2019-09-09T01:02:03.457001Z", dataType), getTimestampPredicate("<=", col("ts"), "2019-09-09T01:02:03.455999Z", dataType), getTimestampPredicate( - ">=", nestedCol("nested.ts"), "2019-09-09T01:02:03.457001Z", dataType), + ">=", + nestedCol("nested.ts"), + "2019-09-09T01:02:03.457001Z", + dataType), getTimestampPredicate( - "<=", nestedCol("nested.ts"), "2019-09-09T01:02:03.455999Z", dataType) - ) - ) + "<=", + nestedCol("nested.ts"), + "2019-09-09T01:02:03.455999Z", + dataType))) } } } @@ -864,7 +803,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with equals(col("c1"), ofInt(1)), equals(col("c2"), ofString("2")), lessThan(col("c3"), ofFloat(1.5f)), - greaterThan(col("c4"), ofFloat(1.0F)), + greaterThan(col("c4"), ofFloat(1.0f)), equals(col("c6"), ofDate(InternalUtils.daysSinceEpoch(Date.valueOf("2002-02-02")))), // Binary Column doesn't support delta statistics. equals(col("c7"), ofBinary("1111".getBytes)), @@ -872,8 +811,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with equals(col("c8"), ofBoolean(true)), equals(col("c8"), ofBoolean(false)), greaterThan(col("c9"), ofDecimal(JBigDecimal.valueOf(1.5), 3, 2)), - getTimestampPredicate(">=", col("c5"), "2001-01-01T01:00:00-07:00", "TIMESTAMP") - ), + getTimestampPredicate(">=", col("c5"), "2001-01-01T01:00:00-07:00", "TIMESTAMP")), misses = Seq( equals(col("c1"), ofInt(10)), equals(col("c2"), ofString("4")), @@ -881,9 +819,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with greaterThan(col("c4"), ofFloat(5.0f)), equals(col("c6"), ofDate(InternalUtils.daysSinceEpoch(Date.valueOf("2003-02-02")))), greaterThan(col("c9"), ofDecimal(JBigDecimal.valueOf(2.5), 3, 2)), - getTimestampPredicate(">=", col("c5"), "2003-01-01T01:00:00-07:00", "TIMESTAMP") - ) - ) + getTimestampPredicate(">=", col("c5"), "2003-01-01T01:00:00-07:00", "TIMESTAMP"))) } } @@ -925,8 +861,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with equals(col("cc8"), ofBoolean(true)), equals(col("cc8"), ofBoolean(false)), greaterThan(col("cc9"), ofDecimal(JBigDecimal.valueOf(1.5), 3, 2)), - getTimestampPredicate(">=", col("cc5"), "2001-01-01T01:00:00-07:00", "TIMESTAMP") - ), + getTimestampPredicate(">=", col("cc5"), "2001-01-01T01:00:00-07:00", "TIMESTAMP")), misses = Seq( equals(col("cc1"), ofInt(10)), equals(col("cc2"), ofString("4")), @@ -934,9 +869,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with greaterThan(col("cc4"), ofFloat(5.0f)), equals(col("cc6"), ofDate(InternalUtils.daysSinceEpoch(Date.valueOf("2003-02-02")))), getTimestampPredicate(">=", col("cc5"), "2003-01-01T01:00:00-07:00", "TIMESTAMP"), - greaterThan(col("cc9"), ofDecimal(JBigDecimal.valueOf(2.5), 3, 2)) - ) - ) + greaterThan(col("cc9"), ofDecimal(JBigDecimal.valueOf(2.5), 3, 2)))) } } @@ -974,8 +907,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with equals(col("c6"), ofDate(InternalUtils.daysSinceEpoch(Date.valueOf("2002-02-02")))), greaterThan(col("c9"), ofDecimal(JBigDecimal.valueOf(1.5), 3, 2)), getTimestampPredicate(">=", col("c5"), "2001-01-01T01:00:00-07:00", "TIMESTAMP"), - getTimestampPredicate(">=", col("c10"), "2001-01-01T01:00:00-07:00", "TIMESTAMP_NTZ") - ), + getTimestampPredicate(">=", col("c10"), "2001-01-01T01:00:00-07:00", "TIMESTAMP_NTZ")), misses = Seq( equals(col("c1"), ofInt(10)), lessThan(col("c3"), ofFloat(0.5f)), @@ -983,37 +915,39 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with equals(col("c6"), ofDate(InternalUtils.daysSinceEpoch(Date.valueOf("2003-02-02")))), greaterThan(col("c9"), ofDecimal(JBigDecimal.valueOf(2.5), 3, 2)), getTimestampPredicate(">=", col("c5"), "2003-01-01T01:00:00-07:00", "TIMESTAMP"), - getTimestampPredicate(">=", col("c10"), "2003-01-01T01:00:00-07:00", "TIMESTAMP_NTZ") - ) - ) + getTimestampPredicate(">=", col("c10"), "2003-01-01T01:00:00-07:00", "TIMESTAMP_NTZ"))) } } test("data skipping by partition and data values - nulls") { withTempDir { tableDir => - val dataSeqs = Seq( // each sequence produce a single file - Seq((null, null)), - Seq((null, "a")), - Seq((null, "b")), - Seq(("a", "a"), ("a", null)), - Seq(("b", null)) - ) + val dataSeqs = + Seq( // each sequence produce a single file + Seq((null, null)), + Seq((null, "a")), + Seq((null, "b")), + Seq(("a", "a"), ("a", null)), + Seq(("b", null))) dataSeqs.foreach { seq => seq.toDF("key", "value").coalesce(1) .write.format("delta").partitionBy("key").mode("append").save(tableDir.getCanonicalPath) } def checkResults( - predicate: Predicate, expNumPartitions: Int, expNumFiles: Long): Unit = { + predicate: Predicate, + expNumPartitions: Int, + expNumFiles: Long): Unit = { val snapshot = latestSnapshot(tableDir.getCanonicalPath) val scanFiles = collectScanFileRows( snapshot.getScanBuilder().withFilter(predicate).build()) - assert(scanFiles.length == expNumFiles, + assert( + scanFiles.length == expNumFiles, s"Expected $expNumFiles but found ${scanFiles.length} for $predicate") val partitionValues = scanFiles.map { row => InternalScanFileUtils.getPartitionValues(row) }.distinct - assert(partitionValues.length == expNumPartitions, + assert( + partitionValues.length == expNumPartitions, s"Expected $expNumPartitions partitions but found ${partitionValues.length}") } @@ -1027,40 +961,45 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with checkResults( predicate = isNotNull(col("key")), expNumPartitions = 2, - expNumFiles = 2) // 2 files with key = 'a', and 1 file with key = 'b' + expNumFiles = 2 + ) // 2 files with key = 'a', and 1 file with key = 'b' checkResults( predicate = equals(col("key"), ofString("a")), expNumPartitions = 1, - expNumFiles = 1) // 1 files with key = 'a' - + expNumFiles = 1 + ) // 1 files with key = 'a' checkResults( predicate = equals(col("key"), ofString("b")), expNumPartitions = 1, - expNumFiles = 1) // 1 files with key = 'b' + expNumFiles = 1 + ) // 1 files with key = 'b' // TODO shouldn't partition filters on unsupported expressions just not prune instead of fail? checkResults( predicate = isNull(col("key")), expNumPartitions = 1, - expNumFiles = 3) // 3 files with key = null - + expNumFiles = 3 + ) // 3 files with key = null checkResults( predicate = nullSafeEquals(col("key"), ofNull(STRING)), expNumPartitions = 1, - expNumFiles = 3) // 3 files with key = null + expNumFiles = 3 + ) // 3 files with key = null checkResults( predicate = nullSafeEquals(col("key"), ofString("a")), expNumPartitions = 1, - expNumFiles = 1) // 1 files with key <=> 'a' + expNumFiles = 1 + ) // 1 files with key <=> 'a' checkResults( predicate = nullSafeEquals(col("key"), ofString("b")), expNumPartitions = 1, - expNumFiles = 1) // 1 files with key <=> 'b' + expNumFiles = 1 + ) // 1 files with key <=> 'b' // Conditions on partitions keys and values checkResults( @@ -1071,7 +1010,8 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with checkResults( predicate = isNotNull(col("value")), expNumPartitions = 2, // one of the partitions has no files left after data skipping - expNumFiles = 3) // files with all NULL values get skipped + expNumFiles = 3 + ) // files with all NULL values get skipped checkResults( predicate = nullSafeEquals(col("value"), ofNull(STRING)), @@ -1086,7 +1026,8 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with checkResults( predicate = equals(col("value"), ofString("a")), expNumPartitions = 3, // should be 2 if we can correctly skip "value = 'a'" for nulls - expNumFiles = 4) // should be 2 if we can correctly skip "value = 'a'" for nulls + expNumFiles = 4 + ) // should be 2 if we can correctly skip "value = 'a'" for nulls checkResults( predicate = nullSafeEquals(col("value"), ofString("a")), @@ -1101,12 +1042,14 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with checkResults( predicate = notEquals(col("value"), ofString("a")), expNumPartitions = 3, // should be 1 once <> is supported - expNumFiles = 5) // should be 1 once <> is supported + expNumFiles = 5 + ) // should be 1 once <> is supported checkResults( predicate = equals(col("value"), ofString("b")), expNumPartitions = 2, // should be 1 if we can correctly skip "value = 'b'" for nulls - expNumFiles = 3) // should be 1 if we can correctly skip "value = 'a'" for nulls + expNumFiles = 3 + ) // should be 1 if we can correctly skip "value = 'a'" for nulls checkResults( predicate = nullSafeEquals(col("value"), ofString("b")), @@ -1126,18 +1069,21 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with ofNull(STRING))), expNumPartitions = 1, expNumFiles = 1) // 3 files with key = null, but only 1 with val = null. - */ + */ checkResults( predicate = new And(isNotNull(col("key")), isNotNull(col("value"))), expNumPartitions = 1, - expNumFiles = 1) // 1 file with (*, a) + expNumFiles = 1 + ) // 1 file with (*, a) checkResults( predicate = new Or( - nullSafeEquals(col("key"), ofNull(STRING)), nullSafeEquals(col("value"), ofNull(STRING))), + nullSafeEquals(col("key"), ofNull(STRING)), + nullSafeEquals(col("value"), ofNull(STRING))), expNumPartitions = 3, - expNumFiles = 5) // all 5 files + expNumFiles = 5 + ) // all 5 files } } @@ -1155,22 +1101,22 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with "as_float" -> (ofFloat(0), ofFloat(-1), ofFloat(1)), "as_double" -> (ofDouble(0), ofDouble(-1), ofDouble(1)), "as_string" -> (ofString("0"), ofString("!"), ofString("1")), - "as_date" -> (ofDate(InternalUtils.daysSinceEpoch(Date.valueOf("2000-01-01"))), + "as_date" -> ( + ofDate(InternalUtils.daysSinceEpoch(Date.valueOf("2000-01-01"))), ofDate(InternalUtils.daysSinceEpoch(Date.valueOf("1999-01-01"))), ofDate(InternalUtils.daysSinceEpoch(Date.valueOf("2000-01-02")))), // TODO (delta-io/delta#2462) add Timestamp once we support skipping for TimestampType - "as_big_decimal" -> (ofDecimal(JBigDecimal.valueOf(0), 1, 0), + "as_big_decimal" -> ( + ofDecimal(JBigDecimal.valueOf(0), 1, 0), ofDecimal(JBigDecimal.valueOf(-1), 1, 0), - ofDecimal(JBigDecimal.valueOf(1), 1, 0)) - ) + ofDecimal(JBigDecimal.valueOf(1), 1, 0))) val misses = colToLits.flatMap { case (colName, (value, small, big)) => Seq( equals(col(colName), small), greaterThan(col(colName), value), greaterThanOrEqual(col(colName), big), lessThan(col(colName), value), - lessThanOrEqual(col(colName), small) - ) + lessThanOrEqual(col(colName), small)) }.toSeq val hits = colToLits.flatMap { case (colName, (value, small, big)) => Seq( @@ -1178,20 +1124,17 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with greaterThan(col(colName), small), greaterThanOrEqual(col(colName), value), lessThan(col(colName), big), - lessThanOrEqual(col(colName), value) - ) + lessThanOrEqual(col(colName), value)) }.toSeq Seq( "data-skipping-basic-stats-all-types", "data-skipping-basic-stats-all-types-columnmapping-name", "data-skipping-basic-stats-all-types-columnmapping-id", - "data-skipping-basic-stats-all-types-checkpoint" - ).foreach { goldenTable => + "data-skipping-basic-stats-all-types-checkpoint").foreach { goldenTable => checkSkipping( goldenTablePath(goldenTable), hits, - misses - ) + misses) } } @@ -1200,13 +1143,10 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with goldenTablePath("data-skipping-basic-stats-all-types"), hits = Seq( equals(col("as_short"), ofFloat(0f)), - equals(col("as_float"), ofShort(0)) - ), + equals(col("as_float"), ofShort(0))), misses = Seq( equals(col("as_short"), ofFloat(1f)), - equals(col("as_float"), ofShort(1)) - ) - ) + equals(col("as_float"), ofShort(1)))) } test("data skipping - incompatible schema change doesn't break") { @@ -1221,12 +1161,9 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with checkSkipping( tablePath, hits = Seq( - equals(col("value"), ofString("1")) - ), + equals(col("value"), ofString("1"))), misses = Seq( - equals(col("value"), ofString("3")) - ) - ) + equals(col("value"), ofString("3")))) } } @@ -1234,8 +1171,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with checkSkipping( goldenTablePath("data-skipping-basic-stats-all-types"), hits = Seq(equals(col("foo"), ofInt(1))), - misses = Seq() - ) + misses = Seq()) } // todo add a test with dvs where tightBounds=false @@ -1248,8 +1184,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with greaterThan(col("part"), ofInt(0)), greaterThan(col("id"), ofInt(0)) ) -> 1 // should prune 3 files from partition + data filter - ) - ) + )) } test("data skipping - stats collected changing across versions") { @@ -1262,8 +1197,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with equals(col("col1"), ofInt(1)), equals(col("col2"), ofInt(1)) ) -> 1 // should prune 2 files - ) - ) + )) } test("data skipping - range of ints") { @@ -1279,8 +1213,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with lessThanOrEqual(col("id"), ofInt(9)), greaterThan(col("id"), ofInt(3)), greaterThan(col("id"), ofInt(-1)), - greaterThanOrEqual(col("id"), ofInt(0)) - ), + greaterThanOrEqual(col("id"), ofInt(0))), misses = Seq( equals(col("id"), ofInt(10)), lessThan(col("id"), ofInt(0)), @@ -1288,9 +1221,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with lessThanOrEqual(col("id"), ofInt(-1)), greaterThan(col("id"), ofInt(10)), greaterThan(col("id"), ofInt(11)), - greaterThanOrEqual(col("id"), ofInt(11)) - ) - ) + greaterThanOrEqual(col("id"), ofInt(11)))) } } @@ -1317,21 +1248,17 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with not(isNull(col("map_col"))), not(isNull(col("struct_col"))), not(isNull(nestedCol("struct_col.field1"))), - isNull(col("struct_col")) - ), + isNull(col("struct_col"))), misses = Seq( equals(col("id"), ofInt(1)), not(isNotNull(col("id"))), not(isNotNull(col("arr_col"))), not(isNotNull(col("map_col"))), not(isNotNull(nestedCol("struct_col.field1"))), - isNull(col("id")), isNull(col("arr_col")), isNull(col("map_col")), - isNull(nestedCol("struct_col.field1")) - ) - ) + isNull(nestedCol("struct_col.field1")))) } } @@ -1357,9 +1284,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with isNotNull(col("id")), isNotNull(col("arr_col")), isNotNull(col("map_col")), - isNotNull(nestedCol("struct_col.field1")) - ) - ) + isNotNull(nestedCol("struct_col.field1")))) } } @@ -1385,10 +1310,8 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with isNotNull(col("arr_col")), isNotNull(col("map_col")), isNotNull(col("struct_col")), - isNotNull(nestedCol("struct_col.field1")) - ), - misses = Seq() - ) + isNotNull(nestedCol("struct_col.field1"))), + misses = Seq()) } } @@ -1415,10 +1338,8 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with tempDir.getCanonicalPath, hits = Seq( isNotNull(col("col2")), - isNotNull(col("col1")) - ), - misses = Seq() - ) + isNotNull(col("col1"))), + misses = Seq()) } // remove no rows @@ -1444,12 +1365,9 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with checkSkipping( tempDir.getCanonicalPath, hits = Seq( - isNotNull(col("col1")) - ), + isNotNull(col("col1"))), misses = Seq( - isNotNull(col("col2")) - ) - ) + isNotNull(col("col2")))) } // write initial table with all nulls for col2 val data = SparkRow(0, null) :: SparkRow(1, null) :: Nil @@ -1507,11 +1425,13 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with test("data skipping - prune schema correctly for various predicates") { def structTypeToLeafColumns( - schema: StructType, parentPath: Seq[String] = Seq()): Set[Column] = { + schema: StructType, + parentPath: Seq[String] = Seq()): Set[Column] = { schema.fields().asScala.flatMap { field => field.getDataType() match { case nestedSchema: StructType => - assert(nestedSchema.fields().size() > 0, + assert( + nestedSchema.fields().size() > 0, "Schema should not have field of type StructType with no child fields") structTypeToLeafColumns(nestedSchema, parentPath ++ Seq(field.getName())) case _ => @@ -1533,9 +1453,9 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with lessThanOrEqual(col("as_int"), ofInt(0)) -> Set(nestedCol("minValues.as_int")), new And( lessThan(col("as_int"), ofInt(0)), - greaterThan(col("as_long"), ofInt(0)) - ) -> Set(nestedCol("minValues.as_int"), nestedCol("maxValues.as_long")) - ).foreach { case (predicate, expectedCols) => + greaterThan(col("as_long"), ofInt(0))) -> Set( + nestedCol("minValues.as_int"), + nestedCol("maxValues.as_long"))).foreach { case (predicate, expectedCols) => val engine = engineVerifyJsonParseSchema(verifySchema(expectedCols)) collectScanFileRows( Table.forPath(engine, path).getLatestSnapshot(engine) @@ -1560,8 +1480,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with } // No query filter checkStatsPresent( - latestSnapshot(tempDir.getCanonicalPath).getScanBuilder().build() - ) + latestSnapshot(tempDir.getCanonicalPath).getScanBuilder().build()) // Query filter but no valid data skipping filter checkStatsPresent( latestSnapshot(tempDir.getCanonicalPath) @@ -1569,69 +1488,62 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with .withFilter( greaterThan( new ScalarExpression("+", Seq(col("id"), ofInt(10)).asJava), - ofInt(100) - ) - ).build() - ) + ofInt(100))).build()) // With valid data skipping filter present checkStatsPresent( latestSnapshot(tempDir.getCanonicalPath) .getScanBuilder() .withFilter(greaterThan(col("id"), ofInt(0))) - .build() - ) + .build()) } } Seq( "spark-variant-checkpoint", - "spark-variant-stable-feature-checkpoint" - ).foreach { tableName => + "spark-variant-stable-feature-checkpoint").foreach { tableName => Seq( ("version 0 no predicate", None, Some(0), 2), ("latest version (has checkpoint) no predicate", None, None, 4), - ("version 0 with predicate", Some(equals(col("id"), ofLong(10))), Some(0), 1) - ).foreach { case (nameSuffix, predicate, snapshotVersion, expectedNumFiles) => - test(s"read scan files with variant - $nameSuffix - $tableName") { - val path = getTestResourceFilePath(tableName) - val table = Table.forPath(defaultEngine, path) - val snapshot = snapshotVersion match { - case Some(version) => table.getSnapshotAsOfVersion(defaultEngine, version) - case None => table.getLatestSnapshot(defaultEngine) - } - val snapshotSchema = snapshot.getSchema() - - val expectedSchema = new StructType() - .add("id", LongType.LONG, true) - .add("v", VariantType.VARIANT, true) - .add("array_of_variants", new ArrayType(VariantType.VARIANT, true), true) - .add("struct_of_variants", new StructType().add("v", VariantType.VARIANT, true)) - .add("map_of_variants", new MapType(StringType.STRING, VariantType.VARIANT, true), true) - .add( - "array_of_struct_of_variants", - new ArrayType(new StructType().add("v", VariantType.VARIANT, true), true), - true - ) - .add( - "struct_of_array_of_variants", - new StructType().add("v", new ArrayType(VariantType.VARIANT, true), true), - true - ) - - assert(snapshotSchema == expectedSchema) - - val scanBuilder = snapshot.getScanBuilder() - val scan = predicate match { - case Some(pred) => scanBuilder.withFilter(pred).build() - case None => scanBuilder.build() - } - val scanFiles = scan.asInstanceOf[ScanImpl].getScanFiles(defaultEngine, true) - var numFiles: Int = 0 - scanFiles.forEach { s => - numFiles += s.getRows().toSeq.length + ("version 0 with predicate", Some(equals(col("id"), ofLong(10))), Some(0), 1)).foreach { + case (nameSuffix, predicate, snapshotVersion, expectedNumFiles) => + test(s"read scan files with variant - $nameSuffix - $tableName") { + val path = getTestResourceFilePath(tableName) + val table = Table.forPath(defaultEngine, path) + val snapshot = snapshotVersion match { + case Some(version) => table.getSnapshotAsOfVersion(defaultEngine, version) + case None => table.getLatestSnapshot(defaultEngine) + } + val snapshotSchema = snapshot.getSchema() + + val expectedSchema = new StructType() + .add("id", LongType.LONG, true) + .add("v", VariantType.VARIANT, true) + .add("array_of_variants", new ArrayType(VariantType.VARIANT, true), true) + .add("struct_of_variants", new StructType().add("v", VariantType.VARIANT, true)) + .add("map_of_variants", new MapType(StringType.STRING, VariantType.VARIANT, true), true) + .add( + "array_of_struct_of_variants", + new ArrayType(new StructType().add("v", VariantType.VARIANT, true), true), + true) + .add( + "struct_of_array_of_variants", + new StructType().add("v", new ArrayType(VariantType.VARIANT, true), true), + true) + + assert(snapshotSchema == expectedSchema) + + val scanBuilder = snapshot.getScanBuilder() + val scan = predicate match { + case Some(pred) => scanBuilder.withFilter(pred).build() + case None => scanBuilder.build() + } + val scanFiles = scan.asInstanceOf[ScanImpl].getScanFiles(defaultEngine, true) + var numFiles: Int = 0 + scanFiles.forEach { s => + numFiles += s.getRows().toSeq.length + } + assert(numFiles == expectedNumFiles) } - assert(numFiles == expectedNumFiles) - } } } } @@ -1684,7 +1596,9 @@ object ScanSuite { new DefaultEngine(hadoopConf) { override def getJsonHandler: JsonHandler = { new DefaultJsonHandler(hadoopConf) { - override def parseJson(stringVector: ColumnVector, schema: StructType, + override def parseJson( + stringVector: ColumnVector, + schema: StructType, selectionVector: Optional[ColumnVector]): ColumnarBatch = { verifyFx(schema) super.parseJson(stringVector, schema, selectionVector) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/SnapshotSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/SnapshotSuite.scala index a7373bc0acf..530eb7acda7 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/SnapshotSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/SnapshotSuite.scala @@ -16,13 +16,14 @@ package io.delta.kernel.defaults +import scala.collection.JavaConverters._ + import io.delta.kernel.{Operation, Table} import io.delta.kernel.defaults.utils.TestUtils import io.delta.kernel.types.{IntegerType, StructField, StructType} import io.delta.kernel.utils.CloseableIterable -import org.scalatest.funsuite.AnyFunSuite -import scala.collection.JavaConverters._ +import org.scalatest.funsuite.AnyFunSuite class SnapshotSuite extends AnyFunSuite with TestUtils { @@ -38,7 +39,7 @@ class SnapshotSuite extends AnyFunSuite with TestUtils { val table = Table.forPath(defaultEngine, dir.getCanonicalPath) val columns = (partCols ++ Seq("col1", "col2")).map { colName => - new StructField(colName, IntegerType.INTEGER, true /* nullable */) + new StructField(colName, IntegerType.INTEGER, true /* nullable */ ) } val schema = new StructType(columns.asJava) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/TableChangesSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/TableChangesSuite.scala index 5d6950d856e..618b4139fa0 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/TableChangesSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/TableChangesSuite.scala @@ -20,22 +20,23 @@ import java.io.File import scala.collection.JavaConverters._ import io.delta.golden.GoldenTableUtils.goldenTablePath -import io.delta.kernel.data.Row +import io.delta.kernel.Table import io.delta.kernel.data.ColumnarBatch +import io.delta.kernel.data.Row import io.delta.kernel.defaults.utils.TestUtils -import io.delta.kernel.utils.CloseableIterator -import io.delta.kernel.internal.DeltaLogActionUtils.DeltaAction -import io.delta.kernel.internal.actions.{AddCDCFile, AddFile, CommitInfo, Metadata, Protocol, RemoveFile} -import io.delta.kernel.internal.util.{FileNames, VectorUtils} -import io.delta.kernel.Table import io.delta.kernel.exceptions.{KernelException, TableNotFoundException} +import io.delta.kernel.internal.DeltaLogActionUtils.DeltaAction import io.delta.kernel.internal.TableImpl +import io.delta.kernel.internal.actions.{AddCDCFile, AddFile, CommitInfo, Metadata, Protocol, RemoveFile} import io.delta.kernel.internal.fs.Path +import io.delta.kernel.internal.util.{FileNames, VectorUtils} +import io.delta.kernel.utils.CloseableIterator -import org.apache.spark.sql.delta.actions.{Action => SparkAction, AddCDCFile => SparkAddCDCFile, AddFile => SparkAddFile, CommitInfo => SparkCommitInfo, Metadata => SparkMetadata, Protocol => SparkProtocol, RemoveFile => SparkRemoveFile} import org.apache.spark.sql.delta.DeltaLog -import org.scalatest.funsuite.AnyFunSuite +import org.apache.spark.sql.delta.actions.{Action => SparkAction, AddCDCFile => SparkAddCDCFile, AddFile => SparkAddFile, CommitInfo => SparkCommitInfo, Metadata => SparkMetadata, Protocol => SparkProtocol, RemoveFile => SparkRemoveFile} + import org.apache.spark.sql.functions.col +import org.scalatest.funsuite.AnyFunSuite class TableChangesSuite extends AnyFunSuite with TestUtils { @@ -51,10 +52,10 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { * TableImpl.getChangesByVersion and compare results with Spark */ def testGetChangesVsSpark( - tablePath: String, - startVersion: Long, - endVersion: Long, - actionSet: Set[DeltaAction]): Unit = { + tablePath: String, + startVersion: Long, + endVersion: Long, + actionSet: Set[DeltaAction]): Unit = { val sparkChanges = DeltaLog.forTable(spark, tablePath) .getChanges(startVersion) @@ -82,48 +83,41 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { tablePath, 0, 2, - Set(DeltaAction.REMOVE) - ) + Set(DeltaAction.REMOVE)) testGetChangesVsSpark( tablePath, 0, 2, - Set(DeltaAction.ADD) - ) + Set(DeltaAction.ADD)) testGetChangesVsSpark( tablePath, 0, 2, - Set(DeltaAction.ADD, DeltaAction.REMOVE, DeltaAction.METADATA, DeltaAction.PROTOCOL) - ) + Set(DeltaAction.ADD, DeltaAction.REMOVE, DeltaAction.METADATA, DeltaAction.PROTOCOL)) // request full actions, various versions testGetChangesVsSpark( tablePath, 0, 2, - FULL_ACTION_SET - ) + FULL_ACTION_SET) testGetChangesVsSpark( tablePath, 1, 2, - FULL_ACTION_SET - ) + FULL_ACTION_SET) testGetChangesVsSpark( tablePath, 0, 0, - FULL_ACTION_SET - ) + FULL_ACTION_SET) } } test("getChanges - returns correct timestamps") { withTempDir { tempDir => - def generateCommits(path: String, commits: Long*): Unit = { commits.zipWithIndex.foreach { case (ts, i) => - spark.range(i*10, i*10 + 10).write.format("delta").mode("append").save(path) + spark.range(i * 10, i * 10 + 10).write.format("delta").mode("append").save(path) val file = new File(FileNames.deltaFile(new Path(path, "_delta_log"), i)) file.setLastModified(ts) } @@ -131,13 +125,15 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { val start = 1540415658000L val minuteInMilliseconds = 60000L - generateCommits(tempDir.getCanonicalPath, start, start + 20 * minuteInMilliseconds, + generateCommits( + tempDir.getCanonicalPath, + start, + start + 20 * minuteInMilliseconds, start + 40 * minuteInMilliseconds) val versionToTimestamp: Map[Long, Long] = Map( 0L -> start, 1L -> (start + 20 * minuteInMilliseconds), - 2L -> (start + 40 * minuteInMilliseconds) - ) + 2L -> (start + 40 * minuteInMilliseconds)) // Check the timestamps are returned correctly Table.forPath(defaultEngine, tempDir.getCanonicalPath) @@ -148,7 +144,8 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { .foreach { row => val version = row.getLong(0) val timestamp = row.getLong(1) - assert(timestamp == versionToTimestamp(version), + assert( + timestamp == versionToTimestamp(version), f"Expected timestamp ${versionToTimestamp(version)} for version $version but" + f"Kernel returned timestamp $timestamp") } @@ -158,8 +155,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { tempDir.getCanonicalPath, 0, 2, - FULL_ACTION_SET - ) + FULL_ACTION_SET) } } @@ -206,29 +202,30 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { test("getChanges - golden table deltalog-getChanges invalid queries") { withGoldenTable("deltalog-getChanges") { tablePath => def getChangesByVersion( - startVersion: Long, endVersion: Long): CloseableIterator[ColumnarBatch] = { + startVersion: Long, + endVersion: Long): CloseableIterator[ColumnarBatch] = { Table.forPath(defaultEngine, tablePath) .asInstanceOf[TableImpl] .getChanges(defaultEngine, startVersion, endVersion, FULL_ACTION_SET.asJava) } // startVersion after latest available version - assert(intercept[KernelException]{ + assert(intercept[KernelException] { getChangesByVersion(3, 8) }.getMessage.contains("no log files found in the requested version range")) // endVersion larger than latest available version - assert(intercept[KernelException]{ + assert(intercept[KernelException] { getChangesByVersion(0, 8) }.getMessage.contains("no log file found for version 8")) // invalid start version - assert(intercept[KernelException]{ + assert(intercept[KernelException] { getChangesByVersion(-1, 2) }.getMessage.contains("Invalid version range")) // invalid end version - assert(intercept[KernelException]{ + assert(intercept[KernelException] { getChangesByVersion(2, 1) }.getMessage.contains("Invalid version range")) } @@ -240,13 +237,14 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { val tablePath = tempDir.getCanonicalPath // Write versions [0, 10] (inclusive) including a checkpoint (0 to 10).foreach { i => - spark.range(i*10, i*10 + 10).write + spark.range(i * 10, i * 10 + 10).write .format("delta") .mode("append") .save(tablePath) } val log = org.apache.spark.sql.delta.DeltaLog.forTable( - spark, new org.apache.hadoop.fs.Path(tablePath)) + spark, + new org.apache.hadoop.fs.Path(tablePath)) val deltaCommitFileProvider = org.apache.spark.sql.delta.util.DeltaCommitFileProvider( log.unsafeVolatileSnapshot) // Delete the log files for versions 0-9, truncating the table history to version 10 @@ -285,14 +283,12 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { tablePath, 10, 12, - FULL_ACTION_SET - ) + FULL_ACTION_SET) testGetChangesVsSpark( tablePath, 11, 12, - FULL_ACTION_SET - ) + FULL_ACTION_SET) } } @@ -338,15 +334,13 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { tempDir.getCanonicalPath, 0, 6, - FULL_ACTION_SET - ) + FULL_ACTION_SET) // Check some subset of actions testGetChangesVsSpark( tempDir.getCanonicalPath, 0, 6, - Set(DeltaAction.ADD) - ) + Set(DeltaAction.ADD)) } } @@ -368,6 +362,17 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { }.getMessage.contains("Unsupported Delta protocol reader version")) } + withGoldenTable("commit-info-containing-arbitrary-operationParams-types") { tablePath => + test("getChanges - commit info with arbitrary operationParams types") { + // Check all actions are correctly retrieved + testGetChangesVsSpark( + tablePath, + 0, + 2, + FULL_ACTION_SET) + } + } + ////////////////////////////////////////////////////////////////////////////////// // Helpers to compare actions returned between Kernel and Spark ////////////////////////////////////////////////////////////////////////////////// @@ -376,43 +381,42 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { trait StandardAction case class StandardRemove( - path: String, - dataChange: Boolean, - partitionValues: Map[String, String]) extends StandardAction + path: String, + dataChange: Boolean, + partitionValues: Map[String, String]) extends StandardAction case class StandardAdd( - path: String, - partitionValues: Map[String, String], - size: Long, - modificationTime: Long, - dataChange: Boolean) extends StandardAction + path: String, + partitionValues: Map[String, String], + size: Long, + modificationTime: Long, + dataChange: Boolean) extends StandardAction case class StandardMetadata( - id: String, - schemaString: String, - partitionColumns: Seq[String], - configuration: Map[String, String]) extends StandardAction + id: String, + schemaString: String, + partitionColumns: Seq[String], + configuration: Map[String, String]) extends StandardAction case class StandardProtocol( - minReaderVersion: Int, - minWriterVersion: Int, - readerFeatures: Set[String], - writerFeatures: Set[String]) extends StandardAction + minReaderVersion: Int, + minWriterVersion: Int, + readerFeatures: Set[String], + writerFeatures: Set[String]) extends StandardAction case class StandardCommitInfo( - operation: String, - operationMetrics: Map[String, String]) extends StandardAction + operation: String, + operationMetrics: Map[String, String]) extends StandardAction case class StandardCdc( - path: String, - partitionValues: Map[String, String], - size: Long, - tags: Map[String, String]) extends StandardAction + path: String, + partitionValues: Map[String, String], + size: Long, + tags: Map[String, String]) extends StandardAction def standardizeKernelAction(row: Row): Option[StandardAction] = { val actionIdx = (2 until row.getSchema.length()).find(!row.isNullAt(_)).getOrElse( - return None - ) + return None) row.getSchema.at(actionIdx).getName match { case DeltaAction.REMOVE.colName => @@ -428,8 +432,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { Some(StandardRemove( removeRow.getString(RemoveFile.FULL_SCHEMA.indexOf("path")), removeRow.getBoolean(RemoveFile.FULL_SCHEMA.indexOf("dataChange")), - partitionValues - )) + partitionValues)) case DeltaAction.ADD.colName => val addRow = row.getStruct(actionIdx) @@ -439,8 +442,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { addRow.getMap(AddFile.FULL_SCHEMA.indexOf("partitionValues"))).asScala.toMap, addRow.getLong(AddFile.FULL_SCHEMA.indexOf("size")), addRow.getLong(AddFile.FULL_SCHEMA.indexOf("modificationTime")), - addRow.getBoolean(AddFile.FULL_SCHEMA.indexOf("dataChange")) - )) + addRow.getBoolean(AddFile.FULL_SCHEMA.indexOf("dataChange")))) case DeltaAction.METADATA.colName => val metadataRow = row.getStruct(actionIdx) @@ -450,8 +452,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { VectorUtils.toJavaList( metadataRow.getArray(Metadata.FULL_SCHEMA.indexOf("partitionColumns"))).asScala, VectorUtils.toJavaMap[String, String]( - metadataRow.getMap(Metadata.FULL_SCHEMA.indexOf("configuration"))).asScala.toMap - )) + metadataRow.getMap(Metadata.FULL_SCHEMA.indexOf("configuration"))).asScala.toMap)) case DeltaAction.PROTOCOL.colName => val protocolRow = row.getStruct(actionIdx) @@ -474,8 +475,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { protocolRow.getInt(Protocol.FULL_SCHEMA.indexOf("minReaderVersion")), protocolRow.getInt(Protocol.FULL_SCHEMA.indexOf("minWriterVersion")), readerFeatures.toSet, - writerFeatures.toSet - )) + writerFeatures.toSet)) case DeltaAction.COMMITINFO.colName => val commitInfoRow = row.getStruct(actionIdx) @@ -489,8 +489,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { } else { VectorUtils.toJavaMap[String, String]( commitInfoRow.getMap(operationMetricsIdx)).asScala.toMap - } - )) + })) case DeltaAction.CDC.colName => val cdcRow = row.getStruct(actionIdx) @@ -507,8 +506,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { VectorUtils.toJavaMap[String, String]( cdcRow.getMap(AddCDCFile.FULL_SCHEMA.indexOf("partitionValues"))).asScala.toMap, cdcRow.getLong(AddCDCFile.FULL_SCHEMA.indexOf("size")), - tags - )) + tags)) case _ => throw new RuntimeException("Encountered an action that hasn't been added as an option yet") @@ -520,22 +518,27 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { Some(StandardRemove(remove.path, remove.dataChange, remove.partitionValues)) case add: SparkAddFile => Some(StandardAdd( - add.path, add.partitionValues, add.size, add.modificationTime, add.dataChange)) + add.path, + add.partitionValues, + add.size, + add.modificationTime, + add.dataChange)) case metadata: SparkMetadata => Some(StandardMetadata( - metadata.id, metadata.schemaString, metadata.partitionColumns, metadata.configuration)) + metadata.id, + metadata.schemaString, + metadata.partitionColumns, + metadata.configuration)) case protocol: SparkProtocol => Some(StandardProtocol( protocol.minReaderVersion, protocol.minWriterVersion, protocol.readerFeatures.getOrElse(Set.empty), - protocol.writerFeatures.getOrElse(Set.empty) - )) + protocol.writerFeatures.getOrElse(Set.empty))) case commitInfo: SparkCommitInfo => Some(StandardCommitInfo( commitInfo.operation, - commitInfo.operationMetrics.getOrElse(Map.empty) - )) + commitInfo.operationMetrics.getOrElse(Map.empty))) case cdc: SparkAddCDCFile => Some(StandardCdc(cdc.path, cdc.partitionValues, cdc.size, cdc.tags)) case _ => None @@ -547,10 +550,11 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { * match `actionSet`. */ def pruneSparkActionsByActionSet( - sparkActions: Iterator[(Long, Seq[SparkAction])], - actionSet: Set[DeltaAction]): Iterator[(Long, Seq[SparkAction])] = { + sparkActions: Iterator[(Long, Seq[SparkAction])], + actionSet: Set[DeltaAction]): Iterator[(Long, Seq[SparkAction])] = { sparkActions.map { case (version, actions) => - (version, + ( + version, actions.filter { case _: SparkRemoveFile => actionSet.contains(DeltaAction.REMOVE) case _: SparkAddFile => actionSet.contains(DeltaAction.ADD) @@ -559,14 +563,13 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { case _: SparkCommitInfo => actionSet.contains(DeltaAction.COMMITINFO) case _: SparkAddCDCFile => actionSet.contains(DeltaAction.CDC) case _ => false - } - ) + }) } } def compareActions( - kernelActions: Seq[ColumnarBatch], - sparkActions: Iterator[(Long, Seq[SparkAction])]): Unit = { + kernelActions: Seq[ColumnarBatch], + sparkActions: Iterator[(Long, Seq[SparkAction])]): Unit = { val standardKernelActions: Seq[(Long, StandardAction)] = { kernelActions.flatMap(_.getRows.toSeq) @@ -580,10 +583,10 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { actions.map(standardizeSparkAction(_)).flatten.map((version, _)) }.toSeq - assert(standardKernelActions sameElements standardSparkActions, + assert( + standardKernelActions sameElements standardSparkActions, f"Kernel actions did not match Spark actions.\n" + f"Kernel actions: $standardKernelActions\n" + - f"Spark actions: $standardSparkActions" - ) + f"Spark actions: $standardSparkActions") } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/TablePropertiesSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/TablePropertiesSuite.scala index 1523a882104..68b4abdc8e1 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/TablePropertiesSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/TablePropertiesSuite.scala @@ -15,13 +15,13 @@ */ package io.delta.kernel.defaults +import scala.collection.immutable.Seq + import io.delta.kernel.Table import io.delta.kernel.exceptions.UnknownConfigurationException import io.delta.kernel.internal.SnapshotImpl import io.delta.kernel.utils.CloseableIterable.emptyIterable -import scala.collection.immutable.Seq - /** * Suite to set or get table properties. * TODO: for now we just have the support for `set`. API `get` will be added in the next PRs. @@ -35,8 +35,7 @@ class TablePropertiesSuite extends DeltaTableWriteSuiteBase { createUpdateTableWithProps( tablePath, createTable = true, - props = Map("my key" -> "10", "my key2" -> "20") - ) + props = Map("my key" -> "10", "my key2" -> "20")) assertHasProp(tablePath, expProps = Map("my key" -> "10", "my key2" -> "20")) // update table by modifying the arbitrary properties and check if they are updated @@ -75,7 +74,8 @@ class TablePropertiesSuite extends DeltaTableWriteSuiteBase { test("create/update table - delta configs are stored with same case as defined in TableConfig") { withTempDir { tempFile => val tablePath = tempFile.getAbsolutePath - createUpdateTableWithProps(tablePath, + createUpdateTableWithProps( + tablePath, createTable = true, Map("delta.CHECKPOINTINTERVAL" -> "20")) assertHasProp(tablePath, expProps = Map("delta.checkpointInterval" -> "20")) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala index 49ea61a5aa8..4ffbd755d45 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala @@ -21,6 +21,7 @@ import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types.LongType.LONG import io.delta.kernel.types.StringType.STRING import io.delta.kernel.types.StructType + import org.scalatest.funsuite.AnyFunSuite class DefaultExpressionHandlerSuite extends AnyFunSuite with ExpressionTestUtils { diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultFileSystemClientSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultFileSystemClientSuite.scala index 53a64003605..6c408fbef85 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultFileSystemClientSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultFileSystemClientSuite.scala @@ -20,6 +20,7 @@ import java.io.FileNotFoundException import scala.collection.mutable.ArrayBuffer import io.delta.kernel.defaults.utils.TestUtils + import org.apache.hadoop.fs.{FileSystem, Path} import org.scalatest.funsuite.AnyFunSuite @@ -36,10 +37,9 @@ class DefaultFileSystemClientSuite extends AnyFunSuite with TestUtils { val files = fsClient.listFrom(listFrom) try { fsClient.listFrom(listFrom).forEach(f => actListOutput += f.getPath) - } - finally if (files != null) { - files.close() - } + } finally if (files != null) { + files.close() + } val expListOutput = Seq(basePath + "/2.json", basePath + "/3.json") diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultJsonHandlerSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultJsonHandlerSuite.scala index 5d0af8cf735..4cacff4c1f0 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultJsonHandlerSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultJsonHandlerSuite.scala @@ -16,17 +16,20 @@ package io.delta.kernel.defaults.engine import java.math.{BigDecimal => JBigDecimal} +import java.nio.file.FileAlreadyExistsException import java.util.Optional + import scala.collection.JavaConverters._ + import io.delta.kernel.data.ColumnVector import io.delta.kernel.defaults.utils.{DefaultVectorTestUtils, TestRow, TestUtils} +import io.delta.kernel.internal.actions.CommitInfo import io.delta.kernel.internal.util.InternalUtils.singletonStringColumnVector import io.delta.kernel.types._ + import org.apache.hadoop.conf.Configuration import org.scalatest.funsuite.AnyFunSuite -import java.nio.file.FileAlreadyExistsException - class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVectorTestUtils { val jsonHandler = new DefaultJsonHandler(new Configuration { @@ -39,35 +42,33 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec ///////////////////////////////////////////////////////////////////////////////////////////////// def testJsonParserWithSchema( - jsonString: String, - schema: StructType, - expectedRow: TestRow): Unit = { + jsonString: String, + schema: StructType, + expectedRow: TestRow): Unit = { val batchRows = jsonHandler.parseJson( singletonStringColumnVector(jsonString), schema, - Optional.empty() - ).getRows.toSeq + Optional.empty()).getRows.toSeq checkAnswer(batchRows, Seq(expectedRow)) } def testJsonParserForSingleType( - jsonString: String, - dataType: DataType, - numColumns: Int, - expectedRow: TestRow): Unit = { + jsonString: String, + dataType: DataType, + numColumns: Int, + expectedRow: TestRow): Unit = { val schema = new StructType( (1 to numColumns).map(i => new StructField(s"col$i", dataType, true)).asJava) testJsonParserWithSchema(jsonString, schema, expectedRow) } def testOutOfRangeValue(stringValue: String, dataType: DataType): Unit = { - val e = intercept[RuntimeException]{ + val e = intercept[RuntimeException] { testJsonParserForSingleType( jsonString = s"""{"col1":$stringValue}""", dataType = dataType, numColumns = 1, - expectedRow = TestRow() - ) + expectedRow = TestRow()) } assert(e.getMessage.contains(s"Couldn't decode $stringValue")) } @@ -77,8 +78,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec jsonString = """{"col1":0,"col2":-127,"col3":127, "col4":null}""", dataType = ByteType.BYTE, 4, - TestRow(0.toByte, -127.toByte, 127.toByte, null) - ) + TestRow(0.toByte, -127.toByte, 127.toByte, null)) testOutOfRangeValue("128", ByteType.BYTE) testOutOfRangeValue("-129", ByteType.BYTE) testOutOfRangeValue("2147483648", ByteType.BYTE) @@ -89,8 +89,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec jsonString = """{"col1":-32767,"col2":8,"col3":32767, "col4":null}""", dataType = ShortType.SHORT, 4, - TestRow(-32767.toShort, 8.toShort, 32767.toShort, null) - ) + TestRow(-32767.toShort, 8.toShort, 32767.toShort, null)) testOutOfRangeValue("32768", ShortType.SHORT) testOutOfRangeValue("-32769", ShortType.SHORT) testOutOfRangeValue("2147483648", ShortType.SHORT) @@ -101,8 +100,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec jsonString = """{"col1":-2147483648,"col2":8,"col3":2147483647, "col4":null}""", dataType = IntegerType.INTEGER, 4, - TestRow(-2147483648, 8, 2147483647, null) - ) + TestRow(-2147483648, 8, 2147483647, null)) testOutOfRangeValue("2147483648", IntegerType.INTEGER) testOutOfRangeValue("-2147483649", IntegerType.INTEGER) } @@ -110,11 +108,10 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec test("parse long type") { testJsonParserForSingleType( jsonString = - """{"col1":-9223372036854775808,"col2":8,"col3":9223372036854775807, "col4":null}""", + """{"col1":-9223372036854775808,"col2":8,"col3":9223372036854775807, "col4":null}""", dataType = LongType.LONG, 4, - TestRow(-9223372036854775808L, 8L, 9223372036854775807L, null) - ) + TestRow(-9223372036854775808L, 8L, 9223372036854775807L, null)) testOutOfRangeValue("9223372036854775808", LongType.LONG) testOutOfRangeValue("-9223372036854775809", LongType.LONG) } @@ -127,8 +124,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec |"col4":1.23E-7,"col5":0.004444444, "col6":null}""".stripMargin, dataType = FloatType.FLOAT, 6, - TestRow(-9223.33F, 0.4F, 120000000.0F, 0.000000123F, 0.004444444F, null) - ) + TestRow(-9223.33f, 0.4f, 120000000.0f, 0.000000123f, 0.004444444f, null)) testOutOfRangeValue("3.4028235E+39", FloatType.FLOAT) } @@ -140,17 +136,15 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec |"col4":1.234444444E-7,"col5":0.0444444444, "col6":null}""".stripMargin, dataType = DoubleType.DOUBLE, 6, - TestRow(-922333333.33D, 0.4D, 120000000.0D, 0.0000001234444444D, 0.0444444444D, null) - ) + TestRow(-922333333.33d, 0.4d, 120000000.0d, 0.0000001234444444d, 0.0444444444d, null)) // For some reason out-of-range doubles are parsed initially as Double.INFINITY instead of // a BigDecimal - val e = intercept[RuntimeException]{ + val e = intercept[RuntimeException] { testJsonParserForSingleType( jsonString = s"""{"col1":1.7976931348623157E+309}""", dataType = DoubleType.DOUBLE, numColumns = 1, - expectedRow = TestRow() - ) + expectedRow = TestRow()) } assert(e.getMessage.contains(s"Couldn't decode")) } @@ -160,8 +154,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec jsonString = """{"col1": "foo", "col2": "", "col3": null}""", dataType = StringType.STRING, 3, - TestRow("foo", "", null) - ) + TestRow("foo", "", null)) } test("parse decimal type") { @@ -189,9 +182,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec new JBigDecimal("123456789123456789123456789123456789"), new JBigDecimal("1234567891234567891234567891.2345678900"), new JBigDecimal("1.23"), - null - ) - ) + null)) } test("parse date type") { @@ -199,8 +190,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec jsonString = """{"col1":"2020-12-31", "col2":"1965-01-31", "col3": null}""", dataType = DateType.DATE, 3, - TestRow(18627, -1796, null) - ) + TestRow(18627, -1796, null)) } test("parse timestamp type") { @@ -216,8 +206,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec | """.stripMargin, dataType = TimestampType.TIMESTAMP, numColumns = 4, - TestRow(2524636800000000L, 23423523000L, -315583200000000L, null) - ) + TestRow(2524636800000000L, 23423523000L, -315583200000000L, null)) } test("parse null input") { @@ -227,8 +216,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec val batch = jsonHandler.parseJson( singletonStringColumnVector(null), schema, - Optional.empty() - ) + Optional.empty()) assert(batch.getColumnVector(0).getChild(0).isNullAt(0)) } @@ -239,19 +227,23 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec schema = new StructType() .add("col1", FloatType.FLOAT) .add("col2", DoubleType.DOUBLE), - output - ) + output) } testSpecifiedString("""{"col1":"NaN","col2":"NaN"}""", TestRow(Float.NaN, Double.NaN)) - testSpecifiedString("""{"col1":"+INF","col2":"+INF"}""", + testSpecifiedString( + """{"col1":"+INF","col2":"+INF"}""", TestRow(Float.PositiveInfinity, Double.PositiveInfinity)) - testSpecifiedString("""{"col1":"+Infinity","col2":"+Infinity"}""", + testSpecifiedString( + """{"col1":"+Infinity","col2":"+Infinity"}""", TestRow(Float.PositiveInfinity, Double.PositiveInfinity)) - testSpecifiedString("""{"col1":"Infinity","col2":"Infinity"}""", + testSpecifiedString( + """{"col1":"Infinity","col2":"Infinity"}""", TestRow(Float.PositiveInfinity, Double.PositiveInfinity)) - testSpecifiedString("""{"col1":"-INF","col2":"-INF"}""", + testSpecifiedString( + """{"col1":"-INF","col2":"-INF"}""", TestRow(Float.NegativeInfinity, Double.NegativeInfinity)) - testSpecifiedString("""{"col1":"-Infinity","col2":"-Infinity"}""", + testSpecifiedString( + """{"col1":"-Infinity","col2":"-Infinity"}""", TestRow(Float.NegativeInfinity, Double.NegativeInfinity)) } @@ -263,8 +255,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec jsonVector, new StructType() .add("col1", IntegerType.INTEGER), - Optional.of(selectionVector) - ).getRows.toSeq + Optional.of(selectionVector)).getRows.toSeq assert(!batchRows(0).isNullAt(0) && batchRows(0).getInt(0) == 1) assert(batchRows(1).isNullAt(0) && batchRows(2).isNullAt(0)) } @@ -277,26 +268,20 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec TestRow("part-00000-0441e99a-c421-400e-83a1-212aa6c84c73-c000.snappy.parquet", 650L, true), TestRow("part-00001-34c8c673-3f44-4fa7-b94e-07357ec28a7d-c000.snappy.parquet", 650L, true), TestRow("part-00000-842017c2-3e02-44b5-a3d6-5b9ae1745045-c000.snappy.parquet", 649L, true), - TestRow("part-00001-e62ca5a1-923c-4ee6-998b-c61d1cfb0b1c-c000.snappy.parquet", 649L, true) - ) + TestRow("part-00001-e62ca5a1-923c-4ee6-998b-c61d1cfb0b1c-c000.snappy.parquet", 649L, true)) Seq( ( fsClient.listFrom(getTestResourceFilePath("json-files/1.json")), - expResults - ), + expResults), ( fsClient.listFrom(getTestResourceFilePath("json-files-with-empty/1.json")), - expResults - ), + expResults), ( fsClient.listFrom(getTestResourceFilePath("json-files-with-empty/5.json")), - expResults.takeRight(2) - ), + expResults.takeRight(2)), ( fsClient.listFrom(getTestResourceFilePath("json-files-all-empty/1.json")), - Seq() - ) - ).foreach { + Seq())).foreach { case (testFiles, expResults) => val actResult = jsonHandler.readJsonFiles( testFiles, @@ -304,8 +289,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec .add("path", StringType.STRING) .add("size", LongType.LONG) .add("dataChange", BooleanType.BOOLEAN), - Optional.empty() - ).toSeq.map(batch => TestRow(batch.getRows.next)) + Optional.empty()).toSeq.map(batch => TestRow(batch.getRows.next)) checkAnswer(actResult, expResults) } @@ -328,8 +312,9 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec .add("dataChange", BooleanType.BOOLEAN) val batch = jsonHandler.parseJson( - singletonStringColumnVector(input), readSchema, Optional.empty[ColumnVector]() - ) + singletonStringColumnVector(input), + readSchema, + Optional.empty[ColumnVector]()) assert(batch.getSize == 1) val actResult = Seq(TestRow(batch.getRows.next)) @@ -337,8 +322,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec "part-00000-d83dafd8-c344-49f0-ab1c-acd944e32493-c000.snappy.parquet", Map("p1" -> "0", "p2" -> "str"), 348L, - true - )) + true)) checkAnswer(actResult, expResult) } @@ -363,21 +347,18 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec new MapType( StringType.STRING, new MapType(StringType.STRING, new ArrayType(IntegerType.INTEGER, true), true), - true - ) - ) + true)) .add( "array_of_struct", new ArrayType( new StructType() .add("field1", StringType.STRING, true) .add("field2", IntegerType.INTEGER, true), - true - ) - ) + true)) val batch = jsonHandler.parseJson( - singletonStringColumnVector(json), schema, Optional.empty[ColumnVector]() - ) + singletonStringColumnVector(json), + schema, + Optional.empty[ColumnVector]()) val actResult = Seq(TestRow(batch.getRows.next)) val expResult = Seq(TestRow( @@ -387,12 +368,9 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec Map( "a" -> Map( "one" -> Vector(), - "two" -> Vector(1, 2, 3) - ), - "b" -> Map() - ), - Vector(TestRow.fromSeq(Seq("foo", 3)), TestRow.fromSeq(Seq(null, null))) - )) + "two" -> Vector(1, 2, 3)), + "b" -> Map()), + Vector(TestRow.fromSeq(Seq("foo", 3)), TestRow.fromSeq(Seq(null, null))))) checkAnswer(actResult, expResult) } @@ -419,8 +397,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec | "dataChange":true | } |} - |""".stripMargin.linesIterator.mkString - ) + |""".stripMargin.linesIterator.mkString) val addRemoveSchema = new StructType() .add("path", StringType.STRING) @@ -441,7 +418,9 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec // read it back and verify the contents are correct val source = scala.io.Source.fromFile(filePath) - val result = try source.getLines().mkString(",") finally source.close() + val result = + try source.getLines().mkString(",") + finally source.close() // remove the whitespaces from the input to compare assert(input.map(_.replaceAll(" ", "")).mkString(",") === result) @@ -451,11 +430,50 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec // Try to write as same file with overwrite as false and expect an error intercept[FileAlreadyExistsException] { - jsonHandler.writeJsonFileAtomically(filePath, batch.getRows, false /* overwrite */) + jsonHandler.writeJsonFileAtomically(filePath, batch.getRows, false /* overwrite */ ) } // Try to write as file with overwrite set to true writeAndVerify(overwrite = true) } } + + test("parse diverse type values in a map[string, string]") { + val input = + """ + |{ + | "inCommitTimestamp":1740009523401, + | "timestamp":1740009523401, + | "engineInfo":"myengine.com", + | "operation":"WRITE", + | "operationParameters": + | {"mode":"Append","statsOnLoad":false,"partitionBy":"[]"}, + | "isBlindAppend":true, + | "txnId":"cb009f42-5da1-4e7e-b4fa-09de3332f52a", + | "operationMetrics": { + | "numFiles":"1", + | "serializedAsNumber":2, + | "serializedAsBoolean":true + | } + |} + |""".stripMargin + + val output = jsonHandler.parseJson( + stringVector(Seq(input)), + CommitInfo.FULL_SCHEMA, + Optional.empty()) + assert(output.getSize == 1) + val actResult = TestRow(output.getRows.next) + val expResult = TestRow( + 1740009523401L, + 1740009523401L, + "myengine.com", + "WRITE", + Map("mode" -> "Append", "statsOnLoad" -> "false", "partitionBy" -> "[]"), + true, + "cb009f42-5da1-4e7e-b4fa-09de3332f52a", + Map("numFiles" -> "1", "serializedAsNumber" -> "2", "serializedAsBoolean" -> "true")) + + checkAnswer(Seq(actResult), Seq(expResult)) + } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultParquetHandlerSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultParquetHandlerSuite.scala index fef6bf07dc1..6bc8a83c2a9 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultParquetHandlerSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultParquetHandlerSuite.scala @@ -15,15 +15,17 @@ */ package io.delta.kernel.defaults.engine +import java.nio.file.FileAlreadyExistsException + +import scala.collection.JavaConverters._ + import io.delta.golden.GoldenTableUtils.goldenTableFile import io.delta.kernel.defaults.internal.parquet.ParquetSuiteBase import io.delta.kernel.internal.util.Utils.toCloseableIterator + import org.apache.hadoop.conf.Configuration import org.scalatest.funsuite.AnyFunSuite -import java.nio.file.FileAlreadyExistsException -import scala.collection.JavaConverters._ - class DefaultParquetHandlerSuite extends AnyFunSuite with ParquetSuiteBase { val parquetHandler = new DefaultParquetHandler(new Configuration { diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluatorSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluatorSuite.scala index abd66231889..ab5ba3da025 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluatorSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluatorSuite.scala @@ -15,23 +15,24 @@ */ package io.delta.kernel.defaults.internal.expressions -import io.delta.kernel.data.{ColumnVector, ColumnarBatch} +import java.lang.{Boolean => BooleanJ} +import java.math.{BigDecimal => BigDecimalJ} +import java.sql.{Date, Timestamp} +import java.util +import java.util.Optional + +import io.delta.kernel.data.{ColumnarBatch, ColumnVector} import io.delta.kernel.defaults.internal.data.DefaultColumnarBatch import io.delta.kernel.defaults.internal.data.vector.{DefaultIntVector, DefaultStructVector} import io.delta.kernel.defaults.utils.DefaultKernelTestUtils.getValueAsObject +import io.delta.kernel.expressions._ import io.delta.kernel.expressions.AlwaysFalse.ALWAYS_FALSE import io.delta.kernel.expressions.AlwaysTrue.ALWAYS_TRUE import io.delta.kernel.expressions.Literal._ -import io.delta.kernel.expressions._ import io.delta.kernel.internal.util.InternalUtils import io.delta.kernel.types._ -import org.scalatest.funsuite.AnyFunSuite -import java.lang.{Boolean => BooleanJ} -import java.math.{BigDecimal => BigDecimalJ} -import java.sql.{Date, Timestamp} -import java.util -import java.util.Optional +import org.scalatest.funsuite.AnyFunSuite class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBase { test("evaluate expression: literal") { @@ -85,8 +86,7 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa } else { assert( literal.getValue === getValueAsObject(outputVector, rowId), - s"invalid value at $rowId for $literal expression" - ) + s"invalid value at $rowId for $literal expression") } } } @@ -143,7 +143,9 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa val batch = new DefaultColumnarBatch(numRows, batchSchema, Array(col1Vector)) def assertTypeAndNullability( - actVector: ColumnVector, expType: DataType, expNullability: Array[Boolean]): Unit = { + actVector: ColumnVector, + expType: DataType, + expNullability: Array[Boolean]): Unit = { assert(actVector.getDataType === expType) assert(actVector.getSize === numRows) Seq.range(0, numRows).foreach { rowId => @@ -224,8 +226,7 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa val notExpression = new Predicate( "NOT", - comparator("=", new Column("child"), Literal.ofBoolean(true)) - ) + comparator("=", new Column("child"), Literal.ofBoolean(true))) val expOutputVector = booleanVector(Seq[BooleanJ](false, true, null)) val actOutputVector = evaluator(schema, notExpression, BooleanType.BOOLEAN).eval(batch) checkBooleanVectors(actOutputVector, expOutputVector) @@ -277,33 +278,43 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa val coalesceEpxr3 = new ScalarExpression( "COALESCE", util.Arrays.asList( - new Column("col1"), new Column("col2"), new Column("col3"))) + new Column("col1"), + new Column("col2"), + new Column("col3"))) val expOutputVector3 = booleanVector(Seq[BooleanJ](true, false, true, null)) val actOutputVector3 = evaluator(schema, coalesceEpxr3, BooleanType.BOOLEAN).eval(batch) checkBooleanVectors(actOutputVector3, expOutputVector3) def checkUnsupportedTypes( - col1Type: DataType, col2Type: DataType, messageContains: String): Unit = { + col1Type: DataType, + col2Type: DataType, + messageContains: String): Unit = { val schema = new StructType() .add("col1", col1Type) .add("col2", col2Type) - val batch = new DefaultColumnarBatch(5, schema, + val batch = new DefaultColumnarBatch( + 5, + schema, Array(testColumnVector(5, col1Type), testColumnVector(5, col2Type))) val e = intercept[UnsupportedOperationException] { evaluator( schema, - new ScalarExpression("COALESCE", + new ScalarExpression( + "COALESCE", util.Arrays.asList(new Column("col1"), new Column("col2"))), - col1Type - ).eval(batch) + col1Type).eval(batch) } assert(e.getMessage.contains(messageContains)) } // TODO support least-common-type resolution - checkUnsupportedTypes(LongType.LONG, IntegerType.INTEGER, + checkUnsupportedTypes( + LongType.LONG, + IntegerType.INTEGER, "Coalesce is only supported for arguments of the same type") // TODO support other types besides boolean - checkUnsupportedTypes(IntegerType.INTEGER, IntegerType.INTEGER, + checkUnsupportedTypes( + IntegerType.INTEGER, + IntegerType.INTEGER, "Coalesce is only supported for boolean type expressions") } @@ -311,32 +322,32 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa val timestampColumn = timestampVector(Seq[Long]( 1577836800000000L, // 2020-01-01 00:00:00.000 1577836800123456L, // 2020-01-01 00:00:00.123456 - -1 // Representing null + -1 // Representing null )) val durationColumn = longVector(Seq[Long]( - 1000, // 1 second in milliseconds - 100, // 0.1 second in milliseconds - -1 - ): _*) + 1000, // 1 second in milliseconds + 100, // 0.1 second in milliseconds + -1): _*) val schema = new StructType() .add("timestamp", TimestampType.TIMESTAMP) .add("duration", LongType.LONG) val batch = new DefaultColumnarBatch( - timestampColumn.getSize, schema, Array(timestampColumn, durationColumn)) + timestampColumn.getSize, + schema, + Array(timestampColumn, durationColumn)) // TimeAdd expression adds milliseconds to timestamps val timeAddExpr = new ScalarExpression( "TIMEADD", - util.Arrays.asList(new Column("timestamp"), new Column("duration")) - ) + util.Arrays.asList(new Column("timestamp"), new Column("duration"))) val expectedTimestamps = Seq[Long]( 1577836801000000L, // 2020-01-01 00:00:01.000 1577836800123456L + 100000, // 2020-01-01 00:00:00.123556 - -1 // Null should propagate + -1 // Null should propagate ) val expOutputVector = timestampVector(expectedTimestamps) @@ -346,17 +357,19 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa } def checkUnsupportedTimeAddTypes( - col1Type: DataType, col2Type: DataType): Unit = { + col1Type: DataType, + col2Type: DataType): Unit = { val schema = new StructType() .add("timestamp", col1Type) .add("duration", col2Type) - val batch = new DefaultColumnarBatch(5, schema, + val batch = new DefaultColumnarBatch( + 5, + schema, Array(testColumnVector(5, col1Type), testColumnVector(5, col2Type))) val timeAddExpr = new ScalarExpression( "TIMEADD", - util.Arrays.asList(new Column("timestamp"), new Column("duration")) - ) + util.Arrays.asList(new Column("timestamp"), new Column("duration"))) val e = intercept[IllegalArgumentException] { val evaluator = new DefaultExpressionEvaluator(schema, timeAddExpr, col1Type) @@ -370,34 +383,55 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa test("TIMEADD with unsupported types") { // Check invalid timestamp column type checkUnsupportedTimeAddTypes( - IntegerType.INTEGER, IntegerType.INTEGER) + IntegerType.INTEGER, + IntegerType.INTEGER) // Check invalid duration column type checkUnsupportedTimeAddTypes( - TimestampType.TIMESTAMP, StringType.STRING) + TimestampType.TIMESTAMP, + StringType.STRING) // Check valid type but with unsupported operations checkUnsupportedTimeAddTypes( - TimestampType.TIMESTAMP, FloatType.FLOAT) + TimestampType.TIMESTAMP, + FloatType.FLOAT) } test("evaluate expression: like") { val col1 = stringVector(Seq[String]( - null, "one", "two", "three", "four", null, null, "seven", "eight")) + null, + "one", + "two", + "three", + "four", + null, + null, + "seven", + "eight")) val col2 = stringVector(Seq[String]( - null, "one", "Two", "thr%", "four%", "f", null, null, "%ght")) + null, + "one", + "Two", + "thr%", + "four%", + "f", + null, + null, + "%ght")) val schema = new StructType() .add("col1", StringType.STRING) .add("col2", StringType.STRING) val input = new DefaultColumnarBatch(col1.getSize, schema, Array(col1, col2)) def checkLike( - input: DefaultColumnarBatch, - likeExpression: Predicate, - expOutputSeq: Seq[BooleanJ]): Unit = { + input: DefaultColumnarBatch, + likeExpression: Predicate, + expOutputSeq: Seq[BooleanJ]): Unit = { val actOutputVector = new DefaultExpressionEvaluator( - schema, likeExpression, BooleanType.BOOLEAN).eval(input) + schema, + likeExpression, + BooleanType.BOOLEAN).eval(input) val expOutputVector = booleanVector(expOutputSeq); checkBooleanVectors(actOutputVector, expOutputVector) } @@ -426,12 +460,16 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa like(new Column("col1"), Literal.ofString("%t%")), Seq[BooleanJ](null, false, true, true, false, null, null, false, true)) - val dummyInput = new DefaultColumnarBatch(1, - new StructType().add("dummy", StringType.STRING), - Array(stringVector(Seq[String]("")))) + val dummyInput = new DefaultColumnarBatch( + 1, + new StructType().add("dummy", StringType.STRING), + Array(stringVector(Seq[String]("")))) - def checkLikeLiteral(left: String, right: String, - escape: Character = null, expOutput: BooleanJ): Unit = { + def checkLikeLiteral( + left: String, + right: String, + escape: Character = null, + expOutput: BooleanJ): Unit = { val expression = like(Literal.ofString(left), Literal.ofString(right), Option(escape)) checkLike(dummyInput, expression, Seq[BooleanJ](expOutput)) } @@ -445,54 +483,57 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa checkLikeLiteral("", "a", null, false) Seq('!', '@', '#').foreach { - escape => { - // simple patterns - checkLikeLiteral("abc", "abc", escape, true) - checkLikeLiteral("a_%b", s"a${escape}__b", escape, true) - checkLikeLiteral("abbc", "a_%c", escape, true) - checkLikeLiteral("abbc", s"a${escape}__c", escape, false) - checkLikeLiteral("abbc", s"a%${escape}%c", escape, false) - checkLikeLiteral("a_%b", s"a%${escape}%b", escape, true) - checkLikeLiteral("abbc", "a%", escape, true) - checkLikeLiteral("abbc", "**", escape, false) - checkLikeLiteral("abc", "a%", escape, true) - checkLikeLiteral("abc", "b%", escape, false) - checkLikeLiteral("abc", "bc%", escape, false) - checkLikeLiteral("a\nb", "a_b", escape, true) - checkLikeLiteral("ab", "a%b", escape, true) - checkLikeLiteral("a\nb", "a%b", escape, true) - checkLikeLiteral("a\nb", "ab", escape, false) - checkLikeLiteral("a\nb", "a\nb", escape, true) - checkLikeLiteral("a\n\nb", "a\nb", escape, false) - checkLikeLiteral("a\n\nb", "a\n_b", escape, true) - - // case - checkLikeLiteral("A", "a%", escape, false) - checkLikeLiteral("a", "a%", escape, true) - checkLikeLiteral("a", "A%", escape, false) - checkLikeLiteral(s"aAa", s"aA_", escape, true) - - // regex - checkLikeLiteral("a([a-b]{2,4})a", "_([a-b]{2,4})%", null, true) - checkLikeLiteral("a([a-b]{2,4})a", "_([a-c]{2,6})_", null, false) - - // %/_ - checkLikeLiteral("a%a", s"%${escape}%%", escape, true) - checkLikeLiteral("a%", s"%${escape}%%", escape, true) - checkLikeLiteral("a%a", s"_${escape}%_", escape, true) - checkLikeLiteral("a_a", s"%${escape}_%", escape, true) - checkLikeLiteral("a_", s"%${escape}_%", escape, true) - checkLikeLiteral("a_a", s"_${escape}__", escape, true) - - // double-escaping - checkLikeLiteral( - s"$escape$escape$escape$escape", s"%${escape}${escape}%", escape, true) - checkLikeLiteral("%%", "%%", escape, true) - checkLikeLiteral(s"${escape}__", s"${escape}${escape}${escape}__", escape, true) - checkLikeLiteral(s"${escape}__", s"%${escape}${escape}%${escape}%", escape, false) - checkLikeLiteral(s"_${escape}${escape}${escape}%", - s"%${escape}${escape}", escape, false) - } + escape => + { + // simple patterns + checkLikeLiteral("abc", "abc", escape, true) + checkLikeLiteral("a_%b", s"a${escape}__b", escape, true) + checkLikeLiteral("abbc", "a_%c", escape, true) + checkLikeLiteral("abbc", s"a${escape}__c", escape, false) + checkLikeLiteral("abbc", s"a%${escape}%c", escape, false) + checkLikeLiteral("a_%b", s"a%${escape}%b", escape, true) + checkLikeLiteral("abbc", "a%", escape, true) + checkLikeLiteral("abbc", "**", escape, false) + checkLikeLiteral("abc", "a%", escape, true) + checkLikeLiteral("abc", "b%", escape, false) + checkLikeLiteral("abc", "bc%", escape, false) + checkLikeLiteral("a\nb", "a_b", escape, true) + checkLikeLiteral("ab", "a%b", escape, true) + checkLikeLiteral("a\nb", "a%b", escape, true) + checkLikeLiteral("a\nb", "ab", escape, false) + checkLikeLiteral("a\nb", "a\nb", escape, true) + checkLikeLiteral("a\n\nb", "a\nb", escape, false) + checkLikeLiteral("a\n\nb", "a\n_b", escape, true) + + // case + checkLikeLiteral("A", "a%", escape, false) + checkLikeLiteral("a", "a%", escape, true) + checkLikeLiteral("a", "A%", escape, false) + checkLikeLiteral(s"aAa", s"aA_", escape, true) + + // regex + checkLikeLiteral("a([a-b]{2,4})a", "_([a-b]{2,4})%", null, true) + checkLikeLiteral("a([a-b]{2,4})a", "_([a-c]{2,6})_", null, false) + + // %/_ + checkLikeLiteral("a%a", s"%${escape}%%", escape, true) + checkLikeLiteral("a%", s"%${escape}%%", escape, true) + checkLikeLiteral("a%a", s"_${escape}%_", escape, true) + checkLikeLiteral("a_a", s"%${escape}_%", escape, true) + checkLikeLiteral("a_", s"%${escape}_%", escape, true) + checkLikeLiteral("a_a", s"_${escape}__", escape, true) + + // double-escaping + checkLikeLiteral( + s"$escape$escape$escape$escape", + s"%${escape}${escape}%", + escape, + true) + checkLikeLiteral("%%", "%%", escape, true) + checkLikeLiteral(s"${escape}__", s"${escape}${escape}${escape}__", escape, true) + checkLikeLiteral(s"${escape}__", s"%${escape}${escape}%${escape}%", escape, false) + checkLikeLiteral(s"_${escape}${escape}${escape}%", s"%${escape}${escape}", escape, false) + } } // check '_' for escape char @@ -513,17 +554,22 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa checkLikeLiteral("abbc", s"a%_%_c", '%', false) def checkUnsupportedTypes( - col1Type: DataType, col2Type: DataType): Unit = { + col1Type: DataType, + col2Type: DataType): Unit = { val schema = new StructType() .add("col1", col1Type) .add("col2", col2Type) val expr = like(new Column("col1"), new Column("col2"), Option(null)) - val input = new DefaultColumnarBatch(5, schema, + val input = new DefaultColumnarBatch( + 5, + schema, Array(testColumnVector(5, col1Type), testColumnVector(5, col2Type))) val e = intercept[UnsupportedOperationException] { new DefaultExpressionEvaluator( - schema, expr, BooleanType.BOOLEAN).eval(input) + schema, + expr, + BooleanType.BOOLEAN).eval(input) } assert(e.getMessage.contains("LIKE is only supported for string type expressions")) } @@ -545,8 +591,11 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa assert(inputCountError1.getMessage.contains(inputCountCheckUserMessage)) val inputCountError2 = intercept[UnsupportedOperationException] { - val expression = like(List(Literal.ofString("a"), Literal.ofString("b"), - Literal.ofString("c"), Literal.ofString("d"))) + val expression = like(List( + Literal.ofString("a"), + Literal.ofString("b"), + Literal.ofString("c"), + Literal.ofString("d"))) checkLike(dummyInput, expression, Seq[BooleanJ](null)) } assert(inputCountError2.getMessage.contains(inputCountCheckUserMessage)) @@ -568,11 +617,14 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa "LIKE expects escape token expression to be a literal of String type")) // empty input checks - val emptyInput = new DefaultColumnarBatch(0, - new StructType().add("dummy", StringType.STRING), - Array(stringVector(Seq[String]("")))) - checkLike(emptyInput, - like(Literal.ofString("abc"), Literal.ofString("abc"), Some('_')), Seq[BooleanJ]()) + val emptyInput = new DefaultColumnarBatch( + 0, + new StructType().add("dummy", StringType.STRING), + Array(stringVector(Seq[String]("")))) + checkLike( + emptyInput, + like(Literal.ofString("abc"), Literal.ofString("abc"), Some('_')), + Seq[BooleanJ]()) // invalid pattern check val invalidPatternError = intercept[IllegalArgumentException] { @@ -592,69 +644,93 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa val startsWithExpressionLiteral = startsWith(new Column("col1"), Literal.ofString("t%")) val expOutputVectorLiteral = - booleanVector(Seq[BooleanJ](false, false, true, false, null, null, false)) - checkBooleanVectors(new DefaultExpressionEvaluator( - schema, startsWithExpressionLiteral, BooleanType.BOOLEAN).eval(input), expOutputVectorLiteral) + booleanVector(Seq[BooleanJ](false, false, true, false, null, null, false)) + checkBooleanVectors( + new DefaultExpressionEvaluator( + schema, + startsWithExpressionLiteral, + BooleanType.BOOLEAN).eval(input), + expOutputVectorLiteral) val startsWithExpressionNullLiteral = startsWith(new Column("col1"), Literal.ofString(null)) val allNullVector = booleanVector(Seq[BooleanJ](null, null, null, null, null, null, null)) - checkBooleanVectors(new DefaultExpressionEvaluator( - schema, startsWithExpressionNullLiteral, BooleanType.BOOLEAN).eval(input), allNullVector) + checkBooleanVectors( + new DefaultExpressionEvaluator( + schema, + startsWithExpressionNullLiteral, + BooleanType.BOOLEAN).eval(input), + allNullVector) // Two literal expressions on both sides val startsWithExpressionAlwaysTrue = startsWith(Literal.ofString("ABC"), Literal.ofString("A")) val allTrueVector = booleanVector(Seq[BooleanJ](true, true, true, true, true, true, true)) - checkBooleanVectors(new DefaultExpressionEvaluator( - schema, startsWithExpressionAlwaysTrue, BooleanType.BOOLEAN).eval(input), allTrueVector) + checkBooleanVectors( + new DefaultExpressionEvaluator( + schema, + startsWithExpressionAlwaysTrue, + BooleanType.BOOLEAN).eval(input), + allTrueVector) val startsWithExpressionAlwaysFalse = startsWith(Literal.ofString("ABC"), Literal.ofString("_B%")) val allFalseVector = booleanVector(Seq[BooleanJ](false, false, false, false, false, false, false)) - checkBooleanVectors(new DefaultExpressionEvaluator( - schema, startsWithExpressionAlwaysFalse, BooleanType.BOOLEAN).eval(input), allFalseVector) + checkBooleanVectors( + new DefaultExpressionEvaluator( + schema, + startsWithExpressionAlwaysFalse, + BooleanType.BOOLEAN).eval(input), + allFalseVector) // scalastyle:off nonascii val colUnicode = stringVector(Seq[String]("中文", "中", "文")) val schemaUnicode = new StructType().add("col", StringType.STRING) - val inputUnicode = new DefaultColumnarBatch(colUnicode.getSize, - schemaUnicode, Array(colUnicode)) + val inputUnicode = + new DefaultColumnarBatch(colUnicode.getSize, schemaUnicode, Array(colUnicode)) val startsWithExpressionUnicode = startsWith(new Column("col"), Literal.ofString("中")) val expOutputVectorLiteralUnicode = booleanVector(Seq[BooleanJ](true, true, false)) - checkBooleanVectors(new DefaultExpressionEvaluator(schemaUnicode, - startsWithExpressionUnicode, - BooleanType.BOOLEAN).eval(inputUnicode), expOutputVectorLiteralUnicode) + checkBooleanVectors( + new DefaultExpressionEvaluator( + schemaUnicode, + startsWithExpressionUnicode, + BooleanType.BOOLEAN).eval(inputUnicode), + expOutputVectorLiteralUnicode) // scalastyle:off nonascii val colSurrogatePair = stringVector(Seq[String]("💕😉💕", "😉💕", "💕")) val schemaSurrogatePair = new StructType().add("col", StringType.STRING) - val inputSurrogatePair = new DefaultColumnarBatch(colSurrogatePair.getSize, - schemaUnicode, Array(colSurrogatePair)) + val inputSurrogatePair = + new DefaultColumnarBatch(colSurrogatePair.getSize, schemaUnicode, Array(colSurrogatePair)) val startsWithExpressionSurrogatePair = startsWith(new Column("col"), Literal.ofString("💕")) val expOutputVectorLiteralSurrogatePair = booleanVector(Seq[BooleanJ](true, false, true)) - checkBooleanVectors(new DefaultExpressionEvaluator(schemaSurrogatePair, - startsWithExpressionSurrogatePair, - BooleanType.BOOLEAN).eval(inputSurrogatePair), expOutputVectorLiteralSurrogatePair) + checkBooleanVectors( + new DefaultExpressionEvaluator( + schemaSurrogatePair, + startsWithExpressionSurrogatePair, + BooleanType.BOOLEAN).eval(inputSurrogatePair), + expOutputVectorLiteralSurrogatePair) val startsWithExpressionExpression = startsWith(new Column("col1"), new Column("col2")) val e = intercept[UnsupportedOperationException] { new DefaultExpressionEvaluator( - schema, startsWithExpressionExpression, BooleanType.BOOLEAN).eval(input) + schema, + startsWithExpressionExpression, + BooleanType.BOOLEAN).eval(input) } assert(e.getMessage.contains("'STARTS_WITH' expects literal as the second input")) - def checkUnsupportedTypes(colType: DataType, literalType: DataType): Unit = { val schema = new StructType() .add("col", colType) val expr = startsWith(new Column("col"), Literal.ofNull(literalType)) - val input = new DefaultColumnarBatch(5, schema, - Array(testColumnVector(5, colType))) + val input = new DefaultColumnarBatch(5, schema, Array(testColumnVector(5, colType))) val e = intercept[UnsupportedOperationException] { new DefaultExpressionEvaluator( - schema, expr, BooleanType.BOOLEAN).eval(input) + schema, + expr, + BooleanType.BOOLEAN).eval(input) } assert(e.getMessage.contains("'STARTS_WITH' expects STRING type inputs")) } @@ -678,21 +754,19 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa (ofShort(1.toShort), ofShort(2.toShort), ofShort(1.toShort), ofNull(ShortType.SHORT)), (ofInt(1), ofInt(2), ofInt(1), ofNull(IntegerType.INTEGER)), (ofLong(1L), ofLong(2L), ofLong(1L), ofNull(LongType.LONG)), - (ofFloat(1.0F), ofFloat(2.0F), ofFloat(1.0F), ofNull(FloatType.FLOAT)), + (ofFloat(1.0f), ofFloat(2.0f), ofFloat(1.0f), ofNull(FloatType.FLOAT)), (ofDouble(1.0), ofDouble(2.0), ofDouble(1.0), ofNull(DoubleType.DOUBLE)), (ofBoolean(false), ofBoolean(true), ofBoolean(false), ofNull(BooleanType.BOOLEAN)), ( ofTimestamp(343L), ofTimestamp(123212312L), ofTimestamp(343L), - ofNull(TimestampType.TIMESTAMP) - ), + ofNull(TimestampType.TIMESTAMP)), ( ofTimestampNtz(323423L), ofTimestampNtz(1232123423312L), ofTimestampNtz(323423L), - ofNull(TimestampNTZType.TIMESTAMP_NTZ) - ), + ofNull(TimestampNTZType.TIMESTAMP_NTZ)), (ofDate(-12123), ofDate(123123), ofDate(-12123), ofNull(DateType.DATE)), (ofString("apples"), ofString("oranges"), ofString("apples"), ofNull(StringType.STRING)), (ofString(""), ofString("a"), ofString(""), ofNull(StringType.STRING)), @@ -703,14 +777,12 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa ofString("Abcabcabc"), ofString("aBcabcabc"), ofString("Abcabcabc"), - ofNull(StringType.STRING) - ), + ofNull(StringType.STRING)), ( ofString("abcabcabC"), ofString("abcabcabc"), ofString("abcabcabC"), - ofNull(StringType.STRING) - ), + ofNull(StringType.STRING)), // scalastyle:off nonascii (ofString("abc"), ofString("世界"), ofString("abc"), ofNull(StringType.STRING)), (ofString("世界"), ofString("你好"), ofString("世界"), ofNull(StringType.STRING)), @@ -722,100 +794,83 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa ofString("abcdef🚀"), ofString(s"abcdef$UTF8_MAX_CHARACTER"), ofString("abcdef🚀"), - ofNull(StringType.STRING) - ), + ofNull(StringType.STRING)), ( ofString("abcde�abcdef�abcdef�abcdef"), ofString(s"abcde�$ASCII_MAX_CHARACTER"), ofString("abcde�abcdef�abcdef�abcdef"), - ofNull(StringType.STRING) - ), + ofNull(StringType.STRING)), ( ofString("abcde�abcdef�abcdef�abcdef"), ofString(s"abcde�$ASCII_MAX_CHARACTER"), ofString("abcde�abcdef�abcdef�abcdef"), - ofNull(StringType.STRING) - ), + ofNull(StringType.STRING)), ( ofString("����"), ofString(s"��$UTF8_MAX_CHARACTER"), ofString("����"), - ofNull(StringType.STRING) - ), + ofNull(StringType.STRING)), ( ofString(s"a${UTF8_MAX_CHARACTER}d"), ofString(s"a$UTF8_MAX_CHARACTER$ASCII_MAX_CHARACTER"), ofString(s"a${UTF8_MAX_CHARACTER}d"), - ofNull(StringType.STRING) - ), + ofNull(StringType.STRING)), ( ofString("abcdefghijklm💞😉💕\n🥀🌹💐🌺🌷🌼🌻🌷🥀"), ofString(s"abcdefghijklm💞😉💕\n🥀🌹💐🌺🌷🌼$UTF8_MAX_CHARACTER"), ofString("abcdefghijklm💞😉💕\n🥀🌹💐🌺🌷🌼🌻🌷🥀"), - ofNull(StringType.STRING) - ), + ofNull(StringType.STRING)), // scalastyle:on nonascii ( ofBinary("apples".getBytes()), ofBinary("oranges".getBytes()), ofBinary("apples".getBytes()), - ofNull(BinaryType.BINARY) - ), + ofNull(BinaryType.BINARY)), ( ofBinary(Array[Byte]()), ofBinary(Array[Byte](5.toByte)), ofBinary(Array[Byte]()), - ofNull(BinaryType.BINARY) - ), + ofNull(BinaryType.BINARY)), ( - ofBinary(Array[Byte](0.toByte)), // 00000000 - ofBinary(Array[Byte](-1.toByte)), // 11111111 + ofBinary(Array[Byte](0.toByte)), // 00000000 + ofBinary(Array[Byte](-1.toByte)), // 11111111 ofBinary(Array[Byte](0.toByte)), - ofNull(BinaryType.BINARY) - ), + ofNull(BinaryType.BINARY)), ( ofBinary(Array[Byte](127.toByte)), // 01111111 - ofBinary(Array[Byte](-1.toByte)), // 11111111 + ofBinary(Array[Byte](-1.toByte)), // 11111111 ofBinary(Array[Byte](127.toByte)), - ofNull(BinaryType.BINARY) - ), + ofNull(BinaryType.BINARY)), ( ofBinary(Array[Byte](5.toByte, 10.toByte)), ofBinary(Array[Byte](6.toByte)), ofBinary(Array[Byte](5.toByte, 10.toByte)), - ofNull(BinaryType.BINARY) - ), + ofNull(BinaryType.BINARY)), ( ofBinary(Array[Byte](5.toByte, 10.toByte)), ofBinary(Array[Byte](5.toByte, 100.toByte)), ofBinary(Array[Byte](5.toByte, 10.toByte)), - ofNull(BinaryType.BINARY) - ), + ofNull(BinaryType.BINARY)), ( ofBinary(Array[Byte](5.toByte, 10.toByte, 5.toByte)), // 00000101 00001010 00000101 - ofBinary(Array[Byte](5.toByte, -3.toByte)), // 00000101 11111101 + ofBinary(Array[Byte](5.toByte, -3.toByte)), // 00000101 11111101 ofBinary(Array[Byte](5.toByte, 10.toByte, 5.toByte)), - ofNull(BinaryType.BINARY) - ), + ofNull(BinaryType.BINARY)), ( ofBinary(Array[Byte](5.toByte, -25.toByte, 5.toByte)), // 00000101 11100111 00000101 - ofBinary(Array[Byte](5.toByte, -9.toByte)), // 00000101 11110111 + ofBinary(Array[Byte](5.toByte, -9.toByte)), // 00000101 11110111 ofBinary(Array[Byte](5.toByte, -25.toByte, 5.toByte)), - ofNull(BinaryType.BINARY) - ), + ofNull(BinaryType.BINARY)), ( ofBinary(Array[Byte](5.toByte, 10.toByte)), ofBinary(Array[Byte](5.toByte, 10.toByte, 0.toByte)), ofBinary(Array[Byte](5.toByte, 10.toByte)), - ofNull(BinaryType.BINARY) - ), + ofNull(BinaryType.BINARY)), ( ofDecimal(BigDecimalJ.valueOf(1.12), 7, 3), ofDecimal(BigDecimalJ.valueOf(5233.232), 7, 3), ofDecimal(BigDecimalJ.valueOf(1.12), 7, 3), - ofNull(new DecimalType(7, 3)) - ) - ) + ofNull(new DecimalType(7, 3)))) // Mapping of comparator to expected results for: // comparator(small, big) @@ -830,8 +885,7 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa ">" -> Seq(false, true, false, null, null, null), ">=" -> Seq(false, true, true, null, null, null), "=" -> Seq(false, false, true, null, null, null), - "IS NOT DISTINCT FROM" -> Seq(false, false, true, false, false, true) - ) + "IS NOT DISTINCT FROM" -> Seq(false, false, true, false, false, true)) literals.foreach { case (small1, big, small2, nullLit) => @@ -855,25 +909,36 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa ofInt(-234), ofLong(223L), ofFloat(-2423423.9f), - ofNull(DoubleType.DOUBLE) - ) + ofNull(DoubleType.DOUBLE)) test("evaluate expression: substring") { // scalastyle:off nonascii val data = Seq[String]( - null, "one", "two", "three", "four", null, null, "seven", "eight", "😉", "ë") + null, + "one", + "two", + "three", + "four", + null, + null, + "seven", + "eight", + "😉", + "ë") val col = stringVector(data) val col_name = "str_col" val schema = new StructType().add(col_name, StringType.STRING) val input = new DefaultColumnarBatch(col.getSize, schema, Array(col)) def checkSubString( - input: DefaultColumnarBatch, - substringExpression: ScalarExpression, - expOutputSeq: Seq[String]): Unit = { + input: DefaultColumnarBatch, + substringExpression: ScalarExpression, + expOutputSeq: Seq[String]): Unit = { val actOutputVector = new DefaultExpressionEvaluator( - schema, substringExpression, StringType.STRING).eval(input) + schema, + substringExpression, + StringType.STRING).eval(input) val expOutputVector = stringVector(expOutputSeq); checkStringVectors(actOutputVector, expOutputVector) } @@ -1004,15 +1069,18 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa val outputVectorForEmptyInput = evaluator( schema, - new ScalarExpression("SUBSTRING", + new ScalarExpression( + "SUBSTRING", util.Arrays.asList( - new Column(col_name), Literal.ofInt(1), Literal.ofInt(1))), - StringType.STRING - ).eval( new DefaultColumnarBatch(/* size= */0, + new Column(col_name), + Literal.ofInt(1), + Literal.ofInt(1))), + StringType.STRING).eval(new DefaultColumnarBatch( + /* size= */ 0, schema, Array( - testColumnVector(/* size= */0, StringType.STRING), - testColumnVector(/* size= */0, BinaryType.BINARY)))) + testColumnVector( /* size= */ 0, StringType.STRING), + testColumnVector( /* size= */ 0, BinaryType.BINARY)))) checkStringVectors(outputVectorForEmptyInput, stringVector(Seq[String]())) def checkUnsupportedColumnTypes(colType: DataType): Unit = { @@ -1022,10 +1090,10 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa val e = intercept[UnsupportedOperationException] { evaluator( schema, - new ScalarExpression("SUBSTRING", + new ScalarExpression( + "SUBSTRING", util.Arrays.asList(new Column(col_name), Literal.ofInt(1))), - StringType.STRING - ).eval(batch) + StringType.STRING).eval(batch) } assert( e.getMessage.contains("Invalid type of first input of SUBSTRING: expects STRING")) @@ -1039,13 +1107,17 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa val badLiteralSize = intercept[UnsupportedOperationException] { evaluator( schema, - new ScalarExpression("SUBSTRING", + new ScalarExpression( + "SUBSTRING", util.Arrays.asList( - new Column(col_name), Literal.ofInt(1), Literal.ofInt(1), Literal.ofInt(1))), - StringType.STRING - ).eval( new DefaultColumnarBatch(/* size= */5, + new Column(col_name), + Literal.ofInt(1), + Literal.ofInt(1), + Literal.ofInt(1))), + StringType.STRING).eval(new DefaultColumnarBatch( + /* size= */ 5, schema, - Array(testColumnVector(/* size= */5, StringType.STRING)))) + Array(testColumnVector( /* size= */ 5, StringType.STRING)))) } assert( badLiteralSize.getMessage.contains( @@ -1054,26 +1126,31 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa val badPosType = intercept[UnsupportedOperationException] { evaluator( schema, - new ScalarExpression("SUBSTRING", + new ScalarExpression( + "SUBSTRING", util.Arrays.asList( - new Column("str_col"), Literal.ofBoolean(true))), - StringType.STRING - ).eval( new DefaultColumnarBatch(/* size= */5, + new Column("str_col"), + Literal.ofBoolean(true))), + StringType.STRING).eval(new DefaultColumnarBatch( + /* size= */ 5, schema, - Array(testColumnVector(/* size= */5, StringType.STRING)))) + Array(testColumnVector( /* size= */ 5, StringType.STRING)))) } assert(badPosType.getMessage.contains("Invalid `pos` argument type for SUBSTRING")) val badLenType = intercept[UnsupportedOperationException] { evaluator( schema, - new ScalarExpression("SUBSTRING", + new ScalarExpression( + "SUBSTRING", util.Arrays.asList( - new Column(col_name), Literal.ofInt(1), Literal.ofBoolean(true))), - StringType.STRING - ).eval( new DefaultColumnarBatch(/* size= */5, + new Column(col_name), + Literal.ofInt(1), + Literal.ofBoolean(true))), + StringType.STRING).eval(new DefaultColumnarBatch( + /* size= */ 5, schema, - Array(testColumnVector(/* size= */5, StringType.STRING)))) + Array(testColumnVector( /* size= */ 5, StringType.STRING)))) } assert(badLenType.getMessage.contains("Invalid `len` argument type for SUBSTRING")) } @@ -1086,8 +1163,7 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa "<=" -> Seq(true, false, true, false, null), ">" -> Seq(false, true, false, true, null), ">=" -> Seq(false, true, false, true, null), - "=" -> Seq(false, false, false, false, null) - ) + "=" -> Seq(false, false, false, false, null)) // Left operand is first literal in [[literal]] which a byte type // Right operands are the remaining literals to the left side of it in [[literal]] @@ -1108,8 +1184,7 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa "<=" -> Seq(false, true, false, null), ">" -> Seq(true, false, true, null), ">=" -> Seq(true, true, true, null), - "=" -> Seq(false, true, false, null) - ) + "=" -> Seq(false, true, false, null)) // Left operand is first literal in [[literal]] which a short type // Right operands are the remaining literals to the left side of it in [[literal]] @@ -1129,8 +1204,7 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa "<=" -> Seq(true, false, null), ">" -> Seq(false, true, null), ">=" -> Seq(false, true, null), - "=" -> Seq(false, false, null) - ) + "=" -> Seq(false, false, null)) // Left operand is first literal in [[literal]] which a int type // Right operands are the remaining literals to the left side of it in [[literal]] @@ -1150,8 +1224,7 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa "<=" -> Seq(false, null), ">" -> Seq(true, null), ">=" -> Seq(true, null), - "=" -> Seq(false, null) - ) + "=" -> Seq(false, null)) // Left operand is fourth literal in [[literal]] which a long type // Right operands are the remaining literals to the left side of it in [[literal]] @@ -1190,8 +1263,7 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa Map("k0" -> "v00", "k1" -> "v01", "k3" -> nullStr, nullStr -> "v04"), Map("k0" -> "v10", "k1" -> nullStr, "k3" -> "v13", nullStr -> "v14"), Map("k0" -> nullStr, "k1" -> "v21", "k3" -> "v23", nullStr -> "v24"), - null - ) + null) val testMapVector = buildMapVector( testMapValues, new MapType(StringType.STRING, StringType.STRING, true)) @@ -1199,8 +1271,7 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa val inputBatch = new DefaultColumnarBatch( testMapVector.getSize, new StructType().add("partitionValues", testMapVector.getDataType), - Seq(testMapVector).toArray - ) + Seq(testMapVector).toArray) Seq("k0", "k1", "k2", null).foreach { lookupKey => val expOutput = testMapValues.map(map => { if (map == null) null @@ -1284,12 +1355,15 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa ("null", BinaryType.BINARY, null), ("2021-11-18", DateType.DATE, InternalUtils.daysSinceEpoch(Date.valueOf("2021-11-18"))), ("null", DateType.DATE, null), - ("2020-02-18 22:00:10", TimestampType.TIMESTAMP, + ( + "2020-02-18 22:00:10", + TimestampType.TIMESTAMP, InternalUtils.microsSinceEpoch(Timestamp.valueOf("2020-02-18 22:00:10"))), - ("2020-02-18 00:00:10.023", TimestampType.TIMESTAMP, + ( + "2020-02-18 00:00:10.023", + TimestampType.TIMESTAMP, InternalUtils.microsSinceEpoch(Timestamp.valueOf("2020-02-18 00:00:10.023"))), - ("null", TimestampType.TIMESTAMP, null) - ) + ("null", TimestampType.TIMESTAMP, null)) val inputBatch = zeroColumnBatch(rowCount = 1) testCases.foreach { testCase => @@ -1322,12 +1396,15 @@ class DefaultExpressionEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBa } private def evaluator(inputSchema: StructType, expression: Expression, outputType: DataType) - : DefaultExpressionEvaluator = { + : DefaultExpressionEvaluator = { new DefaultExpressionEvaluator(inputSchema, expression, outputType) } private def testComparator( - comparator: String, left: Expression, right: Expression, expResult: BooleanJ): Unit = { + comparator: String, + left: Expression, + right: Expression, + expResult: BooleanJ): Unit = { val expression = new Predicate(comparator, left, right) val batch = zeroColumnBatch(rowCount = 1) val outputVector = evaluator(batch.getSchema, expression, BooleanType.BOOLEAN).eval(batch) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/DefaultPredicateEvaluatorSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/DefaultPredicateEvaluatorSuite.scala index a820c420bb7..41f43c1f3ea 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/DefaultPredicateEvaluatorSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/DefaultPredicateEvaluatorSuite.scala @@ -23,6 +23,7 @@ import io.delta.kernel.data.{ColumnarBatch, ColumnVector} import io.delta.kernel.defaults.internal.data.DefaultColumnarBatch import io.delta.kernel.expressions.{Column, Literal} import io.delta.kernel.types.{BooleanType, StructType} + import org.scalatest.funsuite.AnyFunSuite /** @@ -41,7 +42,9 @@ class DefaultPredicateEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBas .add("right", BooleanType.BOOLEAN) private val batch = new DefaultColumnarBatch( - testLeftCol.getSize, testSchema, Array(testLeftCol, testRightCol)) + testLeftCol.getSize, + testSchema, + Array(testLeftCol, testRightCol)) private val left = comparator("=", new Column("left"), Literal.ofBoolean(true)) private val right = comparator("=", new Column("right"), Literal.ofBoolean(true)) @@ -53,7 +56,9 @@ class DefaultPredicateEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBas test("evaluate predicate: with no starting selection vector") { val batch = new DefaultColumnarBatch( - testLeftCol.getSize, testSchema, Array(testLeftCol, testRightCol)) + testLeftCol.getSize, + testSchema, + Array(testLeftCol, testRightCol)) val actOutputVector = evalOr(batch) checkBooleanVectors(actOutputVector, expOrOutput) @@ -87,7 +92,8 @@ class DefaultPredicateEvaluatorSuite extends AnyFunSuite with ExpressionSuiteBas } def evalOr( - batch: ColumnarBatch, existingSelVector: Optional[ColumnVector] = empty()): ColumnVector = { + batch: ColumnarBatch, + existingSelVector: Optional[ColumnVector] = empty()): ColumnVector = { val evaluator = new DefaultPredicateEvaluator(batch.getSchema, orPredicate) evaluator.eval(batch, existingSelVector) } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/ExpressionSuiteBase.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/ExpressionSuiteBase.scala index d4e3ac4f315..f39a36b44dc 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/ExpressionSuiteBase.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/ExpressionSuiteBase.scala @@ -15,16 +15,17 @@ */ package io.delta.kernel.defaults.internal.expressions -import io.delta.kernel.data.{ColumnVector, ColumnarBatch} +import scala.collection.JavaConverters._ + +import io.delta.kernel.data.{ColumnarBatch, ColumnVector} import io.delta.kernel.defaults.internal.data.DefaultColumnarBatch -import io.delta.kernel.defaults.utils.DefaultKernelTestUtils.getValueAsObject import io.delta.kernel.defaults.utils.{DefaultVectorTestUtils, TestUtils} +import io.delta.kernel.defaults.utils.DefaultKernelTestUtils.getValueAsObject import io.delta.kernel.expressions._ import io.delta.kernel.types._ -import scala.collection.JavaConverters._ - trait ExpressionSuiteBase extends TestUtils with DefaultVectorTestUtils { + /** create a columnar batch of given `size` with zero columns in it. */ protected def zeroColumnBatch(rowCount: Int): ColumnarBatch = { new DefaultColumnarBatch(rowCount, new StructType(), new Array[ColumnVector](0)) @@ -40,15 +41,17 @@ trait ExpressionSuiteBase extends TestUtils with DefaultVectorTestUtils { protected def substring(expr: Expression, pos: Int, len: Option[Int] = None): ScalarExpression = { var children = List(expr, Literal.ofInt(pos)) - if(len.isDefined) { - children = children :+ Literal.ofInt(len.get) + if (len.isDefined) { + children = children :+ Literal.ofInt(len.get) } new ScalarExpression("substring", children.asJava) } protected def like( - left: Expression, right: Expression, escape: Option[Character] = None): Predicate = { - if (escape.isDefined && escape.get!=null) { + left: Expression, + right: Expression, + escape: Option[Character] = None): Predicate = { + if (escape.isDefined && escape.get != null) { like(List(left, right, Literal.ofString(escape.get.toString))) } else like(List(left, right)) } @@ -73,8 +76,7 @@ trait ExpressionSuiteBase extends TestUtils with DefaultVectorTestUtils { if (!actual.isNullAt(rowId)) { assert( actual.getBoolean(rowId) === expected.getBoolean(rowId), - s"unexpected value at $rowId" - ) + s"unexpected value at $rowId") } } } @@ -103,8 +105,7 @@ trait ExpressionSuiteBase extends TestUtils with DefaultVectorTestUtils { actual.getString(rowId) === expected.getString(rowId), s"unexpected value at $rowId: " + s"expected: ${expected.getString(rowId)} " + - s"actual: ${actual.getString(rowId)} " - ) + s"actual: ${actual.getString(rowId)} ") } } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/ImplicitCastExpressionSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/ImplicitCastExpressionSuite.scala index da2bf2e558f..79bc81600c5 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/ImplicitCastExpressionSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/expressions/ImplicitCastExpressionSuite.scala @@ -21,6 +21,7 @@ import io.delta.kernel.defaults.utils.DefaultKernelTestUtils.getValueAsObject import io.delta.kernel.defaults.utils.TestUtils import io.delta.kernel.expressions.Column import io.delta.kernel.types._ + import org.scalatest.funsuite.AnyFunSuite class ImplicitCastExpressionSuite extends AnyFunSuite with TestUtils { @@ -30,16 +31,13 @@ class ImplicitCastExpressionSuite extends AnyFunSuite with TestUtils { (ByteType.BYTE, LongType.LONG), (ByteType.BYTE, FloatType.FLOAT), (ByteType.BYTE, DoubleType.DOUBLE), - (ShortType.SHORT, IntegerType.INTEGER), (ShortType.SHORT, LongType.LONG), (ShortType.SHORT, FloatType.FLOAT), (ShortType.SHORT, DoubleType.DOUBLE), - (IntegerType.INTEGER, LongType.LONG), (IntegerType.INTEGER, FloatType.FLOAT), (IntegerType.INTEGER, DoubleType.DOUBLE), - (LongType.LONG, FloatType.FLOAT), (LongType.LONG, DoubleType.DOUBLE), (FloatType.FLOAT, DoubleType.DOUBLE)) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/json/JsonUtilsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/json/JsonUtilsSuite.scala index 4e2420d034d..66546fdda28 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/json/JsonUtilsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/json/JsonUtilsSuite.scala @@ -15,13 +15,14 @@ */ package io.delta.kernel.defaults.internal.json +import scala.Double.NegativeInfinity +import scala.collection.JavaConverters._ + import io.delta.kernel.defaults.utils.{TestRow, TestUtils} import io.delta.kernel.test.VectorTestUtils import io.delta.kernel.types._ -import org.scalatest.funsuite.AnyFunSuite -import scala.Double.NegativeInfinity -import scala.collection.JavaConverters._ +import org.scalatest.funsuite.AnyFunSuite class JsonUtilsSuite extends AnyFunSuite with TestUtils with VectorTestUtils { @@ -32,64 +33,54 @@ class JsonUtilsSuite extends AnyFunSuite with TestUtils with VectorTestUtils { s"""{"c0":false,"c1":true,"c2":null,"c3":false}""", // test JSON TestRow(false, true, null, false), // expected decoded row // expected row serialized as JSON, null values won't be in output - s"""{"c0":false,"c1":true,"c3":false}""" - ), + s"""{"c0":false,"c1":true,"c3":false}"""), ( ByteType.BYTE, s"""{"c0":${Byte.MinValue},"c1":${Byte.MaxValue},"c2":null,"c3":4}""", TestRow(Byte.MinValue, Byte.MaxValue, null, 4.toByte), - s"""{"c0":${Byte.MinValue},"c1":${Byte.MaxValue},"c3":4}""" - ), + s"""{"c0":${Byte.MinValue},"c1":${Byte.MaxValue},"c3":4}"""), ( ShortType.SHORT, s"""{"c0":${Short.MinValue},"c1":${Short.MaxValue},"c2":null,"c3":44}""", TestRow(Short.MinValue, Short.MaxValue, null, 44.toShort), - s"""{"c0":${Short.MinValue},"c1":${Short.MaxValue},"c3":44}""" - ), + s"""{"c0":${Short.MinValue},"c1":${Short.MaxValue},"c3":44}"""), ( IntegerType.INTEGER, s"""{"c0":${Integer.MIN_VALUE},"c1":${Integer.MAX_VALUE},"c2":null,"c3":423423}""", TestRow(Integer.MIN_VALUE, Integer.MAX_VALUE, null, 423423), - s"""{"c0":${Integer.MIN_VALUE},"c1":${Integer.MAX_VALUE},"c3":423423}""" - ), + s"""{"c0":${Integer.MIN_VALUE},"c1":${Integer.MAX_VALUE},"c3":423423}"""), ( LongType.LONG, s"""{"c0":${Long.MinValue},"c1":${Long.MaxValue},"c2":null,"c3":423423}""", TestRow(Long.MinValue, Long.MaxValue, null, 423423.toLong), - s"""{"c0":${Long.MinValue},"c1":${Long.MaxValue},"c3":423423}""" - ), + s"""{"c0":${Long.MinValue},"c1":${Long.MaxValue},"c3":423423}"""), ( FloatType.FLOAT, s"""{"c0":${Float.MinValue},"c1":${Float.MaxValue},"c2":null,"c3":"${Float.NaN}"}""", TestRow(Float.MinValue, Float.MaxValue, null, Float.NaN), - s"""{"c0":${Float.MinValue},"c1":${Float.MaxValue},"c3":"NaN"}""" - ), + s"""{"c0":${Float.MinValue},"c1":${Float.MaxValue},"c3":"NaN"}"""), ( DoubleType.DOUBLE, s"""{"c0":${Double.MinValue},"c1":${Double.MaxValue},"c2":null,"c3":"${NegativeInfinity}"}""", TestRow(Double.MinValue, Double.MaxValue, null, NegativeInfinity), - s"""{"c0":${Double.MinValue},"c1":${Double.MaxValue},"c3":"-Infinity"}""" - ), + s"""{"c0":${Double.MinValue},"c1":${Double.MaxValue},"c3":"-Infinity"}"""), ( StringType.STRING, s"""{"c0":"","c1":"ssdfsdf","c2":null,"c3":"123sdsd"}""", TestRow("", "ssdfsdf", null, "123sdsd"), - s"""{"c0":"","c1":"ssdfsdf","c3":"123sdsd"}""" - ), + s"""{"c0":"","c1":"ssdfsdf","c3":"123sdsd"}"""), ( - new ArrayType(IntegerType.INTEGER, true /* containsNull */), + new ArrayType(IntegerType.INTEGER, true /* containsNull */ ), """{"c0":[23,23],"c1":[1212,null,2332],"c2":null,"c3":[]}""", TestRow(Seq(23, 23), Seq(1212, null, 2332), null, Seq()), - """{"c0":[23,23],"c1":[1212,null,2332],"c3":[]}""" - ), + """{"c0":[23,23],"c1":[1212,null,2332],"c3":[]}"""), ( // array with complex element types new ArrayType( new StructType() .add("cn0", IntegerType.INTEGER) - .add("cn1", - new ArrayType(LongType.LONG, true /* containsNull */)), - true /* containsNull */), + .add("cn1", new ArrayType(LongType.LONG, true /* containsNull */ )), + true /* containsNull */ ), """{ |"c0":[{"cn0":24,"cn1":[23,232]},{"cn0":25,"cn1":[24,237]}], |"c1":[{"cn0":32,"cn1":[37,null,2323]},{"cn0":29,"cn1":[200,111237]}], @@ -99,15 +90,13 @@ class JsonUtilsSuite extends AnyFunSuite with TestUtils with VectorTestUtils { Seq(TestRow(24, Seq(23L, 232L)), TestRow(25, Seq(24L, 237L))), Seq(TestRow(32, Seq(37L, null, 2323L)), TestRow(29, Seq(200L, 111237L))), null, - Seq() - ), + Seq()), """{ |"c0":[{"cn0":24,"cn1":[23,232]},{"cn0":25,"cn1":[24,237]}], |"c1":[{"cn0":32,"cn1":[37,null,2323]},{"cn0":29,"cn1":[200,111237]}], - |"c3":[]}""".stripMargin - ), + |"c3":[]}""".stripMargin), ( - new MapType(StringType.STRING, IntegerType.INTEGER, true /* valueContainsNull */), + new MapType(StringType.STRING, IntegerType.INTEGER, true /* valueContainsNull */ ), """{ |"c0":{"24":200,"25":201}, |"c1":{"27":null,"25":203}, @@ -118,19 +107,16 @@ class JsonUtilsSuite extends AnyFunSuite with TestUtils with VectorTestUtils { Map("24" -> 200, "25" -> 201), Map("27" -> null, "25" -> 203), null, - Map() - ), + Map()), """{ |"c0":{"24":200,"25":201}, |"c1":{"27":null,"25":203}, |"c3":{} - |}""".stripMargin - ), + |}""".stripMargin), ( new StructType() .add("cn0", IntegerType.INTEGER) - .add("cn1", - new ArrayType(LongType.LONG, true /* containsNull */)), + .add("cn1", new ArrayType(LongType.LONG, true /* containsNull */ )), """{ |"c0":{"cn0":24,"cn1":[23,232]}, |"c1":{"cn0":29,"cn1":[200,null,111237]}, @@ -141,15 +127,12 @@ class JsonUtilsSuite extends AnyFunSuite with TestUtils with VectorTestUtils { TestRow(24, Seq(23L, 232L)), TestRow(29, Seq(200L, null, 111237L)), null, - TestRow(null, null) - ), + TestRow(null, null)), """{ |"c0":{"cn0":24,"cn1":[23,232]}, |"c1":{"cn0":29,"cn1":[200,null,111237]}, |"c3":{} - |}""".stripMargin - ) - ).foreach { case (dataType, testJson, expRow, expJson) => + |}""".stripMargin)).foreach { case (dataType, testJson, expRow, expJson) => test(s"JsonUtils.RowSerializer: $dataType") { val schema = new StructType(Seq.range(0, 4).map(colOrdinal => new StructField(s"c$colOrdinal", dataType, true)).asJava) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/logstore/LogStoreProviderSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/logstore/LogStoreProviderSuite.scala index ec30e653fc8..3b0b059b0d2 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/logstore/LogStoreProviderSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/logstore/LogStoreProviderSuite.scala @@ -16,6 +16,7 @@ package io.delta.kernel.defaults.internal.logstore import io.delta.storage._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.scalatest.funsuite.AnyFunSuite @@ -36,8 +37,7 @@ class LogStoreProviderSuite extends AnyFunSuite { "abfs" -> classOf[AzureLogStore].getName, "adl" -> classOf[AzureLogStore].getName, "wasb" -> classOf[AzureLogStore].getName, - "wasbs" -> classOf[AzureLogStore].getName - ).foreach { case (scheme, logStoreClass) => + "wasbs" -> classOf[AzureLogStore].getName).foreach { case (scheme, logStoreClass) => test(s"get the default LogStore for scheme $scheme") { val logStore = LogStoreProvider.getLogStore(hadoopConf, scheme) assert(logStore.getClass.getName === logStoreClass) @@ -62,8 +62,7 @@ class LogStoreProviderSuite extends AnyFunSuite { val hadoopConf = new Configuration() hadoopConf.set(LogStoreProvider.getLogStoreSchemeConfKey("fake"), "java.lang.String") val e = intercept[IllegalArgumentException]( - LogStoreProvider.getLogStore(hadoopConf, "fake") - ) + LogStoreProvider.getLogStore(hadoopConf, "fake")) assert(e.getMessage.contains( "Can not instantiate `LogStore` class (from config): %s".format("java.lang.String"))) } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetFileReaderSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetFileReaderSuite.scala index 7b90439dc18..764e9dadca3 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetFileReaderSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetFileReaderSuite.scala @@ -22,12 +22,13 @@ import io.delta.golden.GoldenTableUtils.{goldenTableFile, goldenTablePath} import io.delta.kernel.defaults.utils.{ExpressionTestUtils, TestRow} import io.delta.kernel.test.VectorTestUtils import io.delta.kernel.types._ + +import org.apache.parquet.io.ParquetDecodingException import org.apache.spark.sql.internal.SQLConf import org.scalatest.funsuite.AnyFunSuite -import org.apache.parquet.io.ParquetDecodingException class ParquetFileReaderSuite extends AnyFunSuite - with ParquetSuiteBase with VectorTestUtils with ExpressionTestUtils { + with ParquetSuiteBase with VectorTestUtils with ExpressionTestUtils { test("decimals encoded using dictionary encoding ") { // Below golden tables contains three decimal columns @@ -36,7 +37,7 @@ class ParquetFileReaderSuite extends AnyFunSuite val decimalDictFileV2 = goldenTableFile("parquet-decimal-dictionaries-v2").getAbsolutePath val expResult = (0 until 1000000).map { i => - TestRow(i, BigDecimal.valueOf(i%5), BigDecimal.valueOf(i%6), BigDecimal.valueOf(i%2)) + TestRow(i, BigDecimal.valueOf(i % 5), BigDecimal.valueOf(i % 6), BigDecimal.valueOf(i % 2)) } val readSchema = tableSchema(decimalDictFileV1) @@ -61,13 +62,12 @@ class ParquetFileReaderSuite extends AnyFunSuite TestRow(i, n.movePointLeft(1).setScale(1), n.setScale(5), n.setScale(5)) } else { val negation = if (i % 33 == 0) -1 else 1 - val n = BigDecimal.valueOf(i*negation) + val n = BigDecimal.valueOf(i * negation) TestRow( i, n.movePointLeft(1), expand(n).movePointLeft(5), - expand(expand(expand(n))).movePointLeft(5) - ) + expand(expand(expand(n))).movePointLeft(5)) } } @@ -79,16 +79,15 @@ class ParquetFileReaderSuite extends AnyFunSuite } Seq( - "parquet-all-types", - "parquet-all-types-legacy-format" - ).foreach { allTypesTableName => + "parquet-all-types", + "parquet-all-types-legacy-format").foreach { allTypesTableName => test(s"read all types of data - $allTypesTableName") { val allTypesFile = goldenTableFile(allTypesTableName).getAbsolutePath val readSchema = tableSchema(allTypesFile) checkAnswer( readParquetFilesUsingKernel(allTypesFile, readSchema), /* actual */ - readParquetFilesUsingSpark(allTypesFile, readSchema) /* expected */) + readParquetFilesUsingSpark(allTypesFile, readSchema) /* expected */ ) } } @@ -115,27 +114,41 @@ class ParquetFileReaderSuite extends AnyFunSuite TestCase("ShortType", DoubleType.DOUBLE, i => if (i % 56 != 0) i.toDouble else null), TestCase("IntegerType", LongType.LONG, i => if (i % 23 != 0) i.toLong else null), TestCase("IntegerType", DoubleType.DOUBLE, i => if (i % 23 != 0) i.toDouble else null), - - TestCase("FloatType", DoubleType.DOUBLE, + TestCase( + "FloatType", + DoubleType.DOUBLE, i => if (i % 28 != 0) (i * 0.234).toFloat.toDouble else null), - TestCase("decimal", new DecimalType(12, 2), + TestCase( + "decimal", + new DecimalType(12, 2), i => if (i % 67 != 0) java.math.BigDecimal.valueOf(i * 12352, 2) else null), - TestCase("decimal", new DecimalType(12, 4), + TestCase( + "decimal", + new DecimalType(12, 4), i => if (i % 67 != 0) java.math.BigDecimal.valueOf(i * 1235200, 4) else null), - TestCase("decimal", new DecimalType(26, 10), - i => if (i % 67 != 0) java.math.BigDecimal.valueOf(i * 12352, 2).setScale(10) - else null), - TestCase("IntegerType", new DecimalType(10, 0), + TestCase( + "decimal", + new DecimalType(26, 10), + i => + if (i % 67 != 0) java.math.BigDecimal.valueOf(i * 12352, 2).setScale(10) + else null), + TestCase( + "IntegerType", + new DecimalType(10, 0), i => if (i % 23 != 0) new java.math.BigDecimal(i) else null), - TestCase("IntegerType", new DecimalType(16, 4), + TestCase( + "IntegerType", + new DecimalType(16, 4), i => if (i % 23 != 0) new java.math.BigDecimal(i).setScale(4) else null), - TestCase("LongType", new DecimalType(20, 0), + TestCase( + "LongType", + new DecimalType(20, 0), i => if (i % 25 != 0) new java.math.BigDecimal(i + 1) else null), - TestCase("LongType", new DecimalType(28, 6), + TestCase( + "LongType", + new DecimalType(28, 6), i => if (i % 25 != 0) new java.math.BigDecimal(i + 1).setScale(6) else null), - - TestCase("BinaryType", StringType.STRING, i => if (i % 59 != 0) i.toString else null) - ) + TestCase("BinaryType", StringType.STRING, i => if (i % 59 != 0) i.toString else null)) // The following conversions are supported by Kernel but not by Spark with parquet-mr. // TODO: We should properly reject these conversions, a lot of them produce wrong results. @@ -153,27 +166,26 @@ class ParquetFileReaderSuite extends AnyFunSuite TestCase("ByteType", DateType.DATE, i => if (i % 72 != 0) i.toByte.toInt else null), TestCase("ShortType", DateType.DATE, i => if (i % 56 != 0) i else null), TestCase("IntegerType", DateType.DATE, i => if (i % 23 != 0) i else null), - TestCase("StringType", BinaryType.BINARY, i => if (i % 57 != 0) i.toString.getBytes else null) - ) + TestCase("StringType", BinaryType.BINARY, i => if (i % 57 != 0) i.toString.getBytes else null)) for (testCase <- supportedConversions ++ kernelOnlyConversions) - test(s"parquet supported conversion - ${testCase.columnName} -> ${testCase.toType.toString}") { - val inputLocation = goldenTablePath("parquet-all-types") - val readSchema = new StructType().add(testCase.columnName, testCase.toType) - val result = readParquetFilesUsingKernel(inputLocation, readSchema) - val expected = (0 until 200) - .map { i => TestRow(testCase.expectedExpr(i))} - checkAnswer(result, expected) - - if (!kernelOnlyConversions.contains(testCase)) { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - val sparkResult = readParquetFilesUsingSpark(inputLocation, readSchema) - checkAnswer(result, sparkResult) + test(s"parquet supported conversion - ${testCase.columnName} -> ${testCase.toType.toString}") { + val inputLocation = goldenTablePath("parquet-all-types") + val readSchema = new StructType().add(testCase.columnName, testCase.toType) + val result = readParquetFilesUsingKernel(inputLocation, readSchema) + val expected = (0 until 200) + .map { i => TestRow(testCase.expectedExpr(i)) } + checkAnswer(result, expected) + + if (!kernelOnlyConversions.contains(testCase)) { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + val sparkResult = readParquetFilesUsingSpark(inputLocation, readSchema) + checkAnswer(result, sparkResult) + } } } - } - test (s"parquet supported conversion - date -> timestamp_ntz") { + test(s"parquet supported conversion - date -> timestamp_ntz") { val timezones = Seq("UTC", "Iceland", "PST", "America/Los_Angeles", "Etc/GMT+9", "Asia/Beirut", "JST") for (fromTimezone <- timezones; toTimezone <- timezones) { @@ -196,15 +208,24 @@ class ParquetFileReaderSuite extends AnyFunSuite // TODO: Uniformize rejecting unsupported conversions. assert( ex.getMessage.contains("Can not read value") || - ex.getMessage.contains("column with Parquet type") || - ex.getMessage.contains("Unable to create Parquet converter for") || - ex.getMessage.contains("Found Delta type Decimal") || - ex.getMessage.contains("cannot be cast to") - ) + ex.getMessage.contains("column with Parquet type") || + ex.getMessage.contains("Unable to create Parquet converter for") || + ex.getMessage.contains("Found Delta type Decimal") || + ex.getMessage.contains("cannot be cast to")) } - for(column <- Seq("BooleanType", "ByteType", "ShortType", "IntegerType", "LongType", - "FloatType", "DoubleType", "StringType", "BinaryType")) { + for ( + column <- Seq( + "BooleanType", + "ByteType", + "ShortType", + "IntegerType", + "LongType", + "FloatType", + "DoubleType", + "StringType", + "BinaryType") + ) { test(s"parquet unsupported conversion from $column") { val inputLocation = goldenTablePath("parquet-all-types") val supportedTypes = (supportedConversions ++ kernelOnlyConversions) @@ -241,14 +262,16 @@ class ParquetFileReaderSuite extends AnyFunSuite .add("booleanType", BooleanType.BOOLEAN) .add("stringType", StringType.STRING) .add("dateType", DateType.DATE) - .add("nested_struct", new StructType() - .add("aa", StringType.STRING) - .add("ac", new StructType().add("aca", IntegerType.INTEGER))) + .add( + "nested_struct", + new StructType() + .add("aa", StringType.STRING) + .add("ac", new StructType().add("aca", IntegerType.INTEGER))) .add("array_of_prims", new ArrayType(IntegerType.INTEGER, true)) checkAnswer( readParquetFilesUsingKernel(tablePath, readSchema), /* actual */ - readParquetFilesUsingSpark(tablePath, readSchema) /* expected */) + readParquetFilesUsingSpark(tablePath, readSchema) /* expected */ ) } test("read subset of columns with missing columns in file") { @@ -259,13 +282,15 @@ class ParquetFileReaderSuite extends AnyFunSuite .add("missing_column_struct", new StructType().add("ab", IntegerType.INTEGER)) .add("longType", LongType.LONG) .add("missing_column_primitive", DateType.DATE) - .add("nested_struct", new StructType() - .add("aa", StringType.STRING) - .add("ac", new StructType().add("aca", IntegerType.INTEGER))) + .add( + "nested_struct", + new StructType() + .add("aa", StringType.STRING) + .add("ac", new StructType().add("aca", IntegerType.INTEGER))) checkAnswer( readParquetFilesUsingKernel(tablePath, readSchema), /* actual */ - readParquetFilesUsingSpark(tablePath, readSchema) /* expected */) + readParquetFilesUsingSpark(tablePath, readSchema) /* expected */ ) } test("read columns with int96 timestamp_ntz") { @@ -277,8 +302,7 @@ class ParquetFileReaderSuite extends AnyFunSuite .add("time", TimestampNTZType.TIMESTAMP_NTZ) checkAnswer( readParquetFilesUsingKernel(filePath, readSchema), /* actual */ - Seq(TestRow(1, 915181200000000L) /* expected */) - ) + Seq(TestRow(1, 915181200000000L) /* expected */ )) } test("request row indices") { @@ -338,22 +362,23 @@ class ParquetFileReaderSuite extends AnyFunSuite // TODO: not working - separate PR to handle 2-level legacy lists // .add("stringsColumn", new ArrayType(StringType.STRING, true /* containsNull */)) // .add("intSetColumn", new ArrayType(IntegerType.INTEGER, true /* containsNull */)) - .add("intToStringColumn", - new MapType(IntegerType.INTEGER, StringType.STRING, true /* valueContainsNull */)) - // TODO: not working - separate PR to handle 2-level legacy lists - // .add("complexColumn", new MapType( - // IntegerType.INTEGER, - // new ArrayType( - // new StructType() - // .add("nestedIntsColumn", new ArrayType(IntegerType.INTEGER, true /* containsNull */)) - // .add("nestedStringColumn", StringType.STRING) - // .add("stringColumn", StringType.STRING), - // true /* containsNull */), - // true /* valueContainsNull */)) + .add( + "intToStringColumn", + new MapType(IntegerType.INTEGER, StringType.STRING, true /* valueContainsNull */ )) + // TODO: not working - separate PR to handle 2-level legacy lists + // .add("complexColumn", new MapType( + // IntegerType.INTEGER, + // new ArrayType( + // new StructType() + // .add("nestedIntsColumn", new ArrayType(IntegerType.INTEGER, true /* containsNull */)) + // .add("nestedStringColumn", StringType.STRING) + // .add("stringColumn", StringType.STRING), + // true /* containsNull */), + // true /* valueContainsNull */)) assert(parquetFileRowCount(parquetFilePath) === 10) checkAnswer( readParquetFilesUsingKernel(parquetFilePath, readSchema), /* actual */ - readParquetFilesUsingSpark(parquetFilePath, readSchema) /* expected */) + readParquetFilesUsingSpark(parquetFilePath, readSchema) /* expected */ ) } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetFileWriterSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetFileWriterSuite.scala index 13fd7906c84..8d3039ab356 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetFileWriterSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetFileWriterSuite.scala @@ -16,6 +16,7 @@ package io.delta.kernel.defaults.internal.parquet import java.lang.{Double => DoubleJ, Float => FloatJ} + import io.delta.golden.GoldenTableUtils.{goldenTableFile, goldenTablePath} import io.delta.kernel.data.{ColumnarBatch, FilteredColumnarBatch} import io.delta.kernel.defaults.internal.DefaultKernelUtils @@ -23,9 +24,10 @@ import io.delta.kernel.defaults.utils.{DefaultVectorTestUtils, ExpressionTestUti import io.delta.kernel.expressions.{Column, Literal, Predicate} import io.delta.kernel.internal.TableConfig import io.delta.kernel.internal.util.ColumnMapping -import io.delta.kernel.internal.util.ColumnMapping.{ColumnMappingMode, convertToPhysicalSchema} +import io.delta.kernel.internal.util.ColumnMapping.{convertToPhysicalSchema, ColumnMappingMode} import io.delta.kernel.types._ import io.delta.kernel.utils.DataFileStatus + import org.apache.spark.sql.{functions => sparkfn} import org.scalatest.funsuite.AnyFunSuite @@ -54,7 +56,7 @@ import org.scalatest.funsuite.AnyFunSuite * 4.3) verify the stats returned in (3) are correct using the Spark Parquet reader */ class ParquetFileWriterSuite extends AnyFunSuite - with ParquetSuiteBase with DefaultVectorTestUtils with ExpressionTestUtils { + with ParquetSuiteBase with DefaultVectorTestUtils with ExpressionTestUtils { Seq( // Test cases reading and writing all types of data with or without stats collection @@ -84,7 +86,8 @@ class ParquetFileWriterSuite extends AnyFunSuite 99998, /* expected number of rows written to Parquet files */ Option.empty[Predicate], // predicate for filtering what rows to write to parquet files leafLevelPrimitiveColumns( - Seq.empty, tableSchema(goldenTablePath("parquet-decimal-type"))), + Seq.empty, + tableSchema(goldenTablePath("parquet-decimal-type"))), 4 // how many columns have the stats collected from given list above ) }, @@ -165,7 +168,8 @@ class ParquetFileWriterSuite extends AnyFunSuite 3, /* expected number of rows written to Parquet files */ Option.empty[Predicate], // predicate for filtering what rows to write to parquet files leafLevelPrimitiveColumns( - Seq.empty, tableSchema(goldenTablePath("decimal-various-scale-precision"))), + Seq.empty, + tableSchema(goldenTablePath("decimal-various-scale-precision"))), 29 // how many columns have the stats collected from given list above ) }, @@ -182,8 +186,7 @@ class ParquetFileWriterSuite extends AnyFunSuite Seq.empty, // list of columns to collect statistics on 0 // how many columns have the stats collected from given list above ) - } - ).flatten.foreach { + }).flatten.foreach { case (name, input, fileSize, expFileCount, expRowCount, predicate, statsCols, expStatsColCnt) => test(s"$name: targetFileSize=$fileSize, predicate=$predicate") { withTempDir { tempPath => @@ -260,45 +263,73 @@ class ParquetFileWriterSuite extends AnyFunSuite // (float values, double values, exp rowCount in files, exp stats (min, max, nullCount) Seq( ( // no stats collection as NaN is present - Seq(Float.NegativeInfinity, Float.MinValue, -1.0f, - -0.0f, 0.0f, 1.0f, null, Float.MaxValue, Float.PositiveInfinity, Float.NaN), - Seq(Double.NegativeInfinity, Double.MinValue, -1.0d, - -0.0d, 0.0d, 1.0d, null, Double.MaxValue, Double.PositiveInfinity, Double.NaN), + Seq( + Float.NegativeInfinity, + Float.MinValue, + -1.0f, + -0.0f, + 0.0f, + 1.0f, + null, + Float.MaxValue, + Float.PositiveInfinity, + Float.NaN), + Seq( + Double.NegativeInfinity, + Double.MinValue, + -1.0d, + -0.0d, + 0.0d, + 1.0d, + null, + Double.MaxValue, + Double.PositiveInfinity, + Double.NaN), 10, (null, null, null), - (null, null, null) - ), + (null, null, null)), ( // Min and max are infinities - Seq(Float.NegativeInfinity, Float.MinValue, -1.0f, - -0.0f, 0.0f, 1.0f, null, Float.MaxValue, Float.PositiveInfinity), - Seq(Double.NegativeInfinity, Double.MinValue, -1.0d, - -0.0d, 0.0d, 1.0d, null, Double.MaxValue, Double.PositiveInfinity), + Seq( + Float.NegativeInfinity, + Float.MinValue, + -1.0f, + -0.0f, + 0.0f, + 1.0f, + null, + Float.MaxValue, + Float.PositiveInfinity), + Seq( + Double.NegativeInfinity, + Double.MinValue, + -1.0d, + -0.0d, + 0.0d, + 1.0d, + null, + Double.MaxValue, + Double.PositiveInfinity), 9, (Float.NegativeInfinity, Float.PositiveInfinity, 1L), - (Double.NegativeInfinity, Double.PositiveInfinity, 1L) - ), + (Double.NegativeInfinity, Double.PositiveInfinity, 1L)), ( // no infinities or NaN - expect stats collected Seq(Float.MinValue, -1.0f, -0.0f, 0.0f, 1.0f, null, Float.MaxValue), Seq(Double.MinValue, -1.0d, -0.0d, 0.0d, 1.0d, null, Double.MaxValue), 7, (Float.MinValue, Float.MaxValue, 1L), - (Double.MinValue, Double.MaxValue, 1L) - ), + (Double.MinValue, Double.MaxValue, 1L)), ( // Only negative numbers. Max is 0.0 instead of -0.0 to avoid PARQUET-1222 Seq(Float.NegativeInfinity, Float.MinValue, -1.0f, -0.0f, null), Seq(Double.NegativeInfinity, Double.MinValue, -1.0d, -0.0d, null), 5, (Float.NegativeInfinity, 0.0f, 1L), - (Double.NegativeInfinity, 0.0d, 1L) - ), + (Double.NegativeInfinity, 0.0d, 1L)), ( // Only positive numbers. Min is -0.0 instead of 0.0 to avoid PARQUET-1222 Seq(0.0f, 1.0f, null, Float.MaxValue, Float.PositiveInfinity), Seq(0.0d, 1.0d, null, Double.MaxValue, Double.PositiveInfinity), 5, (-0.0f, Float.PositiveInfinity, 1L), - (-0.0d, Double.PositiveInfinity, 1L) - ) - ).foreach { + (-0.0d, Double.PositiveInfinity, 1L))).foreach { case (floats: Seq[FloatJ], doubles: Seq[DoubleJ], expRowCount, expFltStats, expDblStats) => withTempDir { tempPath => val targetDir = tempPath.getAbsolutePath @@ -320,8 +351,7 @@ class ParquetFileWriterSuite extends AnyFunSuite ( Option(stats.getMinValues.get(col(column))).map(_.getValue).orNull, Option(stats.getMaxValues.get(col(column))).map(_.getValue).orNull, - Option(stats.getNullCounts.get(col(column))).orNull - ) + Option(stats.getNullCounts.get(col(column))).orNull) assert(getStats("col_0") === expFltStats) assert(getStats("col_1") === expDblStats) @@ -349,11 +379,11 @@ class ParquetFileWriterSuite extends AnyFunSuite } def verifyStatsUsingSpark( - actualFileDir: String, - actualFileStatuses: Seq[DataFileStatus], - fileDataSchema: StructType, - statsColumns: Seq[Column], - expStatsColCount: Int): Unit = { + actualFileDir: String, + actualFileStatuses: Seq[DataFileStatus], + fileDataSchema: StructType, + statsColumns: Seq[Column], + expStatsColCount: Int): Unit = { val actualStatsOutput = actualFileStatuses .map { fileStatus => @@ -379,8 +409,7 @@ class ParquetFileWriterSuite extends AnyFunSuite sparkfn.col("_metadata.file_size").as("size"), // select file size // select mod time and convert to millis sparkfn.unix_timestamp( - sparkfn.col("_metadata.file_modification_time")).as("modificationTime") - ) + sparkfn.col("_metadata.file_modification_time")).as("modificationTime")) .groupBy("path", "size", "modificationTime") val nullStats = Seq(sparkfn.lit(null), sparkfn.lit(null), sparkfn.lit(null)) @@ -401,7 +430,8 @@ class ParquetFileWriterSuite extends AnyFunSuite sparkfn.min(colName).as("min_" + colName), sparkfn.max(colName).as("max_" + colName), sparkfn.sum(sparkfn.when( - sparkfn.col(colName).isNull, 1).otherwise(0)).as("nullCount_" + colName)) + sparkfn.col(colName).isNull, + 1).otherwise(0)).as("nullCount_" + colName)) } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetReaderPredicatePushdownSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetReaderPredicatePushdownSuite.scala index bc55055f210..056217c6e40 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetReaderPredicatePushdownSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetReaderPredicatePushdownSuite.scala @@ -15,20 +15,21 @@ */ package io.delta.kernel.defaults.internal.parquet +import java.nio.file.Files +import java.sql.Date +import java.util.Optional + import io.delta.golden.GoldenTableUtils.goldenTablePath import io.delta.kernel.defaults.utils.{ExpressionTestUtils, TestRow} -import io.delta.kernel.expressions.Literal.{ofBinary, ofBoolean, ofDate, ofDouble, ofFloat, ofInt, ofLong, ofNull, ofString} -import org.scalatest.BeforeAndAfterAll -import org.scalatest.funsuite.AnyFunSuite import io.delta.kernel.expressions._ +import io.delta.kernel.expressions.Literal.{ofBinary, ofBoolean, ofDate, ofDouble, ofFloat, ofInt, ofLong, ofNull, ofString} import io.delta.kernel.internal.util.InternalUtils.daysSinceEpoch +import io.delta.kernel.test.VectorTestUtils import io.delta.kernel.types.{IntegerType, StructType} -import org.apache.spark.sql.{Row, types => sparktypes} -import java.nio.file.Files -import java.sql.Date -import java.util.Optional -import io.delta.kernel.test.VectorTestUtils +import org.apache.spark.sql.{types => sparktypes, Row} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite class ParquetReaderPredicatePushdownSuite extends AnyFunSuite with BeforeAndAfterAll with ParquetSuiteBase with VectorTestUtils with ExpressionTestUtils { @@ -78,8 +79,7 @@ class ParquetReaderPredicatePushdownSuite extends AnyFunSuite sparktypes.StructField("binaryCol", sparktypes.BinaryType), sparktypes.StructField("truncatedBinaryCol", sparktypes.BinaryType), sparktypes.StructField("booleanCol", sparktypes.BooleanType), - sparktypes.StructField("dateCol", sparktypes.DateType) - ).toArray + sparktypes.StructField("dateCol", sparktypes.DateType)).toArray } // supported data type columns as top level columns @@ -155,8 +155,7 @@ class ParquetReaderPredicatePushdownSuite extends AnyFunSuite // dateCol if (rowGroupIdx == 11) null else if (rowGroupIdx == 0) new Date(rowId * 86400000L) - else (if (rowId % 61 != 0) new Date(rowId * 86400000L) else null) - ) + else (if (rowId % 61 != 0) new Date(rowId * 86400000L) else null)) } Seq.range(rowGroupIdx * 100, (rowGroupIdx + 1) * 100).map { rowId => @@ -165,8 +164,7 @@ class ParquetReaderPredicatePushdownSuite extends AnyFunSuite Seq( Row.fromSeq(values(rowId)), // nested column values rowId // row id to help with the test results verification - ) - ) + )) } } @@ -226,14 +224,17 @@ class ParquetReaderPredicatePushdownSuite extends AnyFunSuite eq(col("nested", "booleanCol"), ofBoolean(true)), // expected row groups // 1 has mix of true/false (included), 10 has all nulls (not included) - Seq(0, 1, 2, 4, 6, 8, 12, 14, 16, 18) - ), + Seq(0, 1, 2, 4, 6, 8, 12, 14, 16, 18)), // filter on date type column ( - lte(col("dateCol"), ofDate( - daysSinceEpoch(new Date(500 * 86400000L /* millis in a day */)))), - lte(col("nested", "dateCol"), ofDate( - daysSinceEpoch(new Date(500 * 86400000L /* millis in a day */)))), + lte( + col("dateCol"), + ofDate( + daysSinceEpoch(new Date(500 * 86400000L /* millis in a day */ )))), + lte( + col("nested", "dateCol"), + ofDate( + daysSinceEpoch(new Date(500 * 86400000L /* millis in a day */ )))), Seq(0, 1, 2, 3, 4, 5) // expected row groups ), // filter on string type column @@ -260,8 +261,7 @@ class ParquetReaderPredicatePushdownSuite extends AnyFunSuite lte(col("truncatedBinaryCol"), ofBinary("%060d".format(600).getBytes)), lte(col("nested", "truncatedBinaryCol"), ofBinary("%060d".format(600).getBytes)), Seq(0, 1, 2, 3, 4, 5, 6) // expected row groups - ) - ).foreach { + )).foreach { // boolean, int32, data, int64, float, double, binary, string // Test table has 20 row groups, each with 100 rows. case (predicateTopLevelCol, predicateNestedCol, expRowGroups) => @@ -370,8 +370,7 @@ class ParquetReaderPredicatePushdownSuite extends AnyFunSuite test("OR support") { val predicate = or( eq(col("intCol"), ofInt(20)), - eq(col("longCol"), ofLong(1600)) - ) + eq(col("longCol"), ofLong(1600))) val actData = readUsingKernel(testParquetTable, predicate) checkAnswer(actData, generateExpData(Seq(0, 15))) } @@ -379,8 +378,7 @@ class ParquetReaderPredicatePushdownSuite extends AnyFunSuite test("one end of the OR is not convertible") { val predicate = or( eq(col("intCol"), ofInt(1599)), - eq(col("nonExistentCol"), ofInt(1600)) - ) + eq(col("nonExistentCol"), ofInt(1600))) assertConvertedFilterIsEmpty(predicate, testParquetTable) val actData = readUsingKernel(testParquetTable, predicate) @@ -391,8 +389,7 @@ class ParquetReaderPredicatePushdownSuite extends AnyFunSuite test("AND support") { val predicate = and( eq(col("intCol"), ofInt(1599)), - eq(col("longCol"), ofLong(1600)) - ) + eq(col("longCol"), ofLong(1600))) val actData = readUsingKernel(testParquetTable, predicate) checkAnswer(actData, generateExpData(Seq(15))) } @@ -400,8 +397,7 @@ class ParquetReaderPredicatePushdownSuite extends AnyFunSuite test("one end of the AND is not convertible") { val predicate = and( eq(col("intCol"), ofInt(1599)), - eq(col("nonExistentCol"), ofInt(1600)) - ) + eq(col("nonExistentCol"), ofInt(1600))) val actData = readUsingKernel(testParquetTable, predicate) checkAnswer(actData, generateExpData(Seq(15))) } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSchemaUtilsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSchemaUtilsSuite.scala index bcafa6ac4da..b5187ce9747 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSchemaUtilsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSchemaUtilsSuite.scala @@ -19,9 +19,10 @@ import io.delta.kernel.defaults.internal.parquet.ParquetSchemaUtils.pruneSchema import io.delta.kernel.defaults.utils.TestUtils import io.delta.kernel.internal.util.ColumnMapping import io.delta.kernel.internal.util.ColumnMapping.PARQUET_FIELD_NESTED_IDS_METADATA_KEY +import io.delta.kernel.types.{ArrayType, DoubleType, FieldMetadata, MapType, StructType} import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types.LongType.LONG -import io.delta.kernel.types.{ArrayType, DoubleType, FieldMetadata, MapType, StructType} + import org.apache.parquet.schema.MessageTypeParser import org.scalatest.funsuite.AnyFunSuite @@ -53,7 +54,8 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { // Delta schema corresponding to the above test [[parquetSchema]] private val testParquetFileDeltaSchema = new StructType() - .add("f0", + .add( + "f0", new StructType() .add("f00", INTEGER, fieldMetadata(2)) .add("f01", LONG, fieldMetadata(3)), @@ -66,12 +68,10 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { .add("key_f0", INTEGER, fieldMetadata(10)) .add("key_f1", INTEGER, fieldMetadata(11)), INTEGER, - false - ), + false), fieldMetadata(7)) .add("f3", DoubleType.DOUBLE, fieldMetadata(13)) - test("id mapping mode - delta reads all columns in the parquet file") { val prunedParquetSchema = pruneSchema(testParquetFileSchema, testParquetFileDeltaSchema) assert(prunedParquetSchema === testParquetFileSchema) @@ -84,8 +84,7 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { "f0", new StructType() .add("f00", INTEGER, fieldMetadata(2)), - fieldMetadata(1) - ) + fieldMetadata(1)) val expectedParquetSchema = MessageTypeParser.parseMessageType( """message fileSchema { @@ -112,8 +111,7 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { new StructType() .add("f00", INTEGER, fieldMetadata(2)) .add("f02", INTEGER, fieldMetadata(15)), - fieldMetadata(1) - ) + fieldMetadata(1)) .add("f4", INTEGER, fieldMetadata(14)) // pruned parquet file schema shouldn't have the column "f4" @@ -165,15 +163,14 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { test("id mapping mode - field id matches but not the column name") { val readDeltaSchema = new StructType() - // physical name in the file is f3, but the same field id + // physical name in the file is f3, but the same field id .add("f3_new", DoubleType.DOUBLE, fieldMetadata(13)) .add( "f0", new StructType() // physical name in the file is f00, but the same field id .add("f00_new", INTEGER, fieldMetadata(2)), - fieldMetadata(1) - ) + fieldMetadata(1)) val expectedParquetSchema = MessageTypeParser.parseMessageType( """message fileSchema { @@ -213,8 +210,7 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { "f0", new StructType() .add("f00", INTEGER, fieldMetadata(2)), - fieldMetadata(1) - ) + fieldMetadata(1)) val testParquetFileSchema = MessageTypeParser.parseMessageType( """message fileSchema { @@ -240,10 +236,13 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { "struct with array and map", // Delta schema - input new StructType() - .add("f0", + .add( + "f0", new StructType() .add("f00", new ArrayType(LONG, false), fieldMetadata(2, ("f00.element", 3))) - .add("f01", new MapType(INTEGER, INTEGER, true), + .add( + "f01", + new MapType(INTEGER, INTEGER, true), fieldMetadata(4, ("f01.key", 5), ("f01.value", 6))), fieldMetadata(1)), // Expected parquet schema @@ -262,23 +261,20 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { | } | } | } - |}""".stripMargin) - ), + |}""".stripMargin)), ( "top-level array and map columns", // Delta schema - input new StructType() - .add("f1", - new ArrayType(INTEGER, true), - fieldMetadata(1, ("f1.element", 2))) - .add("f2", + .add("f1", new ArrayType(INTEGER, true), fieldMetadata(1, ("f1.element", 2))) + .add( + "f2", new MapType( new StructType() .add("key_f0", INTEGER, fieldMetadata(6)) .add("key_f1", INTEGER, fieldMetadata(7)), INTEGER, - true - ), + true), fieldMetadata(3, ("f2.key", 4), ("f2.value", 5))), // Expected parquet schema MessageTypeParser.parseMessageType("""message DefaultKernelSchema { @@ -296,29 +292,27 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { | optional int32 value = 5; | } | } - |}""".stripMargin) - ), + |}""".stripMargin)), ( "array/map inside array/map", // Delta schema - input new StructType() - .add("f3", + .add( + "f3", new ArrayType(new ArrayType(INTEGER, false), false), fieldMetadata(0, ("f3.element", 1), ("f3.element.element", 2))) - .add("f4", + .add( + "f4", new MapType( new MapType( new StructType() .add("key_f0", INTEGER, fieldMetadata(3)) .add("key_f1", INTEGER, fieldMetadata(4)), INTEGER, - false - ), + false), INTEGER, - false - ), - fieldMetadata(5, - ("f4.key", 6), ("f4.value", 7), ("f4.key.key", 8), ("f4.key.value", 9))), + false), + fieldMetadata(5, ("f4.key", 6), ("f4.value", 7), ("f4.key.key", 8), ("f4.key.value", 9))), // Expected parquet schema MessageTypeParser.parseMessageType("""message DefaultKernelSchema { | optional group f3 (LIST) = 0 { @@ -344,9 +338,7 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { | required int32 value = 7; | } | } - |}""".stripMargin) - ) - ).foreach { case (testName, deltaSchema, expectedParquetSchema) => + |}""".stripMargin))).foreach { case (testName, deltaSchema, expectedParquetSchema) => test(s"icebergCompatV2 - nested fields are converted to parquet schema - $testName") { val actParquetSchema = ParquetSchemaUtils.toParquetSchema(deltaSchema) assert(actParquetSchema === expectedParquetSchema) @@ -358,96 +350,96 @@ class ParquetSchemaUtilsSuite extends AnyFunSuite with TestUtils { "field id validation: no negative field id", // Delta schema - input new StructType() - .add("f0", + .add( + "f0", new StructType() .add("f00", new ArrayType(LONG, false), fieldMetadata(-1)) .add("f01", new MapType(INTEGER, INTEGER, true), fieldMetadata(4)), fieldMetadata(1)), // Expected error message - "Field id should be non-negative." - ), + "Field id should be non-negative."), ( "field id validation: no negative nested field id", // Delta schema - input new StructType() - .add("f0", + .add( + "f0", new StructType() .add("f00", new ArrayType(LONG, false), fieldMetadata(1, ("f00.element", -1))) .add("f01", new MapType(INTEGER, INTEGER, true), fieldMetadata(4)), fieldMetadata(0)), // Expected error message - "Field id should be non-negative." - ), + "Field id should be non-negative."), ( "field id validation: no duplicate field id", // Delta schema - input new StructType() - .add("f0", + .add( + "f0", new StructType() .add("f00", new ArrayType(LONG, false), fieldMetadata(1, ("f00.element", 1))) .add("f01", new MapType(INTEGER, INTEGER, true), fieldMetadata(4)), fieldMetadata(1)), // Expected error message - "Field id should be unique." - ), + "Field id should be unique."), ( "field id validation: no duplicate nested field id", // Delta schema - input new StructType() - .add("f0", + .add( + "f0", new StructType() .add("f00", new ArrayType(LONG, false), fieldMetadata(1, ("f00.element", 2))) .add("f01", new MapType(INTEGER, INTEGER, true), fieldMetadata(2)), fieldMetadata(1)), // Expected error message - "Field id should be unique." - ), + "Field id should be unique."), ( "field id validation: missing field ids", // Delta schema - input new StructType() - .add("f0", + .add( + "f0", new StructType() .add("f00", new ArrayType(LONG, false)) .add("f01", new MapType(INTEGER, INTEGER, true), fieldMetadata(4)), fieldMetadata(1)), // Expected error message - "Some of the fields are missing field ids." - ), + "Some of the fields are missing field ids."), ( "field id validation: missing nested field ids", // Delta schema - input new StructType() - .add("f0", + .add( + "f0", new StructType() .add("f00", new ArrayType(LONG, false), fieldMetadata(1, ("f00.element", 2))) .add("f01", new MapType(INTEGER, INTEGER, true), fieldMetadata(4)), // missing nested id fieldMetadata(0)), // Expected error message - "Some of the fields are missing field ids." - ), + "Some of the fields are missing field ids."), ( "field id validation: missing field ids but have nested fields", // Delta schema - input new StructType() - .add("f0", + .add( + "f0", new StructType() .add("f00", new ArrayType(LONG, false), fieldMetadata(1, ("f00.element", 2))) .add("f01", new MapType(INTEGER, INTEGER, true), fieldMetadata(4, ("f01.key", 5))) ), // missing field id for f0 // Expected error message - "Some of the fields are missing field ids." - ) - ).foreach { case (testName, deltaSchema, expectedErrorMsg) => - test(testName) { - val ex = intercept[IllegalArgumentException] { - ParquetSchemaUtils.toParquetSchema(deltaSchema) + "Some of the fields are missing field ids.")).foreach { + case (testName, deltaSchema, expectedErrorMsg) => + test(testName) { + val ex = intercept[IllegalArgumentException] { + ParquetSchemaUtils.toParquetSchema(deltaSchema) + } + assert(ex.getMessage.contains(expectedErrorMsg)) } - assert(ex.getMessage.contains(expectedErrorMsg)) - } } - private def fieldMetadata(id: Int, nestedFieldIds: (String, Int) *): FieldMetadata = { + private def fieldMetadata(id: Int, nestedFieldIds: (String, Int)*): FieldMetadata = { val builder = FieldMetadata.builder().putLong(ColumnMapping.PARQUET_FIELD_ID_KEY, id) val nestedFiledMetadata = FieldMetadata.builder() diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSuiteBase.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSuiteBase.scala index bc50b6797dd..958d4f721ad 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSuiteBase.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSuiteBase.scala @@ -17,8 +17,10 @@ package io.delta.kernel.defaults.internal.parquet import java.nio.file.{Files, Paths} import java.util.Optional + import scala.collection.JavaConverters._ import scala.util.control.NonFatal + import io.delta.kernel.data.{ColumnarBatch, FilteredColumnarBatch} import io.delta.kernel.defaults.utils.{TestRow, TestUtils} import io.delta.kernel.expressions.{Column, Predicate} @@ -26,6 +28,7 @@ import io.delta.kernel.internal.util.ColumnMapping import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.types.{ArrayType, DataType, MapType, StructField, StructType} import io.delta.kernel.utils.{DataFileStatus, FileStatus} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.metadata.{ColumnPath, ParquetMetadata} @@ -33,6 +36,7 @@ import org.apache.parquet.hadoop.metadata.{ColumnPath, ParquetMetadata} trait ParquetSuiteBase extends TestUtils { implicit class DataFileStatusOps(dataFileStatus: DataFileStatus) { + /** * Convert the [[DataFileStatus]] to a [[TestRow]]. * (path, size, modification time, numRecords, @@ -54,8 +58,7 @@ trait ParquetSuiteBase extends TestUtils { Seq( Option(stats.getMinValues.get(column)).map(_.getValue).orNull, Option(stats.getMaxValues.get(column)).map(_.getValue).orNull, - Option(stats.getNullCounts.get(column)).orNull - ) + Option(stats.getNullCounts.get(column)).orNull) } else { Seq(null, null, null) } @@ -92,8 +95,8 @@ trait ParquetSuiteBase extends TestUtils { * Use Kernel Parquet reader to read the data from the Parquet files. */ def verifyContentUsingKernelReader( - actualFileDir: String, - expected: Seq[FilteredColumnarBatch]): Unit = { + actualFileDir: String, + expected: Seq[FilteredColumnarBatch]): Unit = { val dataSchema = expected.head.getData.getSchema @@ -112,8 +115,8 @@ trait ParquetSuiteBase extends TestUtils { * Use Spark Parquet reader to read the data from the Parquet files. */ def verifyContentUsingSparkReader( - actualFileDir: String, - expected: Seq[FilteredColumnarBatch]): Unit = { + actualFileDir: String, + expected: Seq[FilteredColumnarBatch]): Unit = { val dataSchema = expected.head.getData.getSchema; @@ -185,8 +188,7 @@ trait ParquetSuiteBase extends TestUtils { verifyNestedFieldId( nearestAncestorStructField, relativePathToNearestAncestor, - elemPathInParquet - ) + elemPathInParquet) } visitDeltaType( elemPathInParquet, @@ -204,13 +206,11 @@ trait ParquetSuiteBase extends TestUtils { verifyNestedFieldId( nearestAncestorStructField, keyRelativePathToNearestAncestor, - keyPathInParquet - ) + keyPathInParquet) verifyNestedFieldId( nearestAncestorStructField, valueRelativePathToNearestAncestor, - valuePathInParquet - ) + valuePathInParquet) } visitDeltaType( @@ -253,14 +253,16 @@ trait ParquetSuiteBase extends TestUtils { * verify the data using the Kernel Parquet reader and Spark Parquet reader. */ def writeToParquetUsingKernel( - filteredData: Seq[FilteredColumnarBatch], - location: String, - targetFileSize: Long = 1024 * 1024, - statsColumns: Seq[Column] = Seq.empty): Seq[DataFileStatus] = { + filteredData: Seq[FilteredColumnarBatch], + location: String, + targetFileSize: Long = 1024 * 1024, + statsColumns: Seq[Column] = Seq.empty): Seq[DataFileStatus] = { val conf = new Configuration(configuration); conf.setLong(ParquetFileWriter.TARGET_FILE_SIZE_CONF, targetFileSize) val parquetWriter = new ParquetFileWriter( - conf, new Path(location), statsColumns.asJava) + conf, + new Path(location), + statsColumns.asJava) parquetWriter.write(toCloseableIterator(filteredData.asJava.iterator())).toSeq } @@ -326,7 +328,8 @@ trait ParquetSuiteBase extends TestUtils { // Read the parquet files in actionFileDir using Spark Parquet reader def readParquetFilesUsingSpark( - actualFileDir: String, readSchema: StructType): Seq[TestRow] = { + actualFileDir: String, + readSchema: StructType): Seq[TestRow] = { spark.read .format("parquet") .parquet(actualFileDir) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/MetricsReportTestUtils.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/MetricsReportTestUtils.scala index 1f115a56a49..5bb6a14732a 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/MetricsReportTestUtils.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/MetricsReportTestUtils.scala @@ -23,6 +23,7 @@ import io.delta.kernel.defaults.engine.DefaultEngine import io.delta.kernel.defaults.utils.TestUtils import io.delta.kernel.engine._ import io.delta.kernel.metrics.MetricsReport + import org.apache.hadoop.conf.Configuration /** @@ -48,13 +49,13 @@ trait MetricsReportTestUtils extends TestUtils { * If [[expectException]], catches any exception thrown by [[f]] and returns it with the reports. */ def collectMetricsReports( - f: Engine => Unit, expectException: Boolean): (Seq[MetricsReport], Option[Exception]) = { + f: Engine => Unit, + expectException: Boolean): (Seq[MetricsReport], Option[Exception]) = { // Initialize a buffer for any metric reports and wrap the engine so that they are recorded val reports = ArrayBuffer.empty[MetricsReport] if (expectException) { val e = intercept[Exception]( - f(new EngineWithInMemoryMetricsReporter(reports, defaultEngine)) - ) + f(new EngineWithInMemoryMetricsReporter(reports, defaultEngine))) (reports, Some(e)) } else { f(new EngineWithInMemoryMetricsReporter(reports, defaultEngine)) @@ -67,16 +68,18 @@ trait MetricsReportTestUtils extends TestUtils { * to the provided in memory buffer. */ class EngineWithInMemoryMetricsReporter(buf: ArrayBuffer[MetricsReport], baseEngine: Engine) - extends Engine { + extends Engine { private val inMemoryMetricsReporter = new MetricsReporter { override def report(report: MetricsReport): Unit = buf.append(report) } - private val metricsReporters = new util.ArrayList[MetricsReporter]() {{ - addAll(baseEngine.getMetricsReporters) - add(inMemoryMetricsReporter) - }} + private val metricsReporters = new util.ArrayList[MetricsReporter]() { + { + addAll(baseEngine.getMetricsReporters) + add(inMemoryMetricsReporter) + } + } override def getExpressionHandler: ExpressionHandler = baseEngine.getExpressionHandler diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/ScanReportSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/ScanReportSuite.scala index 30126936b05..fc44c8a7f8f 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/ScanReportSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/ScanReportSuite.scala @@ -29,13 +29,13 @@ import io.delta.kernel.metrics.{ScanReport, SnapshotReport} import io.delta.kernel.types.{IntegerType, LongType, StructType} import io.delta.kernel.utils.CloseableIterator -import org.apache.spark.sql.functions.col -import org.scalatest.funsuite.AnyFunSuite - import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.stats.StatisticsCollection +import org.apache.spark.sql.functions.col +import org.scalatest.funsuite.AnyFunSuite + class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { /** @@ -49,10 +49,10 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { * @return (ScanReport, durationToConsumeScanFiles, SnapshotReport, ExceptionIfThrown) */ def getScanAndSnapshotReport( - getScan: Engine => Scan, - expectException: Boolean, - consumeScanFiles: CloseableIterator[FilteredColumnarBatch] => Unit - ): (ScanReport, Long, SnapshotReport, Option[Exception]) = { + getScan: Engine => Scan, + expectException: Boolean, + consumeScanFiles: CloseableIterator[FilteredColumnarBatch] => Unit) + : (ScanReport, Long, SnapshotReport, Option[Exception]) = { val timer = new Timer() val (metricsReports, exception) = collectMetricsReports( @@ -61,15 +61,17 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { // Time the actual operation timer.timeCallable(() => consumeScanFiles(scan.getScanFiles(engine))) }, - expectException - ) + expectException) val scanReports = metricsReports.filter(_.isInstanceOf[ScanReport]) assert(scanReports.length == 1, "Expected exactly 1 ScanReport") val snapshotReports = metricsReports.filter(_.isInstanceOf[SnapshotReport]) assert(snapshotReports.length == 1, "Expected exactly 1 SnapshotReport") - (scanReports.head.asInstanceOf[ScanReport], timer.totalDurationNs(), - snapshotReports.head.asInstanceOf[SnapshotReport], exception) + ( + scanReports.head.asInstanceOf[ScanReport], + timer.totalDurationNs(), + snapshotReports.head.asInstanceOf[SnapshotReport], + exception) } /** @@ -92,21 +94,21 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { */ // scalastyle:off def checkScanReport( - path: String, - expectException: Boolean, - expectedNumAddFiles: Long, - expectedNumAddFilesFromDeltaFiles: Long, - expectedNumActiveAddFiles: Long, - expectedNumDuplicateAddFiles: Long = 0, - expectedNumRemoveFilesSeenFromDeltaFiles: Long = 0, - expectedPartitionPredicate: Option[Predicate] = None, - expectedDataSkippingFilter: Option[Predicate] = None, - expectedIsFullyConsumed: Boolean = true, - filter: Option[Predicate] = None, - readSchema: Option[StructType] = None, - // toSeq triggers log replay, consumes the actions and closes the iterator - consumeScanFiles: CloseableIterator[FilteredColumnarBatch] => Unit = iter => iter.toSeq - ): Unit = { + path: String, + expectException: Boolean, + expectedNumAddFiles: Long, + expectedNumAddFilesFromDeltaFiles: Long, + expectedNumActiveAddFiles: Long, + expectedNumDuplicateAddFiles: Long = 0, + expectedNumRemoveFilesSeenFromDeltaFiles: Long = 0, + expectedPartitionPredicate: Option[Predicate] = None, + expectedDataSkippingFilter: Option[Predicate] = None, + expectedIsFullyConsumed: Boolean = true, + filter: Option[Predicate] = None, + readSchema: Option[StructType] = None, + // toSeq triggers log replay, consumes the actions and closes the iterator + consumeScanFiles: CloseableIterator[FilteredColumnarBatch] => Unit = + iter => iter.toSeq): Unit = { // scalastyle:on // We need to save the snapshotSchema to check against the generated scan report // In order to use the utils to collect the reports, we need to generate the snapshot in a anon @@ -127,8 +129,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { scanBuilder.build() }, expectException, - consumeScanFiles - ) + consumeScanFiles) // Verify contents assert(scanReport.getTablePath == defaultEngine.getFileSystemClient.resolvePath(path)) @@ -142,7 +143,8 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { } assert(scanReport.getReportUUID != null) - assert(snapshotReport.getVersion.isPresent, + assert( + snapshotReport.getVersion.isPresent, "Version should be present for success SnapshotReport") assert(scanReport.getTableVersion() == snapshotReport.getVersion.get()) assert(scanReport.getTableSchema() == snapshotSchema) @@ -184,8 +186,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectException = false, expectedNumAddFiles = 1, expectedNumAddFilesFromDeltaFiles = 1, - expectedNumActiveAddFiles = 1 - ) + expectedNumActiveAddFiles = 1) } } @@ -203,8 +204,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumAddFiles = 1, expectedNumAddFilesFromDeltaFiles = 1, expectedNumActiveAddFiles = 1, - readSchema = Some(new StructType().add("id", LongType.LONG)) - ) + readSchema = Some(new StructType().add("id", LongType.LONG))) } } @@ -219,7 +219,9 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { val partFilter = new Predicate("=", new Column("part"), Literal.ofLong(0)) val dataFilter = new Predicate("<=", new Column("id"), Literal.ofLong(0)) val expectedSkippingFilter = new Predicate( - "<=", new Column(Array("minValues", "id")), Literal.ofLong(0)) + "<=", + new Column(Array("minValues", "id")), + Literal.ofLong(0)) // The below metrics are incremented during log replay before any filtering happens and thus // should be the same for all of the following test cases @@ -233,8 +235,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectException = false, expectedNumAddFiles = expectedNumAddFiles, expectedNumAddFilesFromDeltaFiles = expectedNumAddFilesFromDeltaFiles, - expectedNumActiveAddFiles = expectedNumActiveAddFiles - ) + expectedNumActiveAddFiles = expectedNumActiveAddFiles) // With partition filter checkScanReport( @@ -244,8 +245,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumAddFilesFromDeltaFiles = expectedNumAddFilesFromDeltaFiles, expectedNumActiveAddFiles = expectedNumActiveAddFiles, filter = Some(partFilter), - expectedPartitionPredicate = Some(partFilter) - ) + expectedPartitionPredicate = Some(partFilter)) // With data filter checkScanReport( @@ -255,8 +255,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumAddFilesFromDeltaFiles = expectedNumAddFilesFromDeltaFiles, expectedNumActiveAddFiles = expectedNumActiveAddFiles, filter = Some(dataFilter), - expectedDataSkippingFilter = Some(expectedSkippingFilter) - ) + expectedDataSkippingFilter = Some(expectedSkippingFilter)) // With data and partition filter checkScanReport( @@ -267,8 +266,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumActiveAddFiles = expectedNumActiveAddFiles, filter = Some(new Predicate("AND", partFilter, dataFilter)), expectedDataSkippingFilter = Some(expectedSkippingFilter), - expectedPartitionPredicate = Some(partFilter) - ) + expectedPartitionPredicate = Some(partFilter)) } } @@ -317,8 +315,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumActiveAddFiles = 0, expectedIsFullyConsumed = false, filter = Some(partFilter), - expectedPartitionPredicate = Some(partFilter) - ) + expectedPartitionPredicate = Some(partFilter)) } } @@ -340,13 +337,11 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { // Overwrite json file with giberish (this will have a schema mismatch issue for `add`) val giberishRow = new GenericRow( new StructType().add("add", IntegerType.INTEGER), - Collections.singletonMap(0, Integer.valueOf(0)) - ) + Collections.singletonMap(0, Integer.valueOf(0))) defaultEngine.getJsonHandler.writeJsonFileAtomically( FileNames.deltaFile(new Path(tempDir.toString, "_delta_log"), 0), Utils.singletonCloseableIterator(giberishRow), - true - ) + true) checkScanReport( path, @@ -354,8 +349,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumAddFiles = 0, expectedNumAddFilesFromDeltaFiles = 0, expectedNumActiveAddFiles = 0, - expectedIsFullyConsumed = false - ) + expectedIsFullyConsumed = false) } } @@ -375,8 +369,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectException = false, expectedNumAddFiles = 20, // each commit creates 2 files expectedNumAddFilesFromDeltaFiles = 20, - expectedNumActiveAddFiles = 20 - ) + expectedNumActiveAddFiles = 20) } } @@ -398,8 +391,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectException = false, expectedNumAddFiles = 20, // each commit creates 2 files expectedNumAddFilesFromDeltaFiles = 12, // checkpoint is created at version 3 - expectedNumActiveAddFiles = 20 - ) + expectedNumActiveAddFiles = 20) } } } @@ -427,8 +419,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumAddFiles = 5 /* checkpoint */ + 8, /* delta */ expectedNumAddFilesFromDeltaFiles = 8, expectedNumActiveAddFiles = 10, - expectedNumRemoveFilesSeenFromDeltaFiles = 3 - ) + expectedNumRemoveFilesSeenFromDeltaFiles = 3) } } } @@ -458,8 +449,7 @@ class ScanReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumAddFilesFromDeltaFiles = 18, expectedNumActiveAddFiles = 7, expectedNumDuplicateAddFiles = 12, - expectedNumRemoveFilesSeenFromDeltaFiles = 2 - ) + expectedNumRemoveFilesSeenFromDeltaFiles = 2) } } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/SnapshotReportSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/SnapshotReportSuite.scala index 43d4e70afe8..29d2e163980 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/SnapshotReportSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/SnapshotReportSuite.scala @@ -25,6 +25,7 @@ import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.metrics.Timer import io.delta.kernel.internal.util.FileNames import io.delta.kernel.metrics.SnapshotReport + import org.scalatest.funsuite.AnyFunSuite class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { @@ -41,10 +42,9 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { * @returns (SnapshotReport, durationToRunF, ExceptionIfThrown) */ def getSnapshotReport( - f: (Table, Engine) => Snapshot, - path: String, - expectException: Boolean - ): (SnapshotReport, Long, Option[Exception]) = { + f: (Table, Engine) => Snapshot, + path: String, + expectException: Boolean): (SnapshotReport, Long, Option[Exception]) = { val timer = new Timer() val (metricsReports, exception) = collectMetricsReports( @@ -52,8 +52,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { val table = Table.forPath(engine, path) timer.time(() => f(table, engine)) // Time the actual operation }, - expectException - ) + expectException) val snapshotReports = metricsReports.filter(_.isInstanceOf[SnapshotReport]) assert(snapshotReports.length == 1, "Expected exactly 1 SnapshotReport") @@ -81,14 +80,13 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { * exception is thrown before log replay) */ def checkSnapshotReport( - f: (Table, Engine) => Snapshot, - path: String, - expectException: Boolean, - expectedVersion: Optional[Long], - expectedProvidedTimestamp: Optional[Long], - expectNonEmptyTimestampToVersionResolutionDuration: Boolean, - expectNonZeroLoadProtocolAndMetadataDuration: Boolean - ): Unit = { + f: (Table, Engine) => Snapshot, + path: String, + expectException: Boolean, + expectedVersion: Optional[Long], + expectedProvidedTimestamp: Optional[Long], + expectNonEmptyTimestampToVersionResolutionDuration: Boolean, + expectNonZeroLoadProtocolAndMetadataDuration: Boolean): Unit = { val (snapshotReport, duration, exception) = getSnapshotReport(f, path, expectException) @@ -102,7 +100,8 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { case None => assert(!snapshotReport.getException().isPresent) } assert(snapshotReport.getReportUUID != null) - assert(Objects.equals(snapshotReport.getVersion, expectedVersion), + assert( + Objects.equals(snapshotReport.getVersion, expectedVersion), s"Expected version $expectedVersion found ${snapshotReport.getVersion}") assert(Objects.equals(snapshotReport.getProvidedTimestamp, expectedProvidedTimestamp)) @@ -144,8 +143,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.of(1), expectedProvidedTimestamp = Optional.empty(), // No time travel by timestamp expectNonEmptyTimestampToVersionResolutionDuration = false, // No time travel by timestamp - expectNonZeroLoadProtocolAndMetadataDuration = true - ) + expectNonZeroLoadProtocolAndMetadataDuration = true) // Test getSnapshotAsOfVersion checkSnapshotReport( @@ -155,8 +153,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.of(0), expectedProvidedTimestamp = Optional.empty(), // No time travel by timestamp expectNonEmptyTimestampToVersionResolutionDuration = false, // No time travel by timestamp - expectNonZeroLoadProtocolAndMetadataDuration = true - ) + expectNonZeroLoadProtocolAndMetadataDuration = true) // Test getSnapshotAsOfTimestamp checkSnapshotReport( @@ -166,8 +163,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.of(0), expectedProvidedTimestamp = Optional.of(version0timestamp), expectNonEmptyTimestampToVersionResolutionDuration = true, - expectNonZeroLoadProtocolAndMetadataDuration = true - ) + expectNonZeroLoadProtocolAndMetadataDuration = true) } } @@ -187,8 +183,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.of(1), expectedProvidedTimestamp = Optional.empty(), // No time travel by timestamp expectNonEmptyTimestampToVersionResolutionDuration = false, // No time travel by timestamp - expectNonZeroLoadProtocolAndMetadataDuration = false - ) + expectNonZeroLoadProtocolAndMetadataDuration = false) // Test getSnapshotAsOfTimestamp with timestamp=0 (does not exist) // This fails during timestamp -> version resolution @@ -199,8 +194,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.empty(), expectedProvidedTimestamp = Optional.of(0), expectNonEmptyTimestampToVersionResolutionDuration = true, - expectNonZeroLoadProtocolAndMetadataDuration = false - ) + expectNonZeroLoadProtocolAndMetadataDuration = false) // Test getSnapshotAsOfTimestamp with timestamp=currentTime (does not exist) // This fails during timestamp -> version resolution @@ -212,8 +206,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.empty(), expectedProvidedTimestamp = Optional.of(currentTimeMillis), expectNonEmptyTimestampToVersionResolutionDuration = true, - expectNonZeroLoadProtocolAndMetadataDuration = false - ) + expectNonZeroLoadProtocolAndMetadataDuration = false) } } @@ -230,8 +223,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.empty(), expectedProvidedTimestamp = Optional.empty(), // No time travel by timestamp expectNonEmptyTimestampToVersionResolutionDuration = false, // No time travel by timestamp - expectNonZeroLoadProtocolAndMetadataDuration = false - ) + expectNonZeroLoadProtocolAndMetadataDuration = false) // Test getSnapshotAsOfVersion checkSnapshotReport( @@ -241,8 +233,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.of(0), expectedProvidedTimestamp = Optional.empty(), // No time travel by timestamp expectNonEmptyTimestampToVersionResolutionDuration = false, // No time travel by timestamp - expectNonZeroLoadProtocolAndMetadataDuration = false - ) + expectNonZeroLoadProtocolAndMetadataDuration = false) // Test getSnapshotAsOfTimestamp checkSnapshotReport( @@ -252,8 +243,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.empty(), expectedProvidedTimestamp = Optional.of(1000), expectNonEmptyTimestampToVersionResolutionDuration = true, - expectNonZeroLoadProtocolAndMetadataDuration = false - ) + expectNonZeroLoadProtocolAndMetadataDuration = false) } } @@ -262,7 +252,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { val path = tempDir.getCanonicalPath // Set up table with non-contiguous version (0, 2) which will fail during log segment building // for all the following queries - (0 until 3).foreach( _ => + (0 until 3).foreach(_ => spark.range(3).write.format("delta").mode("append").save(path)) assert( new File(FileNames.deltaFile(new Path(tempDir.getCanonicalPath, "_delta_log"), 1)).delete()) @@ -275,8 +265,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.empty(), expectedProvidedTimestamp = Optional.empty(), // No time travel by timestamp expectNonEmptyTimestampToVersionResolutionDuration = false, // No time travel by timestamp - expectNonZeroLoadProtocolAndMetadataDuration = false - ) + expectNonZeroLoadProtocolAndMetadataDuration = false) // Test getSnapshotAsOfVersion checkSnapshotReport( @@ -286,8 +275,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.of(2), expectedProvidedTimestamp = Optional.empty(), // No time travel by timestamp expectNonEmptyTimestampToVersionResolutionDuration = false, // No time travel by timestamp - expectNonZeroLoadProtocolAndMetadataDuration = false - ) + expectNonZeroLoadProtocolAndMetadataDuration = false) // Test getSnapshotAsOfTimestamp val version2Timestamp = new File( @@ -299,8 +287,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.of(2), expectedProvidedTimestamp = Optional.of(version2Timestamp), expectNonEmptyTimestampToVersionResolutionDuration = true, - expectNonZeroLoadProtocolAndMetadataDuration = false - ) + expectNonZeroLoadProtocolAndMetadataDuration = false) } } @@ -316,8 +303,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.of(0), expectedProvidedTimestamp = Optional.empty(), // No time travel by timestamp expectNonEmptyTimestampToVersionResolutionDuration = false, // No time travel by timestamp - expectNonZeroLoadProtocolAndMetadataDuration = true - ) + expectNonZeroLoadProtocolAndMetadataDuration = true) // Test getSnapshotAsOfVersion checkSnapshotReport( @@ -327,8 +313,7 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.of(0), expectedProvidedTimestamp = Optional.empty(), // No time travel by timestamp expectNonEmptyTimestampToVersionResolutionDuration = false, // No time travel by timestamp - expectNonZeroLoadProtocolAndMetadataDuration = true - ) + expectNonZeroLoadProtocolAndMetadataDuration = true) // Test getSnapshotAsOfTimestamp // We use the timestamp of version 0 @@ -341,7 +326,6 @@ class SnapshotReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedVersion = Optional.of(0), expectedProvidedTimestamp = Optional.of(version0Timestamp), expectNonEmptyTimestampToVersionResolutionDuration = true, - expectNonZeroLoadProtocolAndMetadataDuration = true - ) + expectNonZeroLoadProtocolAndMetadataDuration = true) } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/TransactionReportSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/TransactionReportSuite.scala index 0b8cd546ed4..a57d9e07a38 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/TransactionReportSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/TransactionReportSuite.scala @@ -22,14 +22,15 @@ import scala.collection.JavaConverters._ import io.delta.kernel._ import io.delta.kernel.data.Row import io.delta.kernel.engine._ -import io.delta.kernel.internal.metrics.Timer -import io.delta.kernel.internal.util.Utils import io.delta.kernel.internal.TableConfig import io.delta.kernel.internal.actions.{RemoveFile, SingleAction} import io.delta.kernel.internal.data.GenericRow +import io.delta.kernel.internal.metrics.Timer +import io.delta.kernel.internal.util.Utils import io.delta.kernel.metrics.{SnapshotReport, TransactionReport} import io.delta.kernel.types.{IntegerType, StructType} import io.delta.kernel.utils.{CloseableIterable, CloseableIterator, DataFileStatus} + import org.scalatest.funsuite.AnyFunSuite class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { @@ -48,10 +49,10 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { * @return (TransactionReport, durationToCommit, SnapshotReportIfPresent, ExceptionIfThrown) */ def getTransactionAndSnapshotReport( - createTransaction: Engine => Transaction, - generateCommitActions: (Transaction, Engine) => CloseableIterable[Row], - expectException: Boolean - ): (TransactionReport, Long, Option[SnapshotReport], Option[Exception]) = { + createTransaction: Engine => Transaction, + generateCommitActions: (Transaction, Engine) => CloseableIterable[Row], + expectException: Boolean) + : (TransactionReport, Long, Option[SnapshotReport], Option[Exception]) = { val timer = new Timer() val (metricsReports, exception) = collectMetricsReports( @@ -60,15 +61,17 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { val actionsToCommit = generateCommitActions(transaction, engine) timer.time(() => transaction.commit(engine, actionsToCommit)) // Time the actual operation }, - expectException - ) + expectException) val transactionReports = metricsReports.filter(_.isInstanceOf[TransactionReport]) assert(transactionReports.length == 1, "Expected exactly 1 TransactionReport") val snapshotReports = metricsReports.filter(_.isInstanceOf[SnapshotReport]) assert(snapshotReports.length <= 1, "Expected at most 1 SnapshotReport") - (transactionReports.head.asInstanceOf[TransactionReport], timer.totalDurationNs(), - snapshotReports.headOption.map(_.asInstanceOf[SnapshotReport]), exception) + ( + transactionReports.head.asInstanceOf[TransactionReport], + timer.totalDurationNs(), + snapshotReports.headOption.map(_.asInstanceOf[SnapshotReport]), + exception) } /** @@ -91,29 +94,30 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { */ // scalastyle:off def checkTransactionReport( - generateCommitActions: (Transaction, Engine) => CloseableIterable[Row], - path: String, - expectException: Boolean, - expectedBaseSnapshotVersion: Long, - expectedNumAddFiles: Long = 0, - expectedNumRemoveFiles: Long = 0, - expectedNumTotalActions: Long = 0, - expectedCommitVersion: Option[Long] = None, - expectedNumAttempts: Long = 1, - expectedTotalAddFilesSizeInBytes: Long = 0, - buildTransaction: (TransactionBuilder, Engine) => Transaction = (tb, e) => tb.build(e), - engineInfo: String = "test-engine-info", - operation: Operation = Operation.MANUAL_UPDATE - ): Unit = { + generateCommitActions: (Transaction, Engine) => CloseableIterable[Row], + path: String, + expectException: Boolean, + expectedBaseSnapshotVersion: Long, + expectedNumAddFiles: Long = 0, + expectedNumRemoveFiles: Long = 0, + expectedNumTotalActions: Long = 0, + expectedCommitVersion: Option[Long] = None, + expectedNumAttempts: Long = 1, + expectedTotalAddFilesSizeInBytes: Long = 0, + buildTransaction: (TransactionBuilder, Engine) => Transaction = (tb, e) => tb.build(e), + engineInfo: String = "test-engine-info", + operation: Operation = Operation.MANUAL_UPDATE): Unit = { // scalastyle:on assert(expectException == expectedCommitVersion.isEmpty) val (transactionReport, duration, snapshotReportOpt, exception) = getTransactionAndSnapshotReport( - engine => buildTransaction( - Table.forPath(engine, path).createTransactionBuilder(engine, engineInfo, operation), - engine - ), generateCommitActions, expectException) + engine => + buildTransaction( + Table.forPath(engine, path).createTransactionBuilder(engine, engineInfo, operation), + engine), + generateCommitActions, + expectException) // Verify contents assert(transactionReport.getTablePath == defaultEngine.getFileSystemClient.resolvePath(path)) @@ -135,7 +139,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { } else { assert(snapshotReportOpt.exists { snapshotReport => snapshotReport.getVersion.toScala.contains(expectedBaseSnapshotVersion) && - transactionReport.getSnapshotReportUUID.toScala.contains(snapshotReport.getReportUUID) + transactionReport.getSnapshotReportUUID.toScala.contains(snapshotReport.getReportUUID) }) } assert(transactionReport.getCommittedVersion.toScala == expectedCommitVersion) @@ -148,18 +152,21 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { assert(transactionReport.getTransactionMetrics.getNumCommitAttempts == expectedNumAttempts) assert(transactionReport.getTransactionMetrics.getNumAddFiles == expectedNumAddFiles) assert(transactionReport.getTransactionMetrics.getTotalAddFilesSizeInBytes - == expectedTotalAddFilesSizeInBytes) + == expectedTotalAddFilesSizeInBytes) assert(transactionReport.getTransactionMetrics.getNumRemoveFiles == expectedNumRemoveFiles) assert(transactionReport.getTransactionMetrics.getNumTotalActions == expectedNumTotalActions) } - def generateAppendActions(fileStatusIter: CloseableIterator[DataFileStatus]) - (trans: Transaction, engine: Engine): CloseableIterable[Row] = { + def generateAppendActions(fileStatusIter: CloseableIterator[DataFileStatus])( + trans: Transaction, + engine: Engine): CloseableIterable[Row] = { val transState = trans.getTransactionState(engine) CloseableIterable.inMemoryIterable( - Transaction.generateAppendActions(engine, transState, fileStatusIter, - Transaction.getWriteContext(engine, transState, Collections.emptyMap())) - ) + Transaction.generateAppendActions( + engine, + transState, + fileStatusIter, + Transaction.getWriteContext(engine, transState, Collections.emptyMap()))) } test("TransactionReport: Basic append to existing table + update metadata") { @@ -177,8 +184,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumAddFiles = 1, expectedNumTotalActions = 2, // commitInfo + addFile expectedCommitVersion = Some(1), - expectedTotalAddFilesSizeInBytes = 100 - ) + expectedTotalAddFilesSizeInBytes = 100) // Commit 2 AddFiles checkTransactionReport( @@ -191,8 +197,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedCommitVersion = Some(2), engineInfo = "foo", expectedTotalAddFilesSizeInBytes = 200, - operation = Operation.WRITE - ) + operation = Operation.WRITE) // Update metadata only checkTransactionReport( @@ -206,8 +211,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { transBuilder .withTableProperties(engine, Map(TableConfig.CHECKPOINT_INTERVAL.getKey -> "2").asJava) .build(engine) - } - ) + }) } } @@ -226,8 +230,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { transBuilder .withSchema(engine, new StructType().add("id", IntegerType.INTEGER)) .build(engine) - } - ) + }) // Commit 2 AddFiles checkTransactionReport( @@ -238,8 +241,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumAddFiles = 2, expectedNumTotalActions = 3, // commitInfo + addFile expectedCommitVersion = Some(1), - expectedTotalAddFilesSizeInBytes = 200 - ) + expectedTotalAddFilesSizeInBytes = 200) } } @@ -260,8 +262,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { transBuilder .withSchema(engine, new StructType().add("id", IntegerType.INTEGER)) .build(engine) - } - ) + }) } } @@ -274,23 +275,21 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { val removeFileRow: Row = { val fieldMap: Map[Integer, AnyRef] = Map( Integer.valueOf(RemoveFile.FULL_SCHEMA.indexOf("path")) -> "/path/for/remove/file", - Integer.valueOf(RemoveFile.FULL_SCHEMA.indexOf("dataChange")) -> java.lang.Boolean.TRUE - ) + Integer.valueOf(RemoveFile.FULL_SCHEMA.indexOf("dataChange")) -> java.lang.Boolean.TRUE) new GenericRow(RemoveFile.FULL_SCHEMA, fieldMap.asJava) } checkTransactionReport( - generateCommitActions = (_, _) => CloseableIterable.inMemoryIterable( - Utils.toCloseableIterator( - Seq(SingleAction.createRemoveFileSingleAction(removeFileRow)).iterator.asJava - )), + generateCommitActions = (_, _) => + CloseableIterable.inMemoryIterable( + Utils.toCloseableIterator( + Seq(SingleAction.createRemoveFileSingleAction(removeFileRow)).iterator.asJava)), path, expectException = false, expectedBaseSnapshotVersion = 0, expectedNumRemoveFiles = 1, expectedNumTotalActions = 2, // commitInfo + removeFile - expectedCommitVersion = Some(1) - ) + expectedCommitVersion = Some(1)) } } @@ -312,8 +311,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumTotalActions = 2, // commitInfo + removeFile expectedCommitVersion = Some(2), expectedNumAttempts = 2, - expectedTotalAddFilesSizeInBytes = 100 - ) + expectedTotalAddFilesSizeInBytes = 100) } } @@ -330,8 +328,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { }, path, expectException = true, - expectedBaseSnapshotVersion = 0 - ) + expectedBaseSnapshotVersion = 0) } } @@ -344,7 +341,8 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { // This writes a concurrent append everytime the iterable is asked for an iterator. This means // there should be a conflicting transaction committed everytime Kernel tries to commit def actionsIterableWithConcurrentAppend( - trans: Transaction, engine: Engine): CloseableIterable[Row] = { + trans: Transaction, + engine: Engine): CloseableIterable[Row] = { val transState = trans.getTransactionState(engine) val writeContext = Transaction.getWriteContext(engine, transState, Collections.emptyMap()) @@ -365,8 +363,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectException = true, expectedBaseSnapshotVersion = 0, expectedNumAttempts = 6, // 1 first try + 6 retries - buildTransaction = (builder, engine) => builder.withMaxRetries(5).build(engine) - ) + buildTransaction = (builder, engine) => builder.withMaxRetries(5).build(engine)) } } @@ -375,13 +372,10 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { //////////////////// private def fileStatusIter1 = Utils.toCloseableIterator( - Seq(new DataFileStatus("/path/to/file", 100, 100, Optional.empty())).iterator.asJava - ) + Seq(new DataFileStatus("/path/to/file", 100, 100, Optional.empty())).iterator.asJava) private def fileStatusIter2 = Utils.toCloseableIterator( Seq( new DataFileStatus("/path/to/file1", 100, 100, Optional.empty()), - new DataFileStatus("/path/to/file2", 100, 100, Optional.empty()) - ).iterator.asJava - ) + new DataFileStatus("/path/to/file2", 100, 100, Optional.empty())).iterator.asJava) } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/DefaultVectorTestUtils.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/DefaultVectorTestUtils.scala index c5c8e8b6ff1..7a393f15873 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/DefaultVectorTestUtils.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/DefaultVectorTestUtils.scala @@ -16,20 +16,21 @@ package io.delta.kernel.defaults.utils +import io.delta.kernel.data.{ColumnarBatch, ColumnVector} import io.delta.kernel.defaults.internal.data.DefaultColumnarBatch -import io.delta.kernel.data.{ColumnVector, ColumnarBatch} import io.delta.kernel.test.VectorTestUtils import io.delta.kernel.types._ trait DefaultVectorTestUtils extends VectorTestUtils { + /** * Returns a [[ColumnarBatch]] with each given vector is a top-level column col_i where i is * the index of the vector in the input list. */ protected def columnarBatch(vectors: ColumnVector*): ColumnarBatch = { val numRows = vectors.head.getSize - vectors.tail.foreach( - v => require(v.getSize == numRows, "All vectors should have the same size")) + vectors.tail.foreach(v => + require(v.getSize == numRows, "All vectors should have the same size")) val schema = (0 until vectors.length) .foldLeft(new StructType())((s, i) => s.add(s"col_$i", vectors(i).getDataType)) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestRow.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestRow.scala index 661d286a3c9..b0e4556fb34 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestRow.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestRow.scala @@ -15,16 +15,18 @@ */ package io.delta.kernel.defaults.utils +import java.sql.Timestamp +import java.time.{Instant, LocalDate, LocalDateTime, ZoneOffset} +import java.time.ZoneOffset.UTC +import java.time.temporal.ChronoUnit + import scala.collection.JavaConverters._ -import org.apache.spark.sql.{types => sparktypes} -import org.apache.spark.sql.{Row => SparkRow} + import io.delta.kernel.data.{ArrayValue, ColumnVector, MapValue, Row} import io.delta.kernel.types._ -import java.sql.Timestamp -import java.time.ZoneOffset.UTC -import java.time.temporal.ChronoUnit -import java.time.{Instant, LocalDate, LocalDateTime, ZoneOffset} +import org.apache.spark.sql.{Row => SparkRow} +import org.apache.spark.sql.{types => sparktypes} /** * Corresponding Scala class for each Kernel data type: @@ -136,9 +138,9 @@ object TestRow { obj.asInstanceOf[Seq[Any]] .map(decodeCellValue(arrayType.elementType, _)) case mapType: sparktypes.MapType => obj.asInstanceOf[Map[Any, Any]].map { - case (k, v) => - decodeCellValue(mapType.keyType, k) -> decodeCellValue(mapType.valueType, v) - } + case (k, v) => + decodeCellValue(mapType.keyType, k) -> decodeCellValue(mapType.valueType, v) + } case _: sparktypes.StructType => TestRow(obj.asInstanceOf[SparkRow]) case _ => throw new UnsupportedOperationException("unrecognized data type") } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala index 065f59e60aa..d09e87924ab 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala @@ -19,11 +19,13 @@ import java.io.{File, FileNotFoundException} import java.math.{BigDecimal => BigDecimalJ} import java.nio.file.Files import java.util.{Optional, TimeZone, UUID} + import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer + import io.delta.golden.GoldenTableUtils import io.delta.kernel.{Scan, Snapshot, Table} -import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, MapValue, Row} +import io.delta.kernel.data.{ColumnarBatch, ColumnVector, FilteredColumnarBatch, MapValue, Row} import io.delta.kernel.defaults.engine.DefaultEngine import io.delta.kernel.defaults.internal.data.vector.DefaultGenericVector import io.delta.kernel.engine.Engine @@ -34,13 +36,15 @@ import io.delta.kernel.internal.util.{ColumnMapping, Utils} import io.delta.kernel.internal.util.Utils.singletonCloseableIterator import io.delta.kernel.types._ import io.delta.kernel.utils.CloseableIterator + +import org.apache.spark.sql.delta.util.FileNames + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.shaded.org.apache.commons.io.FileUtils -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.{types => sparktypes} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.delta.util.FileNames import org.scalatest.Assertions trait TestUtils extends Assertions with SQLHelper { @@ -128,7 +132,8 @@ trait TestUtils extends Assertions with SQLHelper { } def withTempDirAndEngine( - f: (String, Engine) => Unit, hadoopConf: Map[String, String] = Map.empty): Unit = { + f: (String, Engine) => Unit, + hadoopConf: Map[String, String] = Map.empty): Unit = { val engine = DefaultEngine.create(new Configuration() { { for ((key, value) <- hadoopConf) { @@ -174,8 +179,9 @@ trait TestUtils extends Assertions with SQLHelper { leafLevelPrimitiveColumns( basePath :+ field.getName, field.getDataType.asInstanceOf[StructType]) - case field if !field.getDataType.isInstanceOf[ArrayType] && - !field.getDataType.isInstanceOf[MapType] => + case field + if !field.getDataType.isInstanceOf[ArrayType] && + !field.getDataType.isInstanceOf[MapType] => // for all primitive types Seq(new Column((basePath :+ field.getName).asJava.toArray(new Array[String](0)))); case _ => Seq.empty @@ -188,11 +194,11 @@ trait TestUtils extends Assertions with SQLHelper { } def readSnapshot( - snapshot: Snapshot, - readSchema: StructType = null, - filter: Predicate = null, - expectedRemainingFilter: Predicate = null, - engine: Engine = defaultEngine): Seq[Row] = { + snapshot: Snapshot, + readSchema: StructType = null, + filter: Predicate = null, + expectedRemainingFilter: Predicate = null, + engine: Engine = defaultEngine): Seq[Row] = { val result = ArrayBuffer[Row]() @@ -261,9 +267,9 @@ trait TestUtils extends Assertions with SQLHelper { } def readTableUsingKernel( - engine: Engine, - tablePath: String, - readSchema: StructType): Seq[FilteredColumnarBatch] = { + engine: Engine, + tablePath: String, + readSchema: StructType): Seq[FilteredColumnarBatch] = { val scan = Table.forPath(engine, tablePath) .getLatestSnapshot(engine) .getScanBuilder() @@ -322,15 +328,13 @@ trait TestUtils extends Assertions with SQLHelper { TimestampNTZType.TIMESTAMP_NTZ, StringType.STRING, BinaryType.BINARY, - new DecimalType(10, 5) - ) + new DecimalType(10, 5)) /** All types. Used in parameterized tests where type is one of the test dimensions. */ val ALL_TYPES = SIMPLE_TYPES ++ Seq( new ArrayType(BooleanType.BOOLEAN, true), new MapType(IntegerType.INTEGER, LongType.LONG, true), - new StructType().add("s1", BooleanType.BOOLEAN).add("s2", IntegerType.INTEGER) - ) + new StructType().add("s1", BooleanType.BOOLEAN).add("s2", IntegerType.INTEGER)) /** * Compares the rows in the tables latest snapshot with the expected answer and fails if they @@ -348,17 +352,16 @@ trait TestUtils extends Assertions with SQLHelper { * @param expectedVersion expected version of the latest snapshot for the table */ def checkTable( - path: String, - expectedAnswer: Seq[TestRow], - readCols: Seq[String] = null, - engine: Engine = defaultEngine, - expectedSchema: StructType = null, - filter: Predicate = null, - version: Option[Long] = None, - timestamp: Option[Long] = None, - expectedRemainingFilter: Predicate = null, - expectedVersion: Option[Long] = None - ): Unit = { + path: String, + expectedAnswer: Seq[TestRow], + readCols: Seq[String] = null, + engine: Engine = defaultEngine, + expectedSchema: StructType = null, + filter: Predicate = null, + version: Option[Long] = None, + timestamp: Option[Long] = None, + expectedRemainingFilter: Predicate = null, + expectedVersion: Option[Long] = None): Unit = { assert(version.isEmpty || timestamp.isEmpty, "Cannot provide both a version and timestamp") val snapshot = if (version.isDefined) { @@ -385,12 +388,12 @@ trait TestUtils extends Assertions with SQLHelper { |Expected schema does not match actual schema: |Expected schema: $expectedSchema |Actual schema: ${snapshot.getSchema()} - |""".stripMargin - ) + |""".stripMargin) } expectedVersion.foreach { version => - assert(version == snapshot.getVersion(), + assert( + version == snapshot.getVersion(), s"Expected version $version does not match actual version ${snapshot.getVersion()}") } @@ -425,14 +428,14 @@ trait TestUtils extends Assertions with SQLHelper { case bd: java.math.BigDecimal => BigDecimal(bd) // Equality of WrappedArray differs for AnyVal and AnyRef in Scala 2.12.2+ case seq: Seq[_] => seq.map { - case b: java.lang.Byte => b.byteValue - case s: java.lang.Short => s.shortValue - case i: java.lang.Integer => i.intValue - case l: java.lang.Long => l.longValue - case f: java.lang.Float => f.floatValue - case d: java.lang.Double => d.doubleValue - case x => x - } + case b: java.lang.Byte => b.byteValue + case s: java.lang.Short => s.shortValue + case i: java.lang.Integer => i.intValue + case l: java.lang.Long => l.longValue + case f: java.lang.Float => f.floatValue + case d: java.lang.Double => d.doubleValue + case x => x + } // Convert array to Seq for easy equality check. case b: Array[_] => b.toSeq case r: TestRow => prepareRow(r) @@ -441,33 +444,33 @@ trait TestUtils extends Assertions with SQLHelper { } private def compare(obj1: Any, obj2: Any): Boolean = (obj1, obj2) match { - case (null, null) => true - case (null, _) => false - case (_, null) => false - case (a: Array[_], b: Array[_]) => - a.length == b.length && a.zip(b).forall { case (l, r) => compare(l, r) } - case (a: Map[_, _], b: Map[_, _]) => - a.size == b.size && a.keys.forall { aKey => - b.keys.find(bKey => compare(aKey, bKey)).exists(bKey => compare(a(aKey), b(bKey))) - } - case (a: Iterable[_], b: Iterable[_]) => - a.size == b.size && a.zip(b).forall { case (l, r) => compare(l, r) } - case (a: Product, b: Product) => - compare(a.productIterator.toSeq, b.productIterator.toSeq) - case (a: TestRow, b: TestRow) => - compare(a.toSeq, b.toSeq) - // 0.0 == -0.0, turn float/double to bits before comparison, to distinguish 0.0 and -0.0. - case (a: Double, b: Double) => - java.lang.Double.doubleToRawLongBits(a) == java.lang.Double.doubleToRawLongBits(b) - case (a: Float, b: Float) => - java.lang.Float.floatToRawIntBits(a) == java.lang.Float.floatToRawIntBits(b) - case (a, b) => - if (!a.equals(b)) { - val sds = 200; - } - a.equals(b) - // In scala == does not call equals for boxed numeric classes? - } + case (null, null) => true + case (null, _) => false + case (_, null) => false + case (a: Array[_], b: Array[_]) => + a.length == b.length && a.zip(b).forall { case (l, r) => compare(l, r) } + case (a: Map[_, _], b: Map[_, _]) => + a.size == b.size && a.keys.forall { aKey => + b.keys.find(bKey => compare(aKey, bKey)).exists(bKey => compare(a(aKey), b(bKey))) + } + case (a: Iterable[_], b: Iterable[_]) => + a.size == b.size && a.zip(b).forall { case (l, r) => compare(l, r) } + case (a: Product, b: Product) => + compare(a.productIterator.toSeq, b.productIterator.toSeq) + case (a: TestRow, b: TestRow) => + compare(a.toSeq, b.toSeq) + // 0.0 == -0.0, turn float/double to bits before comparison, to distinguish 0.0 and -0.0. + case (a: Double, b: Double) => + java.lang.Double.doubleToRawLongBits(a) == java.lang.Double.doubleToRawLongBits(b) + case (a: Float, b: Float) => + java.lang.Float.floatToRawIntBits(a) == java.lang.Float.floatToRawIntBits(b) + case (a, b) => + if (!a.equals(b)) { + val sds = 200; + } + a.equals(b) + // In scala == does not call equals for boxed numeric classes? + } private def genErrorMessage(expectedAnswer: Seq[TestRow], result: Seq[TestRow]): String = { // TODO: improve to include schema or Java type information to help debugging @@ -489,7 +492,8 @@ trait TestUtils extends Assertions with SQLHelper { */ protected def withTempDir(f: File => Unit): Unit = { val tempDir = Files.createTempDirectory(UUID.randomUUID().toString).toFile - try f(tempDir) finally { + try f(tempDir) + finally { FileUtils.deleteDirectory(tempDir) } } @@ -498,7 +502,8 @@ trait TestUtils extends Assertions with SQLHelper { * Drops table `tableName` after calling `f`. */ protected def withTable(tableNames: String*)(f: => Unit): Unit = { - try f finally { + try f + finally { tableNames.foreach { name => spark.sql(s"DROP TABLE IF EXISTS $name") } @@ -703,8 +708,7 @@ trait TestUtils extends Assertions with SQLHelper { sparktypes.StructField( field.getName, toSparkType(field.getDataType), - field.isNullable - ) + field.isNullable) }) } } @@ -721,8 +725,7 @@ trait TestUtils extends Assertions with SQLHelper { } def deleteChecksumFileForTable(tablePath: String, versions: Seq[Int]): Unit = - versions.foreach( - v => Files.deleteIfExists( - new File(FileNames.checksumFile(new Path(s"$tablePath/_delta_log"), v).toString).toPath) - ) + versions.foreach(v => + Files.deleteIfExists( + new File(FileNames.checksumFile(new Path(s"$tablePath/_delta_log"), v).toString).toPath)) } diff --git a/project/plugins.sbt b/project/plugins.sbt index 9a49f4bd642..852da22bfbd 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -54,4 +54,6 @@ dependencyOverrides += "com.puppycrawl.tools" % "checkstyle" % "9.3" addSbtPlugin("com.thesamet" % "sbt-protoc" % "1.0.7") -addSbtPlugin("com.lightbend.sbt" % "sbt-java-formatter" % "0.8.0") \ No newline at end of file +addSbtPlugin("com.lightbend.sbt" % "sbt-java-formatter" % "0.8.0") + +addSbtPlugin("org.scalameta" % "sbt-scalafmt" % "2.5.4") diff --git a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceTypeWideningSuite.scala b/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceTypeWideningSuite.scala index 751c1fcc404..6f92ecaf00c 100644 --- a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceTypeWideningSuite.scala +++ b/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceTypeWideningSuite.scala @@ -104,7 +104,6 @@ class DeltaSharingDataSourceTypeWideningSuite .putMetadataArray( "delta.typeChanges", Array( new MetadataBuilder() - .putLong("tableVersion", 2) .putString("fromType", "short") .putString("toType", "integer") .build())) diff --git a/spark/src/main/resources/error/delta-error-classes.json b/spark/src/main/resources/error/delta-error-classes.json index b40bdb3954e..3749045beb3 100644 --- a/spark/src/main/resources/error/delta-error-classes.json +++ b/spark/src/main/resources/error/delta-error-classes.json @@ -869,6 +869,13 @@ ], "sqlState" : "42K03" }, + "DELTA_ENABLING_COLUMN_MAPPING_DISALLOWED_WHEN_COLUMN_MAPPING_METADATA_ALREADY_EXISTS" : { + "message" : [ + "Enabling column mapping when column mapping metadata is already present in schema is not supported.", + "To use column mapping, create a new table and reload the data into it." + ], + "sqlState" : "XXKDS" + }, "DELTA_EXCEED_CHAR_VARCHAR_LIMIT" : { "message" : [ "Value \"\" exceeds char/varchar type length limitation. Failed check: ." @@ -2312,6 +2319,13 @@ "Please check if you want to manually propagate the schema change(s) to the sink table before we proceed with stream processing using the finalized schema at version .", "Once you have fixed the schema of the sink table or have decided there is no need to fix, you can set the following configuration(s) to unblock the non-additive schema change(s) and continue stream processing.", "", + "Using dataframe reader option(s):", + "To unblock for this particular stream just for this series of schema change(s):", + "", + "To unblock for this particular stream:", + "", + "", + "Using SQL configuration(s):", "To unblock for this particular stream just for this series of schema change(s):", "", "To unblock for this particular stream:", @@ -2331,6 +2345,13 @@ "Please check if you want to update your streaming query before we proceed with stream processing using the finalized schema at version .", "Once you have updated your streaming query or have decided there is no need to update it, you can set the following configuration to unblock the type change(s) and continue stream processing.", "", + "Using dataframe reader option:", + "To unblock for this particular stream just for this series of schema change(s):", + "", + "To unblock for this particular stream:", + "", + "", + "Using SQL configuration:", "To unblock for this particular stream just for this series of schema change(s):", "", "To unblock for this particular stream:", diff --git a/spark/src/main/scala-spark-3.5/shims/DecimalPrecisionTypeCoercionShims.scala b/spark/src/main/scala-spark-3.5/shims/DecimalPrecisionTypeCoercionShims.scala new file mode 100644 index 00000000000..41f050828c1 --- /dev/null +++ b/spark/src/main/scala-spark-3.5/shims/DecimalPrecisionTypeCoercionShims.scala @@ -0,0 +1,26 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta + +import org.apache.spark.sql.catalyst.analysis.DecimalPrecision +import org.apache.spark.sql.types.DecimalType + +object DecimalPrecisionTypeCoercionShims { + // Returns the wider decimal type that's wider than both of them + def widerDecimalType(d1: DecimalType, d2: DecimalType): DecimalType = + DecimalPrecision.widerDecimalType(d1, d2) +} diff --git a/spark/src/main/scala-spark-master/shims/DecimalPrecisionTypeCoercionShims.scala b/spark/src/main/scala-spark-master/shims/DecimalPrecisionTypeCoercionShims.scala new file mode 100644 index 00000000000..a907179fdd2 --- /dev/null +++ b/spark/src/main/scala-spark-master/shims/DecimalPrecisionTypeCoercionShims.scala @@ -0,0 +1,26 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta + +import org.apache.spark.sql.catalyst.analysis.DecimalPrecisionTypeCoercion +import org.apache.spark.sql.types.DecimalType + +object DecimalPrecisionTypeCoercionShims { + // Returns the wider decimal type that's wider than both of them + def widerDecimalType(d1: DecimalType, d2: DecimalType): DecimalType = + DecimalPrecisionTypeCoercion.widerDecimalType(d1, d2) +} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala index 10d3b6b0e9f..69142e22277 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala @@ -925,7 +925,7 @@ class DeltaAnalysis(session: SparkSession) if s != t && sNull == tNull => addCastsToArrayStructs(tblName, attr, s, t, sNull, typeWideningMode) case (s: AtomicType, t: AtomicType) - if typeWideningMode.shouldWidenType(fromType = t, toType = s) => + if typeWideningMode.shouldWidenTo(fromType = t, toType = s) => // Keep the type from the query, the target schema will be updated to widen the existing // type to match it. attr @@ -1097,7 +1097,7 @@ class DeltaAnalysis(session: SparkSession) case (StructField(name, sourceType: AtomicType, _, _), i @ TargetIndex(StructField(targetName, targetType: AtomicType, _, targetMetadata))) - if typeWideningMode.shouldWidenType(fromType = targetType, toType = sourceType) => + if typeWideningMode.shouldWidenTo(fromType = targetType, toType = sourceType) => Alias( GetStructField(parent, i, Option(name)), targetName)(explicitMetadata = Option(targetMetadata)) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaColumnMapping.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaColumnMapping.scala index b893d25f3cf..cf3b9493599 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaColumnMapping.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaColumnMapping.scala @@ -174,6 +174,18 @@ trait DeltaColumnMappingBase extends DeltaLogging { } } + // If column mapping was disabled, but there was already column mapping in the schema, it is + // a result of a bug in the previous version of Delta. This should no longer happen with the + // stripping done above. For existing tables with this issue, we should not allow enabling + // column mapping, to prevent further corruption. + if (spark.conf.get(DeltaSQLConf. + DELTA_COLUMN_MAPPING_DISALLOW_ENABLING_WHEN_METADATA_ALREADY_EXISTS)) { + if (oldMappingMode == NoMapping && newMappingMode != NoMapping && + schemaHasColumnMappingMetadata(oldMetadata.schema)) { + throw DeltaErrors.enablingColumnMappingDisallowedWhenColumnMappingMetadataAlreadyExists() + } + } + updatedMetadata = updateColumnMappingMetadata( oldMetadata, updatedMetadata, isChangingModeOnExistingTable, isOverwriteSchema) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala index 386d66caa60..57bdec6c070 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala @@ -2126,6 +2126,12 @@ trait DeltaErrorsBase messageParameters = Array(oldMode, newMode)) } + def enablingColumnMappingDisallowedWhenColumnMappingMetadataAlreadyExists(): Throwable = { + new DeltaColumnMappingUnsupportedException( + errorClass = + "DELTA_ENABLING_COLUMN_MAPPING_DISALLOWED_WHEN_COLUMN_MAPPING_METADATA_ALREADY_EXISTS") + } + def generateManifestWithColumnMappingNotSupported: Throwable = { new DeltaColumnMappingUnsupportedException( errorClass = "DELTA_UNSUPPORTED_MANIFEST_GENERATION_WITH_COLUMN_MAPPING") @@ -3130,7 +3136,14 @@ trait DeltaErrorsBase previousSchemaChangeVersion: Long, currentSchemaChangeVersion: Long, checkpointHash: Int, + readerOptionsUnblock: Seq[String], sqlConfsUnblock: Seq[String]): Throwable = { + val unblockChangeOptions = readerOptionsUnblock.map { option => + s""" .option("$option", "$currentSchemaChangeVersion")""" + }.mkString("\n") + val unblockStreamOptions = readerOptionsUnblock.map { option => + s""" .option("$option", "always")""" + }.mkString("\n") val unblockChangeConfs = sqlConfsUnblock.map { conf => s""" SET $conf.ckpt_$checkpointHash = $currentSchemaChangeVersion;""" }.mkString("\n") @@ -3148,6 +3161,8 @@ trait DeltaErrorsBase previousSchemaChangeVersion.toString, currentSchemaChangeVersion.toString, currentSchemaChangeVersion.toString, + unblockChangeOptions, + unblockStreamOptions, unblockChangeConfs, unblockStreamConfs, unblockAllConfs @@ -3159,6 +3174,7 @@ trait DeltaErrorsBase previousSchemaChangeVersion: Long, currentSchemaChangeVersion: Long, checkpointHash: Int, + readerOptionsUnblock: Seq[String], sqlConfsUnblock: Seq[String], wideningTypeChanges: Seq[TypeChange]): Throwable = { @@ -3167,6 +3183,12 @@ trait DeltaErrorsBase s"${change.toType.sql}" }.mkString("\n") + val unblockChangeOptions = readerOptionsUnblock.map { option => + s""" .option("$option", "$currentSchemaChangeVersion")""" + }.mkString("\n") + val unblockStreamOptions = readerOptionsUnblock.map { option => + s""" .option("$option", "always")""" + }.mkString("\n") val unblockChangeConfs = sqlConfsUnblock.map { conf => s""" SET $conf.ckpt_$checkpointHash = $currentSchemaChangeVersion;""" }.mkString("\n") @@ -3184,6 +3206,8 @@ trait DeltaErrorsBase currentSchemaChangeVersion.toString, wideningTypeChangesStr, currentSchemaChangeVersion.toString, + unblockChangeOptions, + unblockStreamOptions, unblockChangeConfs, unblockStreamConfs, unblockAllConfs diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala index 9fca7192e43..1260c348f87 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala @@ -688,11 +688,15 @@ class DeltaLog private( object DeltaLog extends DeltaLogging { /** - * The key type of `DeltaLog` cache. It's a pair of the canonicalized table path and the file - * system options (options starting with "fs." or "dfs." prefix) passed into - * `DataFrameReader/Writer` + * The key type of `DeltaLog` cache. It consists of + * - The canonicalized table path + * - File system options (options starting with "fs." or "dfs." prefix) passed into + * `DataFrameReader/Writer` */ - private type DeltaLogCacheKey = (Path, Map[String, String]) + case class DeltaLogCacheKey( + path: Path, + fsOptions: Map[String, String] + ) /** The name of the subdirectory that holds Delta metadata files */ private[delta] val LOG_DIR_NAME = "_delta_log" @@ -705,9 +709,8 @@ object DeltaLog extends DeltaLogging { * We create only a single [[DeltaLog]] for any given `DeltaLogCacheKey` to avoid wasted work * in reconstructing the log. */ - type CacheKey = (Path, Map[String, String]) private[delta] def getOrCreateCache(conf: SQLConf): - Cache[CacheKey, DeltaLog] = synchronized { + Cache[DeltaLogCacheKey, DeltaLog] = synchronized { deltaLogCache match { case Some(c) => c case None => @@ -721,12 +724,12 @@ object DeltaLog extends DeltaLogging { // Various layers will throw null pointer if the RDD is already gone. } }) - deltaLogCache = Some(builder.build[CacheKey, DeltaLog]()) + deltaLogCache = Some(builder.build[DeltaLogCacheKey, DeltaLog]()) deltaLogCache.get } } - private var deltaLogCache: Option[Cache[CacheKey, DeltaLog]] = None + private var deltaLogCache: Option[Cache[DeltaLogCacheKey, DeltaLog]] = None /** * Helper to create delta log caches @@ -941,7 +944,11 @@ object DeltaLog extends DeltaLogging { ) } } - def getDeltaLogFromCache(): DeltaLog = { + val cacheKey = DeltaLogCacheKey( + path, + fileSystemOptions) + + def getDeltaLogFromCache: DeltaLog = { // The following cases will still create a new ActionLog even if there is a cached // ActionLog using a different format path: // - Different `scheme` @@ -949,7 +956,7 @@ object DeltaLog extends DeltaLogging { // - Different mount point. try { getOrCreateCache(spark.sessionState.conf) - .get(path -> fileSystemOptions, () => { + .get(cacheKey, () => { createDeltaLog() } ) @@ -959,12 +966,12 @@ object DeltaLog extends DeltaLogging { } } - val deltaLog = getDeltaLogFromCache() + val deltaLog = getDeltaLogFromCache if (Option(deltaLog.sparkContext.get).map(_.isStopped).getOrElse(true)) { // Invalid the cached `DeltaLog` and create a new one because the `SparkContext` of the cached // `DeltaLog` has been stopped. - getOrCreateCache(spark.sessionState.conf).invalidate(path -> fileSystemOptions) - getDeltaLogFromCache() + getOrCreateCache(spark.sessionState.conf).invalidate(cacheKey) + getDeltaLogFromCache } else { deltaLog } @@ -990,13 +997,15 @@ object DeltaLog extends DeltaLogging { val iter = deltaLogCache.asMap().keySet().iterator() while (iter.hasNext) { val key = iter.next() - if (key._1 == path) { + if (key.path == path) { keysToBeRemoved += key } } deltaLogCache.invalidateAll(keysToBeRemoved.asJava) } else { - deltaLogCache.invalidate(path -> Map.empty) + deltaLogCache.invalidate(DeltaLogCacheKey( + path, + fsOptions = Map.empty)) } } catch { case NonFatal(e) => logWarning(e.getMessage, e) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOptions.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOptions.scala index dd01fc0ba2f..626e06e4c88 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOptions.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOptions.scala @@ -212,6 +212,12 @@ trait DeltaReadOptions extends DeltaOptionParser { val schemaTrackingLocation = options.get(SCHEMA_TRACKING_LOCATION) val sourceTrackingId = options.get(STREAMING_SOURCE_TRACKING_ID) + + val allowSourceColumnRename = options.get(ALLOW_SOURCE_COLUMN_RENAME) + + val allowSourceColumnDrop = options.get(ALLOW_SOURCE_COLUMN_DROP) + + val allowSourceColumnTypeChange = options.get(ALLOW_SOURCE_COLUMN_TYPE_CHANGE) } @@ -289,6 +295,10 @@ object DeltaOptions extends DeltaLogging { */ val STREAMING_SOURCE_TRACKING_ID = "streamingSourceTrackingId" + val ALLOW_SOURCE_COLUMN_DROP = "allowSourceColumnDrop" + val ALLOW_SOURCE_COLUMN_RENAME = "allowSourceColumnRename" + val ALLOW_SOURCE_COLUMN_TYPE_CHANGE = "allowSourceColumnTypeChange" + /** * An option to control if delta will write partition columns to data files */ diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala b/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala index af57cbaac90..48062bf9e2a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala @@ -683,6 +683,15 @@ trait OptimisticTransactionImpl extends DeltaTransaction } } + if (spark.sessionState.conf + .getConf(DeltaSQLConf.REMOVE_EXISTS_DEFAULT_FROM_SCHEMA_ON_EVERY_METADATA_CHANGE)) { + val schemaWithRemovedExistsDefaults = + SchemaUtils.removeExistsDefaultMetadata(newMetadataTmp.schema) + if (schemaWithRemovedExistsDefaults != newMetadataTmp.schema) { + newMetadataTmp = newMetadataTmp.copy(schemaString = schemaWithRemovedExistsDefaults.json) + } + } + // Table features Part 2: add manually-supported features specified in table properties, aka // those start with [[FEATURE_PROP_PREFIX]]. // diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/TableFeature.scala b/spark/src/main/scala/org/apache/spark/sql/delta/TableFeature.scala index 52c2cabb5be..10a41968fa4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/TableFeature.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/TableFeature.scala @@ -885,10 +885,20 @@ abstract class TypeWideningTableFeatureBase(name: String) extends ReaderWriterFe /** * Feature used for the preview phase of type widening. Tables that enabled this feature during the - * preview will keep being supported after the preview. + * preview are still supported after the preview. + * + * Note: Users can manually add both the preview and stable features to a table using ADD FEATURE, + * although that's undocumented for type widening. This is allowed: the two feature specifications + * are compatible and supported. */ object TypeWideningPreviewTableFeature extends TypeWideningTableFeatureBase(name = "typeWidening-preview") + +/** + * Stable feature for type widening. + */ +object TypeWideningTableFeature + extends TypeWideningTableFeatureBase(name = "typeWidening") with FeatureAutomaticallyEnabledByMetadata { override def automaticallyUpdateProtocolOfExistingTables: Boolean = true @@ -896,23 +906,11 @@ object TypeWideningPreviewTableFeature protocol: Protocol, metadata: Metadata, spark: SparkSession): Boolean = isTypeWideningSupportNeededByMetadata(metadata) && - // Don't automatically enable the preview feature if the stable feature is already supported. - !protocol.isFeatureSupported(TypeWideningTableFeature) + // Don't automatically enable the stable feature if the preview feature is already supported, to + // avoid possibly breaking old clients that only support the preview feature. + !protocol.isFeatureSupported(TypeWideningPreviewTableFeature) } -/** - * Stable feature for type widening. The stable feature isn't enabled automatically yet - * when setting the type widening table property as the feature is still in preview in this version. - * The feature spec is finalized though and by supporting the stable feature here we guarantee that - * this version can already read any table created in the future. - * - * Note: Users can manually add both the preview and stable features to a table using ADD FEATURE, - * although that's undocumented for type widening. This is allowed: the two feature specifications - * are compatible and supported. - */ -object TypeWideningTableFeature - extends TypeWideningTableFeatureBase(name = "typeWidening") - /** * inCommitTimestamp table feature is a writer feature that makes * every writer write a monotonically increasing timestamp inside the commit file. diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/TypeWideningMode.scala b/spark/src/main/scala/org/apache/spark/sql/delta/TypeWideningMode.scala index eaeefc9358d..88234bda5fd 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/TypeWideningMode.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/TypeWideningMode.scala @@ -16,21 +16,33 @@ package org.apache.spark.sql.delta -import org.apache.spark.sql.types.AtomicType +import org.apache.spark.sql.delta.DecimalPrecisionTypeCoercionShims +import org.apache.spark.sql.util.ScalaExtensions._ + +import org.apache.spark.sql.types.{AtomicType, DecimalType} /** * A type widening mode captures a specific set of type changes that are allowed to be applied. * Currently: * - NoTypeWidening: No type change is allowed. - * - AllTypeWidening: All supported type widening changes are allowed. - * - TypeEvolution(uniformIcebergCompatibleOnly = true): Type changes that are eligible to be - * applied automatically during schema evolution and that are supported by Iceberg are allowed. - * - TypeEvolution(uniformIcebergCompatibleOnly = false): Type changes that are eligible to be - * applied automatically during schema evolution are allowed, even if they are not supported by - * Iceberg. + * - AllTypeWidening: Allows widening to the target type using any supported type change. + * - TypeEvolution: Only allows widening to the target type if the type change is eligible to be + * applied automatically during schema evolution. + * - AllTypeWideningToCommonWiderType: Allows widening to a common (possibly different) wider type + * using any supported type change. + * - TypeEvolutionToCommonWiderType: Allows widening to a common (possibly different) wider type + * using only type changes that are eligible to be applied automatically during schema + * evolution. + * + * TypeEvolution modes can be restricted to only type changes supported by Iceberg by passing + * `uniformIcebergCompatibleOnly = truet`, to ensure that we don't automatically apply a type change + * that would break Iceberg compatibility. */ sealed trait TypeWideningMode { - def shouldWidenType(fromType: AtomicType, toType: AtomicType): Boolean + def getWidenedType(fromType: AtomicType, toType: AtomicType): Option[AtomicType] + + def shouldWidenTo(fromType: AtomicType, toType: AtomicType): Boolean = + getWidenedType(fromType, toType).contains(toType) } object TypeWideningMode { @@ -38,13 +50,13 @@ object TypeWideningMode { * No type change allowed. Typically because type widening and/or schema evolution isn't enabled. */ case object NoTypeWidening extends TypeWideningMode { - override def shouldWidenType(fromType: AtomicType, toType: AtomicType): Boolean = false + override def getWidenedType(fromType: AtomicType, toType: AtomicType): Option[AtomicType] = None } /** All supported type widening changes are allowed. */ case object AllTypeWidening extends TypeWideningMode { - override def shouldWidenType(fromType: AtomicType, toType: AtomicType): Boolean = - TypeWidening.isTypeChangeSupported(fromType = fromType, toType = toType) + override def getWidenedType(fromType: AtomicType, toType: AtomicType): Option[AtomicType] = + Option.when(TypeWidening.isTypeChangeSupported(fromType = fromType, toType = toType))(toType) } /** @@ -52,8 +64,52 @@ object TypeWideningMode { * Can be restricted to only type changes supported by Iceberg. */ case class TypeEvolution(uniformIcebergCompatibleOnly: Boolean) extends TypeWideningMode { - override def shouldWidenType(fromType: AtomicType, toType: AtomicType): Boolean = - TypeWidening.isTypeChangeSupportedForSchemaEvolution( - fromType = fromType, toType = toType, uniformIcebergCompatibleOnly) + override def getWidenedType(fromType: AtomicType, toType: AtomicType): Option[AtomicType] = + Option.when(TypeWidening.isTypeChangeSupportedForSchemaEvolution( + fromType = fromType, toType = toType, uniformIcebergCompatibleOnly))(toType) + } + + /** + * All supported type widening changes are allowed. Unlike [[AllTypeWidening]], this also allows + * widening `to` to `from`, and for decimals, widening to a different decimal type that is wider + * than both input types. Use for example when merging two unrelated schemas and we want just want + * to find a wider schema to use. + */ + case object AllTypeWideningToCommonWiderType extends TypeWideningMode { + override def getWidenedType(left: AtomicType, right: AtomicType): Option[AtomicType] = + (left, right) match { + case (l, r) if TypeWidening.isTypeChangeSupported(l, r) => Some(r) + case (l, r) if TypeWidening.isTypeChangeSupported(r, l) => Some(l) + case (l: DecimalType, r: DecimalType) => + val wider = DecimalPrecisionTypeCoercionShims.widerDecimalType(l, r) + Option.when( + TypeWidening.isTypeChangeSupported(l, wider) && + TypeWidening.isTypeChangeSupported(r, wider))(wider) + case _ => None + } + } + + /** + * Type changes that are eligible to be applied automatically during schema evolution are allowed. + * Can be restricted to only type changes supported by Iceberg. Unlike [[TypeEvolution]], this + * also allows widening `to` to `from`, and for decimals, widening to a different decimal type + * that is wider han both input types. Use for example when merging two unrelated schemas and we + * want just want to find a wider schema to use. + */ + case class TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly: Boolean) + extends TypeWideningMode { + override def getWidenedType(left: AtomicType, right: AtomicType): Option[AtomicType] = { + def typeChangeSupported: (AtomicType, AtomicType) => Boolean = + TypeWidening.isTypeChangeSupportedForSchemaEvolution(_, _, uniformIcebergCompatibleOnly) + + (left, right) match { + case (l, r) if typeChangeSupported(l, r) => Some(r) + case (l, r) if typeChangeSupported(r, l) => Some(l) + case (l: DecimalType, r: DecimalType) => + val wider = DecimalPrecisionTypeCoercionShims.widerDecimalType(l, r) + Option.when(typeChangeSupported(l, wider) && typeChangeSupported(r, wider))(wider) + case _ => None + } + } } } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/actions/InMemoryLogReplay.scala b/spark/src/main/scala/org/apache/spark/sql/delta/actions/InMemoryLogReplay.scala index 99d51324b5f..0352272290a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/actions/InMemoryLogReplay.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/actions/InMemoryLogReplay.scala @@ -47,7 +47,12 @@ class InMemoryLogReplay( private val transactions = new scala.collection.mutable.HashMap[String, SetTransaction]() private val domainMetadatas = collection.mutable.Map.empty[String, DomainMetadata] private val activeFiles = new scala.collection.mutable.HashMap[UniqueFileActionTuple, AddFile]() - private val tombstones = new scala.collection.mutable.HashMap[UniqueFileActionTuple, RemoveFile]() + // RemoveFiles that had cancelled AddFile during replay + private val cancelledRemoveFiles = + new scala.collection.mutable.HashMap[UniqueFileActionTuple, RemoveFile]() + // RemoveFiles that had NOT cancelled any AddFile during replay + private val activeRemoveFiles = + new scala.collection.mutable.HashMap[UniqueFileActionTuple, RemoveFile]() override def append(version: Long, actions: Iterator[Action]): Unit = { assert(currentVersion == -1 || version == currentVersion + 1, @@ -69,11 +74,15 @@ class InMemoryLogReplay( val uniquePath = UniqueFileActionTuple(add.pathAsUri, add.getDeletionVectorUniqueId) activeFiles(uniquePath) = add.copy(dataChange = false) // Remove the tombstone to make sure we only output one `FileAction`. - tombstones.remove(uniquePath) + cancelledRemoveFiles.remove(uniquePath) + // Remove from activeRemoveFiles to handle commits that add a previously-removed file + activeRemoveFiles.remove(uniquePath) case remove: RemoveFile => val uniquePath = UniqueFileActionTuple(remove.pathAsUri, remove.getDeletionVectorUniqueId) - activeFiles.remove(uniquePath) - tombstones(uniquePath) = remove.copy(dataChange = false) + activeFiles.remove(uniquePath) match { + case Some(_) => cancelledRemoveFiles(uniquePath) = remove + case None => activeRemoveFiles(uniquePath) = remove + } case _: CommitInfo => // do nothing case _: AddCDCFile => // do nothing case null => // Some crazy future feature. Ignore @@ -81,7 +90,9 @@ class InMemoryLogReplay( } private def getTombstones: Iterable[FileAction] = { - tombstones.values.filter(_.delTimestamp > minFileRetentionTimestamp) + (cancelledRemoveFiles.values ++ activeRemoveFiles.values) + .filter(_.delTimestamp > minFileRetentionTimestamp) + .map(_.copy(dataChange = false)) } private[delta] def getTransactions: Iterable[SetTransaction] = { diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/redirect/TableRedirect.scala b/spark/src/main/scala/org/apache/spark/sql/delta/redirect/TableRedirect.scala index 4de0a92e21c..c9f64c656d9 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/redirect/TableRedirect.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/redirect/TableRedirect.scala @@ -284,7 +284,9 @@ class TableRedirect(val config: DeltaConfig[Option[String]]) { val txn = deltaLog.startTransaction(catalogTableOpt) val deltaMetadata = txn.snapshot.metadata val currentConfigOpt = getRedirectConfiguration(deltaMetadata) - val tableIdent = catalogTableOpt.get.identifier.quotedString + val tableIdent = catalogTableOpt.map(_.identifier.quotedString).getOrElse { + s"delta.`${deltaLog.dataPath.toString}`" + } // There should be an existing table redirect configuration. if (currentConfigOpt.isEmpty) { DeltaErrors.invalidRedirectStateTransition(tableIdent, NoRedirect, state) @@ -332,7 +334,9 @@ class TableRedirect(val config: DeltaConfig[Option[String]]) { val snapshot = txn.snapshot getRedirectConfiguration(snapshot.metadata).foreach { currentConfig => DeltaErrors.invalidRedirectStateTransition( - catalogTableOpt.get.identifier.quotedString, + catalogTableOpt.map(_.identifier.quotedString).getOrElse { + s"delta.`${deltaLog.dataPath.toString}`" + }, currentConfig.redirectState, EnableRedirectInProgress ) @@ -353,7 +357,9 @@ class TableRedirect(val config: DeltaConfig[Option[String]]) { def remove(deltaLog: DeltaLog, catalogTableOpt: Option[CatalogTable]): Unit = { val txn = deltaLog.startTransaction(catalogTableOpt) val currentConfigOpt = getRedirectConfiguration(txn.snapshot.metadata) - val tableIdent = catalogTableOpt.get.identifier.quotedString + val tableIdent = catalogTableOpt.map(_.identifier.quotedString).getOrElse { + s"delta.`${deltaLog.dataPath.toString}`" + } if (currentConfigOpt.isEmpty) { DeltaErrors.invalidRemoveTableRedirect(tableIdent, NoRedirect) } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala index ada30d2e8b9..200ac9b9519 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala @@ -242,7 +242,8 @@ object SchemaMergingUtils { // If type widening is enabled and the type can be widened, it takes precedence over // keepExistingType. case (current: AtomicType, update: AtomicType) - if typeWideningMode.shouldWidenType(fromType = current, toType = update) => update + if typeWideningMode.getWidenedType(fromType = current, toType = update).isDefined => + typeWideningMode.getWidenedType(fromType = current, toType = update).get // Simply keeps the existing type for primitive types case (current, _) if keepExistingType => current diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala index 1bfc74ee1bc..47f44f95c63 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, GetArrayItem, GetArrayStructFields, GetMapValue, GetStructField} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} -import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, ResolveDefaultColumnsUtils} import org.apache.spark.sql.execution.streaming.IncrementalExecution import org.apache.spark.sql.functions.{col, struct} import org.apache.spark.sql.internal.SQLConf @@ -443,7 +443,7 @@ def normalizeColumnNamesInDataType( isDatatypeReadCompatible(e.keyType, n.keyType) && isDatatypeReadCompatible(e.valueType, n.valueType) case (e: AtomicType, n: AtomicType) - if typeWideningMode.shouldWidenType(fromType = e, toType = n) => true + if typeWideningMode.shouldWidenTo(fromType = e, toType = n) => true case (a, b) => a == b } } @@ -1546,6 +1546,30 @@ def normalizeColumnNamesInDataType( def areLogicalNamesEqual(col1: Seq[String], col2: Seq[String]): Boolean = { col1.length == col2.length && col1.zip(col2).forall(DELTA_COL_RESOLVER.tupled) } + + def removeExistsDefaultMetadata(schema: StructType): StructType = { + // 'EXISTS_DEFAULT' is not used in Delta because it is not allowed to add a column with a + // default value. Spark does though still add the metadata key when a column with a default + // value is added at table creation. + // We remove the metadata field here because it is not part of the Delta protocol and + // having it in the schema prohibits CTAS from a table with a dropped default value. + // @TODO: Clarify if active default values should be propagated to the target table in CTAS or + // not and if not also remove 'CURRENT_DEFAULT' in CTAS. + SchemaUtils.transformSchema(schema) { + case (_, StructType(fields), _) + if fields.exists(_.metadata.contains( + ResolveDefaultColumnsUtils.EXISTS_DEFAULT_COLUMN_METADATA_KEY)) => + val newFields = fields.map { field => + val builder = new MetadataBuilder() + .withMetadata(field.metadata) + .remove(ResolveDefaultColumnsUtils.EXISTS_DEFAULT_COLUMN_METADATA_KEY) + + field.copy(metadata = builder.build()) + } + StructType(newFields) + case (_, other, _) => other + } + } } /** diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala index f9a1e0d1e2c..dab3b4411b8 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala @@ -439,7 +439,6 @@ object DeltaDataSource extends DatabricksLogging { parameters: Map[String, String], sourceMetadataPathOpt: Option[String] = None, mergeConsecutiveSchemaChanges: Boolean = false): Option[DeltaSourceMetadataTrackingLog] = { - val options = new CaseInsensitiveStringMap(parameters.asJava) DeltaDataSource.extractSchemaTrackingLocationConfig(spark, parameters) .map { schemaTrackingLocation => @@ -451,7 +450,7 @@ object DeltaDataSource extends DatabricksLogging { DeltaSourceMetadataTrackingLog.create( spark, schemaTrackingLocation, sourceSnapshot, - Option(options.get(DeltaOptions.STREAMING_SOURCE_TRACKING_ID)), + parameters, sourceMetadataPathOpt, mergeConsecutiveSchemaChanges ) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index ab8b8d6cb43..d7cf7aec852 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -618,6 +618,18 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(true) + val REMOVE_EXISTS_DEFAULT_FROM_SCHEMA_ON_EVERY_METADATA_CHANGE = + buildConf("allowColumnDefaults.removeExistsDefaultFromSchemaOnMetadataChange") + .internal() + .doc("When enabled, remove all field metadata entries using the 'EXISTS_DEFAULT' key " + + "from the schema whenever the table metadata is updated. 'EXISTS_DEFAULT' holds values " + + "that are used in Spark for existing rows when a new column with a default value is " + + "added to a table. Since we do not support adding columns with a default value in " + + "Delta, this configuration should always be removed, also when it was written by an " + + "older version that still put it into the schema.") + .booleanConf + .createWithDefault(true) + ////////////////////////////////////////////// // DynamoDB Commit Coordinator-specific configs ///////////////////////////////////////////// @@ -1953,6 +1965,18 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(true) + val DELTA_COLUMN_MAPPING_DISALLOW_ENABLING_WHEN_METADATA_ALREADY_EXISTS = + buildConf("columnMapping.disallowEnablingWhenColumnMappingMetadataAlreadyExists") + .doc( + """ + |If Delta table already has column mapping metadata before the feature is enabled, it is + |as a result of a corruption or a bug. Enabling column mapping in such a case can lead to + |further corruption of the table and should be disallowed. + |""".stripMargin) + .internal() + .booleanConf + .createWithDefault(true) + val DYNAMIC_PARTITION_OVERWRITE_ENABLED = buildConf("dynamicPartitionOverwrite.enabled") .doc("Whether to overwrite partitions dynamically when 'partitionOverwriteMode' is set to " + @@ -2261,6 +2285,16 @@ trait DeltaSQLConfBase { .checkValue(v => v >= 1, "Must be at least 1.") .createWithDefault(100) + val DELTA_STATS_COLLECTION_FALLBACK_TO_INTERPRETED_PROJECTION = + buildConf("collectStats.fallbackToInterpretedProjection") + .internal() + .doc("When enabled, the updateStats expression will use the standard code path" + + " that falls back to an interpreted expression if codegen fails. This should" + + " always be true. The config only exists to force the old behavior, which was" + + " to always use codegen.") + .booleanConf + .createWithDefault(true) + val DELTA_CONVERT_ICEBERG_STATS = buildConf("collectStats.convertIceberg") .internal() .doc("When enabled, attempts to convert Iceberg stats to Delta stats when cloning from " + diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataEvolutionSupport.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataEvolutionSupport.scala index 9c9687a17ee..9863995e9e5 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataEvolutionSupport.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataEvolutionSupport.scala @@ -449,7 +449,7 @@ object DeltaSourceMetadataEvolutionSupport { /** * Defining the different combinations of non-additive schema changes to detect them and allow - * users to vet and unblock them using a corresponding SQL conf: + * users to vet and unblock them using a corresponding SQL conf or reader option: * - dropping columns * - renaming columns * - widening data types @@ -460,23 +460,28 @@ object DeltaSourceMetadataEvolutionSupport { val isDrop: Boolean val isTypeWidening: Boolean val sqlConfsUnblock: Seq[String] + val readerOptionsUnblock: Seq[String] } // Single types of schema change, typically caused by a single ALTER TABLE operation. private case object SchemaChangeRename extends SchemaChangeType { override val name = "RENAME COLUMN" - override val sqlConfsUnblock: Seq[String] = Seq(SQL_CONF_UNBLOCK_RENAME) override val (isRename, isDrop, isTypeWidening) = (true, false, false) + override val sqlConfsUnblock: Seq[String] = Seq(SQL_CONF_UNBLOCK_RENAME) + override val readerOptionsUnblock: Seq[String] = Seq(DeltaOptions.ALLOW_SOURCE_COLUMN_RENAME) } private case object SchemaChangeDrop extends SchemaChangeType { override val name = "DROP COLUMN" override val (isRename, isDrop, isTypeWidening) = (false, true, false) override val sqlConfsUnblock: Seq[String] = Seq(SQL_CONF_UNBLOCK_DROP) + override val readerOptionsUnblock: Seq[String] = Seq(DeltaOptions.ALLOW_SOURCE_COLUMN_DROP) } private case object SchemaChangeTypeWidening extends SchemaChangeType { override val name = "TYPE WIDENING" override val (isRename, isDrop, isTypeWidening) = (false, false, true) override val sqlConfsUnblock: Seq[String] = Seq(SQL_CONF_UNBLOCK_TYPE_CHANGE) + override val readerOptionsUnblock: Seq[String] = + Seq(DeltaOptions.ALLOW_SOURCE_COLUMN_TYPE_CHANGE) } // Combinations of rename, drop and type change -> can be caused by a complete overwrite. @@ -484,24 +489,32 @@ object DeltaSourceMetadataEvolutionSupport { override val name = "RENAME AND DROP COLUMN" override val (isRename, isDrop, isTypeWidening) = (true, true, false) override val sqlConfsUnblock: Seq[String] = Seq(SQL_CONF_UNBLOCK_RENAME_DROP) + override val readerOptionsUnblock: Seq[String] = + Seq(DeltaOptions.ALLOW_SOURCE_COLUMN_RENAME, DeltaOptions.ALLOW_SOURCE_COLUMN_DROP) } private case object SchemaChangeRenameAndTypeWidening extends SchemaChangeType { override val name = "RENAME AND TYPE WIDENING" override val (isRename, isDrop, isTypeWidening) = (true, false, true) override val sqlConfsUnblock: Seq[String] = Seq(SQL_CONF_UNBLOCK_RENAME, SQL_CONF_UNBLOCK_TYPE_CHANGE) + override val readerOptionsUnblock: Seq[String] = + Seq(DeltaOptions.ALLOW_SOURCE_COLUMN_RENAME, DeltaOptions.ALLOW_SOURCE_COLUMN_DROP) } private case object SchemaChangeDropAndTypeWidening extends SchemaChangeType { override val name = "DROP AND TYPE WIDENING" override val (isRename, isDrop, isTypeWidening) = (false, true, true) override val sqlConfsUnblock: Seq[String] = Seq(SQL_CONF_UNBLOCK_DROP, SQL_CONF_UNBLOCK_TYPE_CHANGE) + override val readerOptionsUnblock: Seq[String] = + Seq(DeltaOptions.ALLOW_SOURCE_COLUMN_DROP, DeltaOptions.ALLOW_SOURCE_COLUMN_TYPE_CHANGE) } private case object SchemaChangeRenameAndDropAndTypeWidening extends SchemaChangeType { override val name = "RENAME, DROP AND TYPE WIDENING" override val (isRename, isDrop, isTypeWidening) = (true, true, true) override val sqlConfsUnblock: Seq[String] = Seq(SQL_CONF_UNBLOCK_RENAME_DROP, SQL_CONF_UNBLOCK_TYPE_CHANGE) + override val readerOptionsUnblock: Seq[String] = + Seq(DeltaOptions.ALLOW_SOURCE_COLUMN_DROP, DeltaOptions.ALLOW_SOURCE_COLUMN_TYPE_CHANGE) } private final val allSchemaChangeTypes = Seq( @@ -541,11 +554,12 @@ object DeltaSourceMetadataEvolutionSupport { /** * Returns whether the given type of non-additive schema change was unblocked by setting one of - * the corresponding SQL confs. + * the corresponding SQL confs or reader options. */ private def isChangeUnblocked( spark: SparkSession, change: SchemaChangeType, + options: DeltaOptions, checkpointHash: Int, schemaChangeVersion: Long): Boolean = { @@ -561,11 +575,20 @@ object DeltaSourceMetadataEvolutionSupport { validConfKeysValuePair.exists(p => getConf(p._1).contains(p._2)) } - val isBlockedRename = change.isRename && !isUnblockedBySQLConf(SQL_CONF_UNBLOCK_RENAME) && + def isUnblockedByReaderOption(readerOption: Option[String]): Boolean = { + readerOption.contains("always") || readerOption.contains(schemaChangeVersion.toString) + } + + val isBlockedRename = change.isRename && + !isUnblockedByReaderOption(options.allowSourceColumnRename) && + !isUnblockedBySQLConf(SQL_CONF_UNBLOCK_RENAME) && !isUnblockedBySQLConf(SQL_CONF_UNBLOCK_RENAME_DROP) - val isBlockedDrop = change.isDrop && !isUnblockedBySQLConf(SQL_CONF_UNBLOCK_DROP) && + val isBlockedDrop = change.isDrop && + !isUnblockedByReaderOption(options.allowSourceColumnDrop) && + !isUnblockedBySQLConf(SQL_CONF_UNBLOCK_DROP) && !isUnblockedBySQLConf(SQL_CONF_UNBLOCK_RENAME_DROP) val isBlockedTypeChange = change.isTypeWidening && + !isUnblockedByReaderOption(options.allowSourceColumnTypeChange) && !isUnblockedBySQLConf(SQL_CONF_UNBLOCK_TYPE_CHANGE) !isBlockedRename && !isBlockedDrop && !isBlockedTypeChange @@ -576,7 +599,7 @@ object DeltaSourceMetadataEvolutionSupport { /** * Whether to accept widening type changes: * - when true, widening type changes cause the stream to fail, requesting user to review and - * unblock them via a SQL conf. + * unblock them via a SQL conf or reader option. * - when false, widening type changes are rejected without possibility to unblock, similar to * any other arbitrary type change. */ @@ -595,34 +618,45 @@ object DeltaSourceMetadataEvolutionSupport { // scalastyle:off /** * Given a non-additive operation type from a previous schema evolution, check we can process - * using the new schema given any SQL conf users have explicitly set to unblock. + * using the new schema given any SQL conf or dataframe reader option users have explicitly set to + * unblock. * The SQL conf can take one of following formats: * 1. spark.databricks.delta.streaming.allowSourceColumn$action = "always" * -> allows non-additive schema change to propagate for all streams. * 2. spark.databricks.delta.streaming.allowSourceColumn$action.$checkpointHash = "always" * -> allows non-additive schema change to propagate for this particular stream. * 3. spark.databricks.delta.streaming.allowSourceColumn$action.$checkpointHash = $deltaVersion - * -> allow non-additive schema change to propagate only for this particular stream source + * -> allow non-additive schema change to propagate only for this particular stream source + * table version. + * The reader options can take one of the following format: + * 1. .option("allowSourceColumn$action", "always") + * -> allows non-additive schema change to propagate for this particular stream. + * 2. .option("allowSourceColumn$action", "$deltaVersion") + * -> allow non-additive schema change to propagate only for this particular stream source * table version. * where `allowSourceColumn$action` is one of: * 1. `allowSourceColumnRename` to allow column renames. * 2. `allowSourceColumnDrop` to allow column drops. - * 3. `allowSourceColumnRenameAndDrop` to allow both column drops and renames. - * 4. `allowSourceColumnTypeChange` to allow widening type changes. + * 3. `allowSourceColumnTypeChange` to allow widening type changes. + * For SQL confs only, action can also be `allowSourceColumnRenameAndDrop` to allow both column + * drops and renames. * * We will check for any of these configs given the non-additive operation, and throw a proper - * error message to instruct the user to set the SQL conf if they would like to unblock. + * error message to instruct the user to set the SQL conf / reader options if they would like to + * unblock. * * @param metadataPath The path to the source-unique metadata location under checkpoint * @param currentSchema The current persisted schema * @param previousSchema The previous persisted schema */ // scalastyle:on - protected[sources] def validateIfSchemaChangeCanBeUnblockedWithSQLConf( + protected[sources] def validateIfSchemaChangeCanBeUnblocked( spark: SparkSession, + parameters: Map[String, String], metadataPath: String, currentSchema: PersistedMetadata, previousSchema: PersistedMetadata): Unit = { + val options = new DeltaOptions(parameters, spark.sessionState.conf) val checkpointHash = getCheckpointHash(metadataPath) // The start version of a possible series of consecutive schema changes. @@ -644,7 +678,7 @@ object DeltaSourceMetadataEvolutionSupport { determineNonAdditiveSchemaChangeType( spark, currentSchema.dataSchema, previousSchema.dataSchema).foreach { change => if (!isChangeUnblocked( - spark, change, checkpointHash, currentSchemaChangeVersion)) { + spark, change, options, checkpointHash, currentSchemaChangeVersion)) { // Throw error to prompt user to set the correct confs change match { case SchemaChangeTypeWidening => @@ -656,6 +690,7 @@ object DeltaSourceMetadataEvolutionSupport { previousSchemaChangeVersion, currentSchemaChangeVersion, checkpointHash, + change.readerOptionsUnblock, change.sqlConfsUnblock, wideningTypeChanges) @@ -665,6 +700,7 @@ object DeltaSourceMetadataEvolutionSupport { previousSchemaChangeVersion, currentSchemaChangeVersion, checkpointHash, + change.readerOptionsUnblock, change.sqlConfsUnblock) } } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataTrackingLog.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataTrackingLog.scala index df3e4317424..9abcbcf94d1 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataTrackingLog.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataTrackingLog.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.delta.sources // scalastyle:off import.ordering.noEmptyLine import java.io.InputStream +import scala.collection.JavaConverters._ import scala.util.control.NonFatal import org.apache.spark.sql.delta.streaming.{JsonSchemaSerializer, PartitionAndDataSchema, SchemaTrackingLog} @@ -33,6 +34,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap // scalastyle:on import.ordering.noEmptyLine /** @@ -240,10 +242,12 @@ object DeltaSourceMetadataTrackingLog extends Logging { sparkSession: SparkSession, rootMetadataLocation: String, sourceSnapshot: SnapshotDescriptor, - sourceTrackingId: Option[String] = None, + parameters: Map[String, String], sourceMetadataPathOpt: Option[String] = None, mergeConsecutiveSchemaChanges: Boolean = false, initMetadataLogEagerly: Boolean = true): DeltaSourceMetadataTrackingLog = { + val options = new CaseInsensitiveStringMap(parameters.asJava) + val sourceTrackingId = Option(options.get(DeltaOptions.STREAMING_SOURCE_TRACKING_ID)) val metadataTrackingLocation = fullMetadataTrackingLocation( rootMetadataLocation, sourceSnapshot.deltaLog.tableId, sourceTrackingId) val log = new DeltaSourceMetadataTrackingLog( @@ -296,7 +300,8 @@ object DeltaSourceMetadataTrackingLog extends Logging { (log.getPreviousTrackedMetadata, log.getCurrentTrackedMetadata, sourceMetadataPathOpt) match { case (Some(prev), Some(curr), Some(metadataPath)) => DeltaSourceMetadataEvolutionSupport - .validateIfSchemaChangeCanBeUnblockedWithSQLConf(sparkSession, metadataPath, curr, prev) + .validateIfSchemaChangeCanBeUnblocked( + sparkSession, parameters, metadataPath, curr, prev) case _ => } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingStatsTracker.scala b/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingStatsTracker.scala index eaff49795ba..407c6bd2fca 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingStatsTracker.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingStatsTracker.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.delta.stats import scala.collection.mutable import org.apache.spark.sql.delta.expressions.JoinedProjection +import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} @@ -27,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration @@ -87,14 +89,23 @@ class DeltaTaskStatisticsTracker( // This projection combines the intermediate results stored by aggBuffer with the values of the // currently processed row and updates aggBuffer in place. - private val updateStats: MutableProjection = GenerateMutableProjection.generate( - expressions = JoinedProjection.bind( - aggBufferAttrs, - dataCols, - aggregates.flatMap(_.updateExpressions)), - inputSchema = Nil, - useSubexprElimination = true - ) + private val updateStats: MutableProjection = { + val aggs = aggregates.flatMap(_.updateExpressions) + val expressions = JoinedProjection.bind(aggBufferAttrs, dataCols, aggs) + if (SQLConf.get.getConf( + DeltaSQLConf.DELTA_STATS_COLLECTION_FALLBACK_TO_INTERPRETED_PROJECTION)) { + MutableProjection.create( + exprs = expressions, + inputSchema = Nil + ) + } else { + GenerateMutableProjection.generate( + expressions = expressions, + inputSchema = Nil, + useSubexprElimination = true + ) + } + } // This executes the whole statsColExpr in order to compute the final stats value for the file. // In order to evaluate it, we have to replace its aggregate functions with the corresponding diff --git a/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala b/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala index 9cba024352f..399b542ef53 100644 --- a/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala +++ b/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala @@ -43,11 +43,6 @@ class DeltaVariantSuite import testImplicits._ - private def getProtocolForTable(table: String): Protocol = { - val deltaLog = DeltaLog.forTable(spark, TableIdentifier(table)) - deltaLog.unsafeVolatileSnapshot.protocol - } - private def assertVariantTypeTableFeatures( tableName: String, expectPreviewFeature: Boolean, diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaColumnMappingSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaColumnMappingSuite.scala index 6504c8943e6..411a7c2a7dc 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaColumnMappingSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaColumnMappingSuite.scala @@ -2106,4 +2106,49 @@ class DeltaColumnMappingSuite extends QueryTest s"Supported modes are: $supportedModes")) } } + + test("enabling column mapping disallowed if column mapping metadata already exists") { + withSQLConf( + // enabling this fixes the issue of committing invalid metadata in the first place + DeltaSQLConf.DELTA_COLUMN_MAPPING_STRIP_METADATA.key -> "false" + ) { + withTempDir { dir => + val path = dir.getCanonicalPath + val deltaLog = DeltaLog.forTable(spark, path) + deltaLog.withNewTransaction(catalogTableOpt = None) { txn => + val schema = + new StructType().add("id", IntegerType, true, withIdAndPhysicalName(0, "col-0")) + val metadata = actions.Metadata( + name = "test_table", + schemaString = schema.json, + configuration = Map(DeltaConfigs.COLUMN_MAPPING_MODE.key -> NoMapping.name) + ) + txn.updateMetadata(metadata) + txn.commit(Seq.empty, DeltaOperations.ManualUpdate) + + // Enabling the config will disallow enabling column mapping. + withSQLConf(DeltaSQLConf + .DELTA_COLUMN_MAPPING_DISALLOW_ENABLING_WHEN_METADATA_ALREADY_EXISTS.key + -> "true") { + val e = intercept[DeltaColumnMappingUnsupportedException] { + alterTableWithProps( + s"delta.`$path`", + Map(DeltaConfigs.COLUMN_MAPPING_MODE.key -> NameMapping.name)) + } + assert(e.getErrorClass == + "DELTA_ENABLING_COLUMN_MAPPING_DISALLOWED_WHEN_COLUMN_MAPPING_METADATA_ALREADY_EXISTS") + } + + // Disabling the config will allow enabling column mapping. + withSQLConf(DeltaSQLConf + .DELTA_COLUMN_MAPPING_DISALLOW_ENABLING_WHEN_METADATA_ALREADY_EXISTS.key + -> "false") { + alterTableWithProps( + s"delta.`$path`", + Map(DeltaConfigs.COLUMN_MAPPING_MODE.key -> NameMapping.name)) + } + } + } + } + } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala index c2ac1addfa0..2f737c68f04 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala @@ -2732,6 +2732,7 @@ trait DeltaErrorsSuiteBase nonAdditiveSchemaChangeOpType = "RENAME AND TYPE WIDENING", previousSchemaChangeVersion = 0, currentSchemaChangeVersion = 1, + readerOptionsUnblock = Seq("allowSourceColumnRename", "allowSourceColumnTypeChange"), sqlConfsUnblock = Seq( "spark.databricks.delta.streaming.allowSourceColumnRename", "spark.databricks.delta.streaming.allowSourceColumnTypeChange"), @@ -2743,6 +2744,12 @@ trait DeltaErrorsSuiteBase "opType" -> "RENAME AND TYPE WIDENING", "previousSchemaChangeVersion" -> "0", "currentSchemaChangeVersion" -> "1", + "unblockChangeOptions" -> + s""" .option("allowSourceColumnRename", "1") + | .option("allowSourceColumnTypeChange", "1")""".stripMargin, + "unblockStreamOptions" -> + s""" .option("allowSourceColumnRename", "always") + | .option("allowSourceColumnTypeChange", "always")""".stripMargin, "unblockChangeConfs" -> s""" SET spark.databricks.delta.streaming.allowSourceColumnRename.ckpt_15 = 1; | SET spark.databricks.delta.streaming.allowSourceColumnTypeChange.ckpt_15 = 1;""".stripMargin, @@ -2760,6 +2767,7 @@ trait DeltaErrorsSuiteBase throw DeltaErrors.cannotContinueStreamingTypeWidening( previousSchemaChangeVersion = 0, currentSchemaChangeVersion = 1, + readerOptionsUnblock = Seq("allowSourceColumnTypeChange"), sqlConfsUnblock = Seq("spark.databricks.delta.streaming.allowSourceColumnTypeChange"), checkpointHash = 15, wideningTypeChanges = Seq(TypeChange(None, IntegerType, LongType, Seq("a")))) @@ -2770,6 +2778,8 @@ trait DeltaErrorsSuiteBase "previousSchemaChangeVersion" -> "0", "currentSchemaChangeVersion" -> "1", "wideningTypeChanges" -> " a: INT -> BIGINT", + "unblockChangeOptions" -> " .option(\"allowSourceColumnTypeChange\", \"1\")", + "unblockStreamOptions" -> " .option(\"allowSourceColumnTypeChange\", \"always\")", "unblockChangeConfs" -> " SET spark.databricks.delta.streaming.allowSourceColumnTypeChange.ckpt_15 = 1;", "unblockStreamConfs" -> diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSchemaEvolutionSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSchemaEvolutionSuite.scala index d697052c49c..bf23b924561 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSchemaEvolutionSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSchemaEvolutionSuite.scala @@ -222,7 +222,8 @@ trait StreamingSchemaEvolutionSuiteBase extends ColumnMappingStreamingTestUtils initializeEagerly: Boolean = true )(implicit log: DeltaLog): DeltaSourceMetadataTrackingLog = DeltaSourceMetadataTrackingLog.create( - spark, getDefaultSchemaLocation.toString, log.update(), sourceTrackingId, + spark, getDefaultSchemaLocation.toString, log.update(), + parameters = sourceTrackingId.map(DeltaOptions.STREAMING_SOURCE_TRACKING_ID -> _).toMap, initMetadataLogEagerly = initializeEagerly) protected def getDefaultCheckpoint(implicit log: DeltaLog): Path = @@ -534,8 +535,10 @@ trait StreamingSchemaEvolutionSuiteBase extends ColumnMappingStreamingTestUtils // of the case; True concurrent execution would require commit service to protected against. val schemaLocation = getDefaultSchemaLocation.toString val snapshot = log.update() - val schemaLog1 = DeltaSourceMetadataTrackingLog.create(spark, schemaLocation, snapshot) - val schemaLog2 = DeltaSourceMetadataTrackingLog.create(spark, schemaLocation, snapshot) + val schemaLog1 = DeltaSourceMetadataTrackingLog.create( + spark, schemaLocation, snapshot, parameters = Map.empty) + val schemaLog2 = DeltaSourceMetadataTrackingLog.create( + spark, schemaLocation, snapshot, Map.empty) val newSchema = PersistedMetadata("1", 1, makeMetadata(new StructType(), partitionSchema = new StructType()), @@ -1607,9 +1610,9 @@ trait StreamingSchemaEvolutionSuiteBase extends ColumnMappingStreamingTestUtils // Both schema log initialized def schemaLog1: DeltaSourceMetadataTrackingLog = DeltaSourceMetadataTrackingLog.create( - spark, schemaLog1Location, log.update()) + spark, schemaLog1Location, log.update(), parameters = Map.empty) def schemaLog2: DeltaSourceMetadataTrackingLog = DeltaSourceMetadataTrackingLog.create( - spark, schemaLog2Location, log.update()) + spark, schemaLog2Location, log.update(), parameters = Map.empty) // The schema log initializes @ v5 with schema testStream(df)( diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableCreationTests.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableCreationTests.scala index 5b5db01d387..5199e19cb9b 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableCreationTests.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableCreationTests.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, ExternalCatalogUtils, SessionCatalog} import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumnsUtils import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, Table, TableCatalog} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -2409,6 +2410,117 @@ class DeltaTableCreationSuite } } } + + private def schemaContainsExistsDefaultKey(testTableName: String): Boolean = { + val (_, snapshot) = DeltaLog.forTableWithSnapshot(spark, TableIdentifier(testTableName)) + snapshot.metadata.schema.fields.exists( + _.metadata.contains(ResolveDefaultColumnsUtils.EXISTS_DEFAULT_COLUMN_METADATA_KEY)) + } + + test("Default column values: A table metadata update removes EXISTS_DEFAULT from a table") { + val testTableName = "test_table" + val metadataOperations = Seq( + s"ALTER TABLE $testTableName ALTER COLUMN int_col SET DEFAULT 2", + s"ALTER TABLE $testTableName CLUSTER BY (int_col)", + s"COMMENT ON TABLE $testTableName IS 'test comment'" + ) + + metadataOperations.foreach { metadataUpdatingQuery => + withTable(testTableName) { + // Create the table and ensure that EXISTS_DEFAULT is part of the schema. + withSQLConf(DeltaSQLConf + .REMOVE_EXISTS_DEFAULT_FROM_SCHEMA_ON_EVERY_METADATA_CHANGE.key -> "false") { + sql(s"""CREATE TABLE $testTableName(int_col INT DEFAULT 2) + |USING delta + |TBLPROPERTIES ('delta.feature.allowColumnDefaults' = 'supported')""".stripMargin) + assert(schemaContainsExistsDefaultKey(testTableName)) + } + + // Execute the metadata operation and assert that it removed EXISTS_DEFAULT from the schema. + withSQLConf(DeltaSQLConf + .REMOVE_EXISTS_DEFAULT_FROM_SCHEMA_ON_EVERY_METADATA_CHANGE.key -> "true") { + sql(metadataUpdatingQuery) + assert(!schemaContainsExistsDefaultKey(testTableName), + s"Operation '$metadataUpdatingQuery' did not remove EXISTS_DEFAULT from the schema.") + } + } + } + } + + test("Default column values: Writes to a table do not remove EXISTS_DEFAULT from a table") { + val testTableName = "test_table" + withTable(testTableName) { + // Add an EXISTS_DEFAULT entry to the schema by disabling the feature flag. + withSQLConf(DeltaSQLConf + .REMOVE_EXISTS_DEFAULT_FROM_SCHEMA_ON_EVERY_METADATA_CHANGE.key -> "false") { + sql(s"""CREATE TABLE $testTableName(int_col INT DEFAULT 2) + |USING delta + |TBLPROPERTIES ('delta.feature.allowColumnDefaults' = 'supported')""".stripMargin) + assert(schemaContainsExistsDefaultKey(testTableName)) + + withSQLConf(DeltaSQLConf + .REMOVE_EXISTS_DEFAULT_FROM_SCHEMA_ON_EVERY_METADATA_CHANGE.key -> "true") { + sql(s"INSERT INTO $testTableName VALUES (1)") + // Validate that EXISTS_DEFAULT is still part of the schema. + assert(schemaContainsExistsDefaultKey(testTableName)) + } + } + } + } + + test("Default column values: CREATE TABLE selecting from a table with dropped column defaults") { + for (sourceTableSchemaContainsKey <- Seq(true, false)) { + withTable("test_table", "test_table_2", "test_table_3") { + // To test with the 'EXISTS_DEFAULT' key present in the source table, we disable removal. + withSQLConf(DeltaSQLConf.REMOVE_EXISTS_DEFAULT_FROM_SCHEMA_ON_EVERY_METADATA_CHANGE.key + -> (!sourceTableSchemaContainsKey).toString) { + // Defaults are only possible for top level columns. + sql("""CREATE TABLE test_table(int_col INT DEFAULT 2) + |USING delta + |TBLPROPERTIES ('delta.feature.allowColumnDefaults' = 'supported')""".stripMargin) + } + + def schemaContainsExistsKey(tableName: String): Boolean = { + val (_, snapshot) = DeltaLog.forTableWithSnapshot(spark, TableIdentifier(tableName)) + snapshot.schema.fields.exists { field => + field.metadata.contains(ResolveDefaultColumnsUtils.EXISTS_DEFAULT_COLUMN_METADATA_KEY) + } + } + + def defaultsTableFeatureEnabled(tableName: String): Boolean = { + val (_, snapshot) = DeltaLog.forTableWithSnapshot(spark, TableIdentifier(tableName)) + val isEnabled = + snapshot.protocol.writerFeatureNames.contains(AllowColumnDefaultsTableFeature.name) + val schemaContainsCurrentDefaultKey = snapshot.schema.fields.exists { field => + field.metadata.contains( + ResolveDefaultColumnsUtils.CURRENT_DEFAULT_COLUMN_METADATA_KEY) + } + assert(schemaContainsCurrentDefaultKey === isEnabled) + isEnabled + } + + assert(schemaContainsExistsKey("test_table") == sourceTableSchemaContainsKey) + assert(defaultsTableFeatureEnabled("test_table")) + + // It is not possible to add a column with a default to a Delta table. + assertThrows[DeltaAnalysisException] { + sql("ALTER TABLE test_table ADD COLUMN new_column_with_a_default INT DEFAULT 0") + } + + // @TODO: It is currently not possible to CTAS from a table with an active column default + // without explicitly enabling the table feature. + assertThrows[AnalysisException] { + sql("CREATE TABLE test_table_2 USING DELTA AS SELECT * FROM test_table") + } + + sql("ALTER TABLE test_table ALTER COLUMN int_col DROP DEFAULT") + sql("CREATE TABLE test_table_3 USING DELTA AS SELECT * FROM test_table") + + assert(schemaContainsExistsKey("test_table_3") === false) + assert(!defaultsTableFeatureEnabled("test_table_3")) + } + } + } } trait DeltaTableCreationColumnMappingSuiteBase extends DeltaColumnMappingSelectedTestMixin { diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/TableRedirectSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/TableRedirectSuite.scala index f8ccc1ee708..6ededce0533 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/TableRedirectSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/TableRedirectSuite.scala @@ -101,197 +101,202 @@ class TableRedirectSuite extends QueryTest Seq(RedirectReaderWriter, RedirectWriterOnly).foreach { feature => val featureName = feature.config.key - redirectTest(s"basic redirect: $featureName") { case (deltaLog, _, dest, catalogTable) => - val snapshot = deltaLog.update() - assert(!feature.isFeatureSet(snapshot.metadata)) - val redirectSpec = new PathBasedRedirectSpec(dest.getCanonicalPath) - val catalogTableOpt = Some(catalogTable) - val redirectType = PathBasedRedirectSpec.REDIRECT_TYPE - // Step-1: Initiate table redirection and set to EnableRedirectInProgress state. - feature.add(deltaLog, catalogTableOpt, redirectType, redirectSpec) - validateState(deltaLog, EnableRedirectInProgress, dest, feature) - // Step-2: Complete table redirection and set to RedirectReady state. - feature.update(deltaLog, catalogTableOpt, RedirectReady, redirectSpec) - validateState(deltaLog, RedirectReady, dest, feature) - // Step-3: Start dropping table redirection and set to DropRedirectInProgress state. - feature.update(deltaLog, catalogTableOpt, DropRedirectInProgress, redirectSpec) - validateState(deltaLog, DropRedirectInProgress, dest, feature) - // Step-4: Finish dropping table redirection and remove the property completely. - feature.remove(deltaLog, catalogTableOpt) - validateRemovedState(deltaLog, feature) - // Step-5: Initiate table redirection and set to EnableRedirectInProgress state one - // more time. - withTempDir { destTablePath2 => - val redirectSpec = new PathBasedRedirectSpec(destTablePath2.getCanonicalPath) - feature.add(deltaLog, catalogTableOpt, redirectType, redirectSpec) - validateState(deltaLog, EnableRedirectInProgress, destTablePath2, feature) - // Step-6: Finish dropping table redirection and remove the property completely. - feature.remove(deltaLog, catalogTableOpt) - validateRemovedState(deltaLog, feature) - } - } - - redirectTest(s"Redirect $featureName: empty no redirect rules") { - case (deltaLog, source, dest, catalogTable) => + Seq(true, false).foreach { hasCatalogTable => + redirectTest(s"basic redirect: $featureName - " + + s"hasCatalogTable: $hasCatalogTable") { case (deltaLog, _, dest, catalogTable) => val snapshot = deltaLog.update() assert(!feature.isFeatureSet(snapshot.metadata)) val redirectSpec = new PathBasedRedirectSpec(dest.getCanonicalPath) - val catalogTableOpt = Some(catalogTable) + val catalogTableOpt = if (hasCatalogTable) Some(catalogTable) else None val redirectType = PathBasedRedirectSpec.REDIRECT_TYPE - // 0. Initialize table redirection by setting table to EnableRedirectInProgress state. + // Step-1: Initiate table redirection and set to EnableRedirectInProgress state. feature.add(deltaLog, catalogTableOpt, redirectType, redirectSpec) validateState(deltaLog, EnableRedirectInProgress, dest, feature) - - // 1. INSERT should hit DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE because table is in - // EnableRedirectInProgress, which doesn't allow any DML and DDL. - val exception1 = intercept[DeltaIllegalStateException] { - sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + // Step-2: Complete table redirection and set to RedirectReady state. + feature.update(deltaLog, catalogTableOpt, RedirectReady, redirectSpec) + validateState(deltaLog, RedirectReady, dest, feature) + // Step-3: Start dropping table redirection and set to DropRedirectInProgress state. + feature.update(deltaLog, catalogTableOpt, DropRedirectInProgress, redirectSpec) + validateState(deltaLog, DropRedirectInProgress, dest, feature) + // Step-4: Finish dropping table redirection and remove the property completely. + feature.remove(deltaLog, catalogTableOpt) + validateRemovedState(deltaLog, feature) + // Step-5: Initiate table redirection and set to EnableRedirectInProgress state one + // more time. + withTempDir { destTablePath2 => + val redirectSpec = new PathBasedRedirectSpec(destTablePath2.getCanonicalPath) + feature.add(deltaLog, catalogTableOpt, redirectType, redirectSpec) + validateState(deltaLog, EnableRedirectInProgress, destTablePath2, feature) + // Step-6: Finish dropping table redirection and remove the property completely. + feature.remove(deltaLog, catalogTableOpt) + validateRemovedState(deltaLog, feature) } - assert(exception1.getErrorClass == "DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE") + } - // 2. DDL should hit DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE because table is in - // EnableRedirectInProgress, which doesn't allow any DML and DDL. - val exception2 = intercept[DeltaIllegalStateException] { - sql(s"alter table delta.`$source` add column c3 long") - } - assert(exception2.getErrorClass == "DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE") + redirectTest(s"Redirect $featureName: empty no redirect rules - " + + s"hasCatalogTable: $hasCatalogTable") { + case (deltaLog, source, dest, catalogTable) => + val snapshot = deltaLog.update() + assert(!feature.isFeatureSet(snapshot.metadata)) + val redirectSpec = new PathBasedRedirectSpec(dest.getCanonicalPath) + val catalogTableOpt = if (hasCatalogTable) Some(catalogTable) else None + val redirectType = PathBasedRedirectSpec.REDIRECT_TYPE + // 0. Initialize table redirection by setting table to EnableRedirectInProgress state. + feature.add(deltaLog, catalogTableOpt, redirectType, redirectSpec) + validateState(deltaLog, EnableRedirectInProgress, dest, feature) - // 3. Move to RedirectReady state. - feature.update(deltaLog, catalogTableOpt, RedirectReady, redirectSpec) + // 1. INSERT should hit DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE because table is in + // EnableRedirectInProgress, which doesn't allow any DML and DDL. + val exception1 = intercept[DeltaIllegalStateException] { + sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + } + assert(exception1.getErrorClass == "DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE") - // 4. INSERT should hit DELTA_NO_REDIRECT_RULES_VIOLATED since the - // no-redirect-rules is empty. - validateState(deltaLog, RedirectReady, dest, feature) - val exception3 = intercept[DeltaIllegalStateException] { - sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") - } - assert(exception3.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") + // 2. DDL should hit DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE because table is in + // EnableRedirectInProgress, which doesn't allow any DML and DDL. + val exception2 = intercept[DeltaIllegalStateException] { + sql(s"alter table delta.`$source` add column c3 long") + } + assert(exception2.getErrorClass == "DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE") - // 5. DDL should hit DELTA_NO_REDIRECT_RULES_VIOLATED since the - // no-redirect-rules is empty. - val exception4 = intercept[DeltaIllegalStateException] { - sql(s"alter table delta.`$source` add column c3 long") - } - assert(exception4.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") + // 3. Move to RedirectReady state. + feature.update(deltaLog, catalogTableOpt, RedirectReady, redirectSpec) - // 6. Move to DropRedirectInProgress state. - feature.update(deltaLog, catalogTableOpt, DropRedirectInProgress, redirectSpec) + // 4. INSERT should hit DELTA_NO_REDIRECT_RULES_VIOLATED since the + // no-redirect-rules is empty. + validateState(deltaLog, RedirectReady, dest, feature) + val exception3 = intercept[DeltaIllegalStateException] { + sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + } + assert(exception3.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") - // 7. INSERT should hit DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE because table is in - // DropRedirectInProgress, which doesn't allow any DML and DDL. - validateState(deltaLog, DropRedirectInProgress, dest, feature) - val exception5 = intercept[DeltaIllegalStateException] { - sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") - } - assert(exception5.getErrorClass == "DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE") + // 5. DDL should hit DELTA_NO_REDIRECT_RULES_VIOLATED since the + // no-redirect-rules is empty. + val exception4 = intercept[DeltaIllegalStateException] { + sql(s"alter table delta.`$source` add column c3 long") + } + assert(exception4.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") - // 8. DDL should hit DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE because table is in - // DropRedirectInProgress, which doesn't allow any DML and DDL. - val exception6 = intercept[DeltaIllegalStateException] { - sql(s"alter table delta.`$source` add column c3 long") - } - assert(exception6.getErrorClass == "DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE") - } + // 6. Move to DropRedirectInProgress state. + feature.update(deltaLog, catalogTableOpt, DropRedirectInProgress, redirectSpec) - redirectTest(s"Redirect $featureName: no redirect rules") { - case (deltaLog, source, dest, catalogTable) => - val snapshot = deltaLog.update() - assert(!feature.isFeatureSet(snapshot.metadata)) - val redirectSpec = new PathBasedRedirectSpec(dest.getCanonicalPath) - val catalogTableOpt = Some(catalogTable) - val redirectType = PathBasedRedirectSpec.REDIRECT_TYPE - sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") - feature.add(deltaLog, catalogTableOpt, redirectType, redirectSpec) - validateState(deltaLog, EnableRedirectInProgress, dest, feature) - // 1. Move table redirect to RedirectReady state with no redirect rules that - // allows WRITE, DELETE, UPDATE. - var noRedirectRules = Set( - NoRedirectRule( - appName = None, - allowedOperations = Set( - DeltaOperations.Write(SaveMode.Append).name, - DeltaOperations.Delete(Seq.empty).name, - DeltaOperations.Update(None).name + // 7. INSERT should hit DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE because table is in + // DropRedirectInProgress, which doesn't allow any DML and DDL. + validateState(deltaLog, DropRedirectInProgress, dest, feature) + val exception5 = intercept[DeltaIllegalStateException] { + sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + } + assert(exception5.getErrorClass == "DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE") + + // 8. DDL should hit DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE because table is in + // DropRedirectInProgress, which doesn't allow any DML and DDL. + val exception6 = intercept[DeltaIllegalStateException] { + sql(s"alter table delta.`$source` add column c3 long") + } + assert(exception6.getErrorClass == "DELTA_COMMIT_INTERMEDIATE_REDIRECT_STATE") + } + + redirectTest(s"Redirect $featureName: no redirect rules - " + + s"hasCatalogTable: $hasCatalogTable") { + case (deltaLog, source, dest, catalogTable) => + val snapshot = deltaLog.update() + assert(!feature.isFeatureSet(snapshot.metadata)) + val redirectSpec = new PathBasedRedirectSpec(dest.getCanonicalPath) + val catalogTableOpt = if (hasCatalogTable) Some(catalogTable) else None + val redirectType = PathBasedRedirectSpec.REDIRECT_TYPE + sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + feature.add(deltaLog, catalogTableOpt, redirectType, redirectSpec) + validateState(deltaLog, EnableRedirectInProgress, dest, feature) + // 1. Move table redirect to RedirectReady state with no redirect rules that + // allows WRITE, DELETE, UPDATE. + var noRedirectRules = Set( + NoRedirectRule( + appName = None, + allowedOperations = Set( + DeltaOperations.Write(SaveMode.Append).name, + DeltaOperations.Delete(Seq.empty).name, + DeltaOperations.Update(None).name + ) ) ) - ) - feature.update(deltaLog, catalogTableOpt, RedirectReady, redirectSpec, noRedirectRules) - validateState(deltaLog, RedirectReady, dest, feature) - sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") - sql(s"update delta.`$source` set c0 = 100") - sql(s"delete from delta.`$source` where c0 = 1") + feature.update(deltaLog, catalogTableOpt, RedirectReady, redirectSpec, noRedirectRules) + validateState(deltaLog, RedirectReady, dest, feature) + sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + sql(s"update delta.`$source` set c0 = 100") + sql(s"delete from delta.`$source` where c0 = 1") - // 2. Move table redirect to RedirectReady state with no-redirect-rules that - // allows UPDATE. - noRedirectRules = Set( - NoRedirectRule( - appName = None, allowedOperations = Set(DeltaOperations.Update(None).name) + // 2. Move table redirect to RedirectReady state with no-redirect-rules that + // allows UPDATE. + noRedirectRules = Set( + NoRedirectRule( + appName = None, allowedOperations = Set(DeltaOperations.Update(None).name) + ) ) - ) - feature.update(deltaLog, catalogTableOpt, RedirectReady, redirectSpec, noRedirectRules) - validateState(deltaLog, RedirectReady, dest, feature) - // 2.1. WRITE should be aborted because no-redirect-rules only allow UPDATE. - val exception1 = intercept[DeltaIllegalStateException] { - sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") - } - assert(exception1.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") + feature.update(deltaLog, catalogTableOpt, RedirectReady, redirectSpec, noRedirectRules) + validateState(deltaLog, RedirectReady, dest, feature) + // 2.1. WRITE should be aborted because no-redirect-rules only allow UPDATE. + val exception1 = intercept[DeltaIllegalStateException] { + sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + } + assert(exception1.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") - // 2.2. UPDATE should pass because no-redirect-rules is fulfilled. - sql(s"update delta.`$source` set c0 = 100") + // 2.2. UPDATE should pass because no-redirect-rules is fulfilled. + sql(s"update delta.`$source` set c0 = 100") - // 2.3. DELETE should be aborted because no-redirect-rules only allow UPDATE. - val exception3 = intercept[DeltaIllegalStateException] { - sql(s"delete from delta.`$source` where c0 = 1") - } - assert(exception3.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") + // 2.3. DELETE should be aborted because no-redirect-rules only allow UPDATE. + val exception3 = intercept[DeltaIllegalStateException] { + sql(s"delete from delta.`$source` where c0 = 1") + } + assert(exception3.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") - // 2.4. Disabling SKIP_REDIRECT_FEATURE should allow all DMLs to pass. - withSQLConf(DeltaSQLConf.SKIP_REDIRECT_FEATURE.key -> "true") { - sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") - sql(s"delete from delta.`$source` where c0 = 1") - } + // 2.4. Disabling SKIP_REDIRECT_FEATURE should allow all DMLs to pass. + withSQLConf(DeltaSQLConf.SKIP_REDIRECT_FEATURE.key -> "true") { + sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + sql(s"delete from delta.`$source` where c0 = 1") + } - // 3. Move table redirect to RedirectReady state with no-redirect-rules that - // allows Write on appName "etl" . - noRedirectRules = Set( - NoRedirectRule( - appName = Some("etl"), - allowedOperations = Set(DeltaOperations.Write(SaveMode.Append).name) + // 3. Move table redirect to RedirectReady state with no-redirect-rules that + // allows Write on appName "etl" . + noRedirectRules = Set( + NoRedirectRule( + appName = Some("etl"), + allowedOperations = Set(DeltaOperations.Write(SaveMode.Append).name) + ) ) - ) - feature.update(deltaLog, catalogTableOpt, RedirectReady, redirectSpec, noRedirectRules) - validateState(deltaLog, RedirectReady, dest, feature) + feature.update(deltaLog, catalogTableOpt, RedirectReady, redirectSpec, noRedirectRules) + validateState(deltaLog, RedirectReady, dest, feature) - // 3.1. The WRITE of appName "dummy" would be aborted because no-redirect-rules - // only allow WRITE on application "etl". - val exception4 = intercept[DeltaIllegalStateException] { - spark.conf.set("spark.app.name", "dummy") - sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") - } - assert(exception4.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") + // 3.1. The WRITE of appName "dummy" would be aborted because no-redirect-rules + // only allow WRITE on application "etl". + val exception4 = intercept[DeltaIllegalStateException] { + spark.conf.set("spark.app.name", "dummy") + sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + } + assert(exception4.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") - // 3.1. WRITE should pass - spark.conf.set("spark.app.name", "etl") - sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + // 3.1. WRITE should pass + spark.conf.set("spark.app.name", "etl") + sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") - // 3.2. UPDATE should be aborted because no-redirect-rules only allow WRITE. - val exception5 = intercept[DeltaIllegalStateException] { - sql(s"update delta.`$source` set c0 = 100") - } - assert(exception5.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") + // 3.2. UPDATE should be aborted because no-redirect-rules only allow WRITE. + val exception5 = intercept[DeltaIllegalStateException] { + sql(s"update delta.`$source` set c0 = 100") + } + assert(exception5.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") - // 3.3. DELETE should be aborted because no-redirect-rules only allow WRITE. - val exception6 = intercept[DeltaIllegalStateException] { - sql(s"delete from delta.`$source` where c0 = 1") - } - assert(exception6.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") + // 3.3. DELETE should be aborted because no-redirect-rules only allow WRITE. + val exception6 = intercept[DeltaIllegalStateException] { + sql(s"delete from delta.`$source` where c0 = 1") + } + assert(exception6.getErrorClass == "DELTA_NO_REDIRECT_RULES_VIOLATED") - // 3.4. Disabling SKIP_REDIRECT_FEATURE should allow all DMLs to pass. - withSQLConf(DeltaSQLConf.SKIP_REDIRECT_FEATURE.key -> "true") { - sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") - sql(s"update delta.`$source` set c0 = 100") - sql(s"delete from delta.`$source` where c0 = 1") - } + // 3.4. Disabling SKIP_REDIRECT_FEATURE should allow all DMLs to pass. + withSQLConf(DeltaSQLConf.SKIP_REDIRECT_FEATURE.key -> "true") { + sql(s"insert into delta.`$source` values(1),(2),(3),(4),(5),(6)") + sql(s"update delta.`$source` set c0 = 100") + sql(s"delete from delta.`$source` where c0 = 1") + } + } } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClientImplSuiteBase.scala b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClientImplSuiteBase.scala index ba7642771e9..3995e6f2f51 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClientImplSuiteBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClientImplSuiteBase.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.concurrent.duration._ import org.apache.spark.sql.delta.DeltaLog -import org.apache.spark.sql.delta.actions.{CommitInfo, Metadata, Protocol} +import org.apache.spark.sql.delta.actions.{Action, CommitInfo, Metadata, Protocol} import org.apache.spark.sql.delta.storage.{LogStore, LogStoreProvider} import org.apache.spark.sql.delta.test.{DeltaSQLCommandTest, DeltaSQLTestUtils} import org.apache.spark.sql.delta.test.DeltaTestImplicits._ @@ -122,7 +122,10 @@ trait CommitCoordinatorClientImplSuiteBase extends QueryTest } protected def writeCommitZero(logPath: Path): Unit = { - store.write(FileNames.unsafeDeltaFile(logPath, 0), Iterator("0", "0"), overwrite = false) + val commitInfo = CommitInfo.empty(version = Some(0)).withTimestamp(0) + .copy(inCommitTimestamp = Some(0)) + val actions = Iterator(commitInfo.json, Metadata().json, Protocol().json) + store.write(FileNames.unsafeDeltaFile(logPath, 0), actions, overwrite = false) } /** @@ -163,7 +166,7 @@ trait CommitCoordinatorClientImplSuiteBase extends QueryTest } tableCommitCoordinatorClient.commit( version, - Iterator(s"$version", s"$timestamp"), + Iterator(commitInfo.json), updatedActions).getCommit } @@ -173,9 +176,13 @@ trait CommitCoordinatorClientImplSuiteBase extends QueryTest timestampOpt: Option[Long] = None): Unit = { val delta = FileNames.unsafeDeltaFile(logPath, version) if (timestampOpt.isDefined) { - assert(store.read(delta, sessionHadoopConf) == Seq(s"$version", s"${timestampOpt.get}")) + val commitInfo = CommitInfo.empty(version = Some(version)) + .withTimestamp(timestampOpt.get) + .copy(inCommitTimestamp = timestampOpt) + assert(store.read(delta, sessionHadoopConf).head == commitInfo.json) } else { - assert(store.read(delta, sessionHadoopConf).take(1) == Seq(s"$version")) + assert(Action.fromJson(store.read(delta, sessionHadoopConf).head) + .isInstanceOf[CommitInfo]) } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala index 5bd6a73c924..a35424c0f47 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala @@ -22,7 +22,7 @@ import java.util.regex.Pattern import scala.annotation.tailrec -import org.apache.spark.sql.delta.{DeltaAnalysisException, DeltaLog, DeltaTestUtils, TypeWideningMode} +import org.apache.spark.sql.delta.{DeltaAnalysisException, DeltaExcludedBySparkVersionTestMixinShims, DeltaLog, DeltaTestUtils, TypeWideningMode} import org.apache.spark.sql.delta.RowCommitVersion import org.apache.spark.sql.delta.RowId import org.apache.spark.sql.delta.commands.cdc.CDCReader @@ -48,8 +48,10 @@ class SchemaUtilsSuite extends QueryTest with SharedSparkSession with GivenWhenThen with DeltaSQLTestUtils - with DeltaSQLCommandTest { + with DeltaSQLCommandTest + with DeltaExcludedBySparkVersionTestMixinShims { import SchemaUtils._ + import TypeWideningMode._ import testImplicits._ private def expectFailure(shouldContain: String*)(f: => Unit): Unit = { @@ -71,8 +73,11 @@ class SchemaUtilsSuite extends QueryTest s"Error message '${e.getMessage}' didn't contain the patterns: $shouldContainPatterns") } - private def expectAnalysisErrorClass(errorClass: String, params: Map[String, String]) - (f: => Unit): Unit = { + private def expectAnalysisErrorClass( + errorClass: String, + params: Map[String, String], + matchPVals: Boolean = true)( + f: => Unit): Unit = { val e = intercept[AnalysisException] { f } @@ -90,7 +95,7 @@ class SchemaUtilsSuite extends QueryTest err.get, errorClass, parameters = params, - matchPVals = true) + matchPVals = matchPVals) } ///////////////////////////// @@ -2569,6 +2574,224 @@ class SchemaUtilsSuite extends QueryTest assert(mergedSchema === expected) } + private val allTypeWideningModes = Set( + NoTypeWidening, + AllTypeWidening, + TypeEvolution(uniformIcebergCompatibleOnly = false), + TypeEvolution(uniformIcebergCompatibleOnly = true), + AllTypeWideningToCommonWiderType, + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = true) + ) + + test("typeWideningMode - byte->short->int is always allowed") { + val narrow = new StructType() + .add("a", ByteType) + .add("b", ByteType) + .add("c", ShortType) + .add("s", new StructType().add("x", ByteType)) + .add("m", MapType(ByteType, ShortType)) + .add("ar", ArrayType(ByteType)) + + val wide = new StructType() + .add("a", ShortType) + .add("b", IntegerType) + .add("c", IntegerType) + .add("s", new StructType().add("x", IntegerType)) + .add("m", MapType(ShortType, IntegerType)) + .add("ar", ArrayType(IntegerType)) + + for (typeWideningMode <- allTypeWideningModes) { + // byte, short, int are all stored as INT64 in parquet, [[mergeSchemas]] always allows + // widening between them. This was already the case before typeWideningMode was introduced. + val merged1 = mergeSchemas(narrow, wide, typeWideningMode = typeWideningMode) + assert(merged1 === wide) + val merged2 = mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + assert(merged2 === wide) + } + } + + // These type changes will only be available once Delta uses Spark 4.0. + for ((fromType, toType) <- Seq( + IntegerType -> LongType, + new StructType().add("x", IntegerType) -> new StructType().add("x", LongType), + MapType(IntegerType, IntegerType) -> MapType(LongType, LongType), + ArrayType(IntegerType) -> ArrayType(LongType) + )) + testSparkMasterOnly(s"typeWideningMode ${fromType.sql} -> ${toType.sql}") { + val narrow = new StructType().add("a", fromType) + val wide = new StructType().add("a", toType) + + for (typeWideningMode <- Seq( + NoTypeWidening, + AllTypeWidening, + TypeEvolution(uniformIcebergCompatibleOnly = false), + TypeEvolution(uniformIcebergCompatibleOnly = true))) { + // Narrowing is not allowed. + expectAnalysisErrorClass("DELTA_MERGE_INCOMPATIBLE_DATATYPE", + Map("currentDataType" -> "LongType", "updateDataType" -> "IntegerType")) { + mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + } + } + + for (typeWideningMode <- Seq( + AllTypeWideningToCommonWiderType, + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = true))) { + // These modes don't enforce an order on the inputs, widening from second schema to first + // is allowed. + val merged = mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + assert(merged === wide) + } + + for (typeWideningMode <- allTypeWideningModes -- Set(NoTypeWidening)) { + // Widening is allowed, unless mode is NoTypeWidening. + val merged = mergeSchemas(narrow, wide, typeWideningMode = typeWideningMode) + assert(merged === wide) + } + expectAnalysisErrorClass("DELTA_MERGE_INCOMPATIBLE_DATATYPE", + Map("currentDataType" -> "LongType", "updateDataType" -> "IntegerType")) { + mergeSchemas(wide, narrow, typeWideningMode = NoTypeWidening) + } + } + + for ((fromType, toType) <- Seq( + ShortType -> DoubleType, + IntegerType -> DecimalType(10, 0) + )) + testSparkMasterOnly( + s"typeWideningMode - blocked type evolution ${fromType.sql} -> ${toType.sql}") { + val narrow = new StructType().add("a", fromType) + val wide = new StructType().add("a", toType) + + for (typeWideningMode <- Seq( + TypeEvolution(uniformIcebergCompatibleOnly = false), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false), + TypeEvolution(uniformIcebergCompatibleOnly = true), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = true))) { + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DATATYPE", + Map("currentDataType" -> fromType.toString, "updateDataType" -> toType.toString), + matchPVals = false) { + mergeSchemas(narrow, wide, typeWideningMode = typeWideningMode) + } + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DATATYPE", + Map("currentDataType" -> toType.toString, "updateDataType" -> fromType.toString), + matchPVals = false) { + mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + } + } + } + + for ((fromType, toType) <- Seq( + DateType -> TimestampNTZType, + DecimalType(10, 2) -> DecimalType(12, 4) + )) + testSparkMasterOnly( + s"typeWideningMode - Uniform Iceberg compatibility ${fromType.sql} -> ${toType.sql}") { + val narrow = new StructType().add("a", fromType) + val wide = new StructType().add("a", toType) + + def checkAnalysisException(f: => Unit): Unit = { + val ex = intercept[DeltaAnalysisException](f).getCause.asInstanceOf[AnalysisException] + // Decimal scale increase return a slightly different error class. + assert(ex.errorClass.contains("DELTA_MERGE_INCOMPATIBLE_DATATYPE") || + ex.errorClass.contains("DELTA_MERGE_INCOMPATIBLE_DECIMAL_TYPE")) + } + + for (typeWideningMode <- Seq( + TypeEvolution(uniformIcebergCompatibleOnly = false), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false))) { + // Unsupported type changes by Iceberg are allowed without Iceberg compatibility. + val merged = mergeSchemas(narrow, wide, typeWideningMode = typeWideningMode) + assert(merged === wide) + } + + for (typeWideningMode <- Seq( + TypeEvolution(uniformIcebergCompatibleOnly = true), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = true))) { + // Widening is blocked for unsupported type changes with Iceberg compatibility. + checkAnalysisException { + mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + } + } + + // These modes don't enforce an order on the inputs, widening from second schema to first + // is allowed without Iceberg compatibility. + val merged = mergeSchemas(wide, narrow, + typeWideningMode = TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false)) + assert(merged === wide) + + for (typeWideningMode <- Seq( + TypeEvolution(uniformIcebergCompatibleOnly = true), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = true), + TypeEvolution(uniformIcebergCompatibleOnly = true))) { + // Rejected either because this is a narrowing type change, or for the bidirectional mode, + // because it is not supported by Iceberg. + checkAnalysisException { + mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + } + } + } + + testSparkMasterOnly( + s"typeWideningMode - widen to common wider decimal") { + val left = new StructType().add("a", DecimalType(10, 2)) + val right = new StructType().add("a", DecimalType(5, 4)) + val wider = new StructType().add("a", DecimalType(12, 4)) + + val modesCanWidenToCommonWiderDecimal = Set( + // Increasing decimal scale isn't supported by Iceberg, so only possible when we don't enforce + // Iceberg compatibility. + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false), + AllTypeWideningToCommonWiderType + ) + + for (typeWideningMode <- modesCanWidenToCommonWiderDecimal) { + assert(mergeSchemas(left, right, typeWideningMode = typeWideningMode) === wider) + assert(mergeSchemas(right, left, typeWideningMode = typeWideningMode) === wider) + } + + for (typeWideningMode <- allTypeWideningModes -- modesCanWidenToCommonWiderDecimal) { + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DECIMAL_TYPE", + Map("decimalRanges" -> "precision 10 and 5 & scale 2 and 4"), + matchPVals = false) { + mergeSchemas(left, right, typeWideningMode = typeWideningMode) + } + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DECIMAL_TYPE", + Map("decimalRanges" -> "precision 5 and 10 & scale 4 and 2"), + matchPVals = false) { + mergeSchemas(right, left, typeWideningMode = typeWideningMode) + } + } + + } + + testSparkMasterOnly( + s"typeWideningMode - widen to common wider decimal exceeds max decimal precision") { + // We'd need a DecimalType(40, 19) to fit both types, which exceeds max decimal precision of 38. + val left = new StructType().add("a", DecimalType(20, 19)) + val right = new StructType().add("a", DecimalType(21, 0)) + + for (typeWideningMode <- allTypeWideningModes) { + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DECIMAL_TYPE", + Map("decimalRanges" -> "precision 20 and 21 & scale 19 and 0"), + matchPVals = false) { + mergeSchemas(left, right, typeWideningMode = typeWideningMode) + } + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DECIMAL_TYPE", + Map("decimalRanges" -> "precision 21 and 20 & scale 0 and 19"), + matchPVals = false) { + mergeSchemas(right, left, typeWideningMode = typeWideningMode) + } + } + } + //////////////////////////// // transformColumns //////////////////////////// diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataEvolutionSupportSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataEvolutionSupportSuite.scala index 4669b1d2666..13bcde4b303 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataEvolutionSupportSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/sources/DeltaSourceMetadataEvolutionSupportSuite.scala @@ -16,7 +16,7 @@ package org.apache.spark.sql.delta.sources -import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaTestUtilsBase, DeltaThrowable} +import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaOptions, DeltaTestUtilsBase, DeltaThrowable} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.sql.test.SharedSparkSession @@ -87,7 +87,7 @@ class DeltaSourceMetadataEvolutionSupportSuite } /** - * Unit test runner covering `validateIfSchemaChangeCanBeUnblockedWithSQLConf()`. Takes as input + * Unit test runner covering `validateIfSchemaChangeCanBeUnblocked()`. Takes as input * an initial schema (from) and an updated schema (to) and checks that: * 1. Non-additive schema changes are correctly detected: matches `expectedResult` * 2. Setting SQL confs to unblock the changes allows the check to succeeds. @@ -117,28 +117,41 @@ class DeltaSourceMetadataEvolutionSupportSuite unblock: Seq[Seq[String]] = Seq.empty, confs: Seq[(String, String)] = Seq.empty): Unit = test(s"$name") { - def validate(): Unit = - DeltaSourceMetadataEvolutionSupport.validateIfSchemaChangeCanBeUnblockedWithSQLConf( + def validate(parameters: Map[String, String]): Unit = + DeltaSourceMetadataEvolutionSupport.validateIfSchemaChangeCanBeUnblocked( spark, - metadataPath = "sourceMetadataPath", + parameters, + metadataPath = "sourceMetadataPath", currentSchema = persistedMetadata(toDDL, toPhysicalNames), previousSchema = persistedMetadata(fromDDL, fromPhysicalNames) ) withSQLConf(confs: _*) { expectedResult match { - case ExpectedResult.Success(_) => validate() + case ExpectedResult.Success(_) => validate(parameters = Map.empty) case ExpectedResult.Failure(checkError) => + // Run first without setting any configuration to unblock and check that the validation + // fails => column dropped, renamed or with changed type. val ex = intercept[DeltaThrowable] { - validate() + validate(parameters = Map.empty) } checkError(ex) // Verify that we can unblock using SQL confs for (u <- unblock) { withSQLConfUnblockedChanges(u) { - validate() + validate(parameters = Map.empty) } } + // Verify that we can unblock using dataframe reader options. + for (u <- unblock) { + val parameters = u.flatMap { + case "allowSourceColumnRenameAndDrop" => + Seq(DeltaOptions.ALLOW_SOURCE_COLUMN_RENAME -> "always", + DeltaOptions.ALLOW_SOURCE_COLUMN_DROP -> "always") + case option => Seq(option -> "always") + } + validate(parameters.toMap) + } } } } @@ -598,12 +611,55 @@ class DeltaSourceMetadataEvolutionSupportSuite test("combining individual SQL confs to unblock is supported") { withSQLConfUnblockedChanges(Seq("allowSourceColumnRename", "allowSourceColumnDrop")) { - DeltaSourceMetadataEvolutionSupport.validateIfSchemaChangeCanBeUnblockedWithSQLConf( + DeltaSourceMetadataEvolutionSupport.validateIfSchemaChangeCanBeUnblocked( + spark, + parameters = Map.empty, + metadataPath = "sourceMetadataPath", + currentSchema = persistedMetadata("a int", Map(Seq("a") -> "b")), + previousSchema = persistedMetadata("a int, b int", Map.empty) + ) + } + } + + test("combining SQL confs and reader options to unblock is supported") { + withSQLConfUnblockedChanges(Seq("allowSourceColumnRename")) { + DeltaSourceMetadataEvolutionSupport.validateIfSchemaChangeCanBeUnblocked( spark, + parameters = Map("allowSourceColumnDrop" -> "always"), metadataPath = "sourceMetadataPath", currentSchema = persistedMetadata("a int", Map(Seq("a") -> "b")), previousSchema = persistedMetadata("a int, b int", Map.empty) ) } } + + test("unblocking column drop for specific version with reader option is supported") { + DeltaSourceMetadataEvolutionSupport.validateIfSchemaChangeCanBeUnblocked( + spark, + parameters = Map("allowSourceColumnDrop" -> "0"), + metadataPath = "sourceMetadataPath", + currentSchema = persistedMetadata("a int", Map.empty), + previousSchema = persistedMetadata("a int, b int", Map.empty) + ) + } + + test("unblocking column rename for specific version with reader option is supported") { + DeltaSourceMetadataEvolutionSupport.validateIfSchemaChangeCanBeUnblocked( + spark, + parameters = Map("allowSourceColumnRename" -> "0"), + metadataPath = "sourceMetadataPath", + currentSchema = persistedMetadata("b int", Map(Seq("b") -> "a")), + previousSchema = persistedMetadata("a int", Map.empty) + ) + } + + test("unblocking column type change for specific version with reader option is supported") { + DeltaSourceMetadataEvolutionSupport.validateIfSchemaChangeCanBeUnblocked( + spark, + parameters = Map("allowSourceColumnTypeChange" -> "0"), + metadataPath = "sourceMetadataPath", + currentSchema = persistedMetadata("a int", Map.empty), + previousSchema = persistedMetadata("a byte", Map.empty) + ) + } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLTestUtils.scala b/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLTestUtils.scala index e16667e886d..6baa4ca12cd 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLTestUtils.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLTestUtils.scala @@ -18,7 +18,17 @@ package org.apache.spark.sql.delta.test import java.io.File +import scala.util.Random + +import org.apache.spark.sql.delta.{DeltaColumnMappingTestUtilsBase, DeltaLog, DeltaTable, Snapshot, TableFeature} +import org.apache.spark.sql.delta.actions.Protocol +import org.apache.spark.sql.delta.stats.{DeltaStatistics, PreparedDeltaFileIndex} +import com.fasterxml.jackson.databind.{JsonNode, ObjectMapper} + +import org.apache.spark.sql.{AnalysisException, DataFrame} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils trait DeltaSQLTestUtils extends SQLTestUtils { @@ -74,4 +84,224 @@ trait DeltaSQLTestUtils extends SQLTestUtils { files.foreach(Utils.deleteRecursively) } } + + /** Returns random alphanumberic string to be used as a unique table name. */ + def uniqueTableName: String = Random.alphanumeric.take(10).mkString + + /** Gets the latest snapshot of the table. */ + def getSnapshot(tableName: String): Snapshot = { + DeltaLog.forTable(spark, TableIdentifier(tableName)).update() + } + + /** Gets the table protocol of the latest snapshot. */ + def getProtocolForTable(tableName: String): Protocol = { + getSnapshot(tableName).protocol + } + /** Gets the `StructField` of `columnPath`. */ + final def getColumnField(schema: StructType, columnPath: Seq[String]): StructField = { + schema.findNestedField(columnPath, includeCollections = true).get._2 + } + + /** Gets the `StructField` of `columnName`. */ + def getColumnField(tableName: String, columnName: String): StructField = { + val deltaLog = DeltaLog.forTable(spark, TableIdentifier(tableName)) + getColumnField(deltaLog.update().schema, columnName.split("\\.")) + } + + /** Gets the `DataType` of `columnPath`. */ + def getColumnType(schema: StructType, columnPath: Seq[String]): DataType = { + getColumnField(schema, columnPath).dataType + } + + /** Gets the `DataType` of `columnName`. */ + def getColumnType(tableName: String, columnName: String): DataType = { + getColumnField(tableName, columnName).dataType + } + + /** + * Gets the stats fields from the AddFiles of `snapshot`. The stats are ordered by the + * modification time of the files they are associated with. + */ + def getUnvalidatedStatsOrderByFileModTime(snapshot: Snapshot): Array[JsonNode] = { + snapshot.allFiles + .orderBy("modificationTime") + .collect() + .map(file => new ObjectMapper().readTree(file.stats)) + } + + /** + * Gets the stats fields from the AddFiles of `tableName`. The stats are ordered by the + * modification time of the files they are associated with. + */ + def getUnvalidatedStatsOrderByFileModTime(tableName: String): Array[JsonNode] = + getUnvalidatedStatsOrderByFileModTime(getSnapshot(tableName)) + + /** Gets the physical column path if there is column mapping metadata in the schema. */ + def getPhysicalColumnPath(tableSchema: StructType, columnName: String): Seq[String] = { + new DeltaColumnMappingTestUtilsBase {}.getPhysicalPathForStats( + columnName.split("\\."), tableSchema + ).get + } + + /** Gets the value of a specified field from `stats` JSON node if it exists. */ + def getStatFieldOpt(stats: JsonNode, path: Seq[String]): Option[JsonNode] = + path.foldLeft(Option(stats)) { + case (Some(node), key) if node.has(key) => Option(node.get(key)) + case _ => None + } + + /** Gets the min/max stats of `columnName` from `stats` if they exist. */ + private def getMinMaxStatsOpt( + tableName: String, + stats: JsonNode, + columnName: String): (Option[String], Option[String]) = { + val columnPath = columnName.split('.') + val schema = getSnapshot(tableName).schema + val colType = getColumnType(schema, columnPath) + assert(colType.isInstanceOf[StringType], s"Expected StringType, got $colType") + + val physicalColumnPath = getPhysicalColumnPath(schema, columnName) + val minStatsPath = DeltaStatistics.MIN +: physicalColumnPath + val maxStatsPath = DeltaStatistics.MAX +: physicalColumnPath + ( + getStatFieldOpt(stats, minStatsPath).map(_.asText()), + getStatFieldOpt(stats, maxStatsPath).map(_.asText())) + } + + /** Gets the min/max stats of `columnName` from `stats`. */ + def getMinMaxStats( + tableName: String, + stats: JsonNode, + columnName: String): (String, String) = { + val (minOpt, maxOpt) = getMinMaxStatsOpt(tableName, stats, columnName) + (minOpt.get, maxOpt.get) + } + + /** Verifies whether there are min/max stats of `columnName` in `stats`. */ + def assertMinMaxStatsPresence( + tableName: String, + stats: JsonNode, + columnName: String, + expectStats: Boolean): Unit = { + val (minStats, maxStats) = getMinMaxStatsOpt(tableName, stats, columnName) + assert(minStats.isDefined === expectStats) + assert(maxStats.isDefined === expectStats) + } + + /** Verifies min/max stats values of `columnName` in `stats`. */ + def assertMinMaxStats( + tableName: String, + stats: JsonNode, + columnName: String, + expectedMin: String, + expectedMax: String): Unit = { + val (min, max) = + getMinMaxStats(tableName, stats, columnName) + assert(min === expectedMin, s"Expected $expectedMin, got $min") + assert(max === expectedMax, s"Expected $expectedMax, got $max") + } + + /** Verifies minReaderVersion and minWriterVersion of the protocol. */ + def assertProtocolVersion( + protocol: Protocol, + minReaderVersion: Int, + minWriterVersion: Int): Unit = { + assert(protocol.minReaderVersion === minReaderVersion) + assert(protocol.minWriterVersion === minWriterVersion) + } + + /** Verifies column is of expected data type. */ + def assertColumnDataType( + tableName: String, + columnName: String, + expectedDataType: DataType): Unit = { + assert(getColumnType(tableName, columnName) === expectedDataType) + } + + /** Verifies `columnName` does not exist in `tableName`. */ + def assertColumnNotExist(tableName: String, columnName: String): Unit = { + val e = intercept[AnalysisException] { + sql(s"SELECT $columnName FROM $tableName") + } + assert(e.getMessage.contains(s"`$columnName` cannot be resolved")) + } + + /** + * Runs `select` query on `tableName` with `predicate` and verifies the number of rows returned + * and files read. + */ + def assertSelectQueryResults( + tableName: String, + predicate: String, + numRows: Int, + numFilesRead: Int): Unit = { + val query = sql(s"SELECT * FROM $tableName WHERE $predicate") + assertSelectQueryResults(query, numRows, numFilesRead) + } + + /** + * Runs `query` and verifies the number of rows returned + * and files read. + */ + def assertSelectQueryResults( + query: DataFrame, + numRows: Int, + numFilesRead: Int): Unit = { + assert(query.count() === numRows, s"Expected $numRows rows, got ${query.count()}") + val filesRead = getNumReadFiles(query) + assert(filesRead === numFilesRead, s"Expected $numFilesRead files read, got $filesRead") + } + + /** Returns the number of read files by the query with given query text. */ + def getNumReadFiles(queryText: String): Int = { + getNumReadFiles(sql(queryText)) + } + + /** Returns the number of read files by the given data frame query. */ + def getNumReadFiles(df: DataFrame): Int = { + val deltaScans = df.queryExecution.optimizedPlan.collect { + case DeltaTable(prepared: PreparedDeltaFileIndex) => prepared.preparedScan + } + assert(deltaScans.size == 1) + deltaScans.head.files.length + } + + /** Drops `columnName` from `tableName`. */ + def dropColumn(tableName: String, columnName: String): Unit = { + sql(s"ALTER TABLE $tableName DROP COLUMN $columnName") + assertColumnNotExist(tableName, columnName) + } + + /** Changes `columnName` to `newType` */ + def alterColumnType(tableName: String, columnName: String, newType: String): Unit = { + sql(s"ALTER TABLE $tableName ALTER COLUMN $columnName TYPE $newType") + } + + /** Whether the table protocol supports the given table feature. */ + def isFeatureSupported(tableName: String, tableFeature: TableFeature): Boolean = { + val protocol = getProtocolForTable(tableName) + protocol.isFeatureSupported(tableFeature) + } + + /** Whether the table protocol supports the given table feature. */ + def isFeatureSupported(tableName: String, featureName: String): Boolean = { + val protocol = getProtocolForTable(tableName) + protocol.readerFeatureNames.contains(featureName) || + protocol.writerFeatureNames.contains(featureName) + } + + /** Enables table feature for `tableName` and given `featureName`. */ + def enableTableFeature(tableName: String, featureName: String): Unit = { + sql(s""" + |ALTER TABLE $tableName + |SET TBLPROPERTIES('delta.feature.$featureName' = 'supported') + |""".stripMargin) + assert(isFeatureSupported(tableName, featureName)) + } + + /** Drops table feature for `tableName` and `featureName`. */ + def dropTableFeature(tableName: String, featureName: String): Unit = { + sql(s"ALTER TABLE $tableName DROP FEATURE `$featureName`") + assert(!isFeatureSupported(tableName, featureName)) + } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningAlterTableNestedSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningAlterTableNestedSuite.scala index 476bfecbdec..3ca4c95e532 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningAlterTableNestedSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningAlterTableNestedSuite.scala @@ -112,7 +112,6 @@ trait TypeWideningAlterTableNestedTests { new MetadataBuilder() .putString("toType", "short") .putString("fromType", "byte") - .putLong("tableVersion", 2) .build() )).build())) .add("m", MapType(IntegerType, IntegerType), nullable = true, metadata = new MetadataBuilder() @@ -120,13 +119,11 @@ trait TypeWideningAlterTableNestedTests { new MetadataBuilder() .putString("toType", "integer") .putString("fromType", "byte") - .putLong("tableVersion", 3) .putString("fieldPath", "key") .build(), new MetadataBuilder() .putString("toType", "integer") .putString("fromType", "short") - .putLong("tableVersion", 4) .putString("fieldPath", "value") .build() )).build()) @@ -135,7 +132,6 @@ trait TypeWideningAlterTableNestedTests { new MetadataBuilder() .putString("toType", "integer") .putString("fromType", "short") - .putLong("tableVersion", 5) .putString("fieldPath", "element") .build() )).build())) @@ -162,7 +158,6 @@ trait TypeWideningAlterTableNestedTests { new MetadataBuilder() .putString("toType", "short") .putString("fromType", "byte") - .putLong("tableVersion", 2) .build() )).build())) .add("m", MapType(IntegerType, IntegerType), nullable = true, metadata = new MetadataBuilder() @@ -170,13 +165,11 @@ trait TypeWideningAlterTableNestedTests { new MetadataBuilder() .putString("toType", "integer") .putString("fromType", "byte") - .putLong("tableVersion", 2) .putString("fieldPath", "key") .build(), new MetadataBuilder() .putString("toType", "integer") .putString("fromType", "short") - .putLong("tableVersion", 2) .putString("fieldPath", "value") .build() )).build()) @@ -185,7 +178,6 @@ trait TypeWideningAlterTableNestedTests { new MetadataBuilder() .putString("toType", "integer") .putString("fromType", "short") - .putLong("tableVersion", 2) .putString("fieldPath", "element") .build() )).build())) diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningAlterTableSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningAlterTableSuite.scala index f772c9ca2aa..74b70ab95f2 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningAlterTableSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningAlterTableSuite.scala @@ -142,7 +142,6 @@ trait TypeWideningAlterTableTests new MetadataBuilder() .putString("toType", "integer") .putString("fromType", "short") - .putLong("tableVersion", 1) .build() )).build())) checkAnswer(readDeltaTable(tempPath), Seq(Row(1), Row(2))) diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningFeatureCompatibilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningFeatureCompatibilitySuite.scala index f29565f1212..e5969efda01 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningFeatureCompatibilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningFeatureCompatibilitySuite.scala @@ -131,7 +131,7 @@ trait TypeWideningCompatibilityTests { assert(readDeltaTable(tempPath).schema === new StructType() .add("a", ShortType, nullable = true, - metadata = typeWideningMetadata(version = 2, ByteType, ShortType)) + metadata = typeWideningMetadata(ByteType, ShortType)) .add("c", StringType, nullable = true, metadata = new MetadataBuilder() .putString("__CHAR_VARCHAR_TYPE_STRING", "char(3)") @@ -149,7 +149,7 @@ trait TypeWideningCompatibilityTests { assert(readDeltaTable(tempPath).schema === new StructType() .add("a", ShortType, nullable = true, - metadata = typeWideningMetadata(version = 2, ByteType, ShortType)) + metadata = typeWideningMetadata(ByteType, ShortType)) .add("c", StringType) .add("v", StringType)) checkAnswer(readDeltaTable(tempPath), diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningInsertSchemaEvolutionSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningInsertSchemaEvolutionSuite.scala index 49099315b08..89b49164819 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningInsertSchemaEvolutionSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningInsertSchemaEvolutionSuite.scala @@ -186,7 +186,7 @@ trait TypeWideningInsertSchemaEvolutionTests expectedResult = ExpectedResult.Success(new StructType() .add("a", IntegerType) .add("b", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType))) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType))) ) testInserts("top-level type evolution with column upcast")( @@ -197,7 +197,7 @@ trait TypeWideningInsertSchemaEvolutionTests expectedResult = ExpectedResult.Success(new StructType() .add("a", IntegerType) .add("b", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType)) .add("c", IntegerType)) ) @@ -209,7 +209,7 @@ trait TypeWideningInsertSchemaEvolutionTests expectedResult = ExpectedResult.Success(new StructType() .add("a", IntegerType) .add("b", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType)) .add("c", IntegerType)), // SQL INSERT by name doesn't support schema evolution. excludeInserts = insertsSQL.intersect(insertsByName) @@ -230,16 +230,14 @@ trait TypeWideningInsertSchemaEvolutionTests .add("s", new StructType() .add("x", ShortType) .add("y", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType))) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType))) .add("m", MapType(StringType, IntegerType), nullable = true, metadata = typeWideningMetadata( - version = 1, from = ShortType, to = IntegerType, path = Seq("value"))) .add("a", ArrayType(IntegerType), nullable = true, metadata = typeWideningMetadata( - version = 1, from = ShortType, to = IntegerType, path = Seq("element")))) @@ -260,17 +258,15 @@ trait TypeWideningInsertSchemaEvolutionTests .add("s", new StructType() .add("x", ShortType) .add("y", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType)) .add("z", IntegerType)) .add("m", MapType(StringType, IntegerType), nullable = true, metadata = typeWideningMetadata( - version = 1, from = ShortType, to = IntegerType, path = Seq("value"))) .add("a", ArrayType(IntegerType), nullable = true, metadata = typeWideningMetadata( - version = 1, from = ShortType, to = IntegerType, path = Seq("element")))) @@ -291,7 +287,7 @@ trait TypeWideningInsertSchemaEvolutionTests .add("s", new StructType() .add("x", IntegerType) .add("y", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)))) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType)))) ) // Interestingly, we introduced a special case to handle schema evolution / casting for structs @@ -311,7 +307,7 @@ trait TypeWideningInsertSchemaEvolutionTests .add("a", ArrayType(new StructType() .add("x", IntegerType) .add("y", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType))))) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType))))) ) // maps now allow type evolution for INSERT by position and name in SQL and dataframe. @@ -330,6 +326,6 @@ trait TypeWideningInsertSchemaEvolutionTests .add("m", MapType(StringType, new StructType() .add("x", IntegerType) .add("y", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType))))) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType))))) ) } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningMergeIntoSchemaEvolutionSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningMergeIntoSchemaEvolutionSuite.scala index 162722fd783..cd83b911a0f 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningMergeIntoSchemaEvolutionSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningMergeIntoSchemaEvolutionSuite.scala @@ -192,7 +192,7 @@ trait TypeWideningMergeIntoSchemaEvolutionTests result = Seq("""{ "a": 1 }""", """{ "a": 10 }"""), resultSchema = new StructType() .add("a", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType)) ) testTypeEvolution("change top-level column short -> int with insert")( @@ -205,7 +205,7 @@ trait TypeWideningMergeIntoSchemaEvolutionTests result = Seq("""{ "a": 0 }""", """{ "a": 10 }""", """{ "a": 20 }"""), resultSchema = new StructType() .add("a", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType)) ) testTypeEvolution("updating using narrower value doesn't evolve schema")( @@ -234,7 +234,7 @@ trait TypeWideningMergeIntoSchemaEvolutionTests """{ "a": 1, "b": 5 }""", """{ "a": 10, "b": 15 }"""), resultSchema = new StructType() .add("a", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType)) .add("b", ShortType) ) @@ -253,7 +253,7 @@ trait TypeWideningMergeIntoSchemaEvolutionTests resultSchema = new StructType() .add("s", new StructType() .add("a", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType))) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType))) ) testTypeEvolution("automatic widening of struct field with field assignment")( @@ -271,7 +271,7 @@ trait TypeWideningMergeIntoSchemaEvolutionTests resultSchema = new StructType() .add("s", new StructType() .add("a", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType))) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType))) ) testTypeEvolution("automatic widening of map value")( @@ -290,7 +290,6 @@ trait TypeWideningMergeIntoSchemaEvolutionTests MapType(StringType, IntegerType), nullable = true, metadata = typeWideningMetadata( - version = 1, from = ShortType, to = IntegerType, path = Seq("value"))) @@ -311,7 +310,6 @@ trait TypeWideningMergeIntoSchemaEvolutionTests ArrayType(IntegerType), nullable = true, metadata = typeWideningMetadata( - version = 1, from = ShortType, to = IntegerType, path = Seq("element"))) @@ -331,9 +329,9 @@ trait TypeWideningMergeIntoSchemaEvolutionTests result = Seq("""{ "a": 1, "b": 4 }""", """{ "a": 5, "b": 6 }"""), resultSchema = new StructType() .add("a", ShortType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ByteType, to = ShortType)) + metadata = typeWideningMetadata(from = ByteType, to = ShortType)) .add("b", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType)) ) for (enabled <- BOOLEAN_DOMAIN) diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningMetadataSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningMetadataSuite.scala index af96dccf410..26ae425fa18 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningMetadataSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningMetadataSuite.scala @@ -629,8 +629,7 @@ trait TypeWideningMetadataEndToEndTests { "metadata": { "delta.typeChanges": [{ "toType": "integer", - "fromType": "short", - "tableVersion": 1 + "fromType": "short" }] } }]}""".stripMargin) @@ -648,12 +647,10 @@ trait TypeWideningMetadataEndToEndTests { "metadata": { "delta.typeChanges": [{ "toType": "short", - "fromType": "byte", - "tableVersion": 1 + "fromType": "byte" },{ "toType": "integer", - "fromType": "short", - "tableVersion": 2 + "fromType": "short" }] } }]}""".stripMargin) @@ -678,8 +675,7 @@ trait TypeWideningMetadataEndToEndTests { "metadata": { "delta.typeChanges": [{ "toType": "short", - "fromType": "byte", - "tableVersion": 2 + "fromType": "byte" }] } }] @@ -691,7 +687,6 @@ trait TypeWideningMetadataEndToEndTests { "delta.typeChanges": [{ "toType": "integer", "fromType": "byte", - "tableVersion": 1, "fieldPath": "key" }] } @@ -717,7 +712,6 @@ trait TypeWideningMetadataEndToEndTests { "delta.typeChanges": [{ "toType": "short", "fromType": "byte", - "tableVersion": 1, "fieldPath": "element" }] } @@ -735,8 +729,7 @@ trait TypeWideningMetadataEndToEndTests { "metadata": { "delta.typeChanges": [{ "toType": "integer", - "fromType": "short", - "tableVersion": 2 + "fromType": "short" }] } }] diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningStreamingSinkSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningStreamingSinkSuite.scala index de5fd6c36ec..645add2a388 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningStreamingSinkSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningStreamingSinkSuite.scala @@ -144,7 +144,7 @@ class TypeWideningStreamingSinkSuite stream.write((12, 3456))("CAST(_1 AS INT) AS a", "CAST(_2 AS DECIMAL(10, 2)) AS b") assert(stream.currentSchema === new StructType() .add("a", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType)) .add("b", DecimalType(10, 2))) checkAnswer(stream.read(), Row(17, null) :: Row(12, 3456) :: Nil) } @@ -161,7 +161,7 @@ class TypeWideningStreamingSinkSuite stream.write((12, -1))("CAST(_1 AS INT) AS a") assert(stream.currentSchema === new StructType() .add("a", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType)) .add("b", LongType)) checkAnswer(stream.read(), Row(17, 45) :: Row(12, null) :: Nil) } @@ -189,7 +189,7 @@ class TypeWideningStreamingSinkSuite stream.write((12, -1))("CAST(_1 AS INT) AS c") assert(stream.currentSchema === new StructType().add("c", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 4, from = ShortType, to = IntegerType))) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType))) checkAnswer(stream.read(), Row(17) :: Row(12) :: Nil) } } @@ -221,7 +221,7 @@ class TypeWideningStreamingSinkSuite sink.addBatch(1, data) val df = spark.read.format("delta").load(tablePath) assert(df.schema === new StructType().add("value", IntegerType, nullable = true, - metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType))) + metadata = typeWideningMetadata(from = ShortType, to = IntegerType))) checkAnswer(df, Row(0) :: Row(1) :: Row(2) :: Row(3) :: Nil) } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningStreamingSourceSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningStreamingSourceSuite.scala index ec93c5c1d43..3db0862dd5a 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningStreamingSourceSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningStreamingSourceSuite.scala @@ -396,7 +396,7 @@ trait TypeWideningStreamingSourceTests ("unblock stream", (hash: Int) => s"allowSourceColumnTypeChange.ckpt_$hash", "always"), ("unblock version", (hash: Int) => s"allowSourceColumnTypeChange.ckpt_$hash", "2") )) { - test(s"unblocking stream after type change - $name") { + test(s"unblocking stream with sql conf after type change - $name") { withTempDir { dir => sql(s"CREATE TABLE delta.`$dir` (widened byte, other byte) USING DELTA") // Getting the checkpoint dir through the delta log to ensure the format is consistent with @@ -436,6 +436,44 @@ trait TypeWideningStreamingSourceTests } } + for ((name, optionValue) <- Seq( + ("unblock stream", "always"), + ("unblock version", "2") + )) { + test(s"unblocking stream with reader option after type change - $name") { + withTempDir { dir => + sql(s"CREATE TABLE delta.`$dir` (widened byte, other byte) USING DELTA") + val checkpointDir = new File(dir, "sink_checkpoint") + + def readWithAgg(options: Map[String, String] = Map.empty): DataFrame = + readStream(dir, checkpointDir, options) + .groupBy("other") + .agg(count(col("widened"))) + + testStream(readWithAgg(), outputMode = OutputMode.Complete())( + StartStream(checkpointLocation = checkpointDir.toString), + Execute { _ => sql(s"INSERT INTO delta.`$dir` VALUES (1, 1)") }, + Execute { _ => sql(s"ALTER TABLE delta.`$dir`ALTER COLUMN widened TYPE int") }, + ExpectMetadataEvolutionException() + ) + + testStream(readWithAgg(), outputMode = OutputMode.Complete())( + StartStream(checkpointLocation = checkpointDir.toString), + ExpectTypeChangeBlockedException() + ) + + testStream( + readWithAgg(Map("allowSourceColumnTypeChange" -> optionValue)), + outputMode = OutputMode.Complete())( + StartStream(checkpointLocation = checkpointDir.toString), + Execute { _ => sql(s"INSERT INTO delta.`$dir` VALUES (123456789, 1)") }, + ProcessAllAvailable(), + CheckLastBatch(Row(1, 2)) + ) + } + } + } + test(s"overwrite schema with type change and dropped column") { withTempDir { dir => sql(s"CREATE TABLE delta.`$dir` (a byte, b int) USING DELTA") @@ -470,6 +508,10 @@ trait TypeWideningStreamingSourceTests "opType" -> "DROP AND TYPE WIDENING", "previousSchemaChangeVersion" -> "0", "currentSchemaChangeVersion" -> "2", + "unblockChangeOptions" -> + ".*allowSourceColumnDrop(.|\\n)*allowSourceColumnTypeChange.*", + "unblockStreamOptions" -> + ".*allowSourceColumnDrop(.|\\n)*allowSourceColumnTypeChange.*", "unblockChangeConfs" -> ".*allowSourceColumnDrop(.|\\n)*allowSourceColumnTypeChange.*", "unblockStreamConfs" -> diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningTableFeatureSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningTableFeatureSuite.scala index ce859fcfead..e263f878cb6 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningTableFeatureSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningTableFeatureSuite.scala @@ -197,12 +197,12 @@ trait TypeWideningTableFeatureTests checkError( intercept[DeltaTableFeatureException] { sql(s"ALTER TABLE $databaseName.$tableName " + - s"DROP FEATURE '${TypeWideningPreviewTableFeature.name}'" + s"DROP FEATURE '${TypeWideningTableFeature.name}'" ).collect() }, "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( - "feature" -> TypeWideningPreviewTableFeature.name, + "feature" -> TypeWideningTableFeature.name, "logRetentionPeriodKey" -> DeltaConfigs.LOG_RETENTION.key, "logRetentionPeriod" -> DeltaConfigs.LOG_RETENTION .fromMetaData(deltaLog.unsafeVolatileMetadata).toString, @@ -471,7 +471,7 @@ trait TypeWideningTableFeatureTests schemaString = new StructType() .add("s", new StructType() .add("a", BooleanType, nullable = true, - metadata = typeWideningMetadata(version = 1, IntegerType, BooleanType))) + metadata = typeWideningMetadata(IntegerType, BooleanType))) .json )), ManualUpdate) @@ -508,20 +508,20 @@ trait TypeWideningTableFeatureTests Seq(txn.snapshot.metadata.copy( schemaString = new StructType() .add("a", StringType, nullable = true, - metadata = typeWideningMetadata(version = 1, StringType, CharType(4))) + metadata = typeWideningMetadata(StringType, CharType(4))) .add("b", StringType, nullable = true, - metadata = typeWideningMetadata(version = 1, StringType, VarcharType(4))) + metadata = typeWideningMetadata(StringType, VarcharType(4))) .add("c", StringType, nullable = true, - metadata = typeWideningMetadata(version = 1, CharType(4), StringType)) + metadata = typeWideningMetadata(CharType(4), StringType)) .add("d", StringType, nullable = true, - metadata = typeWideningMetadata(version = 1, CharType(4), VarcharType(4))) + metadata = typeWideningMetadata(CharType(4), VarcharType(4))) .add("e", StringType, nullable = true, - metadata = typeWideningMetadata(version = 1, VarcharType(4), StringType)) + metadata = typeWideningMetadata(VarcharType(4), StringType)) .add("f", StringType, nullable = true, - metadata = typeWideningMetadata(version = 1, VarcharType(4), CharType(4))) + metadata = typeWideningMetadata(VarcharType(4), CharType(4))) .add("s", new StructType() .add("x", StringType, nullable = true, - metadata = typeWideningMetadata(version = 1, StringType, CharType(4))) + metadata = typeWideningMetadata(StringType, CharType(4))) ) .json )), @@ -538,7 +538,6 @@ trait TypeWideningTableFeatureTests new MetadataBuilder() .putString("toType", "long") .putString("fromType", "int") - .putLong("tableVersion", 1) .build() )).build() @@ -562,7 +561,7 @@ trait TypeWideningTableFeatureTests "fieldPath" -> "a", "fromType" -> "INT", "toType" -> "BIGINT", - "typeWideningFeatureName" -> "typeWidening-preview" + "typeWideningFeatureName" -> "typeWidening" ) ) } @@ -667,7 +666,7 @@ trait TypeWideningTableFeatureTests // transient it will leave files behind that some clients can't read. withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") { val ex = intercept[SparkException] { - sql(s"ALTER TABLE delta.`$tempDir` DROP FEATURE '${TypeWideningPreviewTableFeature.name}'") + sql(s"ALTER TABLE delta.`$tempDir` DROP FEATURE '${TypeWideningTableFeature.name}'") } assert(ex.getMessage.contains("Cannot seek after EOF")) } @@ -703,27 +702,27 @@ trait TypeWideningTableFeatureTests addSingleFile(Seq(1), ByteType) sql(s"ALTER TABLE delta.`$tempPath` CHANGE COLUMN a TYPE int") - // The stable feature isn't supported and can't be dropped. - assertFeatureSupported(preview = true, stable = false) + // The preview feature isn't supported and can't be dropped. + assertFeatureSupported(preview = false, stable = true) dropTableFeature( - feature = TypeWideningTableFeature, + feature = TypeWideningPreviewTableFeature, expectedOutcome = ExpectedOutcome.FAIL_FEATURE_NOT_PRESENT, expectedNumFilesRewritten = 0, expectedColumnTypes = Map("a" -> ByteType) ) - // The preview feature is supported and can be dropped. + // The stable feature is supported and can be dropped. dropTableFeature( - feature = TypeWideningPreviewTableFeature, + feature = TypeWideningTableFeature, expectedOutcome = ExpectedOutcome.FAIL_CURRENT_VERSION_USES_FEATURE, expectedNumFilesRewritten = 1, expectedColumnTypes = Map("a" -> IntegerType) ) - assertFeatureSupported(preview = true, stable = false) + assertFeatureSupported(preview = false, stable = true) advancePastRetentionPeriod() dropTableFeature( - feature = TypeWideningPreviewTableFeature, + feature = TypeWideningTableFeature, expectedOutcome = ExpectedOutcome.SUCCESS, expectedNumFilesRewritten = 0, expectedColumnTypes = Map("a" -> IntegerType) @@ -778,39 +777,39 @@ trait TypeWideningTableFeatureTests assertFeatureSupported(preview = false, stable = false) } - test("tables created with the stable feature aren't automatically enabling the preview feature") { + test("tables created with the preview feature aren't automatically enabling the stable feature") { setupManualClock() sql(s"CREATE TABLE delta.`$tempPath` (a byte) USING DELTA " + s"TBLPROPERTIES ('${DeltaConfigs.ENABLE_TYPE_WIDENING.key}' = 'false')") - addTableFeature(tempPath, TypeWideningTableFeature) - assertFeatureSupported(preview = false, stable = true) + addTableFeature(tempPath, TypeWideningPreviewTableFeature) + assertFeatureSupported(preview = true, stable = false) - // Enable the table property, this should keep the stable feature but not add the preview one. + // Enable the table property, this should keep the preview feature but not add the stable one. enableTypeWidening(tempPath) - assertFeatureSupported(preview = false, stable = true) + assertFeatureSupported(preview = true, stable = false) addSingleFile(Seq(1), ByteType) sql(s"ALTER TABLE delta.`$tempPath` CHANGE COLUMN a TYPE int") dropTableFeature( - feature = TypeWideningPreviewTableFeature, + feature = TypeWideningTableFeature, expectedOutcome = ExpectedOutcome.FAIL_FEATURE_NOT_PRESENT, expectedNumFilesRewritten = 0, expectedColumnTypes = Map("a" -> ByteType) ) - // The stable table feature can be dropped. + // The preview table feature can be dropped. dropTableFeature( - feature = TypeWideningTableFeature, + feature = TypeWideningPreviewTableFeature, expectedOutcome = ExpectedOutcome.FAIL_CURRENT_VERSION_USES_FEATURE, expectedNumFilesRewritten = 1, expectedColumnTypes = Map("a" -> IntegerType) ) - assertFeatureSupported(preview = false, stable = true) + assertFeatureSupported(preview = true, stable = false) advancePastRetentionPeriod() dropTableFeature( - feature = TypeWideningTableFeature, + feature = TypeWideningPreviewTableFeature, expectedOutcome = ExpectedOutcome.SUCCESS, expectedNumFilesRewritten = 0, expectedColumnTypes = Map("a" -> IntegerType) diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningTestMixin.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningTestMixin.scala index 71ce4072162..9c48ccbab93 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningTestMixin.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningTestMixin.scala @@ -66,13 +66,12 @@ trait TypeWideningTestMixin extends DeltaSQLCommandTest with DeltaDMLTestUtils { /** Short-hand to create type widening metadata for struct fields. */ protected def typeWideningMetadata( - version: Long, from: AtomicType, to: AtomicType, path: Seq[String] = Seq.empty): Metadata = new MetadataBuilder() .putMetadataArray( - "delta.typeChanges", Array(TypeChange(Some(version), from, to, path).toMetadata)) + "delta.typeChanges", Array(TypeChange(None, from, to, path).toMetadata)) .build() def addSingleFile[T: Encoder](values: Seq[T], dataType: DataType): Unit = @@ -136,7 +135,7 @@ trait TypeWideningDropFeatureTestMixin * files all contain the expected type for specified columns. */ def dropTableFeature( - feature: TableFeature = TypeWideningPreviewTableFeature, + feature: TableFeature = TypeWideningTableFeature, expectedOutcome: ExpectedOutcome.Value, expectedNumFilesRewritten: Long, expectedColumnTypes: Map[String, DataType]): Unit = { diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/uniform/UniFormE2ESuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/uniform/UniFormE2ESuite.scala index e43c2404a90..3d24d1c3aa3 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/uniform/UniFormE2ESuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/uniform/UniFormE2ESuite.scala @@ -64,7 +64,7 @@ abstract class UniFormE2EIcebergSuiteBase extends UniFormE2ETest { "to_date('2016-12-31', 'yyyy-MM-dd')", "'asdf'", true, - "TIMESTAMP_NTZ'2021-12-06 00:00:00'", + "TIMESTAMP_NTZ'2021-12-06 05:12:34'", "TIMESTAMP'2023-08-18 05:00:00UTC-7'" ) @@ -88,7 +88,7 @@ abstract class UniFormE2EIcebergSuiteBase extends UniFormE2ETest { s"where ${partitionColumnName}=${partitionColumnsAndValues._2}" // Verify against Delta read and Iceberg read checkAnswer(spark.sql(verificationQuery), Seq(Row(123))) - checkAnswer(createReaderSparkSession.sql(verificationQuery), Seq(Row(123))) + assert(read(verificationQuery).sameElements(Seq(Row(123)))) } } } @@ -109,7 +109,7 @@ abstract class UniFormE2EIcebergSuiteBase extends UniFormE2ETest { s"where id=1 and ts=TIMESTAMP'2023-08-18 05:00:00UTC-7'" // Verify against Delta read and Iceberg read checkAnswer(spark.sql(verificationQuery), Seq(Row(123))) - checkAnswer(createReaderSparkSession.sql(verificationQuery), Seq(Row(123))) + assert(read(verificationQuery).sameElements(Seq(Row(123)))) } }