diff --git a/.github/workflows/spark_python_test.yaml b/.github/workflows/spark_python_test.yaml index 5e033ee7871..b183336316f 100644 --- a/.github/workflows/spark_python_test.yaml +++ b/.github/workflows/spark_python_test.yaml @@ -61,7 +61,7 @@ jobs: # `-SNAPSHOT` in version (e.g. `3.3.0-SNAPSHOT`) as the version is picked up from # the`version.sbt` file. pipenv run pip install pip==24.0 setuptools==69.5.1 wheel==0.43.0 - pipenv run pip install pyspark==3.5.0 + pipenv run pip install pyspark==3.5.3 pipenv run pip install flake8==3.5.0 pypandoc==1.3.3 pipenv run pip install black==23.9.1 pipenv run pip install importlib_metadata==3.10.0 diff --git a/.github/workflows/spark_test.yaml b/.github/workflows/spark_test.yaml index f521673f0f7..71893279bc0 100644 --- a/.github/workflows/spark_test.yaml +++ b/.github/workflows/spark_test.yaml @@ -65,7 +65,7 @@ jobs: # `-SNAPSHOT` in version (e.g. `3.3.0-SNAPSHOT`) as the version is picked up from # the`version.sbt` file. pipenv run pip install pip==24.0 setuptools==69.5.1 wheel==0.43.0 - pipenv run pip install pyspark==3.5.2 + pipenv run pip install pyspark==3.5.3 pipenv run pip install flake8==3.5.0 pypandoc==1.3.3 pipenv run pip install black==23.9.1 pipenv run pip install importlib_metadata==3.10.0 diff --git a/Dockerfile b/Dockerfile index 261b6d84a68..6da2b487bf7 100644 --- a/Dockerfile +++ b/Dockerfile @@ -38,7 +38,7 @@ RUN pip3 install --upgrade pip # the`version.sbt` file. RUN pip install pip==24.0 setuptools==69.5.1 wheel==0.43.0 -RUN pip3 install pyspark==3.5.2 +RUN pip3 install pyspark==3.5.3 RUN pip3 install mypy==0.982 diff --git a/benchmarks/build.sbt b/benchmarks/build.sbt index 277a132069b..ef07dd97427 100644 --- a/benchmarks/build.sbt +++ b/benchmarks/build.sbt @@ -20,7 +20,7 @@ scalaVersion := "2.12.18" lazy val root = (project in file(".")) .settings( name := "benchmarks", - libraryDependencies += "org.apache.spark" %% "spark-sql" % "3.5.2" % "provided", + libraryDependencies += "org.apache.spark" %% "spark-sql" % "3.5.3" % "provided", libraryDependencies += "com.github.scopt" %% "scopt" % "4.0.1", libraryDependencies += "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.13.1", diff --git a/build.sbt b/build.sbt index 151af7a67c2..cf58305da7c 100644 --- a/build.sbt +++ b/build.sbt @@ -52,7 +52,7 @@ val all_scala_versions = Seq(scala212, scala213) val default_scala_version = settingKey[String]("Default Scala version") Global / default_scala_version := scala212 -val LATEST_RELEASED_SPARK_VERSION = "3.5.2" +val LATEST_RELEASED_SPARK_VERSION = "3.5.3" val SPARK_MASTER_VERSION = "4.0.0-SNAPSHOT" val sparkVersion = settingKey[String]("Spark version") spark / sparkVersion := getSparkVersion() @@ -176,6 +176,7 @@ def crossSparkSettings(): Seq[Setting[_]] = getSparkVersion() match { Compile / unmanagedSourceDirectories += (Compile / baseDirectory).value / "src" / "main" / "scala-spark-3.5", Test / unmanagedSourceDirectories += (Test / baseDirectory).value / "src" / "test" / "scala-spark-3.5", Antlr4 / antlr4Version := "4.9.3", + Test / javaOptions ++= Seq("-Dlog4j.configurationFile=log4j2.properties"), // Java-/Scala-/Uni-Doc Settings scalacOptions ++= Seq( @@ -204,8 +205,9 @@ def crossSparkSettings(): Seq[Setting[_]] = getSparkVersion() match { "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", "--add-opens=java.base/sun.nio.cs=ALL-UNNAMED", "--add-opens=java.base/sun.security.action=ALL-UNNAMED", - "--add-opens=java.base/sun.util.calendar=ALL-UNNAMED" - ) + "--add-opens=java.base/sun.util.calendar=ALL-UNNAMED", + "-Dlog4j.configurationFile=log4j2_spark_master.properties" + ), // Java-/Scala-/Uni-Doc Settings // This isn't working yet against Spark Master. diff --git a/connectors/.github/workflows/new_pull_request.yaml b/connectors/.github/workflows/new_pull_request.yaml deleted file mode 100644 index 30b9389902a..00000000000 --- a/connectors/.github/workflows/new_pull_request.yaml +++ /dev/null @@ -1,16 +0,0 @@ -name: Add new pull requests to Backlog (External) - -on: - pull_request_target: - types: [opened, reopened] - -jobs: - automate-new-pull-requests: - if: ${{ !contains('allisonport-db dennyglee scottsand-db tdas zsxwing', github.event.sender.login) }} - runs-on: ubuntu-latest - steps: - - uses: alex-page/github-project-automation-plus@v0.8.1 - with: - project: oss-delta-prs - column: Needs Review - repo-token: ${{ secrets.PROJECT_BOARD_AUTOMATION_TOKEN }} diff --git a/connectors/.github/workflows/new_updated_issue.yaml b/connectors/.github/workflows/new_updated_issue.yaml deleted file mode 100644 index 9169e9a4486..00000000000 --- a/connectors/.github/workflows/new_updated_issue.yaml +++ /dev/null @@ -1,19 +0,0 @@ -name: Add new and updated issues to Needs Review - -on: - issues: - types: [opened, reopened] - issue_comment: - types: [created] - - -jobs: - automate-new-updated-issues: - if: ${{ !github.event.issue.pull_request && !contains('allisonport-db dennyglee scottsand-db tdas zsxwing', github.event.sender.login) }} - runs-on: ubuntu-latest - steps: - - uses: alex-page/github-project-automation-plus@v0.8.1 - with: - project: oss-delta-issues - column: Needs Review - repo-token: ${{ secrets.PROJECT_BOARD_AUTOMATION_TOKEN }} diff --git a/connectors/.github/workflows/test.yaml b/connectors/.github/workflows/test.yaml deleted file mode 100644 index 1a27feafb04..00000000000 --- a/connectors/.github/workflows/test.yaml +++ /dev/null @@ -1,43 +0,0 @@ -name: "Delta Lake Connectors Tests" -on: [push, pull_request] -jobs: - build: - name: "Run tests" - runs-on: ubuntu-20.04 - strategy: - matrix: - scala: [2.13.13, 2.12.18, 2.11.12] - steps: - - uses: actions/checkout@v2 - - name: install java - uses: actions/setup-java@v2 - with: - distribution: 'zulu' - java-version: '8' - - name: Cache Scala, SBT - uses: actions/cache@v2 - with: - path: | - ~/.sbt - ~/.ivy2 - ~/.cache/coursier - ~/.m2 - key: build-cache-3-with-scala_${{ matrix.scala }} - - name: Run Scala Style tests on test sources (Scala 2.12 only) - run: build/sbt "++ ${{ matrix.scala }}" testScalastyle - if: startsWith(matrix.scala, '2.12.') - - name: Run sqlDeltaImport tests (Scala 2.12 and 2.13 only) - run: build/sbt "++ ${{ matrix.scala }}" sqlDeltaImport/test - if: ${{ !startsWith(matrix.scala, '2.11.') }} - - name: Run Delta Standalone Compatibility tests (Scala 2.12 only) - run: build/sbt "++ ${{ matrix.scala }}" compatibility/test - if: startsWith(matrix.scala, '2.12.') - - name: Run Delta Standalone tests - run: build/sbt "++ ${{ matrix.scala }}" standalone/test testStandaloneCosmetic/test standaloneParquet/test testParquetUtilsWithStandaloneCosmetic/test - - name: Run Hive 3 tests - run: build/sbt "++ ${{ matrix.scala }}" hiveMR/test hiveTez/test - - name: Run Hive 2 tests - run: build/sbt "++ ${{ matrix.scala }}" hive2MR/test hive2Tez/test - - name: Run Flink tests (Scala 2.12 only) - run: build/sbt -mem 3000 "++ ${{ matrix.scala }}" flink/test - if: ${{ startsWith(matrix.scala, '2.12.') }} diff --git a/connectors/.github/workflows/updated_pull_request.yaml b/connectors/.github/workflows/updated_pull_request.yaml deleted file mode 100644 index d15a0075850..00000000000 --- a/connectors/.github/workflows/updated_pull_request.yaml +++ /dev/null @@ -1,20 +0,0 @@ -name: Move updated pull requests to Needs Review - -on: - issue_comment: - types: [created] - pull_request_target: - types: [synchronize] - -jobs: - automate-updated-pull-requests: - if: ${{ (github.event.issue.pull_request || github.event.pull_request) && - !contains('allisonport-db dennyglee scottsand-db tdas zsxwing', github.event.sender.login) && - (github.event.pull_request.state == 'open' || github.event.issue.state == 'open') }} - runs-on: ubuntu-latest - steps: - - uses: alex-page/github-project-automation-plus@2af3cf061aeca8ac6ab40a960eee1968a7f9ce0e # TODO: update to use a version after fixes are merged & released - with: - project: oss-delta-prs - column: Needs Review - repo-token: ${{ secrets.PROJECT_BOARD_AUTOMATION_TOKEN }} diff --git a/connectors/dev/README.md b/connectors/dev/README.md deleted file mode 100644 index 8a85deb2a6d..00000000000 --- a/connectors/dev/README.md +++ /dev/null @@ -1,38 +0,0 @@ -# Dev README -Below are some helpful IntelliJ configurations you can set to match our coding style and standards. - -## Checkstyle -This project uses checkstyle to format Java code. If developing locally, please setup checkstyle using the following steps. - -1. Add the CheckStyle-IDEA plugin to IntelliJ. -- `Settings > Plugins > Marketplace > CheckStyle-IDEA > INSTALL`. -- Restart your IDE if prompted. - -2. Configure IntelliJ to use the `checkstyle.xml` file provided in this directory. -- Go to `Settings > Tools > Checkstyle` (this tool location may differ based on your version of IntelliJ). -- Set the version to 8.29. -- Under the `Configuration File` heading, click the `+` symbol to add our specific configuration file. -- Give our file a useful description, such as `Delta Connectors Java Checks`, and provide the `connectors/dev/checkstyle.xml` path. -- Click `Next` to add the checkstyle file -- Check `Active` next to it once it has been added -- In the top right, set the Scan Scope to `Only Java sources (including tests)` - -3. Now, on the bottom tab bar, there should be a `CheckStyle` tab that lets you run Java style checks against using the `Check Project` button. - -4. You can also run checkstyle using SBT. For example, `build/sbt checkstyle` to run against all modules or `build/sbt standalone/checkstyle` to test only the `standalone` module. - -## Java Import Order -We use the following import order in our Java files. Please update this in `Settings > Editor > Code Style > Java > Imports > Import Layout`: - -``` -import java.* -import javax.* - -import scala.* - -import all other imports - -import io.delta.standalone.* -import io.delta.standalone.internal.* -``` - \ No newline at end of file diff --git a/docs/environment.yml b/docs/environment.yml index 89dda7b1b3a..eda28c4f724 100644 --- a/docs/environment.yml +++ b/docs/environment.yml @@ -33,7 +33,7 @@ dependencies: - packaging==23.2 - py4j==0.10.9.7 - pygments==2.16.1 - - pyspark==3.5.2 + - pyspark==3.5.3 - pytz==2023.3.post1 - requests==2.31.0 - six==1.16.0 diff --git a/examples/scala/build.sbt b/examples/scala/build.sbt index e253cba3b42..3bc6e285353 100644 --- a/examples/scala/build.sbt +++ b/examples/scala/build.sbt @@ -46,7 +46,7 @@ val lookupSparkVersion: PartialFunction[(Int, Int), String] = { // version 4.0.0-preview1 case (major, minor) if major >= 4 => "4.0.0-preview1" // versions 3.3.x+ - case (major, minor) if major >= 3 && minor >=3 => "3.5.2" + case (major, minor) if major >= 3 && minor >=3 => "3.5.3" // versions 3.0.0 to 3.2.x case (major, minor) if major >= 3 && minor <=2 => "3.5.0" // versions 2.4.x 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 deccf470b18..4c2949c8565 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 @@ -255,7 +255,7 @@ class IcebergConversionTransaction( } def getExpireSnapshotHelper(): ExpireSnapshotHelper = { - val ret = new ExpireSnapshotHelper(txn.expireSnapshots().cleanExpiredFiles(false)) + val ret = new ExpireSnapshotHelper(txn.expireSnapshots()) fileUpdates += ret ret } diff --git a/iceberg/src/test/scala/org/apache/spark/sql/delta/ConvertToIcebergSuite.scala b/iceberg/src/test/scala/org/apache/spark/sql/delta/ConvertToIcebergSuite.scala index ad60dcc6061..1f645a30db4 100644 --- a/iceberg/src/test/scala/org/apache/spark/sql/delta/ConvertToIcebergSuite.scala +++ b/iceberg/src/test/scala/org/apache/spark/sql/delta/ConvertToIcebergSuite.scala @@ -25,11 +25,9 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkContext import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogStorageFormat} import org.apache.spark.sql.delta.actions.Metadata -import org.apache.spark.sql.delta.icebergShaded.IcebergTransactionUtils -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, StringType, StructType, StructField} import org.apache.spark.util.Utils /** @@ -111,7 +109,6 @@ class ConvertToIcebergSuite extends QueryTest with Eventually { runDeltaSql( s"""CREATE TABLE `${testTableName}` (col1 INT) USING DELTA |TBLPROPERTIES ( - | 'delta.enableIcebergCompatV2' = 'true', | 'delta.columnMapping.mode' = 'name', | 'delta.universalFormat.enabledFormats' = 'iceberg' |)""".stripMargin) @@ -126,7 +123,6 @@ class ConvertToIcebergSuite extends QueryTest with Eventually { withDefaultTablePropsInSQLConf { deltaSpark.range(10).write.format("delta") .option("path", testTablePath) - .option("delta.enableIcebergCompatV2", "true") .saveAsTable(testTableName) } } @@ -134,77 +130,14 @@ class ConvertToIcebergSuite extends QueryTest with Eventually { deltaSpark.range(10, 20, 1) .write.format("delta").mode("append") .option("path", testTablePath) - .option("delta.enableIcebergCompatV2", "true") .saveAsTable(testTableName) } verifyReadWithIceberg(testTableName, 0 to 19 map (Row(_))) } } - test("Expire Snapshots") { - if (hmsReady(PORT)) { - runDeltaSql( - s"""CREATE TABLE `${testTableName}` (col1 INT) USING DELTA - |TBLPROPERTIES ( - | 'delta.enableIcebergCompatV2' = 'true', - | 'delta.columnMapping.mode' = 'name', - | 'delta.universalFormat.enabledFormats' = 'iceberg' - |)""".stripMargin) - - val icebergTable = loadIcebergTable() - icebergTable.updateProperties().set("history.expire.max-snapshot-age-ms", "1").commit() - - for (i <- 0 to 7) { - runDeltaSql(s"INSERT INTO ${testTableName} VALUES (${i})", - DeltaSQLConf.DELTA_UNIFORM_ICEBERG_SYNC_CONVERT_ENABLED.key -> "true") - } - - // Sleep past snapshot retention duration - Thread.sleep(5) - withIcebergSparkSession { icebergSpark => { - icebergSpark.sql(s"REFRESH TABLE $testTableName") - val manifestListsBeforeExpiration = icebergSpark - .sql(s"SELECT * FROM default.${testTableName}.snapshots") - .select("manifest_list") - .collect() - - assert(manifestListsBeforeExpiration.length == 8) - - // Trigger snapshot expiration - runDeltaSql(s"OPTIMIZE ${testTableName}") - icebergSpark.sql(s"REFRESH TABLE $testTableName") - - val manifestListsAfterExpiration = icebergSpark - .sql(s"SELECT * FROM default.${testTableName}.snapshots") - .select("manifest_list") - .collect() - - assert(manifestListsAfterExpiration.length == 1) - // Manifests from earlier snapshots should not be removed - manifestListsBeforeExpiration.toStream.foreach( - manifestList => assert( - icebergTable.io().newInputFile(manifestList.get(0).asInstanceOf[String]).exists())) - }} - } - } - - private def loadIcebergTable(): shadedForDelta.org.apache.iceberg.Table = { - withDeltaSparkSession { deltaSpark => { - val log = DeltaLog.forTable(deltaSpark, testTablePath) - val hiveCatalog = IcebergTransactionUtils.createHiveCatalog( - log.newDeltaHadoopConf() - ) - val table = hiveCatalog.loadTable( - shadedForDelta.org.apache.iceberg.catalog.TableIdentifier - .of("default", testTableName) - ) - table - }} - } - - def runDeltaSql(sqlStr: String, conf: (String, String)*): Unit = { + def runDeltaSql(sqlStr: String): Unit = { withDeltaSparkSession { deltaSpark => - conf.foreach(c => deltaSpark.conf.set(c._1, c._2)) deltaSpark.sql(sqlStr) } } diff --git a/icebergShaded/iceberg_src_patches/0002-iceberg-core-must-not-delete-any-delta-data-files.patch b/icebergShaded/iceberg_src_patches/0002-iceberg-core-must-not-delete-any-delta-data-files.patch new file mode 100644 index 00000000000..a181f065040 --- /dev/null +++ b/icebergShaded/iceberg_src_patches/0002-iceberg-core-must-not-delete-any-delta-data-files.patch @@ -0,0 +1,177 @@ +iceberg core must NOT delete any delta data files + +--- + .../iceberg/IncrementalFileCleanup.java | 8 +-- + .../apache/iceberg/ReachableFileCleanup.java | 5 +- + .../apache/iceberg/TestRemoveSnapshots.java | 57 +++++++++++-------- + 3 files changed, 40 insertions(+), 30 deletions(-) + +diff --git a/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java b/connector/iceberg-core/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java +index d894dcbf36d..ead7ea6b076 100644 +--- a/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java ++++ b/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java +@@ -256,10 +256,10 @@ class IncrementalFileCleanup extends FileCleanupStrategy { + } + }); + +- Set filesToDelete = +- findFilesToDelete(manifestsToScan, manifestsToRevert, validIds, afterExpiration); +- +- deleteFiles(filesToDelete, "data"); ++ // iceberg core MUST NOT delete any data files which are managed by delta ++ // Set filesToDelete = ++ // findFilesToDelete(manifestsToScan, manifestsToRevert, validIds, afterExpiration); ++ // deleteFiles(filesToDelete, "data"); + LOG.warn("Manifests to delete: {}", Joiner.on(", ").join(manifestsToDelete)); + LOG.warn("Manifests Lists to delete: {}", Joiner.on(", ").join(manifestListsToDelete)); + deleteFiles(manifestsToDelete, "manifest"); +diff --git a/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java b/connector/iceberg-core/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java +index ccbee78e27b..da888a63b3d 100644 +--- a/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java ++++ b/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java +@@ -72,8 +72,9 @@ class ReachableFileCleanup extends FileCleanupStrategy { + snapshotsAfterExpiration, deletionCandidates, currentManifests::add); + + if (!manifestsToDelete.isEmpty()) { +- Set dataFilesToDelete = findFilesToDelete(manifestsToDelete, currentManifests); +- deleteFiles(dataFilesToDelete, "data"); ++ // iceberg core MUST NOT delete any data files which are managed by delta ++ // Set dataFilesToDelete = findFilesToDelete(manifestsToDelete, currentManifests); ++ // deleteFiles(dataFilesToDelete, "data"); + Set manifestPathsToDelete = + manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet()); + deleteFiles(manifestPathsToDelete, "manifest"); +diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/connector/iceberg-core/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +index 53e5af520d9..95fa8e41de1 100644 +--- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java ++++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +@@ -147,8 +147,9 @@ public class TestRemoveSnapshots extends TableTestBase { + secondSnapshot + .allManifests(table.io()) + .get(0) +- .path(), // manifest contained only deletes, was dropped +- FILE_A.path()), // deleted ++ .path() // manifest contained only deletes, was dropped ++ // FILE_A.path() should NOT delete data files ++ ), // deleted + deletedFiles); + } + +@@ -209,8 +210,9 @@ public class TestRemoveSnapshots extends TableTestBase { + .allManifests(table.io()) + .get(0) + .path(), // manifest was rewritten for delete +- secondSnapshot.manifestListLocation(), // snapshot expired +- FILE_A.path()), // deleted ++ secondSnapshot.manifestListLocation() // snapshot expired ++ // FILE_A.path() should not delete any data files ++ ), + deletedFiles); + } + +@@ -309,8 +311,9 @@ public class TestRemoveSnapshots extends TableTestBase { + Sets.newHashSet( + secondSnapshot.manifestListLocation(), // snapshot expired + Iterables.getOnlyElement(secondSnapshotManifests) +- .path(), // manifest is no longer referenced +- FILE_B.path()), // added, but rolled back ++ .path() // manifest is no longer referenced ++ // FILE_B.path() should not delete any data files ++ ), + deletedFiles); + } + +@@ -686,7 +689,8 @@ public class TestRemoveSnapshots extends TableTestBase { + + removeSnapshots(table).expireOlderThan(t3).deleteWith(deletedFiles::add).commit(); + +- Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); ++ Assert.assertTrue("FILE_A should NOT be deleted", ++ !deletedFiles.contains(FILE_A.path().toString())); + } + + @Test +@@ -712,7 +716,8 @@ public class TestRemoveSnapshots extends TableTestBase { + + removeSnapshots(table).expireOlderThan(t3).deleteWith(deletedFiles::add).commit(); + +- Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); ++ Assert.assertTrue("FILE_A should NOT be deleted", ++ !deletedFiles.contains(FILE_A.path().toString())); + } + + @Test +@@ -749,8 +754,10 @@ public class TestRemoveSnapshots extends TableTestBase { + + removeSnapshots(table).expireOlderThan(t4).deleteWith(deletedFiles::add).commit(); + +- Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); +- Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString())); ++ Assert.assertTrue("FILE_A should NOT be deleted", ++ !deletedFiles.contains(FILE_A.path().toString())); ++ Assert.assertTrue("FILE_B should NOT be deleted", ++ !deletedFiles.contains(FILE_B.path().toString())); + } + + @Test +@@ -824,9 +831,11 @@ public class TestRemoveSnapshots extends TableTestBase { + Sets.newHashSet( + "remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3")); + +- Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); +- Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString())); +- Assert.assertTrue("Thread should be created in provided pool", planThreadsIndex.get() > 0); ++ Assert.assertTrue("FILE_A should NOT be deleted", ++ !deletedFiles.contains(FILE_A.path().toString())); ++ Assert.assertTrue("FILE_B should NOT be deleted", ++ !deletedFiles.contains(FILE_B.path().toString())); ++ // Assert.assertTrue("Thread should be created in provided pool", planThreadsIndex.get() > 0); + } + + @Test +@@ -885,13 +894,13 @@ public class TestRemoveSnapshots extends TableTestBase { + Set expectedDeletes = Sets.newHashSet(); + expectedDeletes.add(snapshotA.manifestListLocation()); + +- // Files should be deleted of dangling staged snapshot +- snapshotB +- .addedDataFiles(table.io()) +- .forEach( +- i -> { +- expectedDeletes.add(i.path().toString()); +- }); ++ // Files should NOT be deleted of dangling staged snapshot ++ // snapshotB ++ // .addedDataFiles(table.io()) ++ // .forEach( ++ // i -> { ++ // expectedDeletes.add(i.path().toString()); ++ // }); + + // ManifestList should be deleted too + expectedDeletes.add(snapshotB.manifestListLocation()); +@@ -1144,10 +1153,10 @@ public class TestRemoveSnapshots extends TableTestBase { + removeSnapshots(table).expireOlderThan(fourthSnapshotTs).deleteWith(deletedFiles::add).commit(); + + Assert.assertEquals( +- "Should remove old delete files and delete file manifests", ++ "Should only delete file manifests", + ImmutableSet.builder() +- .add(FILE_A.path()) +- .add(FILE_A_DELETES.path()) ++ // .add(FILE_A.path()) ++ // .add(FILE_A_DELETES.path()) + .add(firstSnapshot.manifestListLocation()) + .add(secondSnapshot.manifestListLocation()) + .add(thirdSnapshot.manifestListLocation()) +@@ -1501,7 +1510,7 @@ public class TestRemoveSnapshots extends TableTestBase { + expectedDeletes.addAll(manifestPaths(appendA, table.io())); + expectedDeletes.add(branchDelete.manifestListLocation()); + expectedDeletes.addAll(manifestPaths(branchDelete, table.io())); +- expectedDeletes.add(FILE_A.path().toString()); ++ // expectedDeletes.add(FILE_A.path().toString()); + + Assert.assertEquals(2, Iterables.size(table.snapshots())); + Assert.assertEquals(expectedDeletes, deletedFiles); +-- +2.39.2 (Apple Git-143) diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/data/ColumnarBatch.java b/kernel/kernel-api/src/main/java/io/delta/kernel/data/ColumnarBatch.java index f954b2e652b..fb9214f8588 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/data/ColumnarBatch.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/data/ColumnarBatch.java @@ -86,17 +86,6 @@ default ColumnarBatch withNewSchema(StructType newSchema) { throw new UnsupportedOperationException("Not yet implemented"); } - /** - * Return a slice of the current batch. - * - * @param start Starting record index to include in the returned columnar batch - * @param end Ending record index (exclusive) to include in the returned columnar batch - * @return a columnar batch containing the records between [start, end) - */ - default ColumnarBatch slice(int start, int end) { - throw new UnsupportedOperationException("Not yet implemented!"); - } - /** @return iterator of {@link Row}s in this batch */ default CloseableIterator getRows() { final ColumnarBatch batch = this; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/engine/FileSystemClient.java b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/FileSystemClient.java index 8814a86602c..769a29ee48f 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/engine/FileSystemClient.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/FileSystemClient.java @@ -76,4 +76,13 @@ CloseableIterator readFiles(CloseableIterator unsupportedFeatures) { String message = String.format( - "Unsupported Delta reader feature: table `%s` requires reader table feature \"%s\" " + "Unsupported Delta reader features: table `%s` requires reader table features [%s] " + "which is unsupported by this version of Delta Kernel.", - tablePath, readerFeature); + tablePath, String.join(", ", unsupportedFeatures)); return new KernelException(message); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java index 04ee11f6803..ce47da8327e 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java @@ -38,6 +38,8 @@ import java.io.UncheckedIOException; import java.util.*; import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Exposes APIs to read the raw actions within the *commit files* of the _delta_log. This is used @@ -47,13 +49,15 @@ public class DeltaLogActionUtils { private DeltaLogActionUtils() {} + private static final Logger logger = LoggerFactory.getLogger(DeltaLogActionUtils.class); + ///////////////// // Public APIs // ///////////////// /** * Represents a Delta action. This is used to request which actions to read from the commit files - * in {@link TableImpl#getChangesByVersion(Engine, long, long, Set)}. + * in {@link TableImpl#getChanges(Engine, long, long, Set)}. * *

See the Delta protocol for more details * https://github.com/delta-io/delta/blob/master/PROTOCOL.md#actions @@ -262,14 +266,25 @@ private static CloseableIterator listLogDir( private static List listCommitFiles( Engine engine, Path tablePath, long startVersion, long endVersion) { + // TODO update to support coordinated commits; suggested to load the Snapshot at endVersion + // and get the backfilled/unbackfilled commits from the LogSegment to combine with commit files + // listed from [startVersion, LogSegment.checkpointVersion] + logger.info( + "{}: Listing the commit files for versions [{}, {}]", tablePath, startVersion, endVersion); + long startTimeMillis = System.currentTimeMillis(); final List output = new ArrayList<>(); try (CloseableIterator fsIter = listLogDir(engine, tablePath, startVersion)) { while (fsIter.hasNext()) { FileStatus fs = fsIter.next(); if (!FileNames.isCommitFile(getName(fs.getPath()))) { + logger.debug("Ignoring non-commit file {}", fs.getPath()); continue; } if (FileNames.getFileVersion(new Path(fs.getPath())) > endVersion) { + logger.debug( + "Stopping listing found file {} with version > {}=endVersion", + fs.getPath(), + endVersion); break; } output.add(fs); @@ -277,6 +292,12 @@ private static List listCommitFiles( } catch (IOException e) { throw new UncheckedIOException("Unable to close resource", e); } + logger.info( + "{}: Took {} ms to list the commit files for versions [{}, {}]", + tablePath, + System.currentTimeMillis() - startTimeMillis, + startVersion, + endVersion); return output; } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableConfig.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableConfig.java index e68c84dff91..acebf4d521c 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableConfig.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableConfig.java @@ -78,6 +78,30 @@ public class TableConfig { "needs to be a positive integer.", true); + /** + * The shortest duration we have to keep delta/checkpoint files around before deleting them. We + * can only delete delta files that are before a checkpoint. + */ + public static final TableConfig LOG_RETENTION = + new TableConfig<>( + "delta.logRetentionDuration", + "interval 30 days", + (engineOpt, v) -> IntervalParserUtils.safeParseIntervalAsMillis(v), + value -> true, + "needs to be provided as a calendar interval such as '2 weeks'. Months " + + "and years are not accepted. You may specify '365 days' for a year instead.", + true /* editable */); + + /** Whether to clean up expired checkpoints and delta logs. */ + public static final TableConfig EXPIRED_LOG_CLEANUP_ENABLED = + new TableConfig<>( + "delta.enableExpiredLogCleanup", + "true", + (engineOpt, v) -> Boolean.valueOf(v), + value -> true, + "needs to be a boolean.", + true /* editable */); + /** * This table property is used to track the enablement of the {@code inCommitTimestamps}. * diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableFeatures.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableFeatures.java index 7b595d9fbc9..4aa66ec2fff 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableFeatures.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableFeatures.java @@ -24,10 +24,7 @@ import io.delta.kernel.internal.util.ColumnMapping; import io.delta.kernel.internal.util.Tuple2; import io.delta.kernel.types.StructType; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; +import java.util.*; import java.util.stream.Collectors; /** Contains utility methods related to the Delta table feature support in protocol. */ @@ -43,34 +40,40 @@ public class TableFeatures { } }); + private static final Set SUPPORTED_READER_FEATURES = + Collections.unmodifiableSet( + new HashSet() { + { + add("columnMapping"); + add("deletionVectors"); + add("timestampNtz"); + add("vacuumProtocolCheck"); + add("variantType-preview"); + add("v2Checkpoint"); + } + }); + //////////////////// // Helper Methods // //////////////////// public static void validateReadSupportedTable( - Protocol protocol, Metadata metadata, String tablePath) { + Protocol protocol, String tablePath, Optional metadata) { switch (protocol.getMinReaderVersion()) { case 1: break; case 2: - ColumnMapping.throwOnUnsupportedColumnMappingMode(metadata); + metadata.ifPresent(ColumnMapping::throwOnUnsupportedColumnMappingMode); break; case 3: List readerFeatures = protocol.getReaderFeatures(); - for (String readerFeature : readerFeatures) { - switch (readerFeature) { - case "columnMapping": - ColumnMapping.throwOnUnsupportedColumnMappingMode(metadata); - break; - case "deletionVectors": // fall through - case "timestampNtz": // fall through - case "vacuumProtocolCheck": // fall through - case "variantType-preview": // fall through - case "v2Checkpoint": - break; - default: - throw DeltaErrors.unsupportedReaderFeature(tablePath, readerFeature); - } + if (!SUPPORTED_READER_FEATURES.containsAll(readerFeatures)) { + Set unsupportedFeatures = new HashSet<>(readerFeatures); + unsupportedFeatures.removeAll(SUPPORTED_READER_FEATURES); + throw DeltaErrors.unsupportedReaderFeature(tablePath, unsupportedFeatures); + } + if (readerFeatures.contains("columnMapping")) { + metadata.ifPresent(ColumnMapping::throwOnUnsupportedColumnMappingMode); } break; default: 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 2dd6c6b2877..d71ef4c30be 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 @@ -18,11 +18,13 @@ import static io.delta.kernel.internal.DeltaErrors.wrapEngineExceptionThrowsIO; import io.delta.kernel.*; +import io.delta.kernel.data.ColumnVector; import io.delta.kernel.data.ColumnarBatch; import io.delta.kernel.engine.Engine; import io.delta.kernel.exceptions.CheckpointAlreadyExistsException; import io.delta.kernel.exceptions.KernelException; import io.delta.kernel.exceptions.TableNotFoundException; +import io.delta.kernel.internal.actions.Protocol; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.snapshot.SnapshotManager; import io.delta.kernel.internal.util.Clock; @@ -32,11 +34,18 @@ import io.delta.kernel.utils.FileStatus; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TableImpl implements Table { + + private static final Logger logger = LoggerFactory.getLogger(TableImpl.class); + public static Table forPath(Engine engine, String path) { return forPath(engine, path, System::currentTimeMillis); } @@ -99,7 +108,7 @@ public Snapshot getSnapshotAsOfTimestamp(Engine engine, long millisSinceEpochUTC @Override public void checkpoint(Engine engine, long version) throws TableNotFoundException, CheckpointAlreadyExistsException, IOException { - snapshotManager.checkpoint(engine, version); + snapshotManager.checkpoint(engine, clock, version); } @Override @@ -112,6 +121,66 @@ public Clock getClock() { return clock; } + /** + * Returns delta actions for each version between startVersion and endVersion. Only returns the + * actions requested in actionSet. + * + *

For the returned columnar batches: + * + *

    + *
  • Each row within the same batch is guaranteed to have the same commit version + *
  • The batch commit versions are monotonically increasing + *
  • The top-level columns include "version", "timestamp", and the actions requested in + * actionSet. "version" and "timestamp" are the first and second columns in the schema, + * respectively. The remaining columns are based on the actions requested and each have the + * schema found in {@code DeltaAction.schema}. + *
+ * + * @param engine {@link Engine} instance to use in Delta Kernel. + * @param startVersion start version (inclusive) + * @param endVersion end version (inclusive) + * @param actionSet the actions to read and return from the JSON log files + * @return an iterator of batches where each row in the batch has exactly one non-null action and + * its commit version and timestamp + * @throws TableNotFoundException if the table does not exist or if it is not a delta table + * @throws KernelException if a commit file does not exist for any of the versions in the provided + * range + * @throws KernelException if provided an invalid version range + * @throws KernelException if the version range contains a version with reader protocol that is + * unsupported by Kernel + */ + public CloseableIterator getChanges( + Engine engine, + long startVersion, + long endVersion, + Set actionSet) { + // Create a new action set that always contains protocol + Set copySet = new HashSet<>(actionSet); + copySet.add(DeltaLogActionUtils.DeltaAction.PROTOCOL); + // If protocol is not in the original requested actions we drop the column before returning + boolean shouldDropProtocolColumn = + !actionSet.contains(DeltaLogActionUtils.DeltaAction.PROTOCOL); + + return getRawChanges(engine, startVersion, endVersion, copySet) + .map( + batch -> { + int protocolIdx = batch.getSchema().indexOf("protocol"); // must exist + ColumnVector protocolVector = batch.getColumnVector(protocolIdx); + for (int rowId = 0; rowId < protocolVector.getSize(); rowId++) { + if (!protocolVector.isNullAt(rowId)) { + Protocol protocol = Protocol.fromColumnVector(protocolVector, rowId); + TableFeatures.validateReadSupportedTable( + protocol, getDataPath().toString(), Optional.empty()); + } + } + if (shouldDropProtocolColumn) { + return batch.withDeletedColumnAt(protocolIdx); + } else { + return batch; + } + }); + } + protected Path getDataPath() { return new Path(tablePath); } @@ -226,12 +295,14 @@ public long getVersionAtOrAfterTimestamp(Engine engine, long millisSinceEpochUTC * range * @throws KernelException if provided an invalid version range */ - public CloseableIterator getChangesByVersion( + private CloseableIterator getRawChanges( Engine engine, long startVersion, long endVersion, Set actionSet) { + logger.info( + "{}: Getting the commit files for versions [{}, {}]", tablePath, startVersion, endVersion); List commitFiles = DeltaLogActionUtils.getCommitFilesForVersionRange( engine, new Path(tablePath), startVersion, endVersion); @@ -242,6 +313,7 @@ public CloseableIterator getChangesByVersion( .map(action -> new StructField(action.colName, action.schema, true)) .collect(Collectors.toList())); + logger.info("{}: Reading the commit files with readSchema {}", tablePath, readSchema); return DeltaLogActionUtils.readCommitFiles(engine, commitFiles, readSchema); } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/lang/ListUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/lang/ListUtils.java index 78c6970db29..06ba6cab887 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/lang/ListUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/lang/ListUtils.java @@ -18,6 +18,7 @@ import io.delta.kernel.internal.util.Tuple2; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -34,4 +35,22 @@ public static Tuple2, List> partition( public static T last(List list) { return list.get(list.size() - 1); } + + /** Remove once supported JDK (build) version is 21 or above */ + public static T getFirst(List list) { + if (list.isEmpty()) { + throw new NoSuchElementException(); + } else { + return list.get(0); + } + } + + /** Remove once supported JDK (build) version is 21 or above */ + public static T getLast(List list) { + if (list.isEmpty()) { + throw new NoSuchElementException(); + } else { + return list.get(list.size() - 1); + } + } } 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 1d1a229ca31..170dad9ac1f 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 @@ -233,7 +233,8 @@ protected Tuple2 loadTableProtocolAndMetadata( if (protocol != null) { // Stop since we have found the latest Protocol and Metadata. - TableFeatures.validateReadSupportedTable(protocol, metadata, dataPath.toString()); + TableFeatures.validateReadSupportedTable( + protocol, dataPath.toString(), Optional.of(metadata)); return new Tuple2<>(protocol, metadata); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/MetadataCleanup.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/MetadataCleanup.java new file mode 100644 index 00000000000..0b405426751 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/MetadataCleanup.java @@ -0,0 +1,200 @@ +/* + * Copyright (2024) 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.snapshot; + +import static io.delta.kernel.internal.DeltaErrors.wrapEngineExceptionThrowsIO; +import static io.delta.kernel.internal.checkpoints.Checkpointer.getLatestCompleteCheckpointFromList; +import static io.delta.kernel.internal.lang.ListUtils.getFirst; +import static io.delta.kernel.internal.lang.ListUtils.getLast; +import static io.delta.kernel.internal.util.Preconditions.checkArgument; +import static java.util.stream.Collectors.toList; + +import io.delta.kernel.engine.Engine; +import io.delta.kernel.internal.checkpoints.CheckpointInstance; +import io.delta.kernel.internal.fs.Path; +import io.delta.kernel.internal.util.Clock; +import io.delta.kernel.internal.util.FileNames; +import io.delta.kernel.utils.CloseableIterator; +import io.delta.kernel.utils.FileStatus; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class MetadataCleanup { + private static final Logger logger = LoggerFactory.getLogger(MetadataCleanup.class); + + private MetadataCleanup() {} + + /** + * Delete the Delta log files (delta and checkpoint files) that are expired according to the table + * metadata retention settings. While deleting the log files, it makes sure the time travel + * continues to work for all unexpired table versions. + * + *

Here is algorithm: + * + *

    + *
  • Initial the potential delete file list: `potentialFilesToDelete` as an empty list + *
  • Initialize the last seen checkpoint file list: `lastSeenCheckpointFiles`. There could be + * one or more checkpoint files for a given version. + *
  • List the delta log files starting with prefix "00000000000000000000." (%020d). For each + * file: + *
      + *
    • Step 1: Check if the `lastSeenCheckpointFiles` contains a complete checkpoint, then + *
        + *
      • Step 1.1: delete all files in `potentialFilesToDelete`. Now we know there is + * a checkpoint that contains the compacted Delta log up to the checkpoint + * version and all commit/checkpoint files before this checkpoint version are + * not needed. + *
      • Step 1.2: add `lastCheckpointFiles` to `potentialFileStoDelete` list. This + * checkpoint is potential candidate to delete later if we find another + * checkpoint + *
      + *
    • Step 2: If the timestamp falls within the retention period, stop + *
    • Step 3: If the file is a delta log file, add it to the `potentialFilesToDelete` + * list + *
    • Step 4: If the file is a checkpoint file, add it to the `lastSeenCheckpointFiles` + *
    + *
+ * + * @param engine {@link Engine} instance to delete the expired log files + * @param clock {@link Clock} instance to get the current time. Useful in testing to mock the + * current time. + * @param tablePath Table location + * @param retentionMillis Log file retention period in milliseconds + * @return number of log files deleted + * @throws IOException if an error occurs while deleting the log files + */ + public static long cleanupExpiredLogs( + Engine engine, Clock clock, Path tablePath, long retentionMillis) throws IOException { + checkArgument(retentionMillis >= 0, "Retention period must be non-negative"); + + List potentialLogFilesToDelete = new ArrayList<>(); + long lastSeenCheckpointVersion = -1; // -1 indicates no checkpoint seen yet + List lastSeenCheckpointFiles = new ArrayList<>(); + + long fileCutOffTime = clock.getTimeMillis() - retentionMillis; + logger.info("{}: Starting the deletion of log files older than {}", tablePath, fileCutOffTime); + long numDeleted = 0; + try (CloseableIterator files = listDeltaLogs(engine, tablePath)) { + while (files.hasNext()) { + // Step 1: Check if the `lastSeenCheckpointFiles` contains a complete checkpoint + Optional lastCompleteCheckpoint = + getLatestCompleteCheckpointFromList( + lastSeenCheckpointFiles.stream().map(CheckpointInstance::new).collect(toList()), + CheckpointInstance.MAX_VALUE); + + if (lastCompleteCheckpoint.isPresent()) { + // Step 1.1: delete all files in `potentialFilesToDelete`. Now we know there is a + // checkpoint that contains the compacted Delta log up to the checkpoint version and all + // commit/checkpoint files before this checkpoint version are not needed. add + // `lastCheckpointFiles` to `potentialFileStoDelete` list. This checkpoint is potential + // candidate to delete later if we find another checkpoint + if (!potentialLogFilesToDelete.isEmpty()) { + logger.info( + "{}: Deleting log files (start = {}, end = {}) because a checkpoint at " + + "version {} indicates that these log files are no longer needed.", + tablePath, + getFirst(potentialLogFilesToDelete), + getLast(potentialLogFilesToDelete), + lastSeenCheckpointVersion); + + numDeleted += deleteLogFiles(engine, potentialLogFilesToDelete); + potentialLogFilesToDelete.clear(); + } + + // Step 1.2: add `lastCheckpointFiles` to `potentialFileStoDelete` list. This checkpoint + // is potential candidate to delete later if we find another checkpoint + potentialLogFilesToDelete.addAll(lastSeenCheckpointFiles); + lastSeenCheckpointFiles.clear(); + lastSeenCheckpointVersion = -1; + } + + FileStatus nextFile = files.next(); + + // Step 2: If the timestamp is earlier than the retention period, stop + if (nextFile.getModificationTime() > fileCutOffTime) { + if (!potentialLogFilesToDelete.isEmpty()) { + logger.info( + "{}: Skipping deletion of expired log files {}, because there is no checkpoint " + + "file that indicates that the log files are no longer needed. ", + tablePath, + potentialLogFilesToDelete.size()); + } + break; + } + + if (FileNames.isCommitFile(nextFile.getPath())) { + // Step 3: If the file is a delta log file, add it to the `potentialFilesToDelete` list + // We can't delete these files until we encounter a checkpoint later that indicates + // that the log files are no longer needed. + potentialLogFilesToDelete.add(nextFile.getPath()); + } else if (FileNames.isCheckpointFile(nextFile.getPath())) { + // Step 4: If the file is a checkpoint file, add it to the `lastSeenCheckpointFiles` + long newLastSeenCheckpointVersion = FileNames.checkpointVersion(nextFile.getPath()); + checkArgument( + lastSeenCheckpointVersion == -1 + || newLastSeenCheckpointVersion >= lastSeenCheckpointVersion); + + if (lastSeenCheckpointVersion != -1 + && newLastSeenCheckpointVersion > lastSeenCheckpointVersion) { + // We have found checkpoint file for a new version. This means the files gathered for + // the last checkpoint version are not complete (most likely an incomplete multipart + // checkpoint). We should delete the files gathered so far and start fresh + // last seen checkpoint state + logger.info( + "{}: Incomplete checkpoint files found at version {}, ignoring the checkpoint" + + " files and adding them to potential log file delete list", + tablePath, + lastSeenCheckpointVersion); + potentialLogFilesToDelete.addAll(lastSeenCheckpointFiles); + lastSeenCheckpointFiles.clear(); + } + + lastSeenCheckpointFiles.add(nextFile.getPath()); + lastSeenCheckpointVersion = newLastSeenCheckpointVersion; + } + // Ignore non-delta and non-checkpoint files. + } + } + logger.info("{}: Deleted {} log files older than {}", tablePath, numDeleted, fileCutOffTime); + return numDeleted; + } + + private static CloseableIterator listDeltaLogs(Engine engine, Path tablePath) + throws IOException { + Path logPath = new Path(tablePath, "_delta_log"); + // TODO: Currently we don't update the timestamps of files to be monotonically increasing. + // In future we can do something similar to Delta Spark to make the timestamps monotonically + // increasing. See `BufferingLogDeletionIterator` in Delta Spark. + return engine.getFileSystemClient().listFrom(FileNames.listingPrefix(logPath, 0)); + } + + private static int deleteLogFiles(Engine engine, List logFiles) throws IOException { + int numDeleted = 0; + for (String logFile : logFiles) { + if (wrapEngineExceptionThrowsIO( + () -> engine.getFileSystemClient().delete(logFile), + "Failed to delete the log file as part of the metadata cleanup %s", + logFile)) { + numDeleted++; + } + } + return numDeleted; + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java index 82f00bd5e54..8c760dbaf0c 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java @@ -17,10 +17,13 @@ package io.delta.kernel.internal.snapshot; import static io.delta.kernel.internal.DeltaErrors.wrapEngineExceptionThrowsIO; +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.TableFeatures.validateWriteSupportedTable; import static io.delta.kernel.internal.checkpoints.Checkpointer.findLastCompleteCheckpointBefore; import static io.delta.kernel.internal.fs.Path.getName; import static io.delta.kernel.internal.replay.LogReplayUtils.assertLogFilesBelongToTable; +import static io.delta.kernel.internal.snapshot.MetadataCleanup.cleanupExpiredLogs; import static io.delta.kernel.internal.util.Preconditions.checkArgument; import static java.lang.String.format; @@ -31,11 +34,13 @@ import io.delta.kernel.exceptions.InvalidTableException; import io.delta.kernel.exceptions.TableNotFoundException; import io.delta.kernel.internal.*; +import io.delta.kernel.internal.actions.Metadata; import io.delta.kernel.internal.checkpoints.*; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.lang.ListUtils; import io.delta.kernel.internal.replay.CreateCheckpointIterator; import io.delta.kernel.internal.replay.LogReplay; +import io.delta.kernel.internal.util.Clock; import io.delta.kernel.internal.util.FileNames; import io.delta.kernel.internal.util.Tuple2; import io.delta.kernel.utils.CloseableIterator; @@ -186,7 +191,8 @@ public Snapshot getSnapshotForTimestamp(Engine engine, long millisSinceEpochUTC) return getSnapshotAt(engine, versionToRead); } - public void checkpoint(Engine engine, long version) throws TableNotFoundException, IOException { + public void checkpoint(Engine engine, Clock clock, long version) + throws TableNotFoundException, IOException { logger.info("{}: Starting checkpoint for version: {}", tablePath, version); // Get the snapshot corresponding the version SnapshotImpl snapshot = (SnapshotImpl) getSnapshotAt(engine, version); @@ -231,6 +237,15 @@ public void checkpoint(Engine engine, long version) throws TableNotFoundExceptio logger.info("{}: Last checkpoint metadata file is written for version: {}", tablePath, version); logger.info("{}: Finished checkpoint for version: {}", tablePath, version); + + // Clean up delta log files if enabled. + Metadata metadata = snapshot.getMetadata(); + if (EXPIRED_LOG_CLEANUP_ENABLED.fromMetadata(engine, metadata)) { + cleanupExpiredLogs(engine, clock, tablePath, LOG_RETENTION.fromMetadata(engine, metadata)); + } else { + logger.info( + "{}: Log cleanup is disabled. Skipping the deletion of expired log files", tablePath); + } } //////////////////// diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/types/CollationIdentifier.java b/kernel/kernel-api/src/main/java/io/delta/kernel/types/CollationIdentifier.java new file mode 100644 index 00000000000..713a141f0a4 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/types/CollationIdentifier.java @@ -0,0 +1,111 @@ +/* + * Copyright (2024) 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.types; + +import static io.delta.kernel.internal.util.Preconditions.checkArgument; + +import io.delta.kernel.annotation.Evolving; +import java.util.Objects; +import java.util.Optional; + +/** + * Identifies collation for string type. + * Collation identifiers + * + * @since 3.3.0 + */ +@Evolving +public class CollationIdentifier { + + private final String provider; + private final String name; + private final Optional version; + + private CollationIdentifier(String provider, String collationName) { + this(provider, collationName, Optional.empty()); + } + + private CollationIdentifier(String provider, String collationName, Optional version) { + Objects.requireNonNull(provider, "Collation provider cannot be null."); + Objects.requireNonNull(collationName, "Collation name cannot be null."); + Objects.requireNonNull(version, "Collation version cannot be null."); + + this.provider = provider.toUpperCase(); + this.name = collationName.toUpperCase(); + this.version = version.map(String::toUpperCase); + } + + /** @return collation provider. */ + public String getProvider() { + return provider; + } + + /** @return collation name. */ + public String getName() { + return name; + } + + /** @return collation version. */ + public Optional getVersion() { + return version; + } + + /** + * @param identifier collation identifier in string form of
+ * {@code PROVIDER.COLLATION_NAME[.COLLATION_VERSION]}. + * @return appropriate collation identifier object + */ + public static CollationIdentifier fromString(String identifier) { + long numDots = identifier.chars().filter(ch -> ch == '.').count(); + checkArgument(numDots > 0, String.format("Invalid collation identifier: %s", identifier)); + if (numDots == 1) { + String[] parts = identifier.split("\\."); + return new CollationIdentifier(parts[0], parts[1]); + } else { + String[] parts = identifier.split("\\.", 3); + return new CollationIdentifier(parts[0], parts[1], Optional.of(parts[2])); + } + } + + /** Collation identifiers are identical when the provider, name, and version are the same. */ + @Override + public boolean equals(Object o) { + if (!(o instanceof CollationIdentifier)) { + return false; + } + + CollationIdentifier other = (CollationIdentifier) o; + return this.provider.equals(other.provider) + && this.name.equals(other.name) + && this.version.equals(other.version); + } + + /** @return collation identifier in form of {@code PROVIDER.COLLATION_NAME}. */ + public String toStringWithoutVersion() { + return String.format("%s.%s", provider, name); + } + + /** @return collation identifier in form of {@code PROVIDER.COLLATION_NAME[.COLLATION_VERSION]} */ + @Override + public String toString() { + if (version.isPresent()) { + return String.format("%s.%s.%s", provider, name, version.get()); + } else { + return String.format("%s.%s", provider, name); + } + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/types/StringType.java b/kernel/kernel-api/src/main/java/io/delta/kernel/types/StringType.java index 08b5bbd1df7..a18d93cf804 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/types/StringType.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/types/StringType.java @@ -24,9 +24,42 @@ */ @Evolving public class StringType extends BasePrimitiveType { - public static final StringType STRING = new StringType(); + public static final StringType STRING = + new StringType(CollationIdentifier.fromString("SPARK.UTF8_BINARY")); - private StringType() { + private final CollationIdentifier collationIdentifier; + + /** + * @param collationIdentifier An identifier representing the collation to be used for string + * comparison and sorting. This determines how strings will be ordered and compared in query + * operations. + */ + public StringType(CollationIdentifier collationIdentifier) { super("string"); + this.collationIdentifier = collationIdentifier; + } + + /** + * @param collationName name of collation in which this StringType will be observed. In form of + * {@code PROVIDER.COLLATION_NAME[.VERSION]} + */ + public StringType(String collationName) { + super("string"); + this.collationIdentifier = CollationIdentifier.fromString(collationName); + } + + /** @return StringType's collation identifier */ + public CollationIdentifier getCollationIdentifier() { + return collationIdentifier; + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof StringType)) { + return false; + } + + StringType that = (StringType) o; + return collationIdentifier.equals(that.collationIdentifier); } } 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 new file mode 100644 index 00000000000..04afb67bd68 --- /dev/null +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/MetadataCleanupSuite.scala @@ -0,0 +1,308 @@ +/* + * Copyright (2023) 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.snapshot + +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 + +/** + * Test suite for the metadata cleanup logic in the Delta log directory. It mocks the + * `FileSystemClient` to test the cleanup logic for various combinations of delta files and + * checkpoint files. Utility methods in `MockFileSystemClientUtils` are used to generate the + * log file statuses which usually have modification time as the `version * 10`. + */ +class MetadataCleanupSuite extends AnyFunSuite with MockFileSystemClientUtils { + + import MetadataCleanupSuite._ + + /* ------------------- TESTS ------------------ */ + + // Simple case where the Delta log directory contains only delta files and no checkpoint files + Seq( + ( + "no files should be deleted even some of them are expired", + DeletedFileList(), // expected deleted files - none of them should be deleted + 70, // current time + 30 // retention period + ), + ( + "no files should be deleted as none of them are expired", + DeletedFileList(), // expected deleted files - none of them should be deleted + 200, // current time + 200 // retention period + ), + ( + "no files should be deleted as none of them are expired", + DeletedFileList(), // expected deleted files - none of them should be deleted + 200, // current time + 0 // retention period + ) + ).foreach { + case (testName, expectedDeletedFiles, currentTime, retentionPeriod) => + // _deltalog directory contents - contains only delta files + val logFiles = deltaFileStatuses(Seq(0, 1, 2, 3, 4, 5, 6)) + test(s"metadataCleanup: $testName: $currentTime, $retentionPeriod") { + cleanupAndVerify(logFiles, expectedDeletedFiles.fileList(), currentTime, retentionPeriod) + } + } + + // with various checkpoint types + Seq("classic", "multi-part", "v2", "hybrid").foreach { checkpointType => + // _deltalog directory contains a combination of delta files and checkpoint files + + val logFiles = deltaFileStatuses(Seq(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) ++ + (checkpointType match { + case "classic" => + singularCheckpointFileStatuses(Seq(3, 6, 9, 12)) + case "multi-part" => + multiCheckpointFileStatuses(Seq(3, 6, 9, 12), multiPartCheckpointPartsSize) + case "v2" => + v2CPFileStatuses(Seq[Long](3, 6, 9, 12)) + case "hybrid" => + singularCheckpointFileStatuses(Seq(3)) ++ + multiCheckpointFileStatuses(Seq(6), numParts = multiPartCheckpointPartsSize) ++ + v2CPFileStatuses(Seq[Long](9)) ++ + singularCheckpointFileStatuses(Seq(12)) + }) + + // test cases + Seq( + ( + "delete expired delta files up to the checkpoint version, " + + "not all expired delta files are deleted", + Seq(0L, 1L, 2L), // expDeletedDeltaVersions, + Seq(), // expDeletedCheckpointVersions, + 130, // current time + 80 // retention period + ), + ( + "expired delta files + expired checkpoint should be deleted", + Seq(0L, 1L, 2L, 3L, 4L, 5L), // expDeletedDeltaVersions, + Seq(3L), // expDeletedCheckpointVersions, + 130, // current time + 60 // retention period + ), + ( + "expired delta files + expired checkpoints should be deleted", + Seq(0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L), // expDeletedDeltaVersions, + Seq(3L, 6L), // expDeletedCheckpointVersions, + 130, // current time + 40 // retention period + ), + ( + "all delta/checkpoint files should be except the last checkpoint file", + Seq(0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L), // expDeletedDeltaVersions, + Seq(3L, 6L, 9L), // expDeletedCheckpointVersions, + 130, // current time + 0 // retention period + ), + ( + "no delta/checkpoint files should be deleted as none expired", + Seq(), // expDeletedDeltaVersions + Seq(), // expDeletedDeltaVersions + 200, // current time + 200 // retention period + ) + ).foreach { + case (testName, expDeletedDeltaVersions, expDeletedCheckpointVersions, + currentTime, retentionPeriod) => + + val expectedDeletedFiles = DeletedFileList( + deltaVersions = expDeletedDeltaVersions, + classicCheckpointVersions = checkpointType match { + case "classic" => expDeletedCheckpointVersions + case "hybrid" => expDeletedCheckpointVersions.filter(Seq(3, 12).contains(_)) + case _ => Seq.empty + }, + multipartCheckpointVersions = checkpointType match { + case "multi-part" => expDeletedCheckpointVersions + case "hybrid" => expDeletedCheckpointVersions.filter(_ == 6) + case _ => Seq.empty + }, + v2CheckpointVersions = checkpointType match { + case "v2" => expDeletedCheckpointVersions + case "hybrid" => expDeletedCheckpointVersions.filter(_ == 9) + case _ => Seq.empty + } + ) + + test(s"metadataCleanup: $checkpointType: $testName: $currentTime, $retentionPeriod") { + cleanupAndVerify(logFiles, expectedDeletedFiles.fileList(), currentTime, retentionPeriod) + } + } + } + + test("first log entry is a checkpoint") { + val logFiles = multiCheckpointFileStatuses(Seq(25), multiPartCheckpointPartsSize) ++ + singularCheckpointFileStatuses(Seq(29)) ++ + deltaFileStatuses(Seq(25, 26, 27, 28, 29, 30, 31, 32)) + + Seq( + ( + 330, // current time + 50, // retention period + DeletedFileList() // expected deleted files - none of them should be deleted + ), + ( + 330, // current time + 30, // retention period + DeletedFileList( + deltaVersions = Seq(25, 26, 27, 28), + multipartCheckpointVersions = Seq(25) + ) + ), + ( + 330, // current time + 10, // retention period + DeletedFileList( + deltaVersions = Seq(25, 26, 27, 28), + multipartCheckpointVersions = Seq(25) + ) + ) + ).foreach { + case (currentTime, retentionPeriod, expectedDeletedFiles) => + cleanupAndVerify(logFiles, expectedDeletedFiles.fileList(), currentTime, retentionPeriod) + } + } + + /* ------------------- NEGATIVE TESTS ------------------ */ + test("metadataCleanup: invalid retention period") { + val e = intercept[IllegalArgumentException] { + cleanupExpiredLogs( + mockEngine(mockFsClient(Seq.empty)), + new ManualClock(100), + logPath, + -1 /* retentionPeriod */ + ) + } + + assert(e.getMessage.contains("Retention period must be non-negative")) + } + + test("incomplete checkpoints should not be considered") { + val logFiles = deltaFileStatuses(Seq(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) ++ + multiCheckpointFileStatuses(Seq(3), multiPartCheckpointPartsSize) + // delete the third part of the checkpoint + .filterNot(_.getPath.contains(s"%010d.%010d".format(2, 4))) ++ + multiCheckpointFileStatuses(Seq(6), multiPartCheckpointPartsSize) ++ + v2CPFileStatuses(Seq(9)) + + // test cases + Seq( + ( + Seq[Long](), // expDeletedDeltaVersions, + Seq[Long](), // expDeletedCheckpointVersions, + 130, // current time + 80 // retention period + ), + ( + Seq(0L, 1L, 2L, 3L, 4L, 5L), // expDeletedDeltaVersions, + Seq(3L), // expDeletedCheckpointVersions, + 130, // current time + 60 // retention period + ), + ( + Seq(0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L), // expDeletedDeltaVersions, + Seq(3L, 6L), // expDeletedCheckpointVersions, + 130, // current time + 20 // retention period + ) + ).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)) + }).map(_.getPath) + + cleanupAndVerify(logFiles, expectedDeletedFiles, currentTime, retentionPeriod) + } + } + + /* ------------------- HELPER UTILITIES/CONSTANTS ------------------ */ + /** + * Cleanup the metadata log files and verify the expected deleted files. + * + * @param logFiles List of log files in the _delta_log directory + * @param expectedDeletedFiles List of expected deleted file paths + * @param currentTimeMillis Current time in millis + * @param retentionPeriodMillis Retention period in millis + */ + def cleanupAndVerify( + logFiles: Seq[FileStatus], + expectedDeletedFiles: Seq[String], + currentTimeMillis: Long, + retentionPeriodMillis: Long): Unit = { + val fsClient = mockFsClient(logFiles) + val resultDeletedCount = cleanupExpiredLogs( + mockEngine(fsClient), + new ManualClock(currentTimeMillis), + logPath, + retentionPeriodMillis + ) + + assert(resultDeletedCount === expectedDeletedFiles.size) + assert(fsClient.getDeleteCalls.toSet === expectedDeletedFiles.toSet) + } +} + +object MetadataCleanupSuite extends MockFileSystemClientUtils { + /* ------------------- HELPER UTILITIES/CONSTANTS ------------------ */ + private val multiPartCheckpointPartsSize = 4 + + /** 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) { + + def fileList(): Seq[String] = { + (deltaFileStatuses(deltaVersions) ++ + singularCheckpointFileStatuses(classicCheckpointVersions) ++ + multiCheckpointFileStatuses(multipartCheckpointVersions, multiPartCheckpointPartsSize) ++ + v2CPFileStatuses(v2CheckpointVersions) + ).sortBy(_.getPath).map(_.getPath) + } + } + + def mockFsClient(logFiles: Seq[FileStatus]): MockListFromDeleteFileSystemClient = { + new MockListFromDeleteFileSystemClient(logFiles) + } + + def v2CPFileStatuses(versions: Seq[Long]): Seq[FileStatus] = { + // Replace the UUID with a standard UUID to make the test deterministic + val standardUUID = "123e4567-e89b-12d3-a456-426614174000" + val uuidPattern = + "[0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}".r + + 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)) + } +} 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 671272bdabb..c7eb9caf7f9 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 @@ -159,4 +159,7 @@ trait BaseMockFileSystemClient extends FileSystemClient { override def mkdirs(path: String): Boolean = throw new UnsupportedOperationException("not supported in this test suite") + + override def delete(path: String): Boolean = + throw new UnsupportedOperationException("not supported in this test suite") } 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 c3bbda3b117..0e8ed2c56b9 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 @@ -161,3 +161,28 @@ class MockListFromResolvePathFileSystemClient(listFromProvider: String => Seq[Fi def getListFromCalls: Seq[String] = listFromCalls } + +/** + * A mock [[FileSystemClient]] that answers `listFrom` call from the given list of file statuses + * and tracks the delete calls. + * @param listContents List of file statuses to be returned by `listFrom` call. + */ +class MockListFromDeleteFileSystemClient(listContents: Seq[FileStatus]) + extends BaseMockFileSystemClient { + private val listOfFiles: Seq[String] = listContents.map(_.getPath).toSeq + private var isListFromAlreadyCalled = false + private var deleteCalls: Seq[String] = Seq.empty + + override def listFrom(filePath: String): CloseableIterator[FileStatus] = { + assert(!isListFromAlreadyCalled, "listFrom should be called only once") + isListFromAlreadyCalled = true + toCloseableIterator(listContents.sortBy(_.getPath).asJava.iterator()) + } + + override def delete(path: String): Boolean = { + deleteCalls = deleteCalls :+ path + listOfFiles.contains(path) + } + + def getDeleteCalls: Seq[String] = deleteCalls +} 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 new file mode 100644 index 00000000000..2e4f8c29947 --- /dev/null +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/types/CollationIdentifierSuite.scala @@ -0,0 +1,100 @@ +/* + * Copyright (2024) 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.types + +import org.scalatest.funsuite.AnyFunSuite + +import java.util.Optional + +class CollationIdentifierSuite extends AnyFunSuite { + val PROVIDER_SPARK = "SPARK" + val PROVIDER_ICU = "ICU" + val DEFAULT_COLLATION_NAME = "UTF8_BINARY" + val DEFAULT_COLLATION_IDENTIFIER = CollationIdentifier.fromString("SPARK.UTF8_BINARY") + + test("check fromString with valid string") { + Seq( + ( + s"$PROVIDER_SPARK.$DEFAULT_COLLATION_NAME", + DEFAULT_COLLATION_IDENTIFIER + ), + ( + 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) => + assert(CollationIdentifier.fromString(stringIdentifier).equals(collationIdentifier)) + } + } + + test("check fromString with invalid string") { + Seq( + PROVIDER_SPARK, + s"${PROVIDER_SPARK}_sr_Cyrl_SRB" + ).foreach { + stringIdentifier => + val e = intercept[IllegalArgumentException] { + val collationIdentifier = CollationIdentifier.fromString(stringIdentifier) + } + assert(e.getMessage == String.format("Invalid collation identifier: %s", stringIdentifier)) + } + } + + test("check toStringWithoutVersion") { + Seq( + ( + DEFAULT_COLLATION_IDENTIFIER, + s"$PROVIDER_SPARK.$DEFAULT_COLLATION_NAME" + ), + ( + CollationIdentifier.fromString(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) => + assert(collationIdentifier.toStringWithoutVersion == toStringWithoutVersion) + } + } + + test("check toString") { + Seq( + ( + DEFAULT_COLLATION_IDENTIFIER, + s"$PROVIDER_SPARK.$DEFAULT_COLLATION_NAME" + ), + ( + CollationIdentifier.fromString(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) => + assert(collationIdentifier.toString == toString) + } + } +} 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 new file mode 100644 index 00000000000..d6acfa47e93 --- /dev/null +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/types/StringTypeSuite.scala @@ -0,0 +1,59 @@ +/* + * Copyright (2024) 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.types + +import org.scalatest.funsuite.AnyFunSuite + +class StringTypeSuite extends AnyFunSuite { + test("check equals") { + // Testcase: (instance1, instance2, expected value for `instance1 == instance2`) + Seq( + ( + StringType.STRING, + StringType.STRING, + true + ), + ( + StringType.STRING, + new StringType("sPark.UTF8_bINary"), + true + ), + ( + StringType.STRING, + new StringType("SPARK.UTF8_LCASE"), + false + ), + ( + new StringType("ICU.UNICODE"), + new StringType("SPARK.UTF8_LCASE"), + false + ), + ( + new StringType("ICU.UNICODE"), + new StringType("ICU.UNICODE_CI"), + false + ), + ( + new StringType("ICU.UNICODE_CI"), + new StringType("icU.uniCODe_Ci"), + true + ) + ).foreach { + case (st1, st2, expResult) => + assert(st1.equals(st2) == expResult) + } + } +} diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultFileSystemClient.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultFileSystemClient.java index e80923554a1..c226e61a5ad 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultFileSystemClient.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultFileSystemClient.java @@ -99,6 +99,13 @@ public boolean mkdirs(String path) throws IOException { return fs.mkdirs(pathObject); } + @Override + public boolean delete(String path) throws IOException { + Path pathObject = new Path(path); + FileSystem fs = pathObject.getFileSystem(hadoopConf); + return fs.delete(pathObject, false); + } + private ByteArrayInputStream getStream(String filePath, int offset, int size) { Path path = new Path(filePath); try { 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 34b57308258..5d6950d856e 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 @@ -19,6 +19,7 @@ import java.io.File import scala.collection.JavaConverters._ +import io.delta.golden.GoldenTableUtils.goldenTablePath import io.delta.kernel.data.Row import io.delta.kernel.data.ColumnarBatch import io.delta.kernel.defaults.utils.TestUtils @@ -49,7 +50,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { * For the given parameters, read the table changes from Kernel using * TableImpl.getChangesByVersion and compare results with Spark */ - def testGetChangesByVersionVsSpark( + def testGetChangesVsSpark( tablePath: String, startVersion: Long, endVersion: Long, @@ -61,7 +62,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { val kernelChanges = Table.forPath(defaultEngine, tablePath) .asInstanceOf[TableImpl] - .getChangesByVersion(defaultEngine, startVersion, endVersion, actionSet.asJava) + .getChanges(defaultEngine, startVersion, endVersion, actionSet.asJava) .toSeq // Check schema is as expected (version + timestamp column + the actions requested) @@ -74,41 +75,41 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { } // Golden table from Delta Standalone test - test("getChangesByVersion - golden table deltalog-getChanges valid queries") { + test("getChanges - golden table deltalog-getChanges valid queries") { withGoldenTable("deltalog-getChanges") { tablePath => // request subset of actions - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tablePath, 0, 2, Set(DeltaAction.REMOVE) ) - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tablePath, 0, 2, Set(DeltaAction.ADD) ) - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tablePath, 0, 2, Set(DeltaAction.ADD, DeltaAction.REMOVE, DeltaAction.METADATA, DeltaAction.PROTOCOL) ) // request full actions, various versions - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tablePath, 0, 2, FULL_ACTION_SET ) - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tablePath, 1, 2, FULL_ACTION_SET ) - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tablePath, 0, 0, @@ -117,7 +118,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { } } - test("getChangesByVersion - returns correct timestamps") { + test("getChanges - returns correct timestamps") { withTempDir { tempDir => def generateCommits(path: String, commits: Long*): Unit = { @@ -141,7 +142,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { // Check the timestamps are returned correctly Table.forPath(defaultEngine, tempDir.getCanonicalPath) .asInstanceOf[TableImpl] - .getChangesByVersion(defaultEngine, 0, 2, Set(DeltaAction.ADD).asJava) + .getChanges(defaultEngine, 0, 2, Set(DeltaAction.ADD).asJava) .toSeq .flatMap(_.getRows.toSeq) .foreach { row => @@ -153,7 +154,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { } // Check contents as well - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tempDir.getCanonicalPath, 0, 2, @@ -162,53 +163,53 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { } } - test("getChangesByVersion - empty _delta_log folder") { + test("getChanges - empty _delta_log folder") { withTempDir { tempDir => new File(tempDir, "delta_log").mkdirs() intercept[TableNotFoundException] { Table.forPath(defaultEngine, tempDir.getCanonicalPath) .asInstanceOf[TableImpl] - .getChangesByVersion(defaultEngine, 0, 2, FULL_ACTION_SET.asJava) + .getChanges(defaultEngine, 0, 2, FULL_ACTION_SET.asJava) } } } - test("getChangesByVersion - empty folder no _delta_log dir") { + test("getChanges - empty folder no _delta_log dir") { withTempDir { tempDir => intercept[TableNotFoundException] { Table.forPath(defaultEngine, tempDir.getCanonicalPath) .asInstanceOf[TableImpl] - .getChangesByVersion(defaultEngine, 0, 2, FULL_ACTION_SET.asJava) + .getChanges(defaultEngine, 0, 2, FULL_ACTION_SET.asJava) } } } - test("getChangesByVersion - non-empty folder not a delta table") { + test("getChanges - non-empty folder not a delta table") { withTempDir { tempDir => spark.range(20).write.format("parquet").mode("overwrite").save(tempDir.getCanonicalPath) intercept[TableNotFoundException] { Table.forPath(defaultEngine, tempDir.getCanonicalPath) .asInstanceOf[TableImpl] - .getChangesByVersion(defaultEngine, 0, 2, FULL_ACTION_SET.asJava) + .getChanges(defaultEngine, 0, 2, FULL_ACTION_SET.asJava) } } } - test("getChangesByVersion - directory does not exist") { + test("getChanges - directory does not exist") { intercept[TableNotFoundException] { Table.forPath(defaultEngine, "/fake/table/path") .asInstanceOf[TableImpl] - .getChangesByVersion(defaultEngine, 0, 2, FULL_ACTION_SET.asJava) + .getChanges(defaultEngine, 0, 2, FULL_ACTION_SET.asJava) } } - test("getChangesByVersion - golden table deltalog-getChanges invalid queries") { + test("getChanges - golden table deltalog-getChanges invalid queries") { withGoldenTable("deltalog-getChanges") { tablePath => def getChangesByVersion( startVersion: Long, endVersion: Long): CloseableIterator[ColumnarBatch] = { Table.forPath(defaultEngine, tablePath) .asInstanceOf[TableImpl] - .getChangesByVersion(defaultEngine, startVersion, endVersion, FULL_ACTION_SET.asJava) + .getChanges(defaultEngine, startVersion, endVersion, FULL_ACTION_SET.asJava) } // startVersion after latest available version @@ -233,7 +234,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { } } - test("getChangesByVersion - with truncated log") { + test("getChanges - with truncated log") { withTempDir { tempDir => // PREPARE TEST TABLE val tablePath = tempDir.getCanonicalPath @@ -269,24 +270,24 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { assert(intercept[KernelException] { Table.forPath(defaultEngine, tablePath) .asInstanceOf[TableImpl] - .getChangesByVersion(defaultEngine, 0, 9, FULL_ACTION_SET.asJava) + .getChanges(defaultEngine, 0, 9, FULL_ACTION_SET.asJava) }.getMessage.contains("no log files found in the requested version range")) // startVersion less than the earliest available version assert(intercept[KernelException] { Table.forPath(defaultEngine, tablePath) .asInstanceOf[TableImpl] - .getChangesByVersion(defaultEngine, 5, 11, FULL_ACTION_SET.asJava) + .getChanges(defaultEngine, 5, 11, FULL_ACTION_SET.asJava) }.getMessage.contains("no log file found for version 5")) // TEST VALID CASES - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tablePath, 10, 12, FULL_ACTION_SET ) - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tablePath, 11, 12, @@ -295,7 +296,7 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { } } - test("getChangesByVersion - table with a lot of changes") { + test("getChanges - table with a lot of changes") { withTempDir { tempDir => spark.sql( f""" @@ -333,14 +334,14 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { |""".stripMargin) // Check all actions are correctly retrieved - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tempDir.getCanonicalPath, 0, 6, FULL_ACTION_SET ) // Check some subset of actions - testGetChangesByVersionVsSpark( + testGetChangesVsSpark( tempDir.getCanonicalPath, 0, 6, @@ -349,6 +350,24 @@ class TableChangesSuite extends AnyFunSuite with TestUtils { } } + test("getChanges - fails when protocol is not readable by Kernel") { + // Existing tests suffice to check if the protocol column is present/dropped correctly + // We test our protocol checks for table features in TableFeaturesSuite + // Min reader version is too high + assert(intercept[KernelException] { + // Use toSeq because we need to consume the iterator to force the exception + Table.forPath(defaultEngine, goldenTablePath("deltalog-invalid-protocol-version")) + .asInstanceOf[TableImpl] + .getChanges(defaultEngine, 0, 0, FULL_ACTION_SET.asJava).toSeq + }.getMessage.contains("Unsupported Delta protocol reader version")) + // We still get an error if we don't request the protocol file action + assert(intercept[KernelException] { + Table.forPath(defaultEngine, goldenTablePath("deltalog-invalid-protocol-version")) + .asInstanceOf[TableImpl] + .getChanges(defaultEngine, 0, 0, Set(DeltaAction.ADD).asJava).toSeq + }.getMessage.contains("Unsupported Delta protocol reader version")) + } + ////////////////////////////////////////////////////////////////////////////////// // Helpers to compare actions returned between Kernel and Spark ////////////////////////////////////////////////////////////////////////////////// diff --git a/setup.py b/setup.py index 323a93d52fe..b810a968022 100644 --- a/setup.py +++ b/setup.py @@ -65,7 +65,7 @@ def run(self): 'delta': ['py.typed'], }, install_requires=[ - 'pyspark>=3.5.2,<3.6.0', + 'pyspark>=3.5.3,<3.6.0', 'importlib_metadata>=1.0.0', ], python_requires='>=3.6', diff --git a/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingFileIndex.scala b/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingFileIndex.scala index e2efeb65daa..0e9e45c0b04 100644 --- a/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingFileIndex.scala +++ b/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingFileIndex.scala @@ -212,12 +212,7 @@ case class DeltaSharingFileIndex( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): TahoeLogFileIndex = { val deltaLog = fetchFilesAndConstructDeltaLog(partitionFilters, dataFilters, None) - new TahoeLogFileIndex( - params.spark, - deltaLog, - deltaLog.dataPath, - deltaLog.unsafeVolatileSnapshot - ) + TahoeLogFileIndex(params.spark, deltaLog) } override def listFiles( diff --git a/spark/src/main/java/io/delta/dynamodbcommitcoordinator/CoordinatedCommitsUtils.java b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/CoordinatedCommitsUtils.java deleted file mode 100644 index 492917aaca1..00000000000 --- a/spark/src/main/java/io/delta/dynamodbcommitcoordinator/CoordinatedCommitsUtils.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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 io.delta.dynamodbcommitcoordinator; - -import io.delta.storage.commit.actions.AbstractMetadata; -import io.delta.storage.commit.UpdatedActions; -import org.apache.hadoop.fs.Path; - -import java.util.UUID; - -public class CoordinatedCommitsUtils { - - private CoordinatedCommitsUtils() {} - - /** The subdirectory in which to store the unbackfilled commit files. */ - final static String COMMIT_SUBDIR = "_commits"; - - /** The configuration key for the coordinated commits owner. */ - private static final String COORDINATED_COMMITS_COORDINATOR_CONF_KEY = - "delta.coordinatedCommits.commitCoordinator-preview"; - - /** - * Creates a new unbackfilled delta file path for the given commit version. - * The path is of the form `tablePath/_delta_log/_commits/00000000000000000001.uuid.json`. - */ - public static Path generateUnbackfilledDeltaFilePath( - Path logPath, - long version) { - String uuid = UUID.randomUUID().toString(); - Path basePath = new Path(logPath, COMMIT_SUBDIR); - return new Path(basePath, String.format("%020d.%s.json", version, uuid)); - } - - /** - * Returns the path to the backfilled delta file for the given commit version. - * The path is of the form `tablePath/_delta_log/00000000000000000001.json`. - */ - public static Path getBackfilledDeltaFilePath( - Path logPath, - Long version) { - return new Path(logPath, String.format("%020d.json", version)); - } - - private static String getCoordinator(AbstractMetadata metadata) { - return metadata - .getConfiguration() - .getOrDefault(COORDINATED_COMMITS_COORDINATOR_CONF_KEY, ""); - } - - /** - * Returns true if the commit is a coordinated commits to filesystem conversion. - */ - public static boolean isCoordinatedCommitsToFSConversion( - Long commitVersion, - UpdatedActions updatedActions) { - boolean oldMetadataHasCoordinatedCommits = - !getCoordinator(updatedActions.getOldMetadata()).isEmpty(); - boolean newMetadataHasCoordinatedCommits = - !getCoordinator(updatedActions.getNewMetadata()).isEmpty(); - return oldMetadataHasCoordinatedCommits && !newMetadataHasCoordinatedCommits && commitVersion > 0; - } -} diff --git a/spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBCommitCoordinatorClient.java b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBCommitCoordinatorClient.java index 05d42595635..f61a980d10c 100644 --- a/spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBCommitCoordinatorClient.java +++ b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBCommitCoordinatorClient.java @@ -416,7 +416,7 @@ private GetCommitsResultInternal getCommitsImpl( Long.parseLong(item.get(DynamoDBTableEntryConstants.TABLE_LATEST_VERSION).getN()); AttributeValue allStoredCommits = item.get(DynamoDBTableEntryConstants.COMMITS); ArrayList commits = new ArrayList<>(); - Path unbackfilledCommitsPath = new Path(logPath, CoordinatedCommitsUtils.COMMIT_SUBDIR); + Path unbackfilledCommitsPath = CoordinatedCommitsUtils.commitDirPath(logPath); for(AttributeValue attr: allStoredCommits.getL()) { java.util.Map commitMap = attr.getM(); long commitVersion = diff --git a/spark/src/main/resources/error/delta-error-classes.json b/spark/src/main/resources/error/delta-error-classes.json index 2258871f245..2fc9ddf763d 100644 --- a/spark/src/main/resources/error/delta-error-classes.json +++ b/spark/src/main/resources/error/delta-error-classes.json @@ -207,6 +207,12 @@ ], "sqlState" : "42809" }, + "DELTA_CANNOT_MODIFY_COORDINATED_COMMITS_DEPENDENCIES" : { + "message" : [ + " cannot override or unset in-commit timestamp table properties because coordinated commits is enabled in this table and depends on them. Please remove them (\"delta.enableInCommitTimestamps\", \"delta.inCommitTimestampEnablementVersion\", \"delta.inCommitTimestampEnablementTimestamp\") from the TBLPROPERTIES clause and then retry the command again." + ], + "sqlState" : "42616" + }, "DELTA_CANNOT_MODIFY_TABLE_PROPERTY" : { "message" : [ "The Delta table configuration cannot be specified by the user" @@ -267,6 +273,12 @@ ], "sqlState" : "22003" }, + "DELTA_CANNOT_SET_COORDINATED_COMMITS_DEPENDENCIES" : { + "message" : [ + " cannot set in-commit timestamp table properties together with coordinated commits, because the latter depends on the former and sets the former internally. Please remove them (\"delta.enableInCommitTimestamps\", \"delta.inCommitTimestampEnablementVersion\", \"delta.inCommitTimestampEnablementTimestamp\") from the TBLPROPERTIES clause and then retry the command again." + ], + "sqlState" : "42616" + }, "DELTA_CANNOT_SET_LOCATION_MULTIPLE_TIMES" : { "message" : [ "Can't set location multiple times. Found " diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala b/spark/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala index 9091ee6ac8c..844611f1968 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala @@ -41,6 +41,7 @@ import org.apache.spark.TaskContext import org.apache.spark.internal.MDC import org.apache.spark.paths.SparkPath import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{Cast, ElementAt, Literal} import org.apache.spark.sql.execution.SQLExecution @@ -297,13 +298,15 @@ trait Checkpoints extends DeltaLogging { * Note that this function captures and logs all exceptions, since the checkpoint shouldn't fail * the overall commit operation. */ - def checkpoint(snapshotToCheckpoint: Snapshot): Unit = recordDeltaOperation( - this, "delta.checkpoint") { + def checkpoint( + snapshotToCheckpoint: Snapshot, + tableIdentifierOpt: Option[TableIdentifier] = None): Unit = + recordDeltaOperation(this, "delta.checkpoint") { withCheckpointExceptionHandling(snapshotToCheckpoint.deltaLog, "delta.checkpoint.sync.error") { if (snapshotToCheckpoint.version < 0) { throw DeltaErrors.checkpointNonExistTable(dataPath) } - checkpointAndCleanUpDeltaLog(snapshotToCheckpoint) + checkpointAndCleanUpDeltaLog(snapshotToCheckpoint, tableIdentifierOpt) } } @@ -323,8 +326,9 @@ trait Checkpoints extends DeltaLogging { } def checkpointAndCleanUpDeltaLog( - snapshotToCheckpoint: Snapshot): Unit = { - val lastCheckpointInfo = writeCheckpointFiles(snapshotToCheckpoint) + snapshotToCheckpoint: Snapshot, + tableIdentifierOpt: Option[TableIdentifier] = None): Unit = { + val lastCheckpointInfo = writeCheckpointFiles(snapshotToCheckpoint, tableIdentifierOpt) writeLastCheckpointFile( snapshotToCheckpoint.deltaLog, lastCheckpointInfo, LastCheckpointInfo.checksumEnabled(spark)) doLogCleanup(snapshotToCheckpoint) @@ -346,7 +350,9 @@ trait Checkpoints extends DeltaLogging { } } - protected def writeCheckpointFiles(snapshotToCheckpoint: Snapshot): LastCheckpointInfo = { + protected def writeCheckpointFiles( + snapshotToCheckpoint: Snapshot, + tableIdentifierOpt: Option[TableIdentifier] = None): LastCheckpointInfo = { // With Coordinated-Commits, commit files are not guaranteed to be backfilled immediately in the // _delta_log dir. While it is possible to compute a checkpoint file without backfilling, // writing the checkpoint file in the log directory before backfilling the relevant commits @@ -361,9 +367,7 @@ trait Checkpoints extends DeltaLogging { // 00015.json // 00016.json // 00018.checkpoint.parquet - // TODO(table-identifier-plumbing): Plumb the right tableIdentifier from the Checkpoint Hook - // and pass it to `ensureCommitFilesBackfilled`. - snapshotToCheckpoint.ensureCommitFilesBackfilled(tableIdentifierOpt = None) + snapshotToCheckpoint.ensureCommitFilesBackfilled(tableIdentifierOpt) Checkpoints.writeCheckpoint(spark, this, snapshotToCheckpoint) } 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 2ce3cc1b4c3..11ea6e513da 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 @@ -48,6 +48,17 @@ trait DeltaColumnMappingBase extends DeltaLogging { val PARQUET_MAP_KEY_FIELD_NAME = "key" val PARQUET_MAP_VALUE_FIELD_NAME = "value" + /** + * The list of column mapping metadata for each column in the schema. + */ + val COLUMN_MAPPING_METADATA_KEYS: Set[String] = Set( + COLUMN_MAPPING_METADATA_ID_KEY, + COLUMN_MAPPING_PHYSICAL_NAME_KEY, + COLUMN_MAPPING_METADATA_NESTED_IDS_KEY, + PARQUET_FIELD_ID_METADATA_KEY, + PARQUET_MAP_VALUE_FIELD_NAME + ) + /** * This list of internal columns (and only this list) is allowed to have missing * column mapping metadata such as field id and physical name because @@ -116,6 +127,7 @@ trait DeltaColumnMappingBase extends DeltaLogging { * - upgrading to the column mapping Protocol through configurations */ def verifyAndUpdateMetadataChange( + spark: SparkSession, deltaLog: DeltaLog, oldProtocol: Protocol, oldMetadata: Metadata, @@ -136,8 +148,34 @@ trait DeltaColumnMappingBase extends DeltaLogging { oldMappingMode.name, newMappingMode.name) } - val updatedMetadata = updateColumnMappingMetadata( - oldMetadata, newMetadata, isChangingModeOnExistingTable, isOverwriteSchema) + var updatedMetadata = newMetadata + + // If column mapping is disabled, we need to strip any column mapping metadata from the schema, + // because Delta code will use them even when column mapping is not enabled. However, we cannot + // strip column mapping metadata that already exist in the schema, because this would break + // the table. + if (newMappingMode == NoMapping && + schemaHasColumnMappingMetadata(newMetadata.schema)) { + val addsColumnMappingMetadata = !schemaHasColumnMappingMetadata(oldMetadata.schema) + if (addsColumnMappingMetadata && + spark.conf.get(DeltaSQLConf.DELTA_COLUMN_MAPPING_STRIP_METADATA)) { + recordDeltaEvent(deltaLog, opType = "delta.columnMapping.stripMetadata") + val strippedSchema = dropColumnMappingMetadata(newMetadata.schema) + updatedMetadata = newMetadata.copy(schemaString = strippedSchema.json) + } else { + recordDeltaEvent( + deltaLog, + opType = "delta.columnMapping.updateSchema.metadataPresentButFeatureDisabled", + data = Map( + "addsColumnMappingMetadata" -> addsColumnMappingMetadata.toString, + "isCreatingNewTable" -> isCreatingNewTable.toString, + "isOverwriteSchema" -> isOverwriteSchema.toString) + ) + } + } + + updatedMetadata = updateColumnMappingMetadata( + oldMetadata, updatedMetadata, isChangingModeOnExistingTable, isOverwriteSchema) // record column mapping table creation/upgrade if (newMappingMode != NoMapping) { @@ -455,16 +493,12 @@ trait DeltaColumnMappingBase extends DeltaLogging { def dropColumnMappingMetadata(schema: StructType): StructType = { SchemaMergingUtils.transformColumns(schema) { (_, field, _) => - field.copy( - metadata = new MetadataBuilder() - .withMetadata(field.metadata) - .remove(COLUMN_MAPPING_METADATA_ID_KEY) - .remove(COLUMN_MAPPING_METADATA_NESTED_IDS_KEY) - .remove(COLUMN_MAPPING_PHYSICAL_NAME_KEY) - .remove(PARQUET_FIELD_ID_METADATA_KEY) - .remove(PARQUET_FIELD_NESTED_IDS_METADATA_KEY) - .build() - ) + var strippedMetadataBuilder = new MetadataBuilder().withMetadata(field.metadata) + for (key <- COLUMN_MAPPING_METADATA_KEYS) { + strippedMetadataBuilder = strippedMetadataBuilder.remove(key) + } + val strippedMetadata = strippedMetadataBuilder.build() + field.copy(metadata = strippedMetadata) } } @@ -784,6 +818,15 @@ trait DeltaColumnMappingBase extends DeltaLogging { (transform(schema, new MetadataBuilder(), Seq.empty), currFieldId) } + + /** + * Returns whether the schema contains any metadata reserved for column mapping. + */ + def schemaHasColumnMappingMetadata(schema: StructType): Boolean = { + SchemaMergingUtils.explode(schema).exists { case (_, col) => + COLUMN_MAPPING_METADATA_KEYS.exists(k => col.metadata.contains(k)) + } + } } object DeltaColumnMapping extends DeltaColumnMappingBase 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 9d8d242b206..58d7105bae0 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 @@ -1634,14 +1634,14 @@ trait DeltaErrorsBase messageParameters = Array(option, operation)) } - def foundMapTypeColumnException(key: String, value: String, schema: StructType): Throwable = { + def foundMapTypeColumnException(key: String, value: String, schema: DataType): Throwable = { new DeltaAnalysisException( errorClass = "DELTA_FOUND_MAP_TYPE_COLUMN", - messageParameters = Array(key, value, schema.treeString) + messageParameters = Array(key, value, dataTypeToString(schema)) ) } - def columnNotInSchemaException(column: String, schema: StructType): Throwable = { - nonExistentColumnInSchema(column, schema.treeString) + def columnNotInSchemaException(column: String, schema: DataType): Throwable = { + nonExistentColumnInSchema(column, dataTypeToString(schema)) } def metadataAbsentException(): Throwable = { @@ -2690,10 +2690,14 @@ trait DeltaErrorsBase def incorrectArrayAccessByName( rightName: String, wrongName: String, - schema: StructType): Throwable = { + schema: DataType): Throwable = { new DeltaAnalysisException( errorClass = "DELTA_INCORRECT_ARRAY_ACCESS_BY_NAME", - messageParameters = Array(rightName, wrongName, schema.treeString) + messageParameters = Array( + rightName, + wrongName, + dataTypeToString(schema) + ) ) } @@ -2701,14 +2705,14 @@ trait DeltaErrorsBase columnPath: String, other: DataType, column: Seq[String], - schema: StructType): Throwable = { + schema: DataType): Throwable = { new DeltaAnalysisException( errorClass = "DELTA_COLUMN_PATH_NOT_NESTED", messageParameters = Array( s"$columnPath", s"$other", s"${SchemaUtils.prettyFieldName(column)}", - schema.treeString + dataTypeToString(schema) ) ) } @@ -3445,11 +3449,11 @@ trait DeltaErrorsBase } def errorFindingColumnPosition( - columnPath: Seq[String], schema: StructType, extraErrMsg: String): Throwable = { + columnPath: Seq[String], schema: DataType, extraErrMsg: String): Throwable = { new DeltaAnalysisException( errorClass = "_LEGACY_ERROR_TEMP_DELTA_0008", messageParameters = Array( - UnresolvedAttribute(columnPath).name, schema.treeString, extraErrMsg)) + UnresolvedAttribute(columnPath).name, dataTypeToString(schema), extraErrMsg)) } def alterTableClusterByOnPartitionedTableException(): Throwable = { @@ -3481,6 +3485,11 @@ trait DeltaErrorsBase errorClass = "DELTA_UNSUPPORTED_WRITES_WITHOUT_COORDINATOR", messageParameters = Array(coordinatorName)) } + + private def dataTypeToString(dt: DataType): String = dt match { + case s: StructType => s.treeString + case other => other.simpleString + } } object DeltaErrors extends DeltaErrorsBase diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala index e9a31e0a057..659877909c6 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala @@ -191,7 +191,7 @@ class DeltaHistoryManager( if (end - start > 2 * maxKeysPerList) { parallelSearch(time, start, end) } else { - val commits = getCommits( + val commits = getCommitsWithNonIctTimestamps( deltaLog.store, deltaLog.logPath, start, @@ -599,9 +599,11 @@ object DeltaHistoryManager extends DeltaLogging { * Returns the commit version and timestamps of all commits in `[start, end)`. If `end` is not * specified, will return all commits that exist after `start`. Will guarantee that the commits * returned will have both monotonically increasing versions as well as timestamps. - * Exposed for tests. + * Note that this function will return non-ICT timestamps even for commits where + * InCommitTimestamps are enabled. The caller is responsible for ensuring that the appropriate + * timestamps are used. */ - private[delta] def getCommits( + private[delta] def getCommitsWithNonIctTimestamps( logStore: LogStore, logPath: Path, start: Long, @@ -688,7 +690,7 @@ object DeltaHistoryManager extends DeltaLogging { val logStore = LogStore(SparkEnv.get.conf, conf.value) val basePath = new Path(logPath) startVersions.map { startVersion => - val commits = getCommits( + val commits = getCommitsWithNonIctTimestamps( logStore, basePath, startVersion, 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 cb4dfa3e1b8..7d23ec134d5 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 @@ -425,7 +425,7 @@ class DeltaLog private( def assertTableFeaturesMatchMetadata( targetProtocol: Protocol, targetMetadata: Metadata): Unit = { - if (!targetProtocol.supportsReaderFeatures && !targetProtocol.supportsWriterFeatures) return + if (!targetProtocol.supportsTableFeatures) return val protocolEnabledFeatures = targetProtocol.writerFeatureNames .flatMap(TableFeature.featureNameToFeature) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala index 841d950b7f3..1ad915555eb 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala @@ -91,6 +91,21 @@ object DeltaOperations { */ def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean + /** + * Whether the transaction is updating metadata of existing files. + * + * The Delta protocol allows committing AddFile actions for files that already exist on the + * latest version of the table, without committing corresponding RemoveFile actions. This is + * used to update the metadata of existing files, e.g. to recompute statistics or add tags. + * + * Such operations need special handling during conflict checking, especially against + * no-data-change transactions, because the read/delete conflict can be resolved with + * read-file-remapping and because there is no RemoveFile action to trigger a delete/delete + * conflict. In case you are adding such operation, make sure to include a test for conflicts + * with business *and* no-data-change transactions, e.g. optimize. + */ + def isInPlaceFileMetadataUpdate: Option[Boolean] + } abstract class OperationWithPredicates(name: String, val predicates: Seq[Expression]) @@ -153,9 +168,11 @@ object DeltaOperations { } override def changesData: Boolean = true - // This operation shouldn't be introducing AddFile actions with DVs and non tight bounds stats. + // This operation shouldn't be introducing AddFile actions with DVs and tight bounds stats. // DVs can be introduced by the replaceWhere operation. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } case class RemoveColumnMapping( @@ -166,6 +183,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded during streaming inserts. */ @@ -181,8 +200,10 @@ object DeltaOperations { override val operationMetrics: Set[String] = DeltaOperationMetrics.STREAMING_UPDATE override def changesData: Boolean = true - // This operation shouldn't be introducing AddFile actions with DVs and non tight bounds stats. + // This operation shouldn't be introducing AddFile actions with DVs and tight bounds stats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded while deleting certain partitions. */ case class Delete(predicate: Seq[Expression]) @@ -205,8 +226,10 @@ object DeltaOperations { } override def changesData: Boolean = true - // This operation shouldn't be introducing AddFile actions with DVs and non tight bounds stats. + // This operation shouldn't be introducing AddFile actions with DVs and tight bounds stats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when truncating the table. */ case class Truncate() extends Operation("TRUNCATE") { @@ -214,8 +237,10 @@ object DeltaOperations { override val operationMetrics: Set[String] = DeltaOperationMetrics.TRUNCATE override def changesData: Boolean = true - // This operation shouldn't be introducing AddFile actions with DVs and non-tight bounds stats. + // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when converting a table into a Delta table. */ @@ -236,6 +261,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions with DVs and non-tight bounds stats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Represents the predicates and action type (insert, update, delete) for a Merge clause */ @@ -306,6 +333,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions with DVs and non-tight bounds stats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } object Merge { @@ -340,6 +369,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions with DVs and non-tight bounds stats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when the table is created. */ case class CreateTable( @@ -364,6 +395,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions with DVs and non-tight bounds stats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when the table is replaced. */ case class ReplaceTable( @@ -391,6 +424,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions with DVs and non-tight bounds stats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when the table properties are set. */ val OP_SET_TBLPROPERTIES = "SET TBLPROPERTIES" @@ -399,7 +434,11 @@ object DeltaOperations { override val parameters: Map[String, Any] = Map("properties" -> JsonUtils.toJson(properties)) // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. + // Note: This operation may trigger additional actions and additional commits. For example + // RowTrackingBackfill. These are separate transactions, and this check is performed separately. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when the table properties are unset. */ case class UnsetTableProperties( @@ -411,6 +450,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when dropping a table feature. */ case class DropTableFeature( @@ -421,7 +462,11 @@ object DeltaOperations { "truncateHistory" -> truncateHistory) // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. + // Note: this operation may trigger additional actions and additional commits. These would be + // separate transactions, and this check is performed separately. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when columns are added. */ case class AddColumns( @@ -437,6 +482,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when columns are dropped. */ @@ -449,6 +496,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when column is renamed */ @@ -462,6 +511,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when columns are changed. */ @@ -477,6 +528,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when columns are replaced. */ case class ReplaceColumns( @@ -487,6 +540,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } case class UpgradeProtocol(newProtocol: Protocol) extends Operation("UPGRADE PROTOCOL") { @@ -499,6 +554,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } object ManualUpdate extends Operation("Manual Update") { @@ -506,6 +563,10 @@ object DeltaOperations { // Unsafe manual update disables checks. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = false + + // Manual update operations can commit arbitrary actions. In case this field is needed consider + // adding a new Delta operation. For test-only code use TestOperation. + override val isInPlaceFileMetadataUpdate: Option[Boolean] = None } /** A commit without any actions. Could be used to force creation of new checkpoints. */ @@ -514,6 +575,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } case class UpdateColumnMetadata( @@ -528,6 +591,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } case class UpdateSchema(oldSchema: StructType, newSchema: StructType) @@ -538,6 +603,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } case class AddConstraint( @@ -546,6 +613,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } case class DropConstraint( @@ -560,6 +629,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when recomputing stats on the table. */ @@ -569,6 +640,9 @@ object DeltaOperations { // ComputeStats operation commits AddFiles with recomputed stats which are always tight bounds, // even when DVs are present. This check should be disabled. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = false + + // ComputeStats operation only updates statistics of existing files. + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(true) } /** Recorded when restoring a Delta table to an older version. */ @@ -586,6 +660,11 @@ object DeltaOperations { // Restore operation commits AddFiles with files, DVs and stats from the version it restores to. // It can happen that tight bound stats were recomputed before by ComputeStats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = false + + // The restore operation could perform in-place file metadata updates. However, the difference + // between the current and the restored state is computed using only the (path, DV) pairs as + // identifiers, meaning that metadata differences are ignored. + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } sealed abstract class OptimizeOrReorg(override val name: String, predicates: Seq[Expression]) @@ -622,8 +701,10 @@ object DeltaOperations { override val operationMetrics: Set[String] = DeltaOperationMetrics.OPTIMIZE - // This operation shouldn't be introducing AddFile actions with DVs and non tight bounds stats. + // This operation shouldn't be introducing AddFile actions with DVs and tight bounds stats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when cloning a Delta table into a new location. */ @@ -642,6 +723,8 @@ object DeltaOperations { // Clone operation commits AddFiles with files, DVs and stats copied over from the source table. // It can happen that tight bound stats were recomputed before by ComputeStats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = false + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** @@ -662,6 +745,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** @@ -676,6 +761,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when running REORG on the table. */ @@ -688,8 +775,10 @@ object DeltaOperations { override val operationMetrics: Set[String] = DeltaOperationMetrics.OPTIMIZE - // This operation shouldn't be introducing AddFile actions with DVs and non tight bounds stats. + // This operation shouldn't be introducing AddFile actions with DVs and tight bounds stats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when clustering columns are changed on clustered tables. */ @@ -702,6 +791,8 @@ object DeltaOperations { // This operation shouldn't be introducing AddFile actions at all. This check should be trivial. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } /** Recorded when we backfill a Delta table's existing AddFiles with row tracking data. */ @@ -714,6 +805,9 @@ object DeltaOperations { // RowTrackingBackfill operation commits AddFiles with files, DVs and stats copied over. // It can happen that tight bound stats were recomputed before by ComputeStats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = false + + // RowTrackingBackfill only updates tags of existing files. + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(true) } private def structFieldToMap(colPath: Seq[String], field: StructField): Map[String, Any] = { @@ -735,7 +829,10 @@ object DeltaOperations { colPosition: Option[String]) /** Dummy operation only for testing with arbitrary operation names */ - case class TestOperation(operationName: String = "TEST") extends Operation(operationName) { + case class TestOperation( + operationName: String = "TEST", + override val isInPlaceFileMetadataUpdate: Option[Boolean] = None + ) extends Operation(operationName) { override val parameters: Map[String, Any] = Map.empty // Perform the check for testing. @@ -761,8 +858,10 @@ object DeltaOperations { OP_UPGRADE_UNIFORM_BY_REORG) { override val parameters: Map[String, Any] = Map("properties" -> JsonUtils.toJson(properties)) - // This operation shouldn't be introducing AddFile actions with DVs and non tight bounds stats. + // This operation shouldn't be introducing AddFile actions with DVs and tight bounds stats. override def checkAddFileWithDeletionVectorStatsAreNotTightBounds: Boolean = true + + override val isInPlaceFileMetadataUpdate: Option[Boolean] = Some(false) } } 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 4e094fc7cdf..dcafb62bd57 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 @@ -551,6 +551,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite // The `.schema` cannot be generated correctly unless the column mapping metadata is correctly // filled for all the fields. Therefore, the column mapping changes need to happen first. newMetadataTmp = DeltaColumnMapping.verifyAndUpdateMetadataChange( + spark, deltaLog, protocolBeforeUpdate, snapshot.metadata, @@ -700,9 +701,8 @@ trait OptimisticTransactionImpl extends TransactionalWrite } // We are done with protocol versions and features, time to remove related table properties. - val configsWithoutProtocolProps = newMetadataTmp.configuration.filterNot { - case (k, _) => TableFeatureProtocolUtils.isTableProtocolProperty(k) - } + val configsWithoutProtocolProps = + Protocol.filterProtocolPropsFromTableProps(newMetadataTmp.configuration) // Table features Part 3: add automatically-enabled features by looking at the new table // metadata. // @@ -877,25 +877,35 @@ trait OptimisticTransactionImpl extends TransactionalWrite protected def getAssertDeletionVectorWellFormedFunc( spark: SparkSession, op: DeltaOperations.Operation): (Action => Unit) = { - val deletionVectorCreationAllowed = - DeletionVectorUtils.deletionVectorsWritable(snapshot, newProtocol, newMetadata) - val isComputeStatsOperation = op.isInstanceOf[DeltaOperations.ComputeStats] val commitCheckEnabled = spark.conf.get(DeltaSQLConf.DELETION_VECTORS_COMMIT_CHECK_ENABLED) + if (!commitCheckEnabled) { + return _ => {} + } - val deletionVectorDisallowedForAddFiles = - commitCheckEnabled && !isComputeStatsOperation && !deletionVectorCreationAllowed + // Whether DVs are supported, i.e. the table is allowed to contain any DVs. + val deletionVectorsSupported = + DeletionVectorUtils.deletionVectorsReadable(snapshot, newProtocol, newMetadata) + // Whether DVs are enabled, i.e. operations are allowed to create new DVs. + val deletionVectorsEnabled = + DeletionVectorUtils.deletionVectorsWritable(snapshot, newProtocol, newMetadata) + + // If the operation does not define whether it performs in-place metadata updates, we are + // conservative and assume that it is not, which makes the check stricter. + val isInPlaceFileMetadataUpdate = op.isInPlaceFileMetadataUpdate.getOrElse(false) + val deletionVectorAllowedForAddFiles = + deletionVectorsSupported && (deletionVectorsEnabled || isInPlaceFileMetadataUpdate) - val addFileMustHaveWideBounds = deletionVectorCreationAllowed && - op.checkAddFileWithDeletionVectorStatsAreNotTightBounds + val addFileMustHaveWideBounds = op.checkAddFileWithDeletionVectorStatsAreNotTightBounds action => action match { - case a: AddFile => - if (deletionVectorDisallowedForAddFiles && a.deletionVector != null) { + case a: AddFile if a.deletionVector != null => + if (!deletionVectorAllowedForAddFiles) { throw DeltaErrors.addingDeletionVectorsDisallowedException() } + // Protocol requirement checks: // 1. All files with DVs must have `stats` with `numRecords`. - if (a.deletionVector != null && (a.stats == null || a.numPhysicalRecords.isEmpty)) { + if (a.stats == null || a.numPhysicalRecords.isEmpty) { throw DeltaErrors.addFileWithDVsMissingNumRecordsException } @@ -905,7 +915,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite // See that field comment in DeltaOperation for more details. // Note, the absence of the tightBounds column when DVs exist is also an illegal state. if (addFileMustHaveWideBounds && - a.deletionVector != null && // Extra inversion to also catch absent `tightBounds`. !a.tightBounds.contains(false)) { throw DeltaErrors.addFileWithDVsAndTightBoundsException() @@ -1598,6 +1607,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite Some(attemptVersion)) commitEndNano = System.nanoTime() committed = true + executionObserver.beginPostCommit() // NOTE: commitLarge cannot run postCommitHooks (such as the CheckpointHook). // Instead, manually run any necessary actions in updateAndCheckpoint. val postCommitSnapshot = updateAndCheckpoint( @@ -2222,6 +2232,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite commitEndNano = System.nanoTime() + executionObserver.beginPostCommit() val postCommitSnapshot = deltaLog.updateAfterCommit( attemptVersion, commit, diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala b/spark/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala index 1591a99f831..7ed68404544 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala @@ -542,7 +542,7 @@ class Snapshot( } base.put(Protocol.MIN_READER_VERSION_PROP, protocol.minReaderVersion.toString) base.put(Protocol.MIN_WRITER_VERSION_PROP, protocol.minWriterVersion.toString) - if (protocol.supportsReaderFeatures || protocol.supportsWriterFeatures) { + if (protocol.supportsTableFeatures) { val features = protocol.readerAndWriterFeatureNames.map(name => s"${TableFeatureProtocolUtils.FEATURE_PROP_PREFIX}$name" -> TableFeatureProtocolUtils.FEATURE_PROP_SUPPORTED) 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 1edf8d8c024..1c15086b768 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 @@ -540,10 +540,7 @@ object ColumnMappingTableFeature override def validateRemoval(snapshot: Snapshot): Boolean = { val schemaHasNoColumnMappingMetadata = - SchemaMergingUtils.explode(snapshot.schema).forall { case (_, col) => - !DeltaColumnMapping.hasPhysicalName(col) && - !DeltaColumnMapping.hasColumnId(col) - } + !DeltaColumnMapping.schemaHasColumnMappingMetadata(snapshot.schema) val metadataHasNoMappingMode = snapshot.metadata.columnMappingMode match { case NoMapping => true case _ => false diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/TransactionExecutionObserver.scala b/spark/src/main/scala/org/apache/spark/sql/delta/TransactionExecutionObserver.scala index 68268b10ced..1a989ef5775 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/TransactionExecutionObserver.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/TransactionExecutionObserver.scala @@ -77,6 +77,9 @@ trait TransactionExecutionObserver /** Called after publishing the commit file but before the `backfill` attempt. */ def beginBackfill(): Unit + /** Called after backfill but before the `postCommit` attempt. */ + def beginPostCommit(): Unit + /** Called once a commit succeeded. */ def transactionCommitted(): Unit @@ -111,6 +114,8 @@ object NoOpTransactionExecutionObserver extends TransactionExecutionObserver { override def beginBackfill(): Unit = () + override def beginPostCommit(): Unit = () + override def transactionCommitted(): Unit = () override def transactionAborted(): Unit = () diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/actions/TableFeatureSupport.scala b/spark/src/main/scala/org/apache/spark/sql/delta/actions/TableFeatureSupport.scala index b9065499084..2d41fae0f79 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/actions/TableFeatureSupport.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/actions/TableFeatureSupport.scala @@ -39,14 +39,32 @@ import com.fasterxml.jackson.annotation.JsonIgnore */ trait TableFeatureSupport { this: Protocol => - /** Check if this protocol is capable of adding features into its `readerFeatures` field. */ + /** + * Check if this protocol can support arbitrary reader features. If this returns false, + * then the table may still be able to support the "columnMapping" feature. + * See [[canSupportColumnMappingFeature]] below. + */ def supportsReaderFeatures: Boolean = TableFeatureProtocolUtils.supportsReaderFeatures(minReaderVersion) + /** + * Check if this protocol is in table feature representation and can support column mapping. + * Column mapping is the only legacy reader feature and requires special handling in some + * cases. + */ + def canSupportColumnMappingFeature: Boolean = + TableFeatureProtocolUtils.canSupportColumnMappingFeature(minReaderVersion, minWriterVersion) + /** Check if this protocol is capable of adding features into its `writerFeatures` field. */ def supportsWriterFeatures: Boolean = TableFeatureProtocolUtils.supportsWriterFeatures(minWriterVersion) + /** + * As soon as a protocol supports writer features it is considered a table features protocol. + * It is not possible to support reader features without supporting writer features. + */ + def supportsTableFeatures: Boolean = supportsWriterFeatures + /** * 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 @@ -60,7 +78,7 @@ trait TableFeatureSupport { this: Protocol => */ def withFeature(feature: TableFeature): Protocol = { def shouldAddRead: Boolean = { - if (supportsReaderFeatures) return true + if (feature == ColumnMappingTableFeature && canSupportColumnMappingFeature) return true if (feature.minReaderVersion <= minReaderVersion) return false throw DeltaErrors.tableFeatureRequiresHigherReaderProtocolVersion( @@ -111,25 +129,13 @@ trait TableFeatureSupport { this: Protocol => * `writerFeatures` field. * * The method does not require the feature to be recognized by the client, therefore will not - * try keeping the protocol's `readerFeatures` and `writerFeatures` in sync. Use with caution. + * try keeping the protocol's `readerFeatures` and `writerFeatures` in sync. + * Should never be used directly. Always use withFeature(feature: TableFeature): Protocol. */ private[actions] def withFeature( name: String, addToReaderFeatures: Boolean, addToWriterFeatures: Boolean): Protocol = { - if (addToReaderFeatures && !supportsReaderFeatures) { - throw DeltaErrors.tableFeatureRequiresHigherReaderProtocolVersion( - name, - currentVersion = minReaderVersion, - requiredVersion = TableFeatureProtocolUtils.TABLE_FEATURES_MIN_READER_VERSION) - } - if (addToWriterFeatures && !supportsWriterFeatures) { - throw DeltaErrors.tableFeatureRequiresHigherWriterProtocolVersion( - name, - currentVersion = minWriterVersion, - requiredVersion = TableFeatureProtocolUtils.TABLE_FEATURES_MIN_WRITER_VERSION) - } - val addedReaderFeatureOpt = if (addToReaderFeatures) Some(name) else None val addedWriterFeatureOpt = if (addToWriterFeatures) Some(name) else None @@ -143,11 +149,11 @@ trait TableFeatureSupport { this: Protocol => * `readerFeatures` field. * * The method does not require the features to be recognized by the client, therefore will not - * try keeping the protocol's `readerFeatures` and `writerFeatures` in sync. Use with caution. + * try keeping the protocol's `readerFeatures` and `writerFeatures` in sync. + * Intended only for testing. Use with caution. */ private[delta] def withReaderFeatures(names: Iterable[String]): Protocol = { - names.foldLeft(this)( - _.withFeature(_, addToReaderFeatures = true, addToWriterFeatures = false)) + names.foldLeft(this)(_.withFeature(_, addToReaderFeatures = true, addToWriterFeatures = false)) } /** @@ -155,11 +161,11 @@ trait TableFeatureSupport { this: Protocol => * `writerFeatures` field. * * The method does not require the features to be recognized by the client, therefore will not - * try keeping the protocol's `readerFeatures` and `writerFeatures` in sync. Use with caution. + * try keeping the protocol's `readerFeatures` and `writerFeatures` in sync. + * Intended only for testing. Use with caution. */ private[delta] def withWriterFeatures(names: Iterable[String]): Protocol = { - names.foldLeft(this)( - _.withFeature(_, addToReaderFeatures = false, addToWriterFeatures = true)) + names.foldLeft(this)(_.withFeature(_, addToReaderFeatures = false, addToWriterFeatures = true)) } /** @@ -203,14 +209,16 @@ trait TableFeatureSupport { this: Protocol => */ @JsonIgnore lazy val implicitlySupportedFeatures: Set[TableFeature] = { - if (supportsReaderFeatures && supportsWriterFeatures) { - // this protocol uses both reader and writer features, no feature can be implicitly supported + if (supportsTableFeatures) { + // As soon as a protocol supports writer features, all features need to be explicitly defined. + // This includes legacy reader features (the only one is Column Mapping), even if the + // reader protocol is legacy and explicitly supports Column Mapping. Set() } else { TableFeature.allSupportedFeaturesMap.values .filter(_.isLegacyFeature) - .filterNot(supportsReaderFeatures || this.minReaderVersion < _.minReaderVersion) - .filterNot(supportsWriterFeatures || this.minWriterVersion < _.minWriterVersion) + .filter(_.minReaderVersion <= this.minReaderVersion) + .filter(_.minWriterVersion <= this.minWriterVersion) .toSet } } @@ -271,14 +279,11 @@ trait TableFeatureSupport { this: Protocol => val protocols = this +: others val mergedReaderVersion = protocols.map(_.minReaderVersion).max val mergedWriterVersion = protocols.map(_.minWriterVersion).max - val mergedReaderFeatures = protocols.flatMap(_.readerFeatureNames) - val mergedWriterFeatures = protocols.flatMap(_.writerFeatureNames) + val mergedFeatures = protocols.flatMap(_.readerAndWriterFeatures) val mergedImplicitFeatures = protocols.flatMap(_.implicitlySupportedFeatures) val mergedProtocol = Protocol(mergedReaderVersion, mergedWriterVersion) - .withReaderFeatures(mergedReaderFeatures) - .withWriterFeatures(mergedWriterFeatures) - .withFeatures(mergedImplicitFeatures) + .withFeatures(mergedFeatures ++ mergedImplicitFeatures) // The merged protocol is always normalized in order to represent the protocol // with the weakest possible form. This enables backward compatibility. @@ -348,7 +353,7 @@ trait TableFeatureSupport { this: Protocol => */ def normalized: Protocol = { // Normalization can only be applied to table feature protocols. - if (!supportsWriterFeatures) return this + if (!supportsTableFeatures) return this val (minReaderVersion, minWriterVersion) = TableFeatureProtocolUtils.minimumRequiredVersions(readerAndWriterFeatures) @@ -371,7 +376,7 @@ trait TableFeatureSupport { this: Protocol => */ def denormalized: Protocol = { // Denormalization can only be applied to legacy protocols. - if (supportsWriterFeatures) return this + if (supportsTableFeatures) return this val (minReaderVersion, _) = TableFeatureProtocolUtils.minimumRequiredVersions(implicitlySupportedFeatures.toSeq) @@ -419,7 +424,7 @@ object TableFeatureProtocolUtils { /** The string constant "supported" for uses in table properties. */ val FEATURE_PROP_SUPPORTED = "supported" - /** Min reader version that supports reader features. */ + /** Min reader version that supports native reader features. */ val TABLE_FEATURES_MIN_READER_VERSION = 3 /** Min reader version that supports writer features. */ @@ -440,8 +445,20 @@ object TableFeatureProtocolUtils { s"$DEFAULT_FEATURE_PROP_PREFIX$featureName" /** - * Determine whether a [[Protocol]] with the given reader protocol version is capable of adding - * features into its `readerFeatures` field. + * Determine whether a [[Protocol]] with the given reader protocol version can support column + * mapping. All table feature protocols that can support column mapping are capable of adding + * the feature to the `readerFeatures` field. This includes legacy reader protocol version + * (2, 7). + */ + def canSupportColumnMappingFeature(readerVersion: Int, writerVersion: Int): Boolean = { + readerVersion >= ColumnMappingTableFeature.minReaderVersion && + supportsWriterFeatures(writerVersion) + } + + /** + * Determine whether a [[Protocol]] with the given reader protocol version supports + * native features. All protocols that can support native reader features are capable + * of adding the feature to the `readerFeatures` field. */ def supportsReaderFeatures(readerVersion: Int): Boolean = { readerVersion >= TABLE_FEATURES_MIN_READER_VERSION diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala b/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala index d4df3128d0f..390c8a021c2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala @@ -142,13 +142,13 @@ case class Protocol private ( // Correctness check // Reader and writer versions must match the status of reader and writer features require( - supportsReaderFeatures == readerFeatures.isDefined, + (supportsReaderFeatures || canSupportColumnMappingFeature) == readerFeatures.isDefined, "Mismatched minReaderVersion and readerFeatures.") require( supportsWriterFeatures == writerFeatures.isDefined, "Mismatched minWriterVersion and writerFeatures.") - // When reader is on table features, writer must be on table features too + // When reader is on table features, writer must be on table features too. if (supportsReaderFeatures && !supportsWriterFeatures) { throw DeltaErrors.tableFeatureReadRequiresWriteException( TableFeatureProtocolUtils.TABLE_FEATURES_MIN_WRITER_VERSION) @@ -165,7 +165,7 @@ case class Protocol private ( */ @JsonIgnore lazy val simpleString: String = { - if (!supportsReaderFeatures && !supportsWriterFeatures) { + if (!supportsTableFeatures) { s"$minReaderVersion,$minWriterVersion" } else { val readerFeaturesStr = readerFeatures @@ -202,10 +202,12 @@ object Protocol { def apply( minReaderVersion: Int = Action.readerVersion, minWriterVersion: Int = Action.writerVersion): Protocol = { + val shouldAddReaderFeatures = supportsReaderFeatures(minReaderVersion) || + canSupportColumnMappingFeature(minReaderVersion, minWriterVersion) new Protocol( minReaderVersion = minReaderVersion, minWriterVersion = minWriterVersion, - readerFeatures = if (supportsReaderFeatures(minReaderVersion)) Some(Set()) else None, + readerFeatures = if (shouldAddReaderFeatures) Some(Set()) else None, writerFeatures = if (supportsWriterFeatures(minWriterVersion)) Some(Set()) else None) } @@ -213,7 +215,7 @@ object Protocol { def forTableFeature(tf: TableFeature): Protocol = { // Every table feature is a writer feature. val writerFeatures = tf.requiredFeatures + tf - val readerFeatures = writerFeatures.filter(f => f.isReaderWriterFeature && !f.isLegacyFeature) + val readerFeatures = writerFeatures.filter(_.isReaderWriterFeature) val writerFeaturesNames = writerFeatures.map(_.name) val readerFeaturesNames = readerFeatures.map(_.name) @@ -422,6 +424,11 @@ object Protocol { (getReaderVersionFromTableConf(conf), getWriterVersionFromTableConf(conf)) } + def filterProtocolPropsFromTableProps(properties: Map[String, String]): Map[String, String] = + properties.filterNot { + case (k, _) => TableFeatureProtocolUtils.isTableProtocolProperty(k) + } + /** Assert a table metadata contains no protocol-related table properties. */ def assertMetadataContainsNoProtocolProps(metadata: Metadata): Unit = { assert( diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala b/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala index 42f88141d3c..8be72d04ed4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala @@ -347,12 +347,24 @@ class DeltaCatalog extends DelegatingCatalogExtension properties: util.Map[String, String]) : Table = recordFrameProfile("DeltaCatalog", "createTable") { if (DeltaSourceUtils.isDeltaDataSourceName(getProvider(properties))) { + // TODO: we should extract write options from table properties for all the cases. We + // can remove the UC check when we have confidence. + val respectOptions = isUnityCatalog || properties.containsKey("test.simulateUC") + val (props, writeOptions) = if (respectOptions) { + val (props, writeOptions) = getTablePropsAndWriteOptions(properties) + expandTableProps(props, writeOptions, spark.sessionState.conf) + props.remove("test.simulateUC") + (props, writeOptions) + } else { + (properties, Map.empty[String, String]) + } + createDeltaTable( ident, schema, partitions, - properties, - Map.empty, + props, + writeOptions, sourceQuery = None, TableCreationModes.Create ) @@ -523,6 +535,44 @@ class DeltaCatalog extends DelegatingCatalogExtension } } + private def getTablePropsAndWriteOptions(properties: util.Map[String, String]) + : (util.Map[String, String], Map[String, String]) = { + val props = new util.HashMap[String, String]() + // Options passed in through the SQL API will show up both with an "option." prefix and + // without in Spark 3.1, so we need to remove those from the properties + val optionsThroughProperties = properties.asScala.collect { + case (k, _) if k.startsWith(TableCatalog.OPTION_PREFIX) => + k.stripPrefix(TableCatalog.OPTION_PREFIX) + }.toSet + val writeOptions = new util.HashMap[String, String]() + properties.asScala.foreach { case (k, v) => + if (!k.startsWith(TableCatalog.OPTION_PREFIX) && !optionsThroughProperties.contains(k)) { + // Add to properties + props.put(k, v) + } else if (optionsThroughProperties.contains(k)) { + writeOptions.put(k, v) + } + } + (props, writeOptions.asScala.toMap) + } + + private def expandTableProps( + props: util.Map[String, String], + options: Map[String, String], + conf: SQLConf): Unit = { + if (conf.getConf(DeltaSQLConf.DELTA_LEGACY_STORE_WRITER_OPTIONS_AS_PROPS)) { + // Legacy behavior + options.foreach { case (k, v) => props.put(k, v) } + } else { + options.foreach { case (k, v) => + // Continue putting in Delta prefixed options to avoid breaking workloads + if (k.toLowerCase(Locale.ROOT).startsWith("delta.")) { + props.put(k, v) + } + } + } + } + /** * A staged delta table, which creates a HiveMetaStore entry and appends data if this was a * CTAS/RTAS command. We have a ugly way of using this API right now, but it's the best way to @@ -544,35 +594,11 @@ class DeltaCatalog extends DelegatingCatalogExtension override def commitStagedChanges(): Unit = recordFrameProfile( "DeltaCatalog", "commitStagedChanges") { val conf = spark.sessionState.conf - val props = new util.HashMap[String, String]() - // Options passed in through the SQL API will show up both with an "option." prefix and - // without in Spark 3.1, so we need to remove those from the properties - val optionsThroughProperties = properties.asScala.collect { - case (k, _) if k.startsWith("option.") => k.stripPrefix("option.") - }.toSet - val sqlWriteOptions = new util.HashMap[String, String]() - properties.asScala.foreach { case (k, v) => - if (!k.startsWith("option.") && !optionsThroughProperties.contains(k)) { - // Do not add to properties - props.put(k, v) - } else if (optionsThroughProperties.contains(k)) { - sqlWriteOptions.put(k, v) - } - } - if (writeOptions.isEmpty && !sqlWriteOptions.isEmpty) { - writeOptions = sqlWriteOptions.asScala.toMap - } - if (conf.getConf(DeltaSQLConf.DELTA_LEGACY_STORE_WRITER_OPTIONS_AS_PROPS)) { - // Legacy behavior - writeOptions.foreach { case (k, v) => props.put(k, v) } - } else { - writeOptions.foreach { case (k, v) => - // Continue putting in Delta prefixed options to avoid breaking workloads - if (k.toLowerCase(Locale.ROOT).startsWith("delta.")) { - props.put(k, v) - } - } + val (props, sqlWriteOptions) = getTablePropsAndWriteOptions(properties) + if (writeOptions.isEmpty && sqlWriteOptions.nonEmpty) { + writeOptions = sqlWriteOptions } + expandTableProps(props, writeOptions, conf) createDeltaTable( ident, schema, diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala b/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala index 82821b4cb46..15c6fd375c9 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala @@ -96,7 +96,7 @@ case class DeltaTableV2( // as Unity Catalog may add more table storage properties on the fly. We should respect it // and merge the table storage properties and Delta options. val dataSourceOptions = if (catalogTable.isDefined) { - // To be safe, here we only extra file system options from table storage properties and + // To be safe, here we only extract file system options from table storage properties and // the original `options` has higher priority than the table storage properties. val fileSystemOptions = catalogTable.get.storage.properties.filter { case (k, _) => DeltaTableUtils.validDeltaTableHadoopPrefixes.exists(k.startsWith) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala index 3dacf2712ea..213651f6bb4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala @@ -376,13 +376,7 @@ abstract class ConvertToDeltaCommandBase( createdTime = Some(System.currentTimeMillis())) txn.updateMetadataForNewTable(metadata) - // TODO: we have not decided on how to implement CONVERT TO DELTA under column mapping modes - // for some convert targets so we block this feature for them here - checkColumnMapping(txn.metadata, targetTable) - RowTracking.checkStatsCollectedIfRowTrackingSupported( - txn.protocol, - collectStats, - statsEnabled) + checkConversionIsAllowed(txn, targetTable) val numFiles = targetTable.numFiles val addFilesIter = createDeltaActions(spark, manifest, partitionFields, txn, fs) @@ -442,6 +436,18 @@ abstract class ConvertToDeltaCommandBase( } } + /** Check if the conversion is allowed. */ + private def checkConversionIsAllowed( + txn: OptimisticTransaction, + targetTable: ConvertTargetTable): Unit = { + // TODO: we have not decided on how to implement CONVERT TO DELTA under column mapping modes + // for some convert targets so we block this feature for them here + checkColumnMapping(txn.metadata, targetTable) + RowTracking.checkStatsCollectedIfRowTrackingSupported( + txn.protocol, + collectStats, + statsEnabled) + } } case class ConvertToDeltaCommand( diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala index dfef8de28b8..644edbe1350 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.DeltaColumnMapping.{dropColumnMappingMetadata, filterColumnMappingProperties} -import org.apache.spark.sql.delta.actions.{Action, Metadata, Protocol} +import org.apache.spark.sql.delta.actions.{Action, Metadata, Protocol, TableFeatureProtocolUtils} import org.apache.spark.sql.delta.actions.DomainMetadata import org.apache.spark.sql.delta.commands.DMLUtils.TaggedCommitData import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsUtils @@ -134,7 +134,12 @@ case class CreateDeltaTableCommand( } val tableLocation = getDeltaTablePath(tableWithLocation) - val deltaLog = DeltaLog.forTable(sparkSession, tableLocation) + // To be safe, here we only extract file system options from table storage properties, to create + // the DeltaLog. + val fileSystemOptions = table.storage.properties.filter { case (k, _) => + DeltaTableUtils.validDeltaTableHadoopPrefixes.exists(k.startsWith) + } + val deltaLog = DeltaLog.forTable(sparkSession, tableLocation, fileSystemOptions) CoordinatedCommitsUtils.validateConfigurationsForCreateDeltaTableCommand( sparkSession, deltaLog.tableExists, query, tableWithLocation.properties) @@ -541,6 +546,10 @@ case class CreateDeltaTableCommand( // internal column mapping properties for the sake of comparison. var filteredTableProperties = filterColumnMappingProperties( tableDesc.properties) + // We also need to remove any protocol-related properties as we're filtering these + // from the metadata so they won't be present in the table properties. + filteredTableProperties = + Protocol.filterProtocolPropsFromTableProps(filteredTableProperties) var filteredExistingProperties = filterColumnMappingProperties( existingMetadata.configuration) // Clustered table has internal table properties in Metadata configurations and they are @@ -723,7 +732,11 @@ case class CreateDeltaTableCommand( if (txn.readVersion > -1L && isReplace && !dontOverwriteSchema) { // When a table already exists, and we're using the DataFrameWriterV2 API to replace // or createOrReplace a table, we blindly overwrite the metadata. - val newMetadata = getProvidedMetadata(table, schema.json) + var newMetadata = getProvidedMetadata(table, schema.json) + val updatedConfig = UniversalFormat.enforceDependenciesInConfiguration( + newMetadata.configuration, + txn.snapshot) + newMetadata = newMetadata.copy(configuration = updatedConfig) txn.updateMetadataForNewTableInReplace(newMetadata) } } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/VacuumCommand.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/VacuumCommand.scala index ab1242248aa..24113d85765 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/VacuumCommand.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/VacuumCommand.scala @@ -178,6 +178,11 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { None } } + .map { f => + // Below logic will make paths url-encoded + SerializableFileStatus(pathStringtoUrlEncodedString(f.path), f.length, f.isDir, + f.modificationTime) + } } /** @@ -273,6 +278,11 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { ), fileListingParallelism = Option(parallelism) ) + .map { f => + // Below logic will make paths url-encoded + SerializableFileStatus(pathStringtoUrlEncodedString(f.path), f.length, f.isDir, + f.modificationTime) + } } val allFilesAndDirs = allFilesAndDirsWithDuplicates.groupByKey(_.path) .mapGroups { (k, v) => @@ -299,6 +309,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { // 5. We subtract all the valid files and tombstones in our state // 6. We filter all paths with a count of 1, which will correspond to files not in the // state, and empty directories. We can safely delete all of these + val canonicalizedBasePath = SparkPath.fromPathString(basePath).urlEncoded val diff = allFilesAndDirs .where(col("modificationTime") < deleteBeforeTimestamp || col("isDir")) .mapPartitions { fileStatusIterator => @@ -307,16 +318,18 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { fileStatusIterator.flatMap { fileStatus => if (fileStatus.isDir) { Iterator.single(FileNameAndSize( - relativize(fileStatus.getHadoopPath, fs, reservoirBase, isDir = true), 0L)) + relativize(urlEncodedStringToPath(fileStatus.path), fs, + reservoirBase, isDir = true), 0L)) } else { - val dirs = getAllSubdirs(basePath, fileStatus.path, fs) + val dirs = getAllSubdirs(canonicalizedBasePath, fileStatus.path, fs) val dirsWithSlash = dirs.map { p => - val relativizedPath = relativize(new Path(p), fs, reservoirBase, isDir = true) + val relativizedPath = relativize(urlEncodedStringToPath(p), fs, + reservoirBase, isDir = true) FileNameAndSize(relativizedPath, 0L) } dirsWithSlash ++ Iterator( FileNameAndSize(relativize( - fileStatus.getHadoopPath, fs, reservoirBase, isDir = false), + urlEncodedStringToPath(fileStatus.path), fs, reservoirBase, isDir = false), fileStatus.length)) } } @@ -337,9 +350,9 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { .select(col("path")) .as[String] .map { relativePath => - assert(!stringToPath(relativePath).isAbsolute, + assert(!urlEncodedStringToPath(relativePath).isAbsolute, "Shouldn't have any absolute paths for deletion here.") - pathToString(DeltaFileOperations.absolutePath(basePath, relativePath)) + pathToUrlEncodedString(DeltaFileOperations.absolutePath(basePath, relativePath)) } val timeTakenToIdentifyEligibleFiles = System.currentTimeMillis() - startTimeToIdentifyEligibleFiles @@ -369,7 +382,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { log"a total of ${MDC(DeltaLogKeys.NUM_DIRS, dirCounts)} directories " + log"that are safe to delete. Vacuum stats: ${MDC(DeltaLogKeys.STATS, stats)}") - return diffFiles.map(f => stringToPath(f).toString).toDF("path") + return diffFiles.map(f => urlEncodedStringToPath(f).toString).toDF("path") } logVacuumStart( spark, @@ -574,7 +587,7 @@ trait VacuumCommandImpl extends DeltaCommand { fs: FileSystem, reservoirBase: Path, isDir: Boolean): String = { - pathToString(DeltaFileOperations.tryRelativizePath(fs, reservoirBase, path)) + pathToUrlEncodedString(DeltaFileOperations.tryRelativizePath(fs, reservoirBase, path)) } /** @@ -601,21 +614,22 @@ trait VacuumCommandImpl extends DeltaCommand { diff.repartition(parallelPartitions).mapPartitions { files => val fs = new Path(basePath).getFileSystem(hadoopConf.value.value) val filesDeletedPerPartition = - files.map(p => stringToPath(p)).count(f => tryDeleteNonRecursive(fs, f)) + files.map(p => urlEncodedStringToPath(p)).count(f => tryDeleteNonRecursive(fs, f)) Iterator(filesDeletedPerPartition) }.collect().sum } else { val fs = new Path(basePath).getFileSystem(hadoopConf.value.value) val fileResultSet = diff.toLocalIterator().asScala - fileResultSet.map(p => stringToPath(p)).count(f => tryDeleteNonRecursive(fs, f)) + fileResultSet.map(p => urlEncodedStringToPath(p)).count(f => tryDeleteNonRecursive(fs, f)) } } - // scalastyle:off pathfromuri - protected def stringToPath(path: String): Path = new Path(new URI(path)) - // scalastyle:on pathfromuri + protected def urlEncodedStringToPath(path: String): Path = SparkPath.fromUrlString(path).toPath + + protected def pathToUrlEncodedString(path: Path): String = SparkPath.fromPath(path).toString - protected def pathToString(path: Path): String = path.toUri.toString + protected def pathStringtoUrlEncodedString(path: String) = + SparkPath.fromPathString(path).toString /** Returns the relative path of a file action or None if the file lives outside of the table. */ protected def getActionRelativePath( @@ -631,7 +645,7 @@ trait VacuumCommandImpl extends DeltaCommand { fs: FileSystem, basePath: Path, relativizeIgnoreError: Boolean): Option[String] = { - val filePath = stringToPath(path) + val filePath = urlEncodedStringToPath(path) if (filePath.isAbsolute) { val maybeRelative = DeltaFileOperations.tryRelativizePath(fs, basePath, filePath, relativizeIgnoreError) @@ -639,10 +653,10 @@ trait VacuumCommandImpl extends DeltaCommand { // This file lives outside the directory of the table. None } else { - Some(pathToString(maybeRelative)) + Some(pathToUrlEncodedString(maybeRelative)) } } else { - Some(pathToString(filePath)) + Some(pathToUrlEncodedString(filePath)) } } @@ -686,7 +700,7 @@ trait VacuumCommandImpl extends DeltaCommand { case Some(dv) if dv.isOnDisk => if (dv.isRelative) { // We actually want a relative path here. - Some((pathToString(dv.absolutePath(new Path("."))), dv.sizeInBytes)) + Some((pathToUrlEncodedString(dv.absolutePath(new Path("."))), dv.sizeInBytes)) } else { assert(dv.isAbsolute) // This is never going to be a path relative to `basePath` for DVs. diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala index 44476ec68a9..6ef903499c3 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala @@ -153,7 +153,8 @@ case class WriteIntoDelta( // If READ_SIDE_CHAR_PADDING is not enabled, CHAR type is the same as VARCHAR. The change // below makes DESC TABLE to show VARCHAR instead of CHAR. CharVarcharUtils.replaceCharVarcharWithStringInSchema( - replaceCharWithVarchar(CharVarcharUtils.getRawSchema(data.schema)).asInstanceOf[StructType]) + CharVarcharUtils.replaceCharWithVarchar(CharVarcharUtils.getRawSchema(data.schema)) + .asInstanceOf[StructType]) } val finalSchema = schemaInCatalog.getOrElse(dataSchema) if (txn.metadata.schemaString != null) { diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDeltaLike.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDeltaLike.scala index 06350e1b283..3ca82ca2464 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDeltaLike.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDeltaLike.scala @@ -153,20 +153,6 @@ trait WriteIntoDeltaLike { txn.registerSQLMetrics(spark, sqlMetrics) } - import org.apache.spark.sql.types.{ArrayType, CharType, DataType, MapType, VarcharType} - protected def replaceCharWithVarchar(dt: DataType): DataType = dt match { - case ArrayType(et, nullable) => - ArrayType(replaceCharWithVarchar(et), nullable) - case MapType(kt, vt, nullable) => - MapType(replaceCharWithVarchar(kt), replaceCharWithVarchar(vt), nullable) - case StructType(fields) => - StructType(fields.map { field => - field.copy(dataType = replaceCharWithVarchar(field.dataType)) - }) - case CharType(length) => VarcharType(length) - case _ => dt - } - protected def extractConstraints( sparkSession: SparkSession, expr: Seq[Expression]): Seq[Constraint] = { diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/backfill/BackfillCommand.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/backfill/BackfillCommand.scala index b0258add0d1..8254aaa7109 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/backfill/BackfillCommand.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/backfill/BackfillCommand.scala @@ -60,6 +60,7 @@ trait BackfillCommand extends LeafRunnableCommand with DeltaCommand { txn.executionObserver.preparingCommit() txn.executionObserver.beginDoCommit() txn.executionObserver.beginBackfill() + txn.executionObserver.beginPostCommit() val maxNumFilesPerCommit = spark.conf.get(DeltaSQLConf.DELTA_BACKFILL_MAX_NUM_FILES_PER_COMMIT) val metricsOpType = "delta.backfill.materialization.trackerMetrics" diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/cdc/CDCReader.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/cdc/CDCReader.scala index 9f3f85eab32..eed7b665a6c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/cdc/CDCReader.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/cdc/CDCReader.scala @@ -36,13 +36,13 @@ import org.apache.spark.internal.MDC import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Literal} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, Expression, Literal} import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan} +import org.apache.spark.sql.sources.{BaseRelation, CatalystScan, Filter} import org.apache.spark.sql.types.{LongType, StringType, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -113,7 +113,7 @@ object CDCReader extends CDCReaderImpl snapshotWithSchemaMode: SnapshotWithSchemaMode, sqlContext: SQLContext, startingVersion: Option[Long], - endingVersion: Option[Long]) extends BaseRelation with PrunedFilteredScan { + endingVersion: Option[Long]) extends BaseRelation with CatalystScan { private val deltaLog = snapshotWithSchemaMode.snapshot.deltaLog @@ -152,7 +152,7 @@ object CDCReader extends CDCReaderImpl override def unhandledFilters(filters: Array[Filter]): Array[Filter] = Array.empty - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { + override def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] = { val df = changesToBatchDF( deltaLog, startingVersion.get, @@ -163,8 +163,18 @@ object CDCReader extends CDCReaderImpl sqlContext.sparkSession, readSchemaSnapshot = Some(snapshotForBatchSchema)) - val filter = Column(DeltaSourceUtils.translateFilters(filters)) - val projections = requiredColumns.map(SchemaUtils.fieldNameToColumn) + // Rewrite the attributes in the required columns and pushed down filters to match the output + // of the internal DataFrame. + val outputMap = df.queryExecution.analyzed.output.map(a => a.name -> a).toMap + val projections = + requiredColumns.map(a => Column(a.withExprId(outputMap(a.name).exprId))) + val filter = Column( + filters + .map(_.transform { case a: Attribute => a.withExprId(outputMap(a.name).exprId) }) + .reduceOption(And) + .getOrElse(Literal.TrueLiteral) + ) + df.filter(filter).select(projections: _*).rdd } } @@ -406,7 +416,7 @@ trait CDCReaderImpl extends DeltaLogging { spark.sqlContext, startingVersion = None, endingVersion = None) { - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = + override def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] = sqlContext.sparkSession.sparkContext.emptyRDD[Row] } } @@ -459,6 +469,8 @@ trait CDCReaderImpl extends DeltaLogging { * @param start - startingVersion of the changes * @param end - endingVersion of the changes * @param changes - changes is an iterator of all FileActions for a particular commit version. + * Note that for log files where InCommitTimestamps are enabled, the iterator + * must also contain the [[CommitInfo]] action. * @param spark - SparkSession * @param isStreaming - indicates whether the DataFrame returned is a streaming DataFrame * @param useCoarseGrainedCDC - ignores checks related to CDC being disabled in any of the @@ -483,9 +495,11 @@ trait CDCReaderImpl extends DeltaLogging { throw DeltaErrors.endBeforeStartVersionInCDC(start, end) } - // A map from change version to associated commit timestamp. - val timestampsByVersion: Map[Long, Timestamp] = - getTimestampsByVersion(deltaLog, start, end, spark) + // A map from change version to associated file modification timestamps. + // We only need these for non-InCommitTimestamp commits because for InCommitTimestamp commits, + // the timestamps are already stored in the commit info. + val nonICTTimestampsByVersion: Map[Long, Timestamp] = + getNonICTTimestampsByVersion(deltaLog, start, end) val changeFiles = ListBuffer[CDCDataSpec[AddCDCFile]]() val addFiles = ListBuffer[CDCDataSpec[AddFile]]() @@ -605,7 +619,6 @@ trait CDCReaderImpl extends DeltaLogging { // Set up buffers for all action types to avoid multiple passes. val cdcActions = ListBuffer[AddCDCFile]() - val ts = timestampsByVersion.get(v).orNull // Note that the CommitInfo is *not* guaranteed to be generated in 100% of cases. // We are using it only for a hotfix-safe mitigation/defense-in-depth - the value @@ -625,6 +638,18 @@ trait CDCReaderImpl extends DeltaLogging { case i: CommitInfo => commitInfo = Some(i) case _ => // do nothing } + // If the commit has an In-Commit Timestamp, we should use that as the commit timestamp. + // Note that it is technically possible for a commit range to begin with ICT commits + // followed by non-ICT commits, and end with ICT commits again. Ideally, for these commits + // we should use the file modification time for the first two ranges. However, this + // scenario is an edge case not worth optimizing for. + val ts = commitInfo + .flatMap(_.inCommitTimestamp) + .map(ict => new Timestamp(ict)) + .getOrElse(nonICTTimestampsByVersion.get(v).orNull) + // When `isStreaming` = `true` the [CommitInfo] action is only used for passing the + // in-commit timestamp to this method. We should filter them out. + commitInfo = if (isStreaming) None else commitInfo // If there are CDC actions, we read them exclusively if we should not use the // Add and RemoveFiles. @@ -877,22 +902,25 @@ trait CDCReaderImpl extends DeltaLogging { } /** - * Builds a map from commit versions to associated commit timestamps. + * Builds a map from commit versions to associated commit timestamps where the timestamp + * is the modification time of the commit file. Note that this function will not return + * InCommitTimestamps, it is up to the consumer of this function to decide whether the + * file modification time is the correct commit timestamp or whether they need to read the ICT. + * * @param start start commit version - * @param end end commit version + * @param end end commit version (inclusive) */ - def getTimestampsByVersion( + def getNonICTTimestampsByVersion( deltaLog: DeltaLog, start: Long, - end: Long, - spark: SparkSession): Map[Long, Timestamp] = { + end: Long): Map[Long, Timestamp] = { // Correct timestamp values are only available through DeltaHistoryManager.getCommits(). Commit // info timestamps are wrong, and file modification times are wrong because they need to be // monotonized first. This just performs a list (we don't read the contents of the files in // getCommits()) so the performance overhead is minimal. val monotonizationStart = math.max(start - DeltaHistoryManager.POTENTIALLY_UNMONOTONIZED_TIMESTAMPS, 0) - val commits = DeltaHistoryManager.getCommits( + val commits = DeltaHistoryManager.getCommitsWithNonIctTimestamps( deltaLog.store, deltaLog.logPath, monotonizationStart, diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtils.scala index 08ce50c1191..435d5306396 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtils.scala @@ -460,6 +460,20 @@ object CoordinatedCommitsUtils extends DeltaLogging { } } + /** + * Verifies that the property keys do not contain any ICT dependencies for Coordinated Commits. + */ + private def verifyNotContainsICTConfigurations( + propKeys: Seq[String], command: String, errorClass: String): Unit = { + ICT_TABLE_PROPERTY_KEYS.foreach { key => + if (propKeys.contains(key)) { + throw new DeltaIllegalArgumentException( + errorClass, + messageParameters = Array(command)) + } + } + } + /** * Validates the Coordinated Commits configurations in explicit command overrides for * `AlterTableSetPropertiesDeltaCommand`. @@ -467,6 +481,8 @@ object CoordinatedCommitsUtils extends DeltaLogging { * If the table already has Coordinated Commits configurations present, then we do not allow * users to override them via `ALTER TABLE t SET TBLPROPERTIES ...`. Users must downgrade the * table and then upgrade it with the new Coordinated Commits configurations. + * If the table is a Coordinated Commits table or will be one via this ALTER command, then we + * do not allow users to disable any ICT properties that Coordinated Commits depends on. */ def validateConfigurationsForAlterTableSetPropertiesDeltaCommand( existingConfs: Map[String, String], @@ -479,9 +495,15 @@ object CoordinatedCommitsUtils extends DeltaLogging { "DELTA_CANNOT_OVERRIDE_COORDINATED_COMMITS_CONFS", Array("ALTER")) } + verifyNotContainsICTConfigurations(propertyOverrides.keys.toSeq, command = "ALTER", + errorClass = "DELTA_CANNOT_SET_COORDINATED_COMMITS_DEPENDENCIES") verifyContainsOnlyCoordinatorNameAndConf( coordinatedCommitsOverrides, command = "ALTER", fromDefault = false) } + if (existingCoordinatedCommitsConfs.nonEmpty) { + verifyNotContainsICTConfigurations(propertyOverrides.keys.toSeq, command = "ALTER", + errorClass = "DELTA_CANNOT_MODIFY_COORDINATED_COMMITS_DEPENDENCIES") + } } /** @@ -489,7 +511,8 @@ object CoordinatedCommitsUtils extends DeltaLogging { * * If the table already has Coordinated Commits configurations present, then we do not allow users * to unset them via `ALTER TABLE t UNSET TBLPROPERTIES ...`. Users could only downgrade the table - * via `ALTER TABLE t DROP FEATURE ...`. + * via `ALTER TABLE t DROP FEATURE ...`. We also do not allow users to unset any ICT properties + * that Coordinated Commits depends on. */ def validateConfigurationsForAlterTableUnsetPropertiesDeltaCommand( existingConfs: Map[String, String], @@ -504,6 +527,8 @@ object CoordinatedCommitsUtils extends DeltaLogging { "DELTA_CANNOT_UNSET_COORDINATED_COMMITS_CONFS", Array.empty) } + verifyNotContainsICTConfigurations(propKeysToUnset, command = "ALTER", + errorClass = "DELTA_CANNOT_MODIFY_COORDINATED_COMMITS_DEPENDENCIES") } } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/fuzzer/OptimisticTransactionPhases.scala b/spark/src/main/scala/org/apache/spark/sql/delta/fuzzer/OptimisticTransactionPhases.scala index bb59574728e..4fc42e666ce 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/fuzzer/OptimisticTransactionPhases.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/fuzzer/OptimisticTransactionPhases.scala @@ -20,7 +20,8 @@ case class OptimisticTransactionPhases( initialPhase: ExecutionPhaseLock, preparePhase: ExecutionPhaseLock, commitPhase: ExecutionPhaseLock, - backfillPhase: ExecutionPhaseLock) + backfillPhase: ExecutionPhaseLock, + postCommitPhase: ExecutionPhaseLock) object OptimisticTransactionPhases { @@ -30,6 +31,7 @@ object OptimisticTransactionPhases { final val PREPARE_PHASE_LABEL = PREFIX + "PREPARE" final val COMMIT_PHASE_LABEL = PREFIX + "COMMIT" final val BACKFILL_PHASE_LABEL = PREFIX + "BACKFILL" + final val POST_COMMIT_PHASE_LABEL = PREFIX + "POST_COMMIT" def forName(txnName: String): OptimisticTransactionPhases = { @@ -40,6 +42,7 @@ object OptimisticTransactionPhases { initialPhase = ExecutionPhaseLock(toTxnPhaseLabel(INITIAL_PHASE_LABEL)), preparePhase = ExecutionPhaseLock(toTxnPhaseLabel(PREPARE_PHASE_LABEL)), commitPhase = ExecutionPhaseLock(toTxnPhaseLabel(COMMIT_PHASE_LABEL)), - backfillPhase = ExecutionPhaseLock(toTxnPhaseLabel(BACKFILL_PHASE_LABEL))) + backfillPhase = ExecutionPhaseLock(toTxnPhaseLabel(BACKFILL_PHASE_LABEL)), + postCommitPhase = ExecutionPhaseLock(toTxnPhaseLabel(POST_COMMIT_PHASE_LABEL))) } } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/fuzzer/PhaseLockingTransactionExecutionObserver.scala b/spark/src/main/scala/org/apache/spark/sql/delta/fuzzer/PhaseLockingTransactionExecutionObserver.scala index 74fe566799c..62110fc0634 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/fuzzer/PhaseLockingTransactionExecutionObserver.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/fuzzer/PhaseLockingTransactionExecutionObserver.scala @@ -27,7 +27,8 @@ private[delta] class PhaseLockingTransactionExecutionObserver( phases.initialPhase, phases.preparePhase, phases.commitPhase, - phases.backfillPhase) + phases.backfillPhase, + phases.postCommitPhase) override def createChild(): TransactionExecutionObserver = { // Just return the current thread observer. @@ -59,11 +60,16 @@ private[delta] class PhaseLockingTransactionExecutionObserver( phases.backfillPhase.waitToEnter() } + override def beginPostCommit(): Unit = { + phases.backfillPhase.leave() + phases.postCommitPhase.waitToEnter() + } + override def transactionCommitted(): Unit = { if (nextObserver.nonEmpty && autoAdvanceNextObserver) { - waitForCommitPhaseAndAdvanceToNextObserver() + waitForLastPhaseAndAdvanceToNextObserver() } else { - phases.backfillPhase.leave() + phases.postCommitPhase.leave() } } @@ -74,25 +80,31 @@ private[delta] class PhaseLockingTransactionExecutionObserver( } phases.commitPhase.leave() } - if (!phases.backfillPhase.hasEntered) { - phases.backfillPhase.waitToEnter() + if (!phases.backfillPhase.hasLeft) { + if (!phases.backfillPhase.hasEntered) { + phases.backfillPhase.waitToEnter() + } + phases.backfillPhase.leave() + } + if (!phases.postCommitPhase.hasEntered) { + phases.postCommitPhase.waitToEnter() } if (nextObserver.nonEmpty && autoAdvanceNextObserver) { - waitForCommitPhaseAndAdvanceToNextObserver() + waitForLastPhaseAndAdvanceToNextObserver() } else { - phases.backfillPhase.leave() + phases.postCommitPhase.leave() } } /* - * Wait for the backfill phase to pass but do not unblock it so that callers can write tests + * Wait for the last phase to pass but do not unblock it so that callers can write tests * that capture errors caused by code between the end of the last txn and the start of the * new txn. After the commit phase is passed, update the thread observer of the thread to * the next observer. */ - def waitForCommitPhaseAndAdvanceToNextObserver(): Unit = { + def waitForLastPhaseAndAdvanceToNextObserver(): Unit = { require(nextObserver.nonEmpty) - phases.backfillPhase.waitToLeave() + phases.postCommitPhase.waitToLeave() advanceToNextThreadObserver() } @@ -103,7 +115,7 @@ private[delta] class PhaseLockingTransactionExecutionObserver( * Note that when a next observer is set, the caller needs to manually unblock the exit barrier * of the commit phase. * - * For example, see [[waitForCommitPhaseAndAdvanceToNextObserver]]. + * For example, see [[waitForLastPhaseAndAdvanceToNextObserver]]. */ def setNextObserver( nextTxnObserver: TransactionExecutionObserver, diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/CheckpointHook.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/CheckpointHook.scala index e6e99c5b01a..833df53199d 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/CheckpointHook.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/CheckpointHook.scala @@ -40,6 +40,6 @@ object CheckpointHook extends PostCommitHook { committedVersion, lastCheckpointHint = None, lastCheckpointProvider = Some(cp)) - txn.deltaLog.checkpoint(snapshotToCheckpoint) + txn.deltaLog.checkpoint(snapshotToCheckpoint, txn.catalogTable.map(_.identifier)) } } 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 cfd84e9b0f0..fd7172603c0 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 @@ -296,9 +296,9 @@ object SchemaMergingUtils { * @param tf function to apply. * @return the transformed schema. */ - def transformColumns( - schema: StructType)( - tf: (Seq[String], StructField, Resolver) => StructField): StructType = { + def transformColumns[T <: DataType]( + schema: T)( + tf: (Seq[String], StructField, Resolver) => StructField): T = { def transform[E <: DataType](path: Seq[String], dt: E): E = { val newDt = dt match { case StructType(fields) => 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 b346802caf7..0a018e3ca50 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 @@ -59,7 +59,7 @@ object SchemaUtils extends DeltaLogging { * defines whether we should recurse into ArrayType and MapType. */ def filterRecursively( - schema: StructType, + schema: DataType, checkComplexTypes: Boolean)(f: StructField => Boolean): Seq[(Seq[String], StructField)] = { def recurseIntoComplexTypes( complexType: DataType, @@ -699,7 +699,7 @@ def normalizeColumnNamesInDataType( */ def findColumnPosition( column: Seq[String], - schema: StructType, + schema: DataType, resolver: Resolver = DELTA_COL_RESOLVER): Seq[Int] = { def findRecursively( searchPath: Seq[String], @@ -803,7 +803,7 @@ def normalizeColumnNamesInDataType( * @param position A list of ordinals (0-based) representing the path to the nested field in * `parent`. */ - def getNestedTypeFromPosition(schema: StructType, position: Seq[Int]): DataType = + def getNestedTypeFromPosition(schema: DataType, position: Seq[Int]): DataType = getNestedFieldFromPosition(StructField("schema", schema), position).dataType /** @@ -814,7 +814,34 @@ def normalizeColumnNamesInDataType( } /** - * Add `column` to the specified `position` in `schema`. + * Add a column to its child. + * @param parent The parent data type. + * @param column The column to add. + * @param position The position to add the column. + */ + def addColumn[T <: DataType](parent: T, column: StructField, position: Seq[Int]): T = { + if (position.isEmpty) { + throw DeltaErrors.addColumnParentNotStructException(column, parent) + } + parent match { + case struct: StructType => + addColumnToStruct(struct, column, position).asInstanceOf[T] + case map: MapType if position.head == MAP_KEY_INDEX => + map.copy(keyType = addColumn(map.keyType, column, position.tail)).asInstanceOf[T] + case map: MapType if position.head == MAP_VALUE_INDEX => + map.copy(valueType = addColumn(map.valueType, column, position.tail)).asInstanceOf[T] + case array: ArrayType if position.head == ARRAY_ELEMENT_INDEX => + array.copy(elementType = + addColumn(array.elementType, column, position.tail)).asInstanceOf[T] + case _: ArrayType => + throw DeltaErrors.incorrectArrayAccess() + case other => + throw DeltaErrors.addColumnParentNotStructException(column, other) + } + } + + /** + * Add `column` to the specified `position` in a struct `schema`. * @param position A Seq of ordinals on where this column should go. It is a Seq to denote * positions in nested columns (0-based). For example: * @@ -824,26 +851,10 @@ def normalizeColumnNamesInDataType( * will return * result: , b,c:STRUCT> */ - def addColumn(schema: StructType, column: StructField, position: Seq[Int]): StructType = { - def addColumnInChild(parent: DataType, column: StructField, position: Seq[Int]): DataType = { - if (position.isEmpty) { - throw DeltaErrors.addColumnParentNotStructException(column, parent) - } - parent match { - case struct: StructType => - addColumn(struct, column, position) - case map: MapType if position.head == MAP_KEY_INDEX => - map.copy(keyType = addColumnInChild(map.keyType, column, position.tail)) - case map: MapType if position.head == MAP_VALUE_INDEX => - map.copy(valueType = addColumnInChild(map.valueType, column, position.tail)) - case array: ArrayType if position.head == ARRAY_ELEMENT_INDEX => - array.copy(elementType = addColumnInChild(array.elementType, column, position.tail)) - case _: ArrayType => - throw DeltaErrors.incorrectArrayAccess() - case other => - throw DeltaErrors.addColumnParentNotStructException(column, other) - } - } + private def addColumnToStruct( + schema: StructType, + column: StructField, + position: Seq[Int]): StructType = { // If the proposed new column includes a default value, return a specific "not supported" error. // The rationale is that such operations require the data source scan operator to implement // support for filling in the specified default value when the corresponding field is not @@ -877,13 +888,42 @@ def normalizeColumnNamesInDataType( if (!column.nullable && field.nullable) { throw DeltaErrors.nullableParentWithNotNullNestedField } - val mid = field.copy(dataType = addColumnInChild(field.dataType, column, position.tail)) + val mid = field.copy(dataType = addColumn(field.dataType, column, position.tail)) StructType(pre ++ Seq(mid) ++ post.tail) } else { StructType(pre ++ Seq(column) ++ post) } } + /** + * Drop a column from its child. + * @param parent The parent data type. + * @param position The position to drop the column. + */ + def dropColumn[T <: DataType](parent: T, position: Seq[Int]): (T, StructField) = { + if (position.isEmpty) { + throw DeltaErrors.dropNestedColumnsFromNonStructTypeException(parent) + } + parent match { + case struct: StructType => + val (t, s) = dropColumnInStruct(struct, position) + (t.asInstanceOf[T], s) + case map: MapType if position.head == MAP_KEY_INDEX => + val (newKeyType, droppedColumn) = dropColumn(map.keyType, position.tail) + map.copy(keyType = newKeyType).asInstanceOf[T] -> droppedColumn + case map: MapType if position.head == MAP_VALUE_INDEX => + val (newValueType, droppedColumn) = dropColumn(map.valueType, position.tail) + map.copy(valueType = newValueType).asInstanceOf[T] -> droppedColumn + case array: ArrayType if position.head == ARRAY_ELEMENT_INDEX => + val (newElementType, droppedColumn) = dropColumn(array.elementType, position.tail) + array.copy(elementType = newElementType).asInstanceOf[T] -> droppedColumn + case _: ArrayType => + throw DeltaErrors.incorrectArrayAccess() + case other => + throw DeltaErrors.dropNestedColumnsFromNonStructTypeException(other) + } + } + /** * Drop from the specified `position` in `schema` and return with the original column. * @param position A Seq of ordinals on where this column should go. It is a Seq to denote @@ -894,30 +934,9 @@ def normalizeColumnNamesInDataType( * will return * result: , b,c:STRUCT> */ - def dropColumn(schema: StructType, position: Seq[Int]): (StructType, StructField) = { - def dropColumnInChild(parent: DataType, position: Seq[Int]): (DataType, StructField) = { - if (position.isEmpty) { - throw DeltaErrors.dropNestedColumnsFromNonStructTypeException(parent) - } - parent match { - case struct: StructType => - dropColumn(struct, position) - case map: MapType if position.head == MAP_KEY_INDEX => - val (newKeyType, droppedColumn) = dropColumnInChild(map.keyType, position.tail) - map.copy(keyType = newKeyType) -> droppedColumn - case map: MapType if position.head == MAP_VALUE_INDEX => - val (newValueType, droppedColumn) = dropColumnInChild(map.valueType, position.tail) - map.copy(valueType = newValueType) -> droppedColumn - case array: ArrayType if position.head == ARRAY_ELEMENT_INDEX => - val (newElementType, droppedColumn) = dropColumnInChild(array.elementType, position.tail) - array.copy(elementType = newElementType) -> droppedColumn - case _: ArrayType => - throw DeltaErrors.incorrectArrayAccess() - case other => - throw DeltaErrors.dropNestedColumnsFromNonStructTypeException(other) - } - } - + private def dropColumnInStruct( + schema: StructType, + position: Seq[Int]): (StructType, StructField) = { require(position.nonEmpty, "Don't know where to drop the column") val slicePosition = position.head if (slicePosition < 0) { @@ -930,7 +949,7 @@ def normalizeColumnNamesInDataType( val (pre, post) = schema.splitAt(slicePosition) val field = post.head if (position.length > 1) { - val (newType, droppedColumn) = dropColumnInChild(field.dataType, position.tail) + val (newType, droppedColumn) = dropColumn(field.dataType, position.tail) val mid = field.copy(dataType = newType) StructType(pre ++ Seq(mid) ++ post.tail) -> droppedColumn 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 aa46bd9bd0f..41ba27ac219 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 @@ -1561,7 +1561,9 @@ trait DeltaSQLConfBase { |The casting behavior is governed by 'spark.sql.storeAssignmentPolicy'. |""".stripMargin) .booleanConf - .createWithDefault(true) + // This feature doesn't properly support structs with missing fields and is disabled until a + // fix is implemented. + .createWithDefault(false) val DELTA_CDF_UNSAFE_BATCH_READ_ON_INCOMPATIBLE_SCHEMA_CHANGES = buildConf("changeDataFeed.unsafeBatchReadOnIncompatibleSchemaChanges.enabled") @@ -1606,6 +1608,21 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(true) + val DELTA_COLUMN_MAPPING_STRIP_METADATA = + buildConf("columnMapping.stripMetadata") + .doc( + """ + |Transactions might try to update the schema of a table with columns that contain + |column mapping metadata, even when column mapping is not enabled. For example, this + |can happen when transactions copy the schema from another table. When this setting is + |enabled, we will strip the column mapping metadata from the schema before applying it. + |Note that this config applies only when the existing schema of the table does not + |contain any column mapping metadata. + |""".stripMargin) + .internal() + .booleanConf + .createWithDefault(true) + val DYNAMIC_PARTITION_OVERWRITE_ENABLED = buildConf("dynamicPartitionOverwrite.enabled") .doc("Whether to overwrite partitions dynamically when 'partitionOverwriteMode' is set to " + diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala index e5fcb45932a..939ee708777 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.storage.{ClosableIterator, SupportsRewinding} import org.apache.spark.sql.delta.storage.ClosableIterator._ import org.apache.spark.sql.delta.util.{DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.util.ScalaExtensions._ import org.apache.hadoop.fs.FileStatus import org.apache.spark.internal.MDC @@ -792,7 +793,7 @@ case class DeltaSource( val (result, duration) = Utils.timeTakenMs { var iter = if (isInitialSnapshot) { Iterator(1, 2).flatMapWithClose { // so that the filterAndIndexDeltaLogs call is lazy - case 1 => getSnapshotAt(fromVersion).toClosable + case 1 => getSnapshotAt(fromVersion)._1.toClosable case 2 => filterAndIndexDeltaLogs(fromVersion + 1) } } else { @@ -845,8 +846,10 @@ case class DeltaSource( /** * This method computes the initial snapshot to read when Delta Source was initialized on a fresh * stream. + * @return A tuple where the first element is an iterator of IndexedFiles and the second element + * is the in-commit timestamp of the initial snapshot if available. */ - protected def getSnapshotAt(version: Long): Iterator[IndexedFile] = { + protected def getSnapshotAt(version: Long): (Iterator[IndexedFile], Option[Long]) = { if (initialState == null || version != initialStateVersion) { super[DeltaSourceBase].cleanUpSnapshotResources() val snapshot = getSnapshotFromDeltaLog(version) @@ -879,7 +882,12 @@ case class DeltaSource( ) } } - addBeginAndEndIndexOffsetsForVersion(version, initialState.iterator()) + val inCommitTimestampOpt = + Option.when( + DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.fromMetaData(initialState.snapshot.metadata)) { + initialState.snapshot.timestamp + } + (addBeginAndEndIndexOffsetsForVersion(version, initialState.iterator()), inCommitTimestampOpt) } /** diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceCDCSupport.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceCDCSupport.scala index 0e66cf83d9f..fcfbf2d5a71 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceCDCSupport.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceCDCSupport.scala @@ -200,9 +200,9 @@ trait DeltaSourceCDCSupport { self: DeltaSource => val changes = getFileChangesForCDC( startVersion, startIndex, isInitialSnapshot, limits = None, Some(endOffset)) - val groupedFileActions = - changes.map { case (v, indexFiles) => - (v, indexFiles.filter(_.hasFileAction).map(_.getFileAction).toSeq) + val groupedFileAndCommitInfoActions = + changes.map { case (v, indexFiles, commitInfoOpt) => + (v, indexFiles.filter(_.hasFileAction).map(_.getFileAction).toSeq ++ commitInfoOpt) } val (result, duration) = Utils.timeTakenMs { @@ -211,7 +211,7 @@ trait DeltaSourceCDCSupport { self: DeltaSource => readSnapshotDescriptor, startVersion, endOffset.reservoirVersion, - groupedFileActions, + groupedFileAndCommitInfoActions, spark, isStreaming = true) .fileChangeDf @@ -228,7 +228,8 @@ trait DeltaSourceCDCSupport { self: DeltaSource => /** * Get the changes starting from (fromVersion, fromIndex). fromVersion is included. - * It returns an iterator of (log_version, fileActions) + * It returns an iterator of (log_version, fileActions, Optional[CommitInfo]). The commit info + * is needed later on so that the InCommitTimestamp of the log files can be determined. * * If verifyMetadataAction = true, we will break the stream when we detect any read-incompatible * metadata changes. @@ -239,10 +240,12 @@ trait DeltaSourceCDCSupport { self: DeltaSource => isInitialSnapshot: Boolean, limits: Option[AdmissionLimits], endOffset: Option[DeltaSourceOffset], - verifyMetadataAction: Boolean = true): Iterator[(Long, Iterator[IndexedFile])] = { + verifyMetadataAction: Boolean = true + ): Iterator[(Long, Iterator[IndexedFile], Option[CommitInfo])] = { /** Returns matching files that were added on or after startVersion among delta logs. */ - def filterAndIndexDeltaLogs(startVersion: Long): Iterator[(Long, IndexedChangeFileSeq)] = { + def filterAndIndexDeltaLogs( + startVersion: Long): Iterator[(Long, IndexedChangeFileSeq, Option[CommitInfo])] = { // TODO: handle the case when failOnDataLoss = false and we are missing change log files // in that case, we need to recompute the start snapshot and evolve the schema if needed require(options.failOnDataLoss || !trackingMetadataChange, @@ -250,7 +253,7 @@ trait DeltaSourceCDCSupport { self: DeltaSource => deltaLog.getChanges(startVersion, options.failOnDataLoss).map { case (version, actions) => // skipIndexedFile must be applied after creating IndexedFile so that // IndexedFile.index is consistent across all versions. - val (fileActions, skipIndexedFile, metadataOpt, protocolOpt) = + val (fileActions, skipIndexedFile, metadataOpt, protocolOpt, commitInfoOpt) = filterCDCActions( actions, version, fromVersion, endOffset.map(_.reservoirVersion), verifyMetadataAction && !trackingMetadataChange) @@ -278,7 +281,7 @@ trait DeltaSourceCDCSupport { self: DeltaSource => remove = remove, shouldSkip = skipIndexedFile) }) - (version, new IndexedChangeFileSeq(itr, isInitialSnapshot = false)) + (version, new IndexedChangeFileSeq(itr, isInitialSnapshot = false), commitInfoOpt) } } @@ -293,35 +296,43 @@ trait DeltaSourceCDCSupport { self: DeltaSource => } val (result, duration) = Utils.timeTakenMs { - val iter: Iterator[(Long, IndexedChangeFileSeq)] = if (isInitialSnapshot) { - // If we are reading change data from the start of the table we need to - // get the latest snapshot of the table as well. - val snapshot: Iterator[IndexedFile] = getSnapshotAt(fromVersion).map { m => - // When we get the snapshot the dataChange is false for the AddFile actions - // We need to set it to true for it to be considered by the CDCReader. - if (m.add != null) { - m.copy(add = m.add.copy(dataChange = true)) - } else { - m + val iter: Iterator[(Long, IndexedChangeFileSeq, Option[CommitInfo])] = + if (isInitialSnapshot) { + // If we are reading change data from the start of the table we need to + // get the latest snapshot of the table as well. + val (unprocessedSnapshot, snapshotInCommitTimestampOpt) = getSnapshotAt(fromVersion) + val snapshot: Iterator[IndexedFile] = unprocessedSnapshot.map { m => + // When we get the snapshot the dataChange is false for the AddFile actions + // We need to set it to true for it to be considered by the CDCReader. + if (m.add != null) { + m.copy(add = m.add.copy(dataChange = true)) + } else { + m + } } + // This is a hack so that we can easily access the ICT later on. + // This `CommitInfo` action is not useful for anything else and should be filtered + // out later on. + val ictOnlyCommitInfo = Some(CommitInfo.empty(Some(-1)) + .copy(inCommitTimestamp = snapshotInCommitTimestampOpt)) + val snapshotItr: Iterator[(Long, IndexedChangeFileSeq, Option[CommitInfo])] = Iterator(( + fromVersion, + new IndexedChangeFileSeq(snapshot, isInitialSnapshot = true), + ictOnlyCommitInfo + )) + + snapshotItr ++ filterAndIndexDeltaLogs(fromVersion + 1) + } else { + filterAndIndexDeltaLogs(fromVersion) } - val snapshotItr: Iterator[(Long, IndexedChangeFileSeq)] = Iterator(( - fromVersion, - new IndexedChangeFileSeq(snapshot, isInitialSnapshot = true) - )) - - snapshotItr ++ filterAndIndexDeltaLogs(fromVersion + 1) - } else { - filterAndIndexDeltaLogs(fromVersion) - } // In this case, filterFiles will consume the available capacity. We use takeWhile // to stop the iteration when we reach the limit or if endOffset is specified and the // endVersion is reached which will save us from reading unnecessary log files. - iter.takeWhile { case (version, _) => + iter.takeWhile { case (version, _, _) => limits.forall(_.hasCapacity) && versionLessThanEndOffset(version, endOffset) - }.map { case (version, indexItr) => - (version, indexItr.filterFiles(fromVersion, fromIndex, limits, endOffset)) + }.map { case (version, indexItr, ci) => + (version, indexItr.filterFiles(fromVersion, fromIndex, limits, endOffset), ci) } } @@ -351,10 +362,11 @@ trait DeltaSourceCDCSupport { self: DeltaSource => batchStartVersion: Long, batchEndVersionOpt: Option[Long] = None, verifyMetadataAction: Boolean = true - ): (Seq[FileAction], Boolean, Option[Metadata], Option[Protocol]) = { + ): (Seq[FileAction], Boolean, Option[Metadata], Option[Protocol], Option[CommitInfo]) = { var shouldSkipIndexedFile = false var metadataAction: Option[Metadata] = None var protocolAction: Option[Protocol] = None + var commitInfoAction: Option[CommitInfo] = None def checkAndCacheMetadata(m: Metadata): Unit = { if (verifyMetadataAction) { checkReadIncompatibleSchemaChanges(m, version, batchStartVersion, batchEndVersionOpt) @@ -367,6 +379,9 @@ trait DeltaSourceCDCSupport { self: DeltaSource => if (actions.exists(_.isInstanceOf[AddCDCFile])) { (actions.filter { case _: AddCDCFile => true + case commitInfo: CommitInfo => + commitInfoAction = Some(commitInfo) + false case m: Metadata => checkAndCacheMetadata(m) false @@ -374,7 +389,11 @@ trait DeltaSourceCDCSupport { self: DeltaSource => protocolAction = Some(p) false case _ => false - }.asInstanceOf[Seq[FileAction]], shouldSkipIndexedFile, metadataAction, protocolAction) + }.asInstanceOf[Seq[FileAction]], + shouldSkipIndexedFile, + metadataAction, + protocolAction, + commitInfoAction) } else { (actions.filter { case a: AddFile => @@ -392,12 +411,17 @@ trait DeltaSourceCDCSupport { self: DeltaSource => false case commitInfo: CommitInfo => shouldSkipIndexedFile = CDCReader.shouldSkipFileActionsInCommit(commitInfo) + commitInfoAction = Some(commitInfo) false case _: AddCDCFile | _: SetTransaction | _: DomainMetadata => false case null => // Some crazy future feature. Ignore false - }.asInstanceOf[Seq[FileAction]], shouldSkipIndexedFile, metadataAction, protocolAction) + }.asInstanceOf[Seq[FileAction]], + shouldSkipIndexedFile, + metadataAction, + protocolAction, + commitInfoAction) } } } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/util/DatasetRefCache.scala b/spark/src/main/scala/org/apache/spark/sql/delta/util/DatasetRefCache.scala index 3d9bdbdbebe..879a101bb31 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/util/DatasetRefCache.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/util/DatasetRefCache.scala @@ -42,10 +42,12 @@ import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} * * @param creator a function to create [[Dataset]]. */ -class DatasetRefCache[T](creator: () => Dataset[T]) { +class DatasetRefCache[T] private[util](creator: () => Dataset[T]) { private val holder = new AtomicReference[Dataset[T]] + private[delta] def invalidate() = holder.set(null) + def get: Dataset[T] = Option(holder.get()) .filter(_.sparkSession eq SparkSession.active) .getOrElse { @@ -54,4 +56,3 @@ class DatasetRefCache[T](creator: () => Dataset[T]) { df } } - diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/util/DeltaFileOperations.scala b/spark/src/main/scala/org/apache/spark/sql/delta/util/DeltaFileOperations.scala index bf1d4727fd7..8df11b3e18b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/util/DeltaFileOperations.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/util/DeltaFileOperations.scala @@ -243,7 +243,10 @@ object DeltaFileOperations extends DeltaLogging { import org.apache.spark.sql.delta.implicits._ if (subDirs.isEmpty) return spark.emptyDataset[SerializableFileStatus] val listParallelism = fileListingParallelism.getOrElse(spark.sparkContext.defaultParallelism) - val dirsAndFiles = spark.sparkContext.parallelize(subDirs).mapPartitions { dirs => + val subDirsParallelism = subDirs.length.min(spark.sparkContext.defaultParallelism) + val dirsAndFiles = spark.sparkContext.parallelize( + subDirs, + subDirsParallelism).mapPartitions { dirs => val logStore = LogStore(SparkEnv.get.conf, hadoopConf.value.value) listUsingLogStore( logStore, diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/util/StateCache.scala b/spark/src/main/scala/org/apache/spark/sql/delta/util/StateCache.scala index 391cf56710d..86784bfde3c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/util/StateCache.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/util/StateCache.scala @@ -40,6 +40,7 @@ trait StateCache extends DeltaLogging { private var _isCached = true /** A list of RDDs that we need to uncache when we are done with this snapshot. */ private val cached = ArrayBuffer[RDD[_]]() + private val cached_refs = ArrayBuffer[DatasetRefCache[_]]() /** Method to expose the value of _isCached for testing. */ private[delta] def isCached: Boolean = _isCached @@ -47,7 +48,7 @@ trait StateCache extends DeltaLogging { private val storageLevel = StorageLevel.fromString( spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_SNAPSHOT_CACHE_STORAGE_LEVEL)) - class CachedDS[A](ds: Dataset[A], name: String) { + class CachedDS[A] private[StateCache](ds: Dataset[A], name: String) { // While we cache RDD to avoid re-computation in different spark sessions, `Dataset` can only be // reused by the session that created it to avoid session pollution. So we use `DatasetRefCache` // to re-create a new `Dataset` when the active session is changed. This is an optimization for @@ -64,10 +65,10 @@ trait StateCache extends DeltaLogging { rdd.persist(storageLevel) } cached += rdd - val dsCache = new DatasetRefCache(() => { + val dsCache = datasetRefCache { () => val logicalRdd = LogicalRDD(qe.analyzed.output, rdd)(spark) Dataset.ofRows(spark, logicalRdd) - }) + } Some(dsCache) } else { None @@ -110,11 +111,18 @@ trait StateCache extends DeltaLogging { new CachedDS[A](ds, name) } + def datasetRefCache[A](creator: () => Dataset[A]): DatasetRefCache[A] = { + val dsCache = new DatasetRefCache(creator) + cached_refs += dsCache + dsCache + } + /** Drop any cached data for this [[Snapshot]]. */ def uncache(): Unit = cached.synchronized { if (isCached) { _isCached = false cached.foreach(_.unpersist(blocking = false)) + cached_refs.foreach(_.invalidate()) } } } diff --git a/spark/src/test/resources/log4j2.properties b/spark/src/test/resources/log4j2.properties index a35b0f42608..742d9e1fa8c 100644 --- a/spark/src/test/resources/log4j2.properties +++ b/spark/src/test/resources/log4j2.properties @@ -38,13 +38,6 @@ appender.file.append = true appender.file.layout.type = PatternLayout appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n -# Structured Logging Appender -appender.structured.type = File -appender.structured.name = structured -appender.structured.fileName = target/structured.log -appender.structured.layout.type = JsonTemplateLayout -appender.structured.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json - # Pattern Logging Appender appender.pattern.type = File appender.pattern.name = pattern @@ -52,12 +45,6 @@ appender.pattern.fileName = target/pattern.log appender.pattern.layout.type = PatternLayout appender.pattern.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex -# Custom logger for testing structured logging with Spark master -logger.structured_logging.name = org.apache.spark.sql.delta.logging.DeltaStructuredLoggingSuite -logger.structured_logging.level = trace -logger.structured_logging.appenderRefs = structured -logger.structured_logging.appenderRef.structured.ref = structured - # Custom logger for testing structured logging with Spark 3.5 shims logger.pattern_logging.name = org.apache.spark.sql.delta.logging.DeltaPatternLoggingSuite logger.pattern_logging.level = trace @@ -76,4 +63,3 @@ appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose logger.jetty.name = org.sparkproject.jetty logger.jetty.level = warn - diff --git a/spark/src/test/resources/log4j2_spark_master.properties b/spark/src/test/resources/log4j2_spark_master.properties new file mode 100644 index 00000000000..e76ca3383b7 --- /dev/null +++ b/spark/src/test/resources/log4j2_spark_master.properties @@ -0,0 +1,65 @@ +# +# 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. +# +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +rootLogger.level = info +rootLogger.appenderRef.file.ref = ${sys:test.appender:-File} + +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.append = true +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Structured Logging Appender +appender.structured.type = File +appender.structured.name = structured +appender.structured.fileName = target/structured.log +appender.structured.layout.type = JsonTemplateLayout +appender.structured.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json + +# Custom logger for testing structured logging with Spark master +logger.structured_logging.name = org.apache.spark.sql.delta.logging.DeltaStructuredLoggingSuite +logger.structured_logging.level = trace +logger.structured_logging.appenderRefs = structured +logger.structured_logging.appenderRef.structured.ref = structured + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +logger.jetty.name = org.sparkproject.jetty +logger.jetty.level = warn 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 449cd55b9fb..daac96a28e4 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 @@ -100,7 +100,7 @@ class DeltaVariantSuite // check previously thrown error message checkError( e, - errorClass = "DELTA_FEATURES_REQUIRE_MANUAL_ENABLEMENT", + "DELTA_FEATURES_REQUIRE_MANUAL_ENABLEMENT", parameters = Map( "unsupportedFeatures" -> VariantTypeTableFeature.name, "supportedFeatures" -> currentFeatures @@ -123,13 +123,13 @@ class DeltaVariantSuite test("VariantType may not be used as a partition column") { withTable("delta_test") { checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql( """CREATE TABLE delta_test(s STRING, v VARIANT) |USING delta |PARTITIONED BY (v)""".stripMargin) }, - errorClass = "INVALID_PARTITION_COLUMN_DATA_TYPE", + "INVALID_PARTITION_COLUMN_DATA_TYPE", parameters = Map("type" -> "\"VARIANT\"") ) } @@ -516,7 +516,7 @@ class DeltaVariantSuite } checkError( insertException, - errorClass = "DELTA_NOT_NULL_CONSTRAINT_VIOLATED", + "DELTA_NOT_NULL_CONSTRAINT_VIOLATED", parameters = Map("columnName" -> "v") ) @@ -539,7 +539,7 @@ class DeltaVariantSuite } checkError( insertException, - errorClass = "DELTA_VIOLATE_CONSTRAINT_WITH_VALUES", + "DELTA_VIOLATE_CONSTRAINT_WITH_VALUES", parameters = Map( "constraintName" -> "variantgtezero", "expression" -> "(variant_get(v, '$', 'INT') >= 0)", "values" -> " - v : -1" diff --git a/spark/src/test/scala/io/delta/sql/parser/DeltaSqlParserSuite.scala b/spark/src/test/scala/io/delta/sql/parser/DeltaSqlParserSuite.scala index 1052c5528bb..4934a1d8849 100644 --- a/spark/src/test/scala/io/delta/sql/parser/DeltaSqlParserSuite.scala +++ b/spark/src/test/scala/io/delta/sql/parser/DeltaSqlParserSuite.scala @@ -463,9 +463,9 @@ class DeltaSqlParserSuite extends SparkFunSuite with SQLHelper { val parser = new DeltaSqlParser(new SparkSqlParser()) val sql = clusterByStatement(clause, asSelect, "a int, b string", "CLUSTER BY (a) CLUSTER BY (b)") - checkError(exception = intercept[ParseException] { + checkError(intercept[ParseException] { parser.parsePlan(sql) - }, errorClass = "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "CLUSTER BY")) + }, "DUPLICATE_CLAUSES", parameters = Map("clauseName" -> "CLUSTER BY")) } test("CLUSTER BY set clustering column property is ignored - " + @@ -492,9 +492,9 @@ class DeltaSqlParserSuite extends SparkFunSuite with SQLHelper { "CLUSTER BY (a) PARTITIONED BY (b)") val errorMsg = "Clustering and partitioning cannot both be specified. " + "Please remove PARTITIONED BY if you want to create a Delta table with clustering" - checkError(exception = intercept[ParseException] { + checkError(intercept[ParseException] { parser.parsePlan(sql) - }, errorClass = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> errorMsg)) + }, "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> errorMsg)) } test(s"CLUSTER BY with bucketing - $clause TABLE asSelect = $asSelect") { @@ -508,9 +508,9 @@ class DeltaSqlParserSuite extends SparkFunSuite with SQLHelper { val errorMsg = "Clustering and bucketing cannot both be specified. " + "Please remove CLUSTERED BY INTO BUCKETS if you " + "want to create a Delta table with clustering" - checkError(exception = intercept[ParseException] { + checkError(intercept[ParseException] { parser.parsePlan(sql) - }, errorClass = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> errorMsg)) + }, "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> errorMsg)) } } } diff --git a/spark/src/test/scala/io/delta/tables/DeltaTableBuilderSuite.scala b/spark/src/test/scala/io/delta/tables/DeltaTableBuilderSuite.scala index cbeb5b24bc7..4cf9f43f558 100644 --- a/spark/src/test/scala/io/delta/tables/DeltaTableBuilderSuite.scala +++ b/spark/src/test/scala/io/delta/tables/DeltaTableBuilderSuite.scala @@ -492,10 +492,7 @@ class DeltaTableBuilderSuite .execute() } - checkError( - exception = e, - errorClass = "DELTA_CLUSTER_BY_WITH_PARTITIONED_BY" - ) + checkError(e, "DELTA_CLUSTER_BY_WITH_PARTITIONED_BY") } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/ActionSerializerSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/ActionSerializerSuite.scala index e9121cd5ba0..48aa427a539 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/ActionSerializerSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/ActionSerializerSuite.scala @@ -235,7 +235,7 @@ class ActionSerializerSuite extends QueryTest with SharedSparkSession with Delta expectedJson = s"""{"protocol":{"minReaderVersion":$TABLE_FEATURES_MIN_READER_VERSION,""" + s""""minWriterVersion":$TABLE_FEATURES_MIN_WRITER_VERSION,""" + - """"readerFeatures":["testLegacyReaderWriter"],""" + + """"readerFeatures":[],""" + """"writerFeatures":["testLegacyReaderWriter"]}}""") testActionSerDe( @@ -248,7 +248,7 @@ class ActionSerializerSuite extends QueryTest with SharedSparkSession with Delta expectedJson = s"""{"protocol":{"minReaderVersion":$TABLE_FEATURES_MIN_READER_VERSION,""" + s""""minWriterVersion":$TABLE_FEATURES_MIN_WRITER_VERSION,""" + - """"readerFeatures":["testLegacyReaderWriter","testReaderWriter"],""" + + """"readerFeatures":["testReaderWriter"],""" + """"writerFeatures":["testLegacyReaderWriter","testReaderWriter"]}}""") testActionSerDe( diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/CheckCDCAnswer.scala b/spark/src/test/scala/org/apache/spark/sql/delta/CheckCDCAnswer.scala index 77e8730d524..5431c42e8d2 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/CheckCDCAnswer.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/CheckCDCAnswer.scala @@ -47,16 +47,10 @@ trait CheckCDCAnswer extends QueryTest { // Results should match the fully monotonized commits. Note that this map will include // all versions of the table but only the ones in timestampsByVersion are checked for // correctness. - val commits = DeltaHistoryManager.getCommits( - log.store, - log.logPath, - start = 0, - end = None, - log.newDeltaHadoopConf()) - - // Note that the timestamps come from filesystem modification timestamps, so they're - // milliseconds since epoch and we don't need to deal with timezones. - commits.map(f => (f.version -> new Timestamp(f.timestamp))).toMap + val commits = log.history.getHistory(start = 0, end = None) + // Note that the timestamps are in milliseconds since epoch and we don't need to deal + // with timezones. + commits.map(f => (f.getVersion -> f.timestamp)).toMap } timestampsByVersion.keySet.foreach { version => diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaAlterTableTests.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaAlterTableTests.scala index 32098b9c1e4..fa966b36223 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaAlterTableTests.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaAlterTableTests.scala @@ -924,17 +924,17 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { .withColumn("a", map('v1, 'v2)) withDeltaTable(df) { tableName => checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql(s"ALTER TABLE $tableName CHANGE COLUMN a.key COMMENT 'a comment'") }, - errorClass = "DELTA_UNSUPPORTED_COMMENT_MAP_ARRAY", + "DELTA_UNSUPPORTED_COMMENT_MAP_ARRAY", parameters = Map("fieldPath" -> "a.key") ) checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql(s"ALTER TABLE $tableName CHANGE COLUMN a.value COMMENT 'a comment'") }, - errorClass = "DELTA_UNSUPPORTED_COMMENT_MAP_ARRAY", + "DELTA_UNSUPPORTED_COMMENT_MAP_ARRAY", parameters = Map("fieldPath" -> "a.value") ) } @@ -945,10 +945,10 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { .withColumn("a", array('v1)) withDeltaTable(df) { tableName => checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql(s"ALTER TABLE $tableName CHANGE COLUMN a.element COMMENT 'a comment'") }, - errorClass = "DELTA_UNSUPPORTED_COMMENT_MAP_ARRAY", + "DELTA_UNSUPPORTED_COMMENT_MAP_ARRAY", parameters = Map("fieldPath" -> "a.element") ) } @@ -959,20 +959,20 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { .withColumn("a", map('v1, 'v2)) withDeltaTable(df) { tableName => checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"ALTER TABLE $tableName RENAME COLUMN a.key TO key2") }, - errorClass = "INVALID_FIELD_NAME", + "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`a`.`key2`", "path" -> "`a`" ) ) checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"ALTER TABLE $tableName RENAME COLUMN a.value TO value2") }, - errorClass = "INVALID_FIELD_NAME", + "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`a`.`value2`", "path" -> "`a`" @@ -986,10 +986,10 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { .withColumn("a", array('v1)) withDeltaTable(df) { tableName => checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"ALTER TABLE $tableName RENAME COLUMN a.element TO element2") }, - errorClass = "INVALID_FIELD_NAME", + "INVALID_FIELD_NAME", parameters = Map( "fieldName" -> "`a`.`element2`", "path" -> "`a`" @@ -1008,10 +1008,10 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { ddlTest("CHANGE COLUMN - incompatible") { withDeltaTable(Seq((1, "a"), (2, "b")).toDF("v1", "v2")) { tableName => checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql(s"ALTER TABLE $tableName CHANGE COLUMN v1 v1 long") }, - errorClass = "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", + "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", parameters = Map( "fieldPath" -> "v1", "oldField" -> "INT", @@ -1026,10 +1026,10 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { .withColumn("struct", struct("v1", "v2")) withDeltaTable(df) { tableName => checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql(s"ALTER TABLE $tableName CHANGE COLUMN struct.v1 v1 long") }, - errorClass = "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", + "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", parameters = Map( "fieldPath" -> "struct.v1", "oldField" -> "INT", @@ -1044,10 +1044,10 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { .withColumn("a", map('v1, 'v2)) withDeltaTable(df) { tableName => checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql(s"ALTER TABLE $tableName CHANGE COLUMN a.key key long") }, - errorClass = "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", + "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", parameters = Map( "fieldPath" -> "a.key", "oldField" -> "INT NOT NULL", @@ -1062,10 +1062,10 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { .withColumn("a", map('v1, 'v2)) withDeltaTable(df) { tableName => checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql(s"ALTER TABLE $tableName CHANGE COLUMN a.value value long") }, - errorClass = "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", + "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", parameters = Map( "fieldPath" -> "a.value", "oldField" -> "INT", @@ -1080,10 +1080,10 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { .withColumn("a", array('v1)) withDeltaTable(df) { tableName => checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql(s"ALTER TABLE $tableName CHANGE COLUMN a.element element long") }, - errorClass = "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", + "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", parameters = Map( "fieldPath" -> "a.element", "oldField" -> "INT", @@ -1383,8 +1383,8 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { // Changing the nullability of map/array fields is not allowed. var statement = s"ALTER TABLE $tableName CHANGE COLUMN m.key DROP NOT NULL" checkError( - exception = intercept[AnalysisException] { sql(statement) }, - errorClass = "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", + intercept[AnalysisException] { sql(statement) }, + "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", parameters = Map( "fieldPath" -> "m.key", "oldField" -> "INT NOT NULL", @@ -1394,8 +1394,8 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { statement = s"ALTER TABLE $tableName CHANGE COLUMN m.value SET NOT NULL" checkError( - exception = intercept[AnalysisException] { sql(statement) }, - errorClass = "_LEGACY_ERROR_TEMP_2330", + intercept[AnalysisException] { sql(statement) }, + "_LEGACY_ERROR_TEMP_2330", parameters = Map( "fieldName" -> "m.value" ), @@ -1404,8 +1404,8 @@ trait DeltaAlterTableTests extends DeltaAlterTableTestBase { statement = s"ALTER TABLE $tableName CHANGE COLUMN a.element SET NOT NULL" checkError( - exception = intercept[AnalysisException] { sql(statement) }, - errorClass = "_LEGACY_ERROR_TEMP_2330", + intercept[AnalysisException] { sql(statement) }, + "_LEGACY_ERROR_TEMP_2330", parameters = Map( "fieldName" -> "a.element" ), diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCColumnMappingSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCColumnMappingSuite.scala index 4eb5a8cba80..465a748ba65 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCColumnMappingSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCColumnMappingSuite.scala @@ -539,7 +539,8 @@ trait DeltaCDCColumnMappingSuiteBase extends DeltaCDCSuiteBase "add column batch cdc read not blocked", "data type and nullability change batch cdc read blocked", "drop column batch cdc read blocked", - "rename column batch cdc read blocked" + "rename column batch cdc read blocked", + "filters with special characters in name should be pushed down" ) protected def assertBlocked( @@ -618,7 +619,7 @@ trait DeltaCDCColumnMappingSuiteBase extends DeltaCDCSuiteBase } // upgrade to name mode val protocol = deltaLog.snapshot.protocol - val (r, w) = if (protocol.supportsReaderFeatures || protocol.supportsWriterFeatures) { + val (r, w) = if (protocol.supportsTableFeatures) { (TableFeatureProtocolUtils.TABLE_FEATURES_MIN_READER_VERSION, TableFeatureProtocolUtils.TABLE_FEATURES_MIN_WRITER_VERSION) } else { @@ -649,6 +650,28 @@ trait DeltaCDCColumnMappingSuiteBase extends DeltaCDCSuiteBase EndingVersion(deltaLog.update().version.toString)).dropCDCFields, (0 until 10).map(_.toString).toDF("id").withColumn("value", col("id"))) } + + test("filters with special characters in name should be pushed down") { + val tblName = "tbl" + withTable(tblName) { + spark.range(end = 10).withColumn("id with space", col("id")) + .write.format("delta").saveAsTable(tblName) + + val plans = DeltaTestUtils.withAllPlansCaptured(spark) { + val res = cdcRead(new TableName(tblName), StartingVersion("0"), EndingVersion("1")) + .select("id with space", "_change_type") + .where(col("id with space") < lit(5)) + + assert(res.columns === Seq("id with space", "_change_type")) + checkAnswer( + res, + spark.range(end = 5) + .withColumn("_change_type", lit("insert"))) + } + assert(plans.map(_.executedPlan).toString + .contains("PushedFilters: [*IsNotNull(id with space), *LessThan(id with space,5)]")) + } + } } trait DeltaCDCColumnMappingScalaSuiteBase extends DeltaCDCColumnMappingSuiteBase { diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSQLSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSQLSuite.scala index 75ca38b828d..fd1d6c820c4 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSQLSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSQLSuite.scala @@ -289,10 +289,10 @@ class DeltaCDCSQLSuite extends DeltaCDCSuiteBase with DeltaColumnMappingTestUtil withTable(tbl) { spark.range(10).write.format("delta").saveAsTable(tbl) checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"SELECT * FROM table_changes('$tbl', 0, id)") }, - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", parameters = Map("objectName" -> "`id`"), queryContext = Array(ExpectedContext( fragment = "id", diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSuite.scala index 29a95c51b93..710174f5d10 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSuite.scala @@ -25,11 +25,13 @@ import scala.collection.JavaConverters._ // scalastyle:off import.ordering.noEmptyLine import org.apache.spark.sql.delta.DeltaTestUtils.BOOLEAN_DOMAIN import org.apache.spark.sql.delta.commands.cdc.CDCReader._ +import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsBaseSuite import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.DeltaColumnMappingSelectedTestMixin import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import org.apache.spark.sql.delta.test.DeltaTestImplicits._ -import org.apache.spark.sql.delta.util.FileNames +import org.apache.spark.sql.delta.util.{DeltaCommitFileProvider, FileNames} +import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} @@ -1103,3 +1105,25 @@ class DeltaCDCScalaWithDeletionVectorsSuite extends DeltaCDCScalaSuite enableDeletionVectorsForAllSupportedOperations(spark) } } + +class DeltaCDCScalaSuiteWithCoordinatedCommitsBatch10 extends DeltaCDCScalaSuite + with CoordinatedCommitsBaseSuite { + + /** Modify timestamp for a delta commit, used to test timestamp querying */ + override def modifyDeltaTimestamp(deltaLog: DeltaLog, version: Long, time: Long): Unit = { + val fileProvider = DeltaCommitFileProvider(deltaLog.snapshot) + val file = new File(fileProvider.deltaFile(version).toUri) + InCommitTimestampTestUtils.overwriteICTInDeltaFile( + deltaLog, + new Path(file.getPath), + Some(time)) + file.setLastModified(time) + val crc = new File(FileNames.checksumFile(deltaLog.logPath, version).toUri) + if (crc.exists()) { + InCommitTimestampTestUtils.overwriteICTInCrc(deltaLog, version, Some(time)) + crc.setLastModified(time) + } + } + + override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(10) +} 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 6d89818be96..54ba60ba655 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 @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate +import org.apache.spark.sql.delta.DeltaTestUtils.BOOLEAN_DOMAIN import org.apache.spark.sql.delta.actions.{Action, AddCDCFile, AddFile, Metadata => MetadataAction, Protocol, SetTransaction} import org.apache.spark.sql.delta.catalog.DeltaTableV2 import org.apache.spark.sql.delta.schema.SchemaMergingUtils @@ -1944,12 +1945,12 @@ class DeltaColumnMappingSuite extends QueryTest |TBLPROPERTIES('${DeltaConfigs.COLUMN_MAPPING_MODE.key}'='none') |""".stripMargin) } - val errorClass = "DELTA_INVALID_CHARACTERS_IN_COLUMN_NAMES" + val condition = "DELTA_INVALID_CHARACTERS_IN_COLUMN_NAMES" checkError( - exception = e, - errorClass = errorClass, + e, + condition, parameters = DeltaThrowableHelper - .getParameterNames(errorClass, errorSubClass = null) + .getParameterNames(condition, errorSubClass = null) .zip(invalidColumns).toMap ) } @@ -2046,4 +2047,48 @@ class DeltaColumnMappingSuite extends QueryTest } } } + + for (txnIntroducesMetadata <- BOOLEAN_DOMAIN) { + test("column mapping metadata are stripped when feature is disabled - " + + s"txnIntroducesMetadata=$txnIntroducesMetadata") { + withTempDir { dir => + val tablePath = dir.getCanonicalPath + val deltaLog = DeltaLog.forTable(spark, tablePath) + // Create the original table. + val schemaV0 = if (txnIntroducesMetadata) { + new StructType().add("id", LongType, true) + } else { + new StructType().add("id", LongType, true, withIdAndPhysicalName(0, "col-0")) + } + withSQLConf(DeltaSQLConf.DELTA_COLUMN_MAPPING_STRIP_METADATA.key -> "false") { + deltaLog.withNewTransaction(catalogTableOpt = None) { txn => + val metadata = actions.Metadata( + name = "testTable", + schemaString = schemaV0.json, + configuration = Map(DeltaConfigs.COLUMN_MAPPING_MODE.key -> NoMapping.name) + ) + txn.updateMetadata(metadata) + txn.commit(Seq.empty, ManualUpdate) + } + } + val metadataV0 = deltaLog.update().metadata + assert(DeltaColumnMapping.schemaHasColumnMappingMetadata(metadataV0.schema) === + !txnIntroducesMetadata) + + // Update the schema of the existing table. + withSQLConf(DeltaSQLConf.DELTA_COLUMN_MAPPING_STRIP_METADATA.key -> "true") { + deltaLog.withNewTransaction(catalogTableOpt = None) { txn => + val schemaV1 = + schemaV0.add("value", LongType, true, withIdAndPhysicalName(0, "col-0")) + val metadata = metadataV0.copy(schemaString = schemaV1.json) + txn.updateMetadata(metadata) + txn.commit(Seq.empty, ManualUpdate) + } + val metadataV1 = deltaLog.update().metadata + assert(DeltaColumnMapping.schemaHasColumnMappingMetadata(metadataV1.schema) === + !txnIntroducesMetadata) + } + } + } + } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaColumnMappingTestUtils.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaColumnMappingTestUtils.scala index d6ad49e2c03..aced5b0ae1f 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaColumnMappingTestUtils.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaColumnMappingTestUtils.scala @@ -264,7 +264,7 @@ trait DeltaColumnMappingTestUtilsBase extends SharedSparkSession { Protocol.forNewTable(spark, Some(metadata)).minReaderVersion.toString), (Protocol.MIN_WRITER_VERSION_PROP, Protocol.forNewTable(spark, Some(metadata)).minWriterVersion.toString)) - if (snapshot.protocol.supportsReaderFeatures || snapshot.protocol.supportsWriterFeatures) { + if (snapshot.protocol.supportsTableFeatures) { props ++= Protocol.minProtocolComponentsFromAutomaticallyEnabledFeatures( spark, metadata, snapshot.protocol) diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala index 2304da97c16..a4fd227dabf 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.delta.schema.InvariantViolationException import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import org.apache.spark.sql.delta.test.DeltaSQLTestUtils -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{Path, UnsupportedFileSystemException} import org.apache.spark.SparkEnv import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} @@ -41,6 +41,35 @@ class DeltaDDLSuite extends DeltaDDLTestBase with SharedSparkSession exception.getMessage.contains("Cannot change nullable column to non-nullable") } + test("protocol-related properties are not considered during duplicate table creation") { + def createTable(tableName: String, location: String): Unit = { + sql(s""" + |CREATE TABLE $tableName (id INT, val STRING) + |USING DELTA + |LOCATION '$location' + |TBLPROPERTIES ( + | 'delta.columnMapping.mode' = 'name', + | 'delta.minReaderVersion' = '2', + | 'delta.minWriterVersion' = '5' + |)""".stripMargin + ) + } + withTempDir { dir => + val table1 = "t1" + val table2 = "t2" + withTable(table1, table2) { + withSQLConf(DeltaSQLConf.DELTA_UPDATE_CATALOG_ENABLED.key -> "true") { + createTable(table1, dir.getCanonicalPath) + createTable(table2, dir.getCanonicalPath) + val catalogTable1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table1)) + val catalogTable2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table2)) + assert(catalogTable1.properties("delta.columnMapping.mode") == "name") + assert(catalogTable2.properties("delta.columnMapping.mode") == "name") + } + } + } + } + test("table creation with ambiguous paths only allowed with legacy flag") { // ambiguous paths not allowed withTempDir { foo => @@ -84,6 +113,26 @@ class DeltaDDLSuite extends DeltaDDLTestBase with SharedSparkSession assert(spark.table("t").collect().isEmpty) } } + + test("CREATE TABLE with OPTIONS") { + withTempPath { path => + spark.range(10).write.format("delta").save(path.getCanonicalPath) + withTable("t") { + def createTableWithOptions(simulateUC: Boolean): Unit = { + sql( + s""" + |CREATE TABLE t USING delta LOCATION 'fake://${path.getCanonicalPath}' + |${if (simulateUC) "TBLPROPERTIES (test.simulateUC=true)" else ""} + |OPTIONS ( + | fs.fake.impl='${classOf[FakeFileSystem].getName}', + | fs.fake.impl.disable.cache=true) + |""".stripMargin) + } + intercept[UnsupportedFileSystemException](createTableWithOptions(false)) + createTableWithOptions(true) + } + } + } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDataFrameWriterV2Suite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDataFrameWriterV2Suite.scala index ac398604640..9c579f0d524 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDataFrameWriterV2Suite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDataFrameWriterV2Suite.scala @@ -677,8 +677,8 @@ class DeltaDataFrameWriterV2Suite def verifyNotImplicitCasting(f: => Unit): Unit = { val e = intercept[DeltaAnalysisException](f) checkError( - exception = e.getCause.asInstanceOf[DeltaAnalysisException], - errorClass = "DELTA_MERGE_INCOMPATIBLE_DATATYPE", + e.getCause.asInstanceOf[DeltaAnalysisException], + "DELTA_MERGE_INCOMPATIBLE_DATATYPE", parameters = Map("currentDataType" -> "LongType", "updateDataType" -> "IntegerType")) } verifyNotImplicitCasting { diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDropColumnSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDropColumnSuite.scala index 73356c348ae..ece065f4ab0 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDropColumnSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDropColumnSuite.scala @@ -450,10 +450,10 @@ class DeltaDropColumnSuite extends QueryTest field <- Seq("m.key", "m.value", "a.element") } checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"ALTER TABLE delta_test DROP COLUMN $field") }, - errorClass = "DELTA_UNSUPPORTED_DROP_NESTED_COLUMN_FROM_NON_STRUCT_TYPE", + "DELTA_UNSUPPORTED_DROP_NESTED_COLUMN_FROM_NON_STRUCT_TYPE", parameters = Map( "struct" -> "IntegerType" ) 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 b5e8b069f8f..c94b4a884d9 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 @@ -477,12 +477,12 @@ trait DeltaErrorsSuiteBase Some(s"Delta table $table doesn't exist.")) } checkError( - exception = intercept[DeltaIllegalStateException] { + intercept[DeltaIllegalStateException] { throw DeltaErrors.differentDeltaTableReadByStreamingSource( newTableId = "027fb01c-94aa-4cab-87cb-5aab6aec6d17", oldTableId = "2edf2c02-bb63-44e9-a84c-517fad0db296") }, - errorClass = "DIFFERENT_DELTA_TABLE_READ_BY_STREAMING_SOURCE", + "DIFFERENT_DELTA_TABLE_READ_BY_STREAMING_SOURCE", parameters = Map( "oldTableId" -> "2edf2c02-bb63-44e9-a84c-517fad0db296", "newTableId" -> "027fb01c-94aa-4cab-87cb-5aab6aec6d17") @@ -961,12 +961,12 @@ trait DeltaErrorsSuiteBase SchemaMergingUtils.mergeSchemas(s1, s2) } checkError( - exception = e, - errorClass = "DELTA_FAILED_TO_MERGE_FIELDS", + e, + "DELTA_FAILED_TO_MERGE_FIELDS", parameters = Map("currentField" -> "c0", "updateField" -> "c0")) checkError( - exception = e.getCause.asInstanceOf[DeltaAnalysisException], - errorClass = "DELTA_MERGE_INCOMPATIBLE_DATATYPE", + e.getCause.asInstanceOf[DeltaAnalysisException], + "DELTA_MERGE_INCOMPATIBLE_DATATYPE", parameters = Map("currentDataType" -> "IntegerType", "updateDataType" -> "StringType")) } { @@ -997,13 +997,13 @@ trait DeltaErrorsSuiteBase } { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { throw DeltaErrors.alterTableChangeColumnException( fieldPath = "a.b.c", oldField = StructField("c", IntegerType), newField = StructField("c", LongType)) }, - errorClass = "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", + "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", parameters = Map( "fieldPath" -> "a.b.c", "oldField" -> "INT", @@ -1421,14 +1421,14 @@ trait DeltaErrorsSuiteBase } { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { throw DeltaErrors.constraintDataTypeMismatch( columnPath = Seq("a", "x"), columnType = ByteType, dataType = IntegerType, constraints = Map("ck1" -> "a > 0", "ck2" -> "hash(b) > 0")) }, - errorClass = "DELTA_CONSTRAINT_DATA_TYPE_MISMATCH", + "DELTA_CONSTRAINT_DATA_TYPE_MISMATCH", parameters = Map( "columnName" -> "a.x", "columnType" -> "TINYINT", @@ -1438,7 +1438,7 @@ trait DeltaErrorsSuiteBase } { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { throw DeltaErrors.generatedColumnsDataTypeMismatch( columnPath = Seq("a", "x"), columnType = ByteType, @@ -1448,7 +1448,7 @@ trait DeltaErrorsSuiteBase "gen2" -> "3 + a . x" )) }, - errorClass = "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", + "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", parameters = Map( "columnName" -> "a.x", "columnType" -> "TINYINT", @@ -1916,10 +1916,10 @@ trait DeltaErrorsSuiteBase } { checkError( - exception = intercept[DeltaIllegalStateException] { + intercept[DeltaIllegalStateException] { throw MaterializedRowId.missingMetadataException("table_name") }, - errorClass = "DELTA_MATERIALIZED_ROW_TRACKING_COLUMN_NAME_MISSING", + "DELTA_MATERIALIZED_ROW_TRACKING_COLUMN_NAME_MISSING", parameters = Map( "rowTrackingColumn" -> "Row ID", "tableName" -> "table_name" @@ -1928,10 +1928,10 @@ trait DeltaErrorsSuiteBase } { checkError( - exception = intercept[DeltaIllegalStateException] { + intercept[DeltaIllegalStateException] { throw MaterializedRowCommitVersion.missingMetadataException("table_name") }, - errorClass = "DELTA_MATERIALIZED_ROW_TRACKING_COLUMN_NAME_MISSING", + "DELTA_MATERIALIZED_ROW_TRACKING_COLUMN_NAME_MISSING", parameters = Map( "rowTrackingColumn" -> "Row Commit Version", "tableName" -> "table_name" diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoImplicitCastSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoImplicitCastSuite.scala index 93697d2280b..2ab31496221 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoImplicitCastSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoImplicitCastSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.delta import org.apache.spark.sql.delta.sources.DeltaSQLConf - import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -32,6 +32,12 @@ import org.apache.spark.sql.types._ */ class DeltaInsertIntoImplicitCastSuite extends DeltaInsertIntoTest { + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set(DeltaSQLConf.DELTA_STREAMING_SINK_ALLOW_IMPLICIT_CASTS.key, "true") + spark.conf.set(SQLConf.ANSI_ENABLED.key, "true") + } + for (schemaEvolution <- BOOLEAN_DOMAIN) { testInserts("insert with implicit up and down cast on top-level fields, " + s"schemaEvolution=$schemaEvolution")( diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala index b40fa7adf65..18acbc09e0f 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala @@ -213,10 +213,10 @@ class DeltaInsertIntoSQLSuite withTable("t") { sql(s"CREATE TABLE t(i STRING, c string) USING $v2Format PARTITIONED BY (c)") checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql("INSERT OVERWRITE t PARTITION (c='1') (c) VALUES ('2')") }, - errorClass = "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST", + "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST", parameters = Map("staticName" -> "c")) } } @@ -596,22 +596,22 @@ class DeltaColumnDefaultsInsertSuite extends InsertIntoSQLOnlyTests with DeltaSQ // The table feature is not enabled via TBLPROPERTIES. withTable("createTableWithDefaultFeatureNotEnabled") { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql(s"create table createTableWithDefaultFeatureNotEnabled(" + s"i boolean, s bigint, q int default 42) using $v2Format " + "partitioned by (i)") }, - errorClass = "WRONG_COLUMN_DEFAULTS_FOR_DELTA_FEATURE_NOT_ENABLED", + "WRONG_COLUMN_DEFAULTS_FOR_DELTA_FEATURE_NOT_ENABLED", parameters = Map("commandType" -> "CREATE TABLE") ) } withTable("alterTableSetDefaultFeatureNotEnabled") { sql(s"create table alterTableSetDefaultFeatureNotEnabled(a int) using $v2Format") checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql("alter table alterTableSetDefaultFeatureNotEnabled alter column a set default 42") }, - errorClass = "WRONG_COLUMN_DEFAULTS_FOR_DELTA_FEATURE_NOT_ENABLED", + "WRONG_COLUMN_DEFAULTS_FOR_DELTA_FEATURE_NOT_ENABLED", parameters = Map("commandType" -> "ALTER TABLE") ) } @@ -620,19 +620,19 @@ class DeltaColumnDefaultsInsertSuite extends InsertIntoSQLOnlyTests with DeltaSQ sql(s"create table alterTableTest(i boolean, s bigint, q int default 42) using $v2Format " + s"partitioned by (i) $tblPropertiesAllowDefaults") checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql("alter table alterTableTest add column z int default 42") }, - errorClass = "WRONG_COLUMN_DEFAULTS_FOR_DELTA_ALTER_TABLE_ADD_COLUMN_NOT_SUPPORTED" + "WRONG_COLUMN_DEFAULTS_FOR_DELTA_ALTER_TABLE_ADD_COLUMN_NOT_SUPPORTED" ) } // The default value fails to analyze. checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"create table t4 (s int default badvalue) using $v2Format " + s"$tblPropertiesAllowDefaults") }, - errorClass = INVALID_COLUMN_DEFAULT_VALUE_ERROR_MSG, + INVALID_COLUMN_DEFAULT_VALUE_ERROR_MSG, parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`s`", @@ -642,11 +642,11 @@ class DeltaColumnDefaultsInsertSuite extends InsertIntoSQLOnlyTests with DeltaSQ // The error message reports that we failed to execute the command because subquery // expressions are not allowed in DEFAULT values. checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"create table t4 (s int default (select min(x) from badtable)) using $v2Format " + tblPropertiesAllowDefaults) }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`s`", @@ -656,22 +656,22 @@ class DeltaColumnDefaultsInsertSuite extends InsertIntoSQLOnlyTests with DeltaSQ // The error message reports that we failed to execute the command because subquery // expressions are not allowed in DEFAULT values. checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"create table t4 (s int default (select 42 as alias)) using $v2Format " + tblPropertiesAllowDefaults) }, - errorClass = "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`s`", "defaultValue" -> "(select 42 as alias)")) // The default value parses but the type is not coercible. checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"create table t4 (s bigint default false) " + s"using $v2Format $tblPropertiesAllowDefaults") }, - errorClass = "INVALID_DEFAULT_VALUE.DATA_TYPE", + "INVALID_DEFAULT_VALUE.DATA_TYPE", parameters = Map( "statement" -> "CREATE TABLE", "colName" -> "`s`", @@ -702,11 +702,11 @@ class DeltaColumnDefaultsInsertSuite extends InsertIntoSQLOnlyTests with DeltaSQ // Column default values are disabled per configuration in general. withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") { checkError( - exception = intercept[ParseException] { + intercept[ParseException] { sql(s"create table t4 (s int default 41 + 1) using $v2Format " + tblPropertiesAllowDefaults) }, - errorClass = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION", + "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION", parameters = Map.empty, context = ExpectedContext(fragment = "s int default 41 + 1", start = 17, stop = 36)) } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala index b0e34c49ad9..84aebb36023 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala @@ -214,39 +214,38 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest } test("upgrade to support table features - many features") { - withTempDir { path => - val log = createTableWithProtocol(Protocol(2, 5), path) - assert(log.update().protocol === Protocol(2, 5)) - val table = io.delta.tables.DeltaTable.forPath(spark, path.getCanonicalPath) - table.upgradeTableProtocol(2, TABLE_FEATURES_MIN_WRITER_VERSION) - // Setting table feature versions to a protocol without table features is a noop. - assert(log.update().protocol === Protocol(2, 5)) - spark.sql( - s"ALTER TABLE delta.`${path.getPath}` SET TBLPROPERTIES (" + - s" delta.feature.${TestWriterFeature.name}='enabled'" + - s")") - table.upgradeTableProtocol( - TABLE_FEATURES_MIN_READER_VERSION, - TABLE_FEATURES_MIN_WRITER_VERSION) - assert( - log.snapshot.protocol === Protocol( - minReaderVersion = 2, - minWriterVersion = TABLE_FEATURES_MIN_WRITER_VERSION, - readerFeatures = None, - writerFeatures = Some( - Set( - AppendOnlyTableFeature, - ChangeDataFeedTableFeature, - CheckConstraintsTableFeature, - ColumnMappingTableFeature, - GeneratedColumnsTableFeature, - InvariantsTableFeature, - TestLegacyWriterFeature, - TestRemovableLegacyWriterFeature, - TestLegacyReaderWriterFeature, - TestRemovableLegacyReaderWriterFeature, - TestWriterFeature) - .map(_.name)))) + withSQLConf(DeltaSQLConf.TABLE_FEATURES_TEST_FEATURES_ENABLED.key -> false.toString) { + withTempDir { path => + val log = createTableWithProtocol(Protocol(2, 5), path) + assert(log.update().protocol === Protocol(2, 5)) + val table = io.delta.tables.DeltaTable.forPath(spark, path.getCanonicalPath) + table.upgradeTableProtocol(2, TABLE_FEATURES_MIN_WRITER_VERSION) + // Setting table feature versions to a protocol without table features is a noop. + assert(log.update().protocol === Protocol(2, 5)) + spark.sql( + s"ALTER TABLE delta.`${path.getPath}` SET TBLPROPERTIES (" + + s" delta.feature.${RowTrackingFeature.name}='enabled'" + + s")") + table.upgradeTableProtocol( + TABLE_FEATURES_MIN_READER_VERSION, + TABLE_FEATURES_MIN_WRITER_VERSION) + assert( + log.snapshot.protocol === Protocol( + minReaderVersion = 2, + minWriterVersion = TABLE_FEATURES_MIN_WRITER_VERSION, + readerFeatures = Some(Set(ColumnMappingTableFeature.name)), + writerFeatures = Some( + Set( + AppendOnlyTableFeature, + InvariantsTableFeature, + ChangeDataFeedTableFeature, + CheckConstraintsTableFeature, + ColumnMappingTableFeature, + GeneratedColumnsTableFeature, + DomainMetadataTableFeature, + RowTrackingFeature) + .map(_.name)))) + } } } @@ -1313,7 +1312,7 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest deltaLog.snapshot.protocol === Protocol( minReaderVersion = 2, minWriterVersion = TABLE_FEATURES_MIN_WRITER_VERSION, - readerFeatures = None, + readerFeatures = Some(Set.empty), writerFeatures = Some(Set(TestLegacyReaderWriterFeature.name)))) assertPropertiesAndShowTblProperties(deltaLog, tableHasFeatures = true) } @@ -2136,7 +2135,7 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest assert(log.snapshot.protocol === Protocol( 2, TABLE_FEATURES_MIN_WRITER_VERSION, - readerFeatures = None, + readerFeatures = Some(Set.empty), writerFeatures = Some(Set(TestLegacyReaderWriterFeature.name)))) } } @@ -2377,6 +2376,27 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest } } + test("Column mapping appears in reader features") { + withTempDir { dir => + val deltaLog = DeltaLog.forTable(spark, dir) + sql( + s"""CREATE TABLE delta.`${deltaLog.dataPath}` (id bigint) USING delta + |TBLPROPERTIES ( + |delta.feature.${ColumnMappingTableFeature.name} = 'supported', + |delta.feature.${TestWriterFeature.name} = 'supported' + |)""".stripMargin) + assert(deltaLog.update().protocol === Protocol( + minReaderVersion = 2, + minWriterVersion = 7, + readerFeatures = Some(Set(ColumnMappingTableFeature.name)), + writerFeatures = Some(Set( + InvariantsTableFeature.name, + AppendOnlyTableFeature.name, + ColumnMappingTableFeature.name, + TestWriterFeature.name)))) + } + } + def protocolWithFeatures( readerFeatures: Seq[TableFeature] = Seq.empty, writerFeatures: Seq[TableFeature] = Seq.empty): Protocol = { @@ -2434,8 +2454,17 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest val readerVersion = Math.max(feature.minReaderVersion, 1) val expectedWriterFeatures = Some(Set(feature.name, InvariantsTableFeature.name, AppendOnlyTableFeature.name)) + val supportsColumnMapping = + canSupportColumnMappingFeature(readerVersion, TABLE_FEATURES_MIN_WRITER_VERSION) val expectedReaderFeatures: Option[Set[String]] = - if (supportsReaderFeatures(readerVersion)) Some(Set(feature.name)) else None + if ((feature == ColumnMappingTableFeature && supportsColumnMapping) || + supportsReaderFeatures(readerVersion)) { + Some(Set(feature.name)) + } else if (supportsColumnMapping) { + Some(Set.empty) + } else { + None + } assert( deltaLog.update().protocol === Protocol( @@ -2502,8 +2531,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest feature.name).run(spark) } checkError( - exception = e1, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + e1, + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> feature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -2521,8 +2550,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest feature.name).run(spark) } checkError( - exception = e2, - errorClass = "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", + e2, + "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", parameters = Map( "feature" -> feature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -2563,8 +2592,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest dropCommand.run(spark) } checkError( - exception = e3, - errorClass = "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", + e3, + "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", parameters = Map( "feature" -> feature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -2678,8 +2707,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest command.run(spark) } checkError( - exception = e, - errorClass = "DELTA_FEATURE_DROP_NONREMOVABLE_FEATURE", + e, + "DELTA_FEATURE_DROP_NONREMOVABLE_FEATURE", parameters = Map("feature" -> TestWriterMetadataNoAutoUpdateFeature.name)) } } @@ -2701,8 +2730,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest command.run(spark) } checkError( - exception = e, - errorClass = "DELTA_FEATURE_DROP_NONREMOVABLE_FEATURE", + e, + "DELTA_FEATURE_DROP_NONREMOVABLE_FEATURE", parameters = Map("feature" -> AppendOnlyTableFeature.name)) } } @@ -2727,8 +2756,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest command.run(spark) } checkError( - exception = e, - errorClass = "DELTA_FEATURE_DROP_UNSUPPORTED_CLIENT_FEATURE", + e, + "DELTA_FEATURE_DROP_UNSUPPORTED_CLIENT_FEATURE", parameters = Map("feature" -> "NonSupportedFeature")) } } @@ -2753,8 +2782,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest command.run(spark) } checkError( - exception = e, - errorClass = "DELTA_FEATURE_DROP_FEATURE_NOT_PRESENT", + e, + "DELTA_FEATURE_DROP_FEATURE_NOT_PRESENT", parameters = Map("feature" -> TestRemovableWriterFeature.name)) } } @@ -2834,8 +2863,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest TestRemovableWriterFeature.name).run(spark) } checkError( - exception = e1, - errorClass = "DELTA_FEATURE_DROP_DEPENDENT_FEATURE", + e1, + "DELTA_FEATURE_DROP_DEPENDENT_FEATURE", parameters = Map( "feature" -> TestRemovableWriterFeature.name, "dependentFeatures" -> TestRemovableWriterFeatureWithDependency.name)) @@ -2875,8 +2904,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest |TRUNCATE HISTORY""".stripMargin) } checkError( - exception = e, - errorClass = "DELTA_FEATURE_DROP_HISTORY_TRUNCATION_NOT_ALLOWED", + e, + "DELTA_FEATURE_DROP_HISTORY_TRUNCATION_NOT_ALLOWED", parameters = Map.empty) } } @@ -2903,8 +2932,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest TestRemovableReaderWriterFeature.name).run(spark) } checkError( - exception = e1, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + e1, + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> TestRemovableReaderWriterFeature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -2934,8 +2963,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest } checkError( - exception = e2, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + e2, + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> TestRemovableReaderWriterFeature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -2962,8 +2991,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest TestRemovableReaderWriterFeature.name).run(spark) } checkError( - exception = e1, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + e1, + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> TestRemovableReaderWriterFeature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -2982,8 +3011,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest TestRemovableReaderWriterFeature.name).run(spark) } checkError( - exception = e2, - errorClass = "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", + e2, + "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", parameters = Map( "feature" -> TestRemovableReaderWriterFeature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -3034,8 +3063,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest TestRemovableReaderWriterFeature.name).run(spark) } checkError( - exception = e1, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + e1, + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> TestRemovableReaderWriterFeature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -3057,8 +3086,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest TestRemovableReaderWriterFeature.name).run(spark) } checkError( - exception = e2, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + e2, + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> TestRemovableReaderWriterFeature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -3085,8 +3114,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest TestRemovableReaderWriterFeature.name).run(spark) } checkError( - exception = e1, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + e1, + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> TestRemovableReaderWriterFeature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -3502,8 +3531,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest TestRemovableWriterWithHistoryTruncationFeature.name).run(spark) } checkError( - exception = e1, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + e1, + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> TestRemovableWriterWithHistoryTruncationFeature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -3582,8 +3611,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest dropV2CheckpointsTableFeature(spark, targetLog) } checkError( - exception = e1, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + e1, + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> V2CheckpointTableFeature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -3616,8 +3645,8 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest dropV2CheckpointsTableFeature(spark, targetLog) } checkError( - exception = e2, - errorClass = "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", + e2, + "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", parameters = Map( "feature" -> V2CheckpointTableFeature.name, "logRetentionPeriodKey" -> "delta.logRetentionDuration", diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSinkImplicitCastSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSinkImplicitCastSuite.scala index 27966ef23b2..0d2008e52ce 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSinkImplicitCastSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSinkImplicitCastSuite.scala @@ -141,8 +141,8 @@ class DeltaSinkImplicitCastSuite extends DeltaSinkImplicitCastSuiteBase { stream.write(Long.MaxValue)("CAST(value AS LONG)") } checkError( - exception = ex.getCause.asInstanceOf[SparkThrowable], - errorClass = "CANNOT_UP_CAST_DATATYPE", + ex.getCause.asInstanceOf[SparkThrowable], + "CANNOT_UP_CAST_DATATYPE", parameters = Map( "expression" -> "value", "sourceType" -> toSQLType("BIGINT"), @@ -173,8 +173,8 @@ class DeltaSinkImplicitCastSuite extends DeltaSinkImplicitCastSuiteBase { case e => fail(s"Unexpected exception: $e") } checkError( - exception = getSparkArithmeticException(ex), - errorClass = "CAST_OVERFLOW_IN_TABLE_INSERT", + getSparkArithmeticException(ex), + "CAST_OVERFLOW_IN_TABLE_INSERT", parameters = Map( "sourceType" -> "\"BIGINT\"", "targetType" -> "\"INT\"", @@ -276,8 +276,8 @@ class DeltaSinkImplicitCastSuite extends DeltaSinkImplicitCastSuiteBase { stream.write(-12)("array(value) AS s") } checkError( - exception = ex.getCause.asInstanceOf[SparkThrowable], - errorClass = "DELTA_FAILED_TO_MERGE_FIELDS", + ex.getCause.asInstanceOf[SparkThrowable], + "DELTA_FAILED_TO_MERGE_FIELDS", parameters = Map( "currentField" -> "s", "updateField" -> "s") @@ -473,8 +473,8 @@ class DeltaSinkImplicitCastSuite extends DeltaSinkImplicitCastSuiteBase { stream.write(23)("CAST(value AS LONG)") } checkError( - exception = ex.getCause.asInstanceOf[SparkThrowable], - errorClass = "DELTA_FAILED_TO_MERGE_FIELDS", + ex.getCause.asInstanceOf[SparkThrowable], + "DELTA_FAILED_TO_MERGE_FIELDS", parameters = Map( "currentField" -> "value", "updateField" -> "value") diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSinkSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSinkSuite.scala index e643782fcb9..1578c88c120 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSinkSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSinkSuite.scala @@ -407,8 +407,8 @@ class DeltaSinkSuite .save(outputDir.getCanonicalPath) } checkError( - exception = e, - errorClass = "DELTA_FAILED_TO_MERGE_FIELDS", + e, + "DELTA_FAILED_TO_MERGE_FIELDS", parameters = Map("currentField" -> "id", "updateField" -> "id")) } finally { query.stop() @@ -442,8 +442,8 @@ class DeltaSinkSuite } assert(wrapperException.cause.isInstanceOf[AnalysisException]) checkError( - exception = wrapperException.cause.asInstanceOf[AnalysisException], - errorClass = "DELTA_FAILED_TO_MERGE_FIELDS", + wrapperException.cause.asInstanceOf[AnalysisException], + "DELTA_FAILED_TO_MERGE_FIELDS", parameters = Map("currentField" -> "id", "updateField" -> "id")) } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSourceColumnMappingSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSourceColumnMappingSuite.scala index cae6b29f4c2..36c142ace29 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSourceColumnMappingSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSourceColumnMappingSuite.scala @@ -303,7 +303,7 @@ trait ColumnMappingStreamingBlockedWorkflowSuiteBase extends ColumnMappingStream // upgrade to name mode val protocol = deltaLog.snapshot.protocol - val (r, w) = if (protocol.supportsReaderFeatures || protocol.supportsWriterFeatures) { + val (r, w) = if (protocol.supportsTableFeatures) { (TableFeatureProtocolUtils.TABLE_FEATURES_MIN_READER_VERSION, TableFeatureProtocolUtils.TABLE_FEATURES_MIN_WRITER_VERSION) } else { diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala index dc013656bb1..d87b2eebd3c 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala @@ -1305,8 +1305,8 @@ class DeltaSuite extends QueryTest .save(tempDir.toString) } checkError( - exception = e, - errorClass = "DELTA_FAILED_TO_MERGE_FIELDS", + e, + "DELTA_FAILED_TO_MERGE_FIELDS", parameters = Map("currentField" -> "value", "updateField" -> "value")) } } @@ -3000,7 +3000,7 @@ class DeltaNameColumnMappingSuite extends DeltaSuite .save(tempDir.getCanonicalPath) val protocol = DeltaLog.forTable(spark, tempDir).snapshot.protocol - val (r, w) = if (protocol.supportsReaderFeatures || protocol.supportsWriterFeatures) { + val (r, w) = if (protocol.supportsTableFeatures) { (TableFeatureProtocolUtils.TABLE_FEATURES_MIN_READER_VERSION, TableFeatureProtocolUtils.TABLE_FEATURES_MIN_WRITER_VERSION) } else { 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 70fdd0471a5..5b5db01d387 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 @@ -2391,8 +2391,8 @@ class DeltaTableCreationSuite s" LOCATION '${subdir.getCanonicalPath}'") } checkError( - exception = e, - errorClass = "DELTA_METADATA_ABSENT_EXISTING_CATALOG_TABLE", + e, + "DELTA_METADATA_ABSENT_EXISTING_CATALOG_TABLE", parameters = Map( "tableName" -> tableName, "tablePath" -> deltaLog.logPath.toString, diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableFeatureSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableFeatureSuite.scala index 5dcb76e0d17..38f1e275ad3 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableFeatureSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableFeatureSuite.scala @@ -197,9 +197,8 @@ class DeltaTableFeatureSuite val protocol = Protocol(2, TABLE_FEATURES_MIN_WRITER_VERSION).withFeature(TestLegacyReaderWriterFeature) - assert(!protocol.readerFeatures.isDefined) - assert( - protocol.writerFeatures.get === Set(TestLegacyReaderWriterFeature.name)) + assert(protocol.readerFeatures.get === Set.empty) + assert(protocol.writerFeatures.get === Set(TestLegacyReaderWriterFeature.name)) } test("merge protocols") { diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala index 3ab857369b5..6c2bfcb9f9a 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala @@ -140,7 +140,7 @@ class DeltaTimeTravelSuite extends QueryTest start - 2.seconds, // adjusts to start + 4 ms start + 10.seconds) - val commits = DeltaHistoryManager.getCommits( + val commits = DeltaHistoryManager.getCommitsWithNonIctTimestamps( deltaLog.store, deltaLog.logPath, 0, diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala index 917a04ef5f7..20fdd92d18a 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala @@ -424,7 +424,7 @@ trait DescribeDeltaHistorySuiteBase Seq("UPGRADE PROTOCOL", s"""{"minReaderVersion":$readerVersion,""" + s""""minWriterVersion":$writerVersion,""" + - s""""readerFeatures":["${TestLegacyReaderWriterFeature.name}"],""" + + s""""readerFeatures":[],""" + s""""writerFeatures":["${TestLegacyReaderWriterFeature.name}"]}"""), Seq($"operation", $"operationParameters.newProtocol")) // scalastyle:on line.size.limit diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala index 2ff8439d546..218b51b677f 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala @@ -719,12 +719,12 @@ trait GeneratedColumnSuiteBase assert(tableSchema == spark.table(table).schema) // Insert a LONG to `c1` should fail rather than changing the `c1` type to LONG. checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { Seq(32767.toLong).toDF("c1").write.format("delta").mode("append") .option("mergeSchema", "true") .saveAsTable(table) }, - errorClass = "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", + "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", parameters = Map( "columnName" -> "c1", "columnType" -> "INT", @@ -754,14 +754,14 @@ trait GeneratedColumnSuiteBase // Insert an INT to `a` should fail rather than changing the `a` type to INT checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { Seq((32767, 32767)).toDF("a", "c1a") .selectExpr("a", "named_struct('a', c1a) as c1") .write.format("delta").mode("append") .option("mergeSchema", "true") .saveAsTable(table) }, - errorClass = "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", + "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", parameters = Map( "columnName" -> "a", "columnType" -> "SMALLINT", diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/IdentityColumnConflictSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/IdentityColumnConflictSuite.scala index 0caff6363b3..e56b57aa21b 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/IdentityColumnConflictSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/IdentityColumnConflictSuite.scala @@ -162,7 +162,7 @@ trait IdentityColumnConflictSuiteBase if (currentTxn.isInstanceOf[RowTrackingEnablementOnlyTestCase]) { txnObserver.setNextObserver(metadataUpdateObserver, autoAdvance = true) unblockAllPhases(txnObserver) - txnObserver.phases.backfillPhase.exitBarrier.unblock() + txnObserver.phases.postCommitPhase.exitBarrier.unblock() txnObserver = metadataUpdateObserver } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/InCommitTimestampSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/InCommitTimestampSuite.scala index 7da3d15f6d5..4e567eb2214 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/InCommitTimestampSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/InCommitTimestampSuite.scala @@ -20,7 +20,7 @@ import java.nio.charset.StandardCharsets.UTF_8 import java.sql.Timestamp import scala.collection.JavaConverters._ -import scala.concurrent.duration.Duration +import scala.concurrent.duration._ import com.databricks.spark.util.{Log4jUsageLogger, UsageRecord} import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate @@ -34,6 +34,8 @@ import org.apache.spark.sql.delta.util.{DeltaCommitFileProvider, FileNames, Json import org.apache.hadoop.fs.Path import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.{ManualClock, SerializableConfiguration, ThreadUtils} @@ -42,7 +44,8 @@ class InCommitTimestampSuite with SharedSparkSession with DeltaSQLCommandTest with DeltaTestUtilsBase - with CoordinatedCommitsTestUtils { + with CoordinatedCommitsTestUtils + with StreamTest { override def beforeAll(): Unit = { super.beforeAll() @@ -58,6 +61,15 @@ class InCommitTimestampSuite commitInfo.get.inCommitTimestamp.get } + private def getFileModificationTimesMap( + deltaLog: DeltaLog, start: Long, end: Long): Map[Long, Long] = { + deltaLog.store.listFrom( + FileNames.listingPrefix(deltaLog.logPath, start), deltaLog.newDeltaHadoopConf()) + .collect { case FileNames.DeltaFile(fs, v) => v -> fs.getModificationTime } + .takeWhile(_._1 <= end) + .toMap + } + test("Enable ICT on commit 0") { withTempDir { tempDir => spark.range(10).write.format("delta").save(tempDir.getAbsolutePath) @@ -205,8 +217,8 @@ class InCommitTimestampSuite latestSnapshot.timestamp } checkError( - exception = e, - errorClass = "DELTA_MISSING_COMMIT_INFO", + e, + "DELTA_MISSING_COMMIT_INFO", parameters = Map( "featureName" -> InCommitTimestampTableFeature.name, "version" -> "1")) @@ -244,8 +256,8 @@ class InCommitTimestampSuite latestSnapshot.timestamp } checkError( - exception = e, - errorClass = "DELTA_MISSING_COMMIT_TIMESTAMP", + e, + "DELTA_MISSING_COMMIT_TIMESTAMP", parameters = Map("featureName" -> InCommitTimestampTableFeature.name, "version" -> "1")) } } @@ -262,24 +274,26 @@ class InCommitTimestampSuite } test("CREATE OR REPLACE should not disable ICT") { - withSQLConf( - DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey -> false.toString - ) { - withTempDir { tempDir => - spark.range(10).write.format("delta").save(tempDir.getAbsolutePath) - spark.sql( - s"ALTER TABLE delta.`${tempDir.getAbsolutePath}` " + - s"SET TBLPROPERTIES ('${DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.key}' = 'true')") - - spark.sql( - s"CREATE OR REPLACE TABLE delta.`${tempDir.getAbsolutePath}` (id long) USING delta") - - val deltaLogAfterCreateOrReplace = - DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) - val snapshot = deltaLogAfterCreateOrReplace.snapshot - assert(DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.fromMetaData(snapshot.metadata)) - assert(snapshot.timestamp == - getInCommitTimestamp(deltaLogAfterCreateOrReplace, snapshot.version)) + withoutCoordinatedCommitsDefaultTableProperties { + withSQLConf( + DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey -> false.toString + ) { + withTempDir { tempDir => + spark.range(10).write.format("delta").save(tempDir.getAbsolutePath) + spark.sql( + s"ALTER TABLE delta.`${tempDir.getAbsolutePath}` " + + s"SET TBLPROPERTIES ('${DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.key}' = 'true')") + + spark.sql( + s"CREATE OR REPLACE TABLE delta.`${tempDir.getAbsolutePath}` (id long) USING delta") + + val deltaLogAfterCreateOrReplace = + DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) + val snapshot = deltaLogAfterCreateOrReplace.snapshot + assert(DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.fromMetaData(snapshot.metadata)) + assert(snapshot.timestamp == + getInCommitTimestamp(deltaLogAfterCreateOrReplace, snapshot.version)) + } } } } @@ -988,6 +1002,98 @@ class InCommitTimestampSuite assert(historySubset.head.timestamp.getTime == getInCommitTimestamp(deltaLog, 2)) } } + + for (ictEnablementVersion <- Seq(1, 4, 7)) + testWithDefaultCommitCoordinatorUnset(s"CDC read with all commits being ICT " + + s"[ictEnablementVersion = $ictEnablementVersion]") { + withSQLConf( + DeltaConfigs.CHANGE_DATA_FEED.defaultTablePropertyKey -> "true", + DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey -> "false" + ) { + withTempDir { tempDir => + val path = tempDir.getCanonicalPath + for (i <- 0 to 7) { + if (i == ictEnablementVersion) { + spark.sql( + s"ALTER TABLE delta.`$path` " + + s"SET TBLPROPERTIES ('${DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.key}' = 'true')") + } else { + spark.range(i, i + 1).write.format("delta").mode("append").save(path) + } + } + val deltaLog = DeltaLog.forTable(spark, new Path(path)) + val result = spark.read + .format("delta") + .option("startingVersion", "1") + .option("endingVersion", "7") + .option("readChangeFeed", "true") + .load(path) + .select("_commit_timestamp", "_commit_version") + .collect() + val fileTimestampsMap = getFileModificationTimesMap(deltaLog, 0, 7) + result.foreach { row => + val v = row.getAs[Long]("_commit_version") + val expectedTimestamp = if (v >= ictEnablementVersion) { + getInCommitTimestamp(deltaLog, v) + } else { + fileTimestampsMap(v) + } + assert(row.getAs[Timestamp]("_commit_timestamp").getTime == expectedTimestamp) + } + } + } + } + + for (ictEnablementVersion <- Seq(1, 4, 7)) + testWithDefaultCommitCoordinatorUnset(s"Streaming query + CDC " + + s"[ictEnablementVersion = $ictEnablementVersion]") { + withSQLConf( + DeltaConfigs.CHANGE_DATA_FEED.defaultTablePropertyKey -> "true", + DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey -> "false" + ) { + withTempDir { tempDir => withTempDir { checkpointDir => withTempDir { streamingSink => + val path = tempDir.getCanonicalPath + spark.range(0).write.format("delta").mode("append").save(path) + + val sourceDeltaLog = DeltaLog.forTable(spark, new Path(path)) + val sinkPath = streamingSink.getCanonicalPath + val streamingQuery = spark.readStream + .format("delta") + .option("readChangeFeed", "true") + .load(path) + .select( + col("_commit_timestamp").alias("source_commit_timestamp"), + col("_commit_version").alias("source_commit_version")) + .writeStream + .format("delta") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .start(sinkPath) + for (i <- 1 to 7) { + if (i == ictEnablementVersion) { + spark.sql(s"ALTER TABLE delta.`$path` " + + s"SET TBLPROPERTIES ('${DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.key}' = 'true')") + } else { + spark.range(i, i + 1).write.format("delta").mode("append").save(path) + } + } + streamingQuery.processAllAvailable() + val fileTimestampsMap = getFileModificationTimesMap(sourceDeltaLog, 0, 7) + val result = spark.read.format("delta") + .load(sinkPath) + .collect() + result.foreach { row => + val v = row.getAs[Long]("source_commit_version") + val expectedTimestamp = if (v >= ictEnablementVersion) { + getInCommitTimestamp(sourceDeltaLog, v) + } else { + fileTimestampsMap(v) + } + assert( + row.getAs[Timestamp]("source_commit_timestamp").getTime == expectedTimestamp) + } + }}} + } + } } class InCommitTimestampWithCoordinatedCommitsSuite diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/S3SingleDriverLogStoreSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/S3SingleDriverLogStoreSuite.scala index 7c869bab726..ba65bd9b886 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/S3SingleDriverLogStoreSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/S3SingleDriverLogStoreSuite.scala @@ -66,86 +66,6 @@ trait S3SingleDriverLogStoreSuiteBase extends LogStoreSuiteBase { } } - test("cache works") { - withTempDir { dir => - val store = createLogStore(spark) - val deltas = - Seq(0, 1, 2, 3, 4).map(i => FileNames.unsafeDeltaFile(new Path(dir.toURI), i)) - store.write(deltas(0), Iterator("zero"), overwrite = false, sessionHadoopConf) - store.write(deltas(1), Iterator("one"), overwrite = false, sessionHadoopConf) - store.write(deltas(2), Iterator("two"), overwrite = false, sessionHadoopConf) - - // delete delta file 2 from file system - val fs = new Path(dir.getCanonicalPath).getFileSystem(sessionHadoopConf) - fs.delete(deltas(2), true) - - // file system listing doesn't see file 2 - checkFileSystemList(fs, deltas(0), Seq(0, 1)) - - // can't re-write because cache says it still exists - intercept[java.nio.file.FileAlreadyExistsException] { - store.write(deltas(2), Iterator("two"), overwrite = false, sessionHadoopConf) - } - - // log store list still sees file 2 as it's cached - checkLogStoreList(store, deltas(0), Seq(0, 1, 2), sessionHadoopConf) - - if (canInvalidateCache) { - // clear the cache - store.invalidateCache() - - // log store list doesn't see file 2 anymore - checkLogStoreList(store, deltas(0), Seq(0, 1), sessionHadoopConf) - - // write a new file 2 - store.write(deltas(2), Iterator("two"), overwrite = false, sessionHadoopConf) - } - - // add a file 3 to cache only - store.write(deltas(3), Iterator("three"), overwrite = false, sessionHadoopConf) - fs.delete(deltas(3), true) - - // log store listing returns a union of: - // 1) file system listing: 0, 1, 2 - // 2a) cache listing - canInvalidateCache=true: 2, 3 - // 2b) cache listing - canInvalidateCache=false: 0, 1, 2, 3 - checkLogStoreList(store, deltas(0), Seq(0, 1, 2, 3), sessionHadoopConf) - } - } - - test("cache works correctly when writing an initial log version") { - withTempDir { rootDir => - val dir = new File(rootDir, "_delta_log") - dir.mkdir() - val store = createLogStore(spark) - val deltas = - Seq(0, 1, 2).map(i => FileNames.unsafeDeltaFile(new Path(dir.toURI), i)) - store.write(deltas(0), Iterator("log version 0"), overwrite = false, sessionHadoopConf) - store.write(deltas(1), Iterator("log version 1"), overwrite = false, sessionHadoopConf) - store.write(deltas(2), Iterator("log version 2"), overwrite = false, sessionHadoopConf) - - val fs = new Path(dir.getCanonicalPath).getFileSystem(sessionHadoopConf) - // delete all log files - fs.delete(deltas(2), true) - fs.delete(deltas(1), true) - fs.delete(deltas(0), true) - - // can't write a new version 1 as it's in cache - intercept[java.nio.file.FileAlreadyExistsException] { - store.write(deltas(1), Iterator("new log version 1"), overwrite = false, sessionHadoopConf) - } - - // all three log files still in cache - checkLogStoreList(store, deltas(0), Seq(0, 1, 2), sessionHadoopConf) - - // can write a new version 0 as it's the initial version of the log - store.write(deltas(0), Iterator("new log version 0"), overwrite = false, sessionHadoopConf) - - // writing a new initial version invalidates all files in that log - checkLogStoreList(store, deltas(0), Seq(0), sessionHadoopConf) - } - } - protected def shouldUseRenameToWriteCheckpoint: Boolean = false /** diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/SchemaValidationSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/SchemaValidationSuite.scala index 0e9e1c47515..70924195f35 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/SchemaValidationSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/SchemaValidationSuite.scala @@ -348,8 +348,8 @@ class SchemaValidationSuite } checkErrorMatchPVals( - exception = e, - errorClass = "DELTA_SCHEMA_CHANGE_SINCE_ANALYSIS", + e, + "DELTA_SCHEMA_CHANGE_SINCE_ANALYSIS", parameters = Map( "schemaDiff" -> ".*id.*", "legacyFlagMessage" -> "" @@ -383,8 +383,8 @@ class SchemaValidationSuite .execute() } checkErrorMatchPVals( - exception = e, - errorClass = "DELTA_SCHEMA_CHANGE_SINCE_ANALYSIS", + e, + "DELTA_SCHEMA_CHANGE_SINCE_ANALYSIS", parameters = Map( "schemaDiff" -> ".*col2.*", "legacyFlagMessage" -> "" diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/UpdateSQLSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/UpdateSQLSuite.scala index fcfd59cca8b..fa046d00cbc 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/UpdateSQLSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/UpdateSQLSuite.scala @@ -117,10 +117,10 @@ class UpdateSQLSuite extends UpdateSuiteBase SQLConf.STORE_ASSIGNMENT_POLICY.key -> StoreAssignmentPolicy.STRICT.toString, DeltaSQLConf.UPDATE_AND_MERGE_CASTING_FOLLOWS_ANSI_ENABLED_FLAG.key -> "false") { checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { executeUpdate(target = s"delta.`$tempPath`", set = "value = 'false'") }, - errorClass = "CANNOT_UP_CAST_DATATYPE", + "CANNOT_UP_CAST_DATATYPE", parameters = Map( "expression" -> "'false'", "sourceType" -> toSQLType("STRING"), @@ -139,11 +139,11 @@ class UpdateSQLSuite extends UpdateSuiteBase SQLConf.STORE_ASSIGNMENT_POLICY.key -> StoreAssignmentPolicy.STRICT.toString, DeltaSQLConf.UPDATE_AND_MERGE_CASTING_FOLLOWS_ANSI_ENABLED_FLAG.key -> "false") { checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { executeUpdate(target = s"delta.`$tempPath`", set = "value = '5'") }, - errorClass = "CANNOT_UP_CAST_DATATYPE", - parameters = Map( + "CANNOT_UP_CAST_DATATYPE", + parameters = Map( "expression" -> "'5'", "sourceType" -> toSQLType("STRING"), "targetType" -> toSQLType("INT"), diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/columnmapping/DropColumnMappingFeatureSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/columnmapping/DropColumnMappingFeatureSuite.scala index 0e9ac064806..be5abc7f376 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/columnmapping/DropColumnMappingFeatureSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/columnmapping/DropColumnMappingFeatureSuite.scala @@ -58,8 +58,7 @@ class DropColumnMappingFeatureSuite extends RemoveColumnMappingSuiteUtils { dropColumnMappingTableFeature() } checkError(e, - errorClass = DeltaErrors.dropTableFeatureFeatureNotSupportedByProtocol(".") - .getErrorClass, + DeltaErrors.dropTableFeatureFeatureNotSupportedByProtocol(".").getErrorClass, parameters = Map("feature" -> "columnMapping")) } @@ -76,7 +75,7 @@ class DropColumnMappingFeatureSuite extends RemoveColumnMappingSuiteUtils { dropColumnMappingTableFeature() } checkError(e, - errorClass = "DELTA_INVALID_COLUMN_NAMES_WHEN_REMOVING_COLUMN_MAPPING", + "DELTA_INVALID_COLUMN_NAMES_WHEN_REMOVING_COLUMN_MAPPING", parameters = Map("invalidColumnNames" -> "col1 with special chars ,;{}()\n\t=")) } @@ -125,7 +124,7 @@ class DropColumnMappingFeatureSuite extends RemoveColumnMappingSuiteUtils { } checkError( e, - errorClass = "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", + "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", parameters = Map( "feature" -> "columnMapping", "logRetentionPeriodKey" -> "delta.logRetentionDuration", @@ -168,7 +167,7 @@ class DropColumnMappingFeatureSuite extends RemoveColumnMappingSuiteUtils { } checkError( e, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> "columnMapping", "logRetentionPeriodKey" -> "delta.logRetentionDuration", diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/commands/backfill/RowTrackingBackfillConflictsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/commands/backfill/RowTrackingBackfillConflictsSuite.scala index 11aaf8e213f..fa991595c50 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/commands/backfill/RowTrackingBackfillConflictsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/commands/backfill/RowTrackingBackfillConflictsSuite.scala @@ -418,7 +418,7 @@ class RowTrackingBackfillConflictsSuite extends RowTrackingBackfillConflictsTest backfillObserver.setNextObserver( NoOpTransactionExecutionObserver, autoAdvance = true) unblockCommit(backfillObserver) - backfillObserver.phases.backfillPhase.exitBarrier.unblock() + backfillObserver.phases.postCommitPhase.exitBarrier.unblock() waitForCommit(backfillObserver) } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/concurrency/TransactionExecutionObserverSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/concurrency/TransactionExecutionObserverSuite.scala index ce7c80a48e5..f6c8070ce73 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/concurrency/TransactionExecutionObserverSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/concurrency/TransactionExecutionObserverSuite.scala @@ -61,11 +61,13 @@ class TransactionExecutionObserverSuite extends QueryTest with SharedSparkSessio assert(!observer.phases.preparePhase.hasEntered) assert(!observer.phases.commitPhase.hasEntered) assert(!observer.phases.backfillPhase.hasEntered) + assert(!observer.phases.postCommitPhase.hasEntered) // allow things to progress observer.phases.preparePhase.entryBarrier.unblock() observer.phases.commitPhase.entryBarrier.unblock() observer.phases.backfillPhase.entryBarrier.unblock() + observer.phases.postCommitPhase.entryBarrier.unblock() val removedFiles = txn.snapshot.allFiles.collect().map(_.remove).toSeq txn.commit(removedFiles, DeltaOperations.ManualUpdate) @@ -75,6 +77,8 @@ class TransactionExecutionObserverSuite extends QueryTest with SharedSparkSessio assert(observer.phases.commitPhase.hasLeft) assert(observer.phases.backfillPhase.hasEntered) assert(observer.phases.backfillPhase.hasLeft) + assert(observer.phases.postCommitPhase.hasEntered) + assert(observer.phases.postCommitPhase.hasLeft) } } val res = spark.read.format("delta").load(tempPath).collect() @@ -126,6 +130,10 @@ class TransactionExecutionObserverSuite extends QueryTest with SharedSparkSessio observer.phases.backfillPhase.entryBarrier.unblock() busyWaitFor(observer.phases.backfillPhase.hasEntered, timeout) busyWaitFor(observer.phases.backfillPhase.hasLeft, timeout) + + observer.phases.postCommitPhase.entryBarrier.unblock() + busyWaitFor(observer.phases.postCommitPhase.hasEntered, timeout) + busyWaitFor(observer.phases.postCommitPhase.hasLeft, timeout) testThread.join(timeout.toMillis) assert(!testThread.isAlive) // should have passed the barrier and completed @@ -155,6 +163,7 @@ class TransactionExecutionObserverSuite extends QueryTest with SharedSparkSessio observer.phases.preparePhase.entryBarrier.unblock() observer.phases.commitPhase.entryBarrier.unblock() observer.phases.backfillPhase.entryBarrier.unblock() + observer.phases.postCommitPhase.entryBarrier.unblock() val removedFiles = txn.snapshot.allFiles.collect().map(_.remove).toSeq txn.commit(removedFiles, DeltaOperations.ManualUpdate) } @@ -165,6 +174,7 @@ class TransactionExecutionObserverSuite extends QueryTest with SharedSparkSessio observer.phases.preparePhase.entryBarrier.unblock() observer.phases.commitPhase.entryBarrier.unblock() observer.phases.backfillPhase.entryBarrier.unblock() + observer.phases.postCommitPhase.entryBarrier.unblock() val removedFiles = txn.snapshot.allFiles.collect().map(_.remove).toSeq txn.commit(removedFiles, DeltaOperations.ManualUpdate) } @@ -221,6 +231,7 @@ class TransactionExecutionObserverSuite extends QueryTest with SharedSparkSessio busyWaitFor(observer.phases.preparePhase.hasLeft, timeout) assert(!observer.phases.commitPhase.hasEntered) assert(!observer.phases.backfillPhase.hasEntered) + assert(!observer.phases.postCommitPhase.hasEntered) assertOperationNotVisible() @@ -228,6 +239,8 @@ class TransactionExecutionObserverSuite extends QueryTest with SharedSparkSessio busyWaitFor(observer.phases.commitPhase.hasLeft, timeout) observer.phases.backfillPhase.entryBarrier.unblock() busyWaitFor(observer.phases.backfillPhase.hasLeft, timeout) + observer.phases.postCommitPhase.entryBarrier.unblock() + busyWaitFor(observer.phases.postCommitPhase.hasLeft, timeout) testThread.join(timeout.toMillis) assert(!testThread.isAlive) // should have passed the barrier and completed @@ -256,7 +269,7 @@ class TransactionExecutionObserverSuite extends QueryTest with SharedSparkSessio TransactionExecutionObserver.withObserver(observer) { deltaLog.withNewTransaction { txn => - observer.phases.backfillPhase.exitBarrier.unblock() + observer.phases.postCommitPhase.exitBarrier.unblock() val removedFiles = txn.snapshot.allFiles.collect().map(_.remove).toSeq txn.commit(removedFiles, DeltaOperations.ManualUpdate) } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/concurrency/TransactionExecutionTestMixin.scala b/spark/src/test/scala/org/apache/spark/sql/delta/concurrency/TransactionExecutionTestMixin.scala index 8833d7464e8..059314aa072 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/concurrency/TransactionExecutionTestMixin.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/concurrency/TransactionExecutionTestMixin.scala @@ -124,6 +124,7 @@ trait TransactionExecutionTestMixin { def unblockCommit(observer: TransactionObserver): Unit = { observer.phases.commitPhase.entryBarrier.unblock() observer.phases.backfillPhase.entryBarrier.unblock() + observer.phases.postCommitPhase.entryBarrier.unblock() } /** Unblocks all phases for [[TransactionObserver]] so that corresponding query can finish. */ @@ -132,6 +133,7 @@ trait TransactionExecutionTestMixin { observer.phases.preparePhase.entryBarrier.unblock() observer.phases.commitPhase.entryBarrier.unblock() observer.phases.backfillPhase.entryBarrier.unblock() + observer.phases.postCommitPhase.entryBarrier.unblock() } def waitForPrecommit(observer: TransactionObserver): Unit = @@ -140,6 +142,7 @@ trait TransactionExecutionTestMixin { def waitForCommit(observer: TransactionObserver): Unit = { busyWaitFor(observer.phases.commitPhase.hasLeft, timeout) busyWaitFor(observer.phases.backfillPhase.hasLeft, timeout) + busyWaitFor(observer.phases.postCommitPhase.hasLeft, timeout) } /** @@ -153,7 +156,7 @@ trait TransactionExecutionTestMixin { waitForPrecommit(observer) unblockCommit(observer) if (hasNextObserver) { - observer.phases.backfillPhase.leave() + observer.phases.postCommitPhase.leave() } waitForCommit(observer) } @@ -222,13 +225,15 @@ trait TransactionExecutionTestMixin { // B starts and commits unblockAllPhases(observerB) - busyWaitFor(observerB.phases.backfillPhase.hasLeft, timeout) + busyWaitFor(observerB.phases.postCommitPhase.hasLeft, timeout) // A commits observerA.phases.commitPhase.entryBarrier.unblock() busyWaitFor(observerA.phases.commitPhase.hasLeft, timeout) observerA.phases.backfillPhase.entryBarrier.unblock() busyWaitFor(observerA.phases.backfillPhase.hasLeft, timeout) + observerA.phases.postCommitPhase.entryBarrier.unblock() + busyWaitFor(observerA.phases.postCommitPhase.hasLeft, timeout) } (futureA, futureB) } @@ -260,17 +265,19 @@ trait TransactionExecutionTestMixin { // B starts and commits unblockAllPhases(observerB) - busyWaitFor(observerB.phases.backfillPhase.hasLeft, timeout) + busyWaitFor(observerB.phases.postCommitPhase.hasLeft, timeout) // C starts and commits unblockAllPhases(observerC) - busyWaitFor(observerC.phases.backfillPhase.hasLeft, timeout) + busyWaitFor(observerC.phases.postCommitPhase.hasLeft, timeout) // A commits observerA.phases.commitPhase.entryBarrier.unblock() busyWaitFor(observerA.phases.commitPhase.hasLeft, timeout) observerA.phases.backfillPhase.entryBarrier.unblock() busyWaitFor(observerA.phases.backfillPhase.hasLeft, timeout) + observerA.phases.postCommitPhase.entryBarrier.unblock() + busyWaitFor(observerA.phases.postCommitPhase.hasLeft, timeout) } (futureA, futureB, futureC) } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsSuite.scala index 5c0c93ae088..ae2643d2050 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsSuite.scala @@ -1322,7 +1322,7 @@ class CoordinatedCommitsSuite tableMutationFn() } checkError(e, - errorClass = "DELTA_UNSUPPORTED_WRITES_WITHOUT_COORDINATOR", + "DELTA_UNSUPPORTED_WRITES_WITHOUT_COORDINATOR", sqlState = "0AKDC", parameters = Map("coordinatorName" -> "tracking-in-memory") ) @@ -1576,17 +1576,17 @@ class CoordinatedCommitsSuite CommitCoordinatorProvider.registerBuilder(InMemoryCommitCoordinatorBuilder(1)) withTempDir { tempDir => - sql(s"CREATE TABLE delta.`${tempDir.getAbsolutePath}` (id LONG) USING delta TBLPROPERTIES" + + sql(s"CREATE TABLE delta.`${tempDir.getAbsolutePath}` (id LONG) USING delta TBLPROPERTIES " + s"('${COORDINATED_COMMITS_COORDINATOR_NAME.key}' = 'tracking-in-memory', " + s"'${COORDINATED_COMMITS_COORDINATOR_CONF.key}' = '${JsonUtils.toJson(Map())}')") val e = interceptWithUnwrapping[DeltaIllegalArgumentException] { - sql(s"ALTER TABLE delta.`${tempDir.getAbsolutePath}` SET TBLPROPERTIES" + + sql(s"ALTER TABLE delta.`${tempDir.getAbsolutePath}` SET TBLPROPERTIES " + s"('${COORDINATED_COMMITS_COORDINATOR_NAME.key}' = 'in-memory', " + s"'${COORDINATED_COMMITS_COORDINATOR_CONF.key}' = '${JsonUtils.toJson(Map())}')") } checkError( - exception = e, - errorClass = "DELTA_CANNOT_OVERRIDE_COORDINATED_COMMITS_CONFS", + e, + "DELTA_CANNOT_OVERRIDE_COORDINATED_COMMITS_CONFS", sqlState = "42616", parameters = Map("Command" -> "ALTER")) } @@ -1594,23 +1594,83 @@ class CoordinatedCommitsSuite test("During ALTER, unsetting Coordinated Commits configurations throws an exception.") { CommitCoordinatorProvider.registerBuilder(TrackingInMemoryCommitCoordinatorBuilder(1)) + withTempDir { tempDir => - sql(s"CREATE TABLE delta.`${tempDir.getAbsolutePath}` (id LONG) USING delta TBLPROPERTIES" + + sql(s"CREATE TABLE delta.`${tempDir.getAbsolutePath}` (id LONG) USING delta TBLPROPERTIES " + s"('${COORDINATED_COMMITS_COORDINATOR_NAME.key}' = 'tracking-in-memory', " + s"'${COORDINATED_COMMITS_COORDINATOR_CONF.key}' = '${JsonUtils.toJson(Map())}')") val e = interceptWithUnwrapping[DeltaIllegalArgumentException] { - sql(s"ALTER TABLE delta.`${tempDir.getAbsolutePath}` UNSET TBLPROPERTIES" + + sql(s"ALTER TABLE delta.`${tempDir.getAbsolutePath}` UNSET TBLPROPERTIES " + s"('${COORDINATED_COMMITS_COORDINATOR_NAME.key}', " + s"'${COORDINATED_COMMITS_COORDINATOR_CONF.key}')") } checkError( - exception = e, - errorClass = "DELTA_CANNOT_UNSET_COORDINATED_COMMITS_CONFS", + e, + "DELTA_CANNOT_UNSET_COORDINATED_COMMITS_CONFS", sqlState = "42616", parameters = Map[String, String]()) } } + test("During ALTER, overriding ICT configurations on (potential) Coordinated Commits tables " + + "throws an exception.") { + CommitCoordinatorProvider.registerBuilder(TrackingInMemoryCommitCoordinatorBuilder(1)) + + // For a table that had Coordinated Commits enabled before the ALTER command. + withTempDir { tempDir => + sql(s"CREATE TABLE delta.`${tempDir.getAbsolutePath}` (id LONG) USING delta TBLPROPERTIES " + + s"('${COORDINATED_COMMITS_COORDINATOR_NAME.key}' = 'tracking-in-memory', " + + s"'${COORDINATED_COMMITS_COORDINATOR_CONF.key}' = '${JsonUtils.toJson(Map())}')") + val e = interceptWithUnwrapping[DeltaIllegalArgumentException] { + sql(s"ALTER TABLE delta.`${tempDir.getAbsolutePath}` SET TBLPROPERTIES " + + s"('${IN_COMMIT_TIMESTAMPS_ENABLED.key}' = 'false')") + } + checkError( + e, + "DELTA_CANNOT_MODIFY_COORDINATED_COMMITS_DEPENDENCIES", + sqlState = "42616", + parameters = Map("Command" -> "ALTER")) + } + + // For a table that is about to enable Coordinated Commits during the same ALTER command. + withoutCoordinatedCommitsDefaultTableProperties { + withTempDir { tempDir => + sql(s"CREATE TABLE delta.`${tempDir.getAbsolutePath}` (id LONG) USING delta") + val e = interceptWithUnwrapping[DeltaIllegalArgumentException] { + sql(s"ALTER TABLE delta.`${tempDir.getAbsolutePath}` SET TBLPROPERTIES " + + s"('${COORDINATED_COMMITS_COORDINATOR_NAME.key}' = 'tracking-in-memory', " + + s"'${COORDINATED_COMMITS_COORDINATOR_CONF.key}' = '${JsonUtils.toJson(Map())}', " + + s"'${IN_COMMIT_TIMESTAMPS_ENABLED.key}' = 'false')") + } + checkError( + e, + "DELTA_CANNOT_SET_COORDINATED_COMMITS_DEPENDENCIES", + sqlState = "42616", + parameters = Map("Command" -> "ALTER")) + } + } + } + + test("During ALTER, unsetting ICT configurations on Coordinated Commits tables throws an " + + "exception.") { + CommitCoordinatorProvider.registerBuilder(TrackingInMemoryCommitCoordinatorBuilder(1)) + + withTempDir { tempDir => + sql(s"CREATE TABLE delta.`${tempDir.getAbsolutePath}` (id LONG) USING delta TBLPROPERTIES " + + s"('${COORDINATED_COMMITS_COORDINATOR_NAME.key}' = 'tracking-in-memory', " + + s"'${COORDINATED_COMMITS_COORDINATOR_CONF.key}' = '${JsonUtils.toJson(Map())}')") + val e = interceptWithUnwrapping[DeltaIllegalArgumentException] { + sql(s"ALTER TABLE delta.`${tempDir.getAbsolutePath}` UNSET TBLPROPERTIES " + + s"('${IN_COMMIT_TIMESTAMPS_ENABLED.key}')") + } + checkError( + e, + "DELTA_CANNOT_MODIFY_COORDINATED_COMMITS_DEPENDENCIES", + sqlState = "42616", + parameters = Map("Command" -> "ALTER")) + } + } + test("During REPLACE, for non-CC tables, default CC configurations are ignored, but default " + "ICT confs are retained, and existing ICT confs are discarded") { // Non-CC table, REPLACE with default CC and ICT confs => Non-CC, but with ICT confs. diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtilsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtilsSuite.scala index 081e2ee7c95..158fe6d5e6d 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtilsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtilsSuite.scala @@ -85,8 +85,8 @@ class CoordinatedCommitsUtilsSuite extends QueryTest spark, propertyOverrides, tableExists, command) } checkError( - exception = e, - errorClass = errorOpt.get.getErrorClass, + e, + errorOpt.get.getErrorClass, sqlState = errorOpt.get.getSqlState, parameters = errorOpt.get.getMessageParameters.asScala.toMap) } else { @@ -260,8 +260,8 @@ class CoordinatedCommitsUtilsSuite extends QueryTest existingConfs, propertyOverrides) } checkError( - exception = e, - errorClass = errorOpt.get.getErrorClass, + e, + errorOpt.get.getErrorClass, sqlState = errorOpt.get.getSqlState, parameters = errorOpt.get.getMessageParameters.asScala.toMap) } else { @@ -328,8 +328,8 @@ class CoordinatedCommitsUtilsSuite extends QueryTest existingConfs, propKeysToUnset) } checkError( - exception = e, - errorClass = errorOpt.get.getErrorClass, + e, + errorOpt.get.getErrorClass, sqlState = errorOpt.get.getSqlState, parameters = errorOpt.get.getMessageParameters.asScala.toMap) } else { diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/rowid/RowTrackingBackfillSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/rowid/RowTrackingBackfillSuite.scala index 8eaaeef98a5..1a2a76e9cdb 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/rowid/RowTrackingBackfillSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/rowid/RowTrackingBackfillSuite.scala @@ -460,7 +460,8 @@ class RowTrackingBackfillSuite assert( afterProtocol.minWriterVersion === TableFeatureProtocolUtils.TABLE_FEATURES_MIN_WRITER_VERSION) - assert(afterProtocol.readerFeatures === None) + assert(afterProtocol.readerFeatures === Some(Set( + ColumnMappingTableFeature.name))) assert( afterProtocol.writerFeatures === Some(( prevProtocol.implicitlyAndExplicitlySupportedFeatures ++ diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/rowtracking/MaterializedColumnSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/rowtracking/MaterializedColumnSuite.scala index 551dd9f3702..6b4802f2c05 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/rowtracking/MaterializedColumnSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/rowtracking/MaterializedColumnSuite.scala @@ -89,7 +89,7 @@ class MaterializedColumnSuite extends RowIdTestUtils sql(s"ALTER TABLE $testTableName " + s"RENAME COLUMN $testDataColumnName TO `$materializedColumnName`") } - checkError(error, errorClass = "DELTA_ADDING_COLUMN_WITH_INTERNAL_NAME_FAILED", + checkError(error, "DELTA_ADDING_COLUMN_WITH_INTERNAL_NAME_FAILED", parameters = Map("colName" -> materializedColumnName)) } } @@ -111,7 +111,7 @@ class MaterializedColumnSuite extends RowIdTestUtils val error = intercept[DeltaRuntimeException] { sql(s"CREATE OR REPLACE TABLE $targetName SHALLOW CLONE $sourceName") } - checkError(error, errorClass = "DELTA_ADDING_COLUMN_WITH_INTERNAL_NAME_FAILED", + checkError(error, "DELTA_ADDING_COLUMN_WITH_INTERNAL_NAME_FAILED", parameters = Map("colName" -> materializedColumnName)) } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/rowtracking/RowTrackingReadWriteSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/rowtracking/RowTrackingReadWriteSuite.scala index 100a5cfa281..80fc7556251 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/rowtracking/RowTrackingReadWriteSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/rowtracking/RowTrackingReadWriteSuite.scala @@ -220,7 +220,7 @@ class RowTrackingReadWriteSuite extends RowIdTestUtils val errorRowIds = intercept[AnalysisException](sql(insertStmt1 + " VALUES(1, 2)")) checkError( errorRowIds, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = errorRowIds.messageParameters, queryContext = Array(ExpectedContext(insertStmt1, 0, insertStmt1.length - 1))) @@ -229,7 +229,7 @@ class RowTrackingReadWriteSuite extends RowIdTestUtils val errorRowCommitVersions = intercept[AnalysisException](sql(insertStmt2 + " VALUES(1, 2)")) checkError( errorRowCommitVersions, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = errorRowCommitVersions.messageParameters, queryContext = Array(ExpectedContext(insertStmt2, 0, insertStmt2.length - 1))) } @@ -285,7 +285,7 @@ class RowTrackingReadWriteSuite extends RowIdTestUtils } checkError( error, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = error.messageParameters) } @@ -303,7 +303,7 @@ class RowTrackingReadWriteSuite extends RowIdTestUtils } checkError( error, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = error.messageParameters) } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/schema/CheckConstraintsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/schema/CheckConstraintsSuite.scala index 7f14b19e3b2..f86fe164e8a 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/schema/CheckConstraintsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/schema/CheckConstraintsSuite.scala @@ -80,7 +80,7 @@ class CheckConstraintsSuite extends QueryTest exception = intercept[AnalysisException] { sql(s"ALTER TABLE $table ADD CONSTRAINT integerVal CHECK (3)") }, - errorClass = "DELTA_NON_BOOLEAN_CHECK_CONSTRAINT", + "DELTA_NON_BOOLEAN_CHECK_CONSTRAINT", parameters = Map( "name" -> "integerVal", "expr" -> "3" @@ -92,10 +92,10 @@ class CheckConstraintsSuite extends QueryTest test("can't add constraint referencing non-existent columns") { withTestTable { table => checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"ALTER TABLE $table ADD CONSTRAINT c CHECK (does_not_exist)") }, - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( "objectName" -> "`does_not_exist`", "proposal" -> "`text`, `num`" @@ -451,7 +451,7 @@ class CheckConstraintsSuite extends QueryTest } checkError( exception, - errorClass = "DELTA_EXCEED_CHAR_VARCHAR_LIMIT", + "DELTA_EXCEED_CHAR_VARCHAR_LIMIT", parameters = Map( "value" -> "a very long string", "expr" -> "((value IS NULL) OR (length(value) <= 12))" @@ -474,7 +474,7 @@ class CheckConstraintsSuite extends QueryTest } checkError( error1, - errorClass = "DELTA_CANNOT_DROP_CHECK_CONSTRAINT_FEATURE", + "DELTA_CANNOT_DROP_CHECK_CONSTRAINT_FEATURE", parameters = Map("constraints" -> "`c1`, `c2`") ) val deltaLog = DeltaLog.forTable(spark, TableIdentifier("table")) @@ -488,7 +488,7 @@ class CheckConstraintsSuite extends QueryTest } checkError( error2, - errorClass = "DELTA_CANNOT_DROP_CHECK_CONSTRAINT_FEATURE", + "DELTA_CANNOT_DROP_CHECK_CONSTRAINT_FEATURE", parameters = Map("constraints" -> "`c2`") ) val featureNames2 = 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 8dfb54c11f8..a8fbfd51ff8 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 @@ -87,8 +87,8 @@ class SchemaUtilsSuite extends QueryTest val err = getError(e) assert(err.isDefined, "exception with the error class not found") checkError( - exception = err.get, - errorClass = errorClass, + err.get, + errorClass, parameters = params, matchPVals = true) } @@ -1258,6 +1258,35 @@ class SchemaUtilsSuite extends QueryTest } } + test("addColumn - top level array") { + val a = StructField("a", IntegerType) + val b = StructField("b", StringType) + val schema = ArrayType(new StructType().add(a).add(b)) + + val x = StructField("x", LongType) + assert(SchemaUtils.addColumn(schema, x, Seq(0, 1)) === + ArrayType(new StructType().add(a).add(x).add(b))) + } + + test("addColumn - top level map") { + val k = StructField("k", IntegerType) + val v = StructField("v", StringType) + val schema = MapType( + keyType = new StructType().add(k), + valueType = new StructType().add(v)) + + val x = StructField("x", LongType) + assert(SchemaUtils.addColumn(schema, x, Seq(0, 1)) === + MapType( + keyType = new StructType().add(k).add(x), + valueType = new StructType().add(v))) + + assert(SchemaUtils.addColumn(schema, x, Seq(1, 1)) === + MapType( + keyType = new StructType().add(k), + valueType = new StructType().add(v).add(x))) + } + //////////////////////////// // dropColumn //////////////////////////// @@ -1511,6 +1540,29 @@ class SchemaUtilsSuite extends QueryTest } } + test("dropColumn - top level array") { + val schema = ArrayType(new StructType().add("a", IntegerType).add("b", StringType)) + + assert(SchemaUtils.dropColumn(schema, Seq(0, 0))._1 === + ArrayType(new StructType().add("b", StringType))) + } + + test("dropColumn - top level map") { + val schema = MapType( + keyType = new StructType().add("k", IntegerType).add("k2", StringType), + valueType = new StructType().add("v", StringType).add("v2", StringType)) + + assert(SchemaUtils.dropColumn(schema, Seq(0, 0))._1 === + MapType( + keyType = new StructType().add("k2", StringType), + valueType = new StructType().add("v", StringType).add("v2", StringType))) + + assert(SchemaUtils.dropColumn(schema, Seq(1, 0))._1 === + MapType( + keyType = new StructType().add("k", IntegerType).add("k2", StringType), + valueType = new StructType().add("v2", StringType))) + } + ///////////////////////////////// // normalizeColumnNamesInDataType ///////////////////////////////// @@ -1680,8 +1732,8 @@ class SchemaUtilsSuite extends QueryTest Seq("x", "Y"), new StructType()) } checkError( - exception = exception, - errorClass = "DELTA_CANNOT_RESOLVE_COLUMN", + exception, + "DELTA_CANNOT_RESOLVE_COLUMN", sqlState = "42703", parameters = Map("columnName" -> "x.Y.bb", "schema" -> "root\n") ) @@ -1948,8 +2000,8 @@ class SchemaUtilsSuite extends QueryTest ) } checkError( - exception = exception, - errorClass = "DELTA_CANNOT_RESOLVE_COLUMN", + exception, + "DELTA_CANNOT_RESOLVE_COLUMN", sqlState = "42703", parameters = Map("columnName" -> "two", "schema" -> tableSchema.treeString) ) @@ -1974,8 +2026,8 @@ class SchemaUtilsSuite extends QueryTest ) } checkError( - exception = exception, - errorClass = "DELTA_CANNOT_RESOLVE_COLUMN", + exception, + "DELTA_CANNOT_RESOLVE_COLUMN", sqlState = "42703", parameters = Map("columnName" -> "s.two", "schema" -> tableSchema.treeString) ) @@ -2348,8 +2400,8 @@ class SchemaUtilsSuite extends QueryTest mergeSchemas(longType, sourceType) } checkError( - exception = e.getCause.asInstanceOf[AnalysisException], - errorClass = "DELTA_MERGE_INCOMPATIBLE_DATATYPE", + e.getCause.asInstanceOf[AnalysisException], + "DELTA_MERGE_INCOMPATIBLE_DATATYPE", parameters = Map("currentDataType" -> "LongType", "updateDataType" -> sourceType.head.dataType.toString)) } @@ -2584,6 +2636,45 @@ class SchemaUtilsSuite extends QueryTest assert(update === res3) } + test("transform top level array type") { + val at = ArrayType( + new StructType() + .add("s1", IntegerType) + ) + + var visitedFields = 0 + val updated = SchemaMergingUtils.transformColumns(at) { + case (_, field, _) => + visitedFields += 1 + field.copy(name = "s1_1", dataType = StringType) + } + + assert(visitedFields === 1) + assert(updated === ArrayType(new StructType().add("s1_1", StringType))) + } + + test("transform top level map type") { + val mt = MapType( + new StructType() + .add("k1", IntegerType), + new StructType() + .add("v1", IntegerType) + ) + + var visitedFields = 0 + val updated = SchemaMergingUtils.transformColumns(mt) { + case (_, field, _) => + visitedFields += 1 + field.copy(name = field.name + "_1", dataType = StringType) + } + + assert(visitedFields === 2) + assert(updated === MapType( + new StructType().add("k1_1", StringType), + new StructType().add("v1_1", StringType) + )) + } + //////////////////////////// // pruneEmptyStructs //////////////////////////// @@ -2637,10 +2728,10 @@ class SchemaUtilsSuite extends QueryTest badCharacters.foreach { char => Seq(s"a${char}b", s"${char}ab", s"ab${char}", char.toString).foreach { name => checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { SchemaUtils.checkFieldNames(Seq(name)) }, - errorClass = "DELTA_INVALID_CHARACTERS_IN_COLUMN_NAME", + "DELTA_INVALID_CHARACTERS_IN_COLUMN_NAME", parameters = Map("columnName" -> s"$name") ) } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableDDLSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableDDLSuite.scala index b522aa16e58..a2fe9f5bd8c 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableDDLSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableDDLSuite.scala @@ -267,8 +267,8 @@ trait ClusteredTableCreateOrReplaceDDLSuiteBase extends QueryTest assert(dataTypeOpt.nonEmpty, s"Can't find column $colName " + s"in schema ${tableSchema.treeString}") checkError( - exception = e, - errorClass = "DELTA_CLUSTERING_COLUMNS_DATATYPE_NOT_SUPPORTED", + e, + "DELTA_CLUSTERING_COLUMNS_DATATYPE_NOT_SUPPORTED", parameters = Map("columnsWithDataTypes" -> s"$colName : ${dataTypeOpt.get.sql}") ) } @@ -287,8 +287,8 @@ trait ClusteredTableCreateOrReplaceDDLSuiteBase extends QueryTest "CREATE", testTable, "a INT, b INT, c INT, d INT, e INT", "a, b, c, d, e") } checkError( - exception = e, - errorClass = "DELTA_CLUSTER_BY_INVALID_NUM_COLUMNS", + e, + "DELTA_CLUSTER_BY_INVALID_NUM_COLUMNS", parameters = Map("numColumnsLimit" -> "4", "actualNumColumns" -> "5") ) } @@ -305,8 +305,8 @@ trait ClusteredTableCreateOrReplaceDDLSuiteBase extends QueryTest "CREATE", testTable, sourceTable, "a, b, c, d, e", location = location) } checkError( - exception = e, - errorClass = "DELTA_CLUSTER_BY_INVALID_NUM_COLUMNS", + e, + "DELTA_CLUSTER_BY_INVALID_NUM_COLUMNS", parameters = Map("numColumnsLimit" -> "4", "actualNumColumns" -> "5") ) } @@ -354,8 +354,8 @@ trait ClusteredTableCreateOrReplaceDDLSuiteBase extends QueryTest indexedColumns, Some(tableSchema))) checkError( - exception = e, - errorClass = "DELTA_CLUSTERING_COLUMN_MISSING_STATS", + e, + "DELTA_CLUSTERING_COLUMN_MISSING_STATS", parameters = Map( "columns" -> "col1.col12, col2", "schema" -> """root @@ -411,8 +411,8 @@ trait ClusteredTableCreateOrReplaceDDLSuiteBase extends QueryTest None, location = Some(dir.getPath))) checkError( - exception = e, - errorClass = "DELTA_CLUSTERING_COLUMN_MISSING_STATS", + e, + "DELTA_CLUSTERING_COLUMN_MISSING_STATS", parameters = Map( "columns" -> "col1.col12, col2", "schema" -> """root @@ -456,8 +456,8 @@ trait ClusteredTableCreateOrReplaceDDLSuiteBase extends QueryTest indexedColumns, Some(nonEligibleTableSchema))) checkError( - exception = e, - errorClass = "DELTA_CLUSTERING_COLUMNS_DATATYPE_NOT_SUPPORTED", + e, + "DELTA_CLUSTERING_COLUMNS_DATATYPE_NOT_SUPPORTED", parameters = Map("columnsWithDataTypes" -> "col1.col11 : ARRAY") ) } @@ -553,8 +553,8 @@ trait ClusteredTableDDLWithColumnMapping sql(s"ALTER TABLE $testTable DROP COLUMNS (col1)") } checkError( - exception = e, - errorClass = "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN", + e, + "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN", parameters = Map("columnList" -> "col1") ) // Drop non-clustering columns are allowed. @@ -568,8 +568,8 @@ trait ClusteredTableDDLWithColumnMapping sql(s"ALTER TABLE $testTable DROP COLUMNS (col1, col2)") } checkError( - exception = e, - errorClass = "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN", + e, + "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN", parameters = Map("columnList" -> "col1,col2") ) } @@ -582,8 +582,8 @@ trait ClusteredTableDDLWithColumnMapping sql(s"ALTER TABLE $testTable DROP COLUMNS (col1, col3)") } checkError( - exception = e, - errorClass = "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN", + e, + "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN", parameters = Map("columnList" -> "col1") ) } @@ -659,7 +659,7 @@ trait ClusteredTableDDLSuiteBase } checkError( e, - errorClass = "DELTA_CLUSTER_BY_INVALID_NUM_COLUMNS", + "DELTA_CLUSTER_BY_INVALID_NUM_COLUMNS", parameters = Map( "numColumnsLimit" -> "4", "actualNumColumns" -> "5") @@ -782,8 +782,8 @@ trait ClusteredTableDDLSuiteBase sql(s"OPTIMIZE $testTable ZORDER BY (a)") } checkError( - exception = e2, - errorClass = "DELTA_CLUSTERING_WITH_ZORDER_BY", + e2, + "DELTA_CLUSTERING_WITH_ZORDER_BY", parameters = Map("zOrderBy" -> "a") ) } @@ -911,7 +911,7 @@ trait ClusteredTableDDLSuiteBase } checkError( e, - errorClass = "DELTA_CANNOT_MODIFY_TABLE_PROPERTY", + "DELTA_CANNOT_MODIFY_TABLE_PROPERTY", parameters = Map("prop" -> "clusteringColumns")) } } @@ -1169,7 +1169,7 @@ trait ClusteredTableDDLDataSourceV2SuiteBase } checkError( e, - errorClass = "DELTA_CREATE_TABLE_WITH_DIFFERENT_CLUSTERING", + "DELTA_CREATE_TABLE_WITH_DIFFERENT_CLUSTERING", parameters = Map( "path" -> dir.toURI.toString.stripSuffix("/"), "specifiedColumns" -> "", @@ -1194,7 +1194,7 @@ trait ClusteredTableDDLDataSourceV2SuiteBase } checkError( e, - errorClass = "DELTA_CREATE_TABLE_WITH_DIFFERENT_CLUSTERING", + "DELTA_CREATE_TABLE_WITH_DIFFERENT_CLUSTERING", parameters = Map( "path" -> dir.toURI.toString.stripSuffix("/"), "specifiedColumns" -> "col2", @@ -1235,7 +1235,7 @@ trait ClusteredTableDDLDataSourceV2SuiteBase } checkError( e, - errorClass = "DELTA_CREATE_TABLE_WITH_DIFFERENT_CLUSTERING", + "DELTA_CREATE_TABLE_WITH_DIFFERENT_CLUSTERING", parameters = Map( "path" -> dir.toURI.toString.stripSuffix("/"), "specifiedColumns" -> "col1", 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 15c0594c8b4..476bfecbdec 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 @@ -57,8 +57,8 @@ trait TypeWideningAlterTableNestedTests { // Running ALTER TABLE CHANGE COLUMN on non-leaf fields is invalid. var alterTableSql = s"ALTER TABLE delta.`$tempPath` CHANGE COLUMN s TYPE struct" checkError( - exception = intercept[AnalysisException] { sql(alterTableSql) }, - errorClass = "CANNOT_UPDATE_FIELD.STRUCT_TYPE", + intercept[AnalysisException] { sql(alterTableSql) }, + "CANNOT_UPDATE_FIELD.STRUCT_TYPE", parameters = Map( "table" -> s"`spark_catalog`.`delta`.`$tempPath`", "fieldName" -> "`s`" @@ -71,8 +71,8 @@ trait TypeWideningAlterTableNestedTests { alterTableSql = s"ALTER TABLE delta.`$tempPath` CHANGE COLUMN m TYPE map" checkError( - exception = intercept[AnalysisException] { sql(alterTableSql) }, - errorClass = "CANNOT_UPDATE_FIELD.MAP_TYPE", + intercept[AnalysisException] { sql(alterTableSql) }, + "CANNOT_UPDATE_FIELD.MAP_TYPE", parameters = Map( "table" -> s"`spark_catalog`.`delta`.`$tempPath`", "fieldName" -> "`m`" @@ -85,8 +85,8 @@ trait TypeWideningAlterTableNestedTests { alterTableSql = s"ALTER TABLE delta.`$tempPath` CHANGE COLUMN a TYPE array" checkError( - exception = intercept[AnalysisException] { sql(alterTableSql) }, - errorClass = "CANNOT_UPDATE_FIELD.ARRAY_TYPE", + intercept[AnalysisException] { sql(alterTableSql) }, + "CANNOT_UPDATE_FIELD.ARRAY_TYPE", parameters = Map( "table" -> s"`spark_catalog`.`delta`.`$tempPath`", "fieldName" -> "`a`" 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 70f19ed4154..f772c9ca2aa 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 @@ -99,10 +99,10 @@ trait TypeWideningAlterTableTests // are rejected in Delta when the ALTER TABLE command is executed. if (Cast.canUpCast(testCase.fromType, testCase.toType)) { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql(alterTableSql) }, - errorClass = "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", + "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", sqlState = None, parameters = Map( "fieldPath" -> "value", @@ -111,10 +111,10 @@ trait TypeWideningAlterTableTests ) } else { checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(alterTableSql) }, - errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", + "NOT_SUPPORTED_CHANGE_COLUMN", sqlState = None, parameters = Map( "table" -> s"`spark_catalog`.`delta`.`$tempPath`", @@ -176,10 +176,10 @@ trait TypeWideningAlterTableTests .mkString(", ") checkError( - exception = intercept[DeltaTableFeatureException] { + intercept[DeltaTableFeatureException] { sql(s"ALTER TABLE delta.`$tempPath` CHANGE COLUMN a TYPE TIMESTAMP_NTZ") }, - errorClass = "DELTA_FEATURES_REQUIRE_MANUAL_ENABLEMENT", + "DELTA_FEATURES_REQUIRE_MANUAL_ENABLEMENT", parameters = Map( "unsupportedFeatures" -> "timestampNtz", "supportedFeatures" -> currentFeatures diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningConstraintsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningConstraintsSuite.scala index 0c2d8cb5cfb..17e75bbb2d8 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningConstraintsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningConstraintsSuite.scala @@ -57,10 +57,10 @@ trait TypeWideningConstraintsTests { self: QueryTest with TypeWideningTestMixin // Changing the type of a column that a CHECK constraint depends on is not allowed. checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql("ALTER TABLE t CHANGE COLUMN a TYPE SMALLINT") }, - errorClass = "DELTA_CONSTRAINT_DEPENDENT_COLUMN_CHANGE", + "DELTA_CONSTRAINT_DEPENDENT_COLUMN_CHANGE", parameters = Map( "columnName" -> "a", "constraints" -> "delta.constraints.ck -> hash ( a ) > 0" @@ -81,10 +81,10 @@ trait TypeWideningConstraintsTests { self: QueryTest with TypeWideningTestMixin checkAnswer(sql("SELECT hash(a.x) FROM t"), Row(1765031574)) checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql("ALTER TABLE t CHANGE COLUMN a.x TYPE SMALLINT") }, - errorClass = "DELTA_CONSTRAINT_DEPENDENT_COLUMN_CHANGE", + "DELTA_CONSTRAINT_DEPENDENT_COLUMN_CHANGE", parameters = Map( "columnName" -> "a.x", "constraints" -> "delta.constraints.ck -> hash ( a . x ) > 0" @@ -105,10 +105,10 @@ trait TypeWideningConstraintsTests { self: QueryTest with TypeWideningTestMixin withSQLConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key -> "true") { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql("INSERT INTO t VALUES (200)") }, - errorClass = "DELTA_CONSTRAINT_DATA_TYPE_MISMATCH", + "DELTA_CONSTRAINT_DATA_TYPE_MISMATCH", parameters = Map( "columnName" -> "a", "columnType" -> "TINYINT", @@ -128,10 +128,10 @@ trait TypeWideningConstraintsTests { self: QueryTest with TypeWideningTestMixin withSQLConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key -> "true") { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql("INSERT INTO t (a) VALUES (named_struct('x', 200, 'y', CAST(5 AS byte)))") }, - errorClass = "DELTA_CONSTRAINT_DATA_TYPE_MISMATCH", + "DELTA_CONSTRAINT_DATA_TYPE_MISMATCH", parameters = Map( "columnName" -> "a.x", "columnType" -> "TINYINT", @@ -157,7 +157,7 @@ trait TypeWideningConstraintsTests { self: QueryTest with TypeWideningTestMixin withSQLConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key -> "true") { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql( s""" | INSERT INTO t (a) VALUES ( @@ -166,7 +166,7 @@ trait TypeWideningConstraintsTests { self: QueryTest with TypeWideningTestMixin |""".stripMargin ) }, - errorClass = "DELTA_CONSTRAINT_DATA_TYPE_MISMATCH", + "DELTA_CONSTRAINT_DATA_TYPE_MISMATCH", parameters = Map( "columnName" -> "a.x.z", "columnType" -> "TINYINT", 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 67250fe0bb6..d524d2fcd62 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 @@ -53,10 +53,10 @@ trait TypeWideningCompatibilityTests { .drop(CDCReader.CDC_COMMIT_VERSION) checkErrorMatchPVals( - exception = intercept[DeltaUnsupportedOperationException] { + intercept[DeltaUnsupportedOperationException] { readCDF(start = 1, end = 1).collect() }, - errorClass = "DELTA_CHANGE_DATA_FEED_INCOMPATIBLE_DATA_SCHEMA", + "DELTA_CHANGE_DATA_FEED_INCOMPATIBLE_DATA_SCHEMA", parameters = Map( "start" -> "1", "end" -> "1", @@ -92,10 +92,10 @@ trait TypeWideningCompatibilityTests { checkAnswer(readCDF(start = 1, end = 1), Seq(Row(1, "insert"), Row(2, "insert"))) checkErrorMatchPVals( - exception = intercept[DeltaUnsupportedOperationException] { + intercept[DeltaUnsupportedOperationException] { readCDF(start = 1, end = 3) }, - errorClass = "DELTA_CHANGE_DATA_FEED_INCOMPATIBLE_SCHEMA_CHANGE", + "DELTA_CHANGE_DATA_FEED_INCOMPATIBLE_SCHEMA_CHANGE", parameters = Map( "start" -> "1", "end" -> "3", diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningGeneratedColumnsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningGeneratedColumnsSuite.scala index 94eec4dae8c..7f8ebc2033d 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningGeneratedColumnsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/typewidening/TypeWideningGeneratedColumnsSuite.scala @@ -48,10 +48,10 @@ trait TypeWideningGeneratedColumnTests extends GeneratedColumnTest { // Changing the type of a column that a generated column depends on is not allowed. checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql("ALTER TABLE t CHANGE COLUMN a TYPE SMALLINT") }, - errorClass = "DELTA_GENERATED_COLUMNS_DEPENDENT_COLUMN_CHANGE", + "DELTA_GENERATED_COLUMNS_DEPENDENT_COLUMN_CHANGE", parameters = Map( "columnName" -> "a", "generatedColumns" -> "gen -> hash(a)" @@ -77,10 +77,10 @@ trait TypeWideningGeneratedColumnTests extends GeneratedColumnTest { checkAnswer(sql("SELECT hash(a.x) FROM t"), Row(1765031574)) checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql("ALTER TABLE t CHANGE COLUMN a.x TYPE SMALLINT") }, - errorClass = "DELTA_GENERATED_COLUMNS_DEPENDENT_COLUMN_CHANGE", + "DELTA_GENERATED_COLUMNS_DEPENDENT_COLUMN_CHANGE", parameters = Map( "columnName" -> "a.x", "generatedColumns" -> "gen -> hash(a.x)" @@ -106,10 +106,10 @@ trait TypeWideningGeneratedColumnTests extends GeneratedColumnTest { withSQLConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key -> "true") { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql("INSERT INTO t (a) VALUES (200)") }, - errorClass = "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", + "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", parameters = Map( "columnName" -> "a", "columnType" -> "TINYINT", @@ -134,10 +134,10 @@ trait TypeWideningGeneratedColumnTests extends GeneratedColumnTest { withSQLConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key -> "true") { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql("INSERT INTO t (a) VALUES (named_struct('x', 200, 'y', CAST(5 AS byte)))") }, - errorClass = "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", + "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", parameters = Map( "columnName" -> "a.x", "columnType" -> "TINYINT", @@ -169,7 +169,7 @@ trait TypeWideningGeneratedColumnTests extends GeneratedColumnTest { withSQLConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key -> "true") { checkError( - exception = intercept[DeltaAnalysisException] { + intercept[DeltaAnalysisException] { sql( s""" | INSERT INTO t (a) VALUES ( @@ -178,7 +178,7 @@ trait TypeWideningGeneratedColumnTests extends GeneratedColumnTest { |""".stripMargin ) }, - errorClass = "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", + "DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH", parameters = Map( "columnName" -> "a.x.z", "columnType" -> "TINYINT", 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 b4ede10f3dd..64805d26027 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 @@ -110,11 +110,11 @@ trait TypeWideningTableFeatureTests extends RowTrackingTestUtils with TypeWideni sql(s"CREATE TABLE delta.`$tempPath` (a int) USING DELTA " + s"TBLPROPERTIES ('${DeltaConfigs.ENABLE_TYPE_WIDENING.key}' = 'false')") checkError( - exception = intercept[SparkException] { + intercept[SparkException] { sql(s"ALTER TABLE delta.`$tempPath` " + s"SET TBLPROPERTIES ('${DeltaConfigs.ENABLE_TYPE_WIDENING.key}' = 'bla')") }, - errorClass = "_LEGACY_ERROR_TEMP_2045", + "_LEGACY_ERROR_TEMP_2045", parameters = Map( "message" -> "For input string: \"bla\"" ) @@ -128,10 +128,10 @@ trait TypeWideningTableFeatureTests extends RowTrackingTestUtils with TypeWideni s"TBLPROPERTIES ('${DeltaConfigs.ENABLE_TYPE_WIDENING.key}' = 'false')") checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"ALTER TABLE delta.`$tempPath` CHANGE COLUMN a TYPE SMALLINT") }, - errorClass = "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", + "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", parameters = Map( "fieldPath" -> "a", "oldField" -> "TINYINT", @@ -147,10 +147,10 @@ trait TypeWideningTableFeatureTests extends RowTrackingTestUtils with TypeWideni s"SET TBLPROPERTIES ('${DeltaConfigs.ENABLE_TYPE_WIDENING.key}' = 'false')") checkError( - exception = intercept[AnalysisException] { + intercept[AnalysisException] { sql(s"ALTER TABLE delta.`$tempPath` CHANGE COLUMN a TYPE INT") }, - errorClass = "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", + "DELTA_UNSUPPORTED_ALTER_TABLE_CHANGE_COL_OP", parameters = Map( "fieldPath" -> "a", "oldField" -> "SMALLINT", @@ -192,12 +192,12 @@ trait TypeWideningTableFeatureTests extends RowTrackingTestUtils with TypeWideni val deltaLog = DeltaLog.forTable(spark, TableIdentifier(tableName, Some(databaseName))) checkError( - exception = intercept[DeltaTableFeatureException] { + intercept[DeltaTableFeatureException] { sql(s"ALTER TABLE $databaseName.$tableName " + s"DROP FEATURE '${TypeWideningPreviewTableFeature.name}'" ).collect() }, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> TypeWideningPreviewTableFeature.name, "logRetentionPeriodKey" -> DeltaConfigs.LOG_RETENTION.key, @@ -442,10 +442,10 @@ trait TypeWideningTableFeatureTests extends RowTrackingTestUtils with TypeWideni } checkError( - exception = intercept[DeltaIllegalStateException] { + intercept[DeltaIllegalStateException] { readDeltaTable(tempPath).collect() }, - errorClass = "DELTA_UNSUPPORTED_TYPE_CHANGE_IN_SCHEMA", + "DELTA_UNSUPPORTED_TYPE_CHANGE_IN_SCHEMA", parameters = Map( "fieldName" -> "a.element", "fromType" -> "INT", 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 b33bb3ad077..71ce4072162 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 @@ -150,8 +150,8 @@ trait TypeWideningDropFeatureTestMixin dropFeature.run(spark) case ExpectedOutcome.FAIL_CURRENT_VERSION_USES_FEATURE => checkError( - exception = intercept[DeltaTableFeatureException] { dropFeature.run(spark) }, - errorClass = "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", + intercept[DeltaTableFeatureException] { dropFeature.run(spark) }, + "DELTA_FEATURE_DROP_WAIT_FOR_RETENTION_PERIOD", parameters = Map( "feature" -> feature.name, "logRetentionPeriodKey" -> DeltaConfigs.LOG_RETENTION.key, @@ -163,8 +163,8 @@ trait TypeWideningDropFeatureTestMixin ) case ExpectedOutcome.FAIL_HISTORICAL_VERSION_USES_FEATURE => checkError( - exception = intercept[DeltaTableFeatureException] { dropFeature.run(spark) }, - errorClass = "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", + intercept[DeltaTableFeatureException] { dropFeature.run(spark) }, + "DELTA_FEATURE_DROP_HISTORICAL_VERSIONS_EXIST", parameters = Map( "feature" -> feature.name, "logRetentionPeriodKey" -> DeltaConfigs.LOG_RETENTION.key, @@ -176,8 +176,8 @@ trait TypeWideningDropFeatureTestMixin ) case ExpectedOutcome.FAIL_FEATURE_NOT_PRESENT => checkError( - exception = intercept[DeltaTableFeatureException] { dropFeature.run(spark) }, - errorClass = "DELTA_FEATURE_DROP_FEATURE_NOT_PRESENT", + intercept[DeltaTableFeatureException] { dropFeature.run(spark) }, + "DELTA_FEATURE_DROP_FEATURE_NOT_PRESENT", parameters = Map("feature" -> feature.name) ) } diff --git a/storage/src/main/java/io/delta/storage/S3SingleDriverLogStore.java b/storage/src/main/java/io/delta/storage/S3SingleDriverLogStore.java index a1c4a296d7b..36d5cd2877e 100644 --- a/storage/src/main/java/io/delta/storage/S3SingleDriverLogStore.java +++ b/storage/src/main/java/io/delta/storage/S3SingleDriverLogStore.java @@ -23,14 +23,8 @@ import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; import java.util.*; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; import com.google.common.io.CountingOutputStream; -import io.delta.storage.internal.FileNameUtils; import io.delta.storage.internal.PathLock; import io.delta.storage.internal.S3LogStoreUtil; import org.apache.hadoop.conf.Configuration; @@ -41,12 +35,12 @@ import org.apache.hadoop.fs.RawLocalFileSystem; /** - * Single Spark-driver/JVM LogStore implementation for S3. + * Single JVM LogStore implementation for S3. *

* We assume the following from S3's {@link FileSystem} implementations: *

    *
  • File writing on S3 is all-or-nothing, whether overwrite or not.
  • - *
  • List-after-write can be inconsistent.
  • + *
  • List-after-write is strongly consistent.
  • *
*

* Regarding file creation, this implementation: @@ -55,12 +49,6 @@ *

  • Failures during stream write may leak resources, but may never result in partial * writes.
  • * - *

    - * Regarding directory listing, this implementation: - *

      - *
    • returns a list by merging the files listed from S3 and recently-written files from the - * cache.
    • - *
    */ public class S3SingleDriverLogStore extends HadoopFileSystemLogStore { @@ -85,16 +73,6 @@ public class S3SingleDriverLogStore extends HadoopFileSystemLogStore { */ private static final PathLock pathLock = new PathLock(); - /** - * A global cache that records the metadata of the files recently written. - * As list-after-write may be inconsistent on S3, we can use the files in the cache - * to fix the inconsistent file listing. - */ - private static final Cache writtenPathCache = - CacheBuilder.newBuilder() - .expireAfterAccess(120, TimeUnit.MINUTES) - .build(); - ///////////////////////////////////////////// // Constructor and Instance Helper Methods // ///////////////////////////////////////////// @@ -103,13 +81,6 @@ public S3SingleDriverLogStore(Configuration hadoopConf) { super(hadoopConf); } - /** - * Check if the path is an initial version of a Delta log. - */ - private boolean isInitialVersion(Path path) { - return FileNameUtils.isDeltaFile(path) && FileNameUtils.deltaVersion(path) == 0L; - } - private Path resolvePath(FileSystem fs, Path path) { return stripUserInfo(fs.makeQualified(path)); } @@ -137,57 +108,6 @@ private Path stripUserInfo(Path path) { } } - /** - * Merge two lists of {@link FileStatus} into a single list ordered by file path name. - * In case both lists have {@link FileStatus}'s for the same file path, keep the one from - * `listWithPrecedence` and discard the other from `list`. - */ - private Iterator mergeFileLists( - List list, - List listWithPrecedence) { - final Map fileStatusMap = new HashMap<>(); - - // insert all elements from `listWithPrecedence` (highest priority) - // and then insert elements from `list` if and only if that key doesn't already exist - Stream.concat(listWithPrecedence.stream(), list.stream()) - .forEach(fs -> fileStatusMap.putIfAbsent(fs.getPath(), fs)); - - return fileStatusMap - .values() - .stream() - .sorted(Comparator.comparing(a -> a.getPath().getName())) - .iterator(); - } - - /** - * List files starting from `resolvedPath` (inclusive) in the same directory. - */ - private List listFromCache( - FileSystem fs, - Path resolvedPath) { - final Path pathKey = stripUserInfo(resolvedPath); - - return writtenPathCache - .asMap() - .entrySet() - .stream() - .filter(e -> { - final Path path = e.getKey(); - return path.getParent().equals(pathKey.getParent()) && - path.getName().compareTo(pathKey.getName()) >= 0; - }).map(e -> { - final Path path = e.getKey(); - final FileMetadata fileMetadata = e.getValue(); - return new FileStatus( - fileMetadata.length, - false, // isDir - 1, // block_replication - fs.getDefaultBlockSize(path), - fileMetadata.modificationTime, - path); - }).collect(Collectors.toList()); - } - /** * List files starting from `resolvedPath` (inclusive) in the same directory, which merges * the file system list and the cache list when `useCache` is on, otherwise @@ -195,8 +115,7 @@ private List listFromCache( */ private Iterator listFromInternal( FileSystem fs, - Path resolvedPath, - boolean useCache) throws IOException { + Path resolvedPath) throws IOException { final Path parentPath = resolvedPath.getParent(); if (!fs.exists(parentPath)) { throw new FileNotFoundException( @@ -214,30 +133,11 @@ private Iterator listFromInternal( statuses = S3LogStoreUtil.s3ListFromArray(fs, resolvedPath, parentPath); } - final List listedFromFs = Arrays + return Arrays .stream(statuses) .filter(s -> s.getPath().getName().compareTo(resolvedPath.getName()) >= 0) - .collect(Collectors.toList()); - - final List listedFromCache = useCache ? - listFromCache(fs, resolvedPath) : Collections.emptyList(); - - // File statuses listed from file system take precedence - return mergeFileLists(listedFromCache, listedFromFs); - } - - /** - * Check if a path exists. Normally we check both the file system and the cache, but when the - * path is the first version of a Delta log, we ignore the cache. - */ - private boolean exists( - FileSystem fs, - Path resolvedPath) throws IOException { - final boolean useCache = !isInitialVersion(resolvedPath); - final Iterator iter = listFromInternal(fs, resolvedPath, useCache); - if (!iter.hasNext()) return false; - - return iter.next().getPath().getName().equals(resolvedPath.getName()); + .sorted(Comparator.comparing(a -> a.getPath().getName())) + .iterator(); } //////////////////////// @@ -255,7 +155,7 @@ public void write( try { pathLock.acquire(resolvedPath); try { - if (exists(fs, resolvedPath) && !overwrite) { + if (fs.exists(resolvedPath) && !overwrite) { throw new java.nio.file.FileAlreadyExistsException( resolvedPath.toUri().toString() ); @@ -268,26 +168,6 @@ public void write( stream.write((actions.next() + "\n").getBytes(StandardCharsets.UTF_8)); } stream.close(); - - // When a Delta log starts afresh, all cached files in that Delta log become - // obsolete, so we remove them from the cache. - if (isInitialVersion(resolvedPath)) { - final List obsoleteFiles = writtenPathCache - .asMap() - .keySet() - .stream() - .filter(p -> p.getParent().equals(resolvedPath.getParent())) - .collect(Collectors.toList()); - - writtenPathCache.invalidateAll(obsoleteFiles); - } - - // Cache the information of written files to help fix the inconsistency in future - // listings - writtenPathCache.put( - resolvedPath, - new FileMetadata(stream.getCount(), System.currentTimeMillis()) - ); } catch (org.apache.hadoop.fs.FileAlreadyExistsException e) { // Convert Hadoop's FileAlreadyExistsException to Java's FileAlreadyExistsException throw new java.nio.file.FileAlreadyExistsException(e.getMessage()); @@ -303,28 +183,11 @@ public void write( public Iterator listFrom(Path path, Configuration hadoopConf) throws IOException { final FileSystem fs = path.getFileSystem(hadoopConf); final Path resolvedPath = resolvePath(fs, path); - return listFromInternal(fs, resolvedPath, true); // useCache=true + return listFromInternal(fs, resolvedPath); } @Override public Boolean isPartialWriteVisible(Path path, Configuration hadoopConf) { return false; } - - ////////////////// - // Helper Class // - ////////////////// - - /** - * The file metadata to be stored in the cache. - */ - private class FileMetadata { - private long length; - private long modificationTime; - - public FileMetadata(long length, long modificationTime) { - this.length = length; - this.modificationTime = modificationTime; - } - } } diff --git a/storage/src/test/scala/io/delta/storage/commit/CoordinatedCommitsUtils.java b/storage/src/main/java/io/delta/storage/commit/CoordinatedCommitsUtils.java similarity index 75% rename from storage/src/test/scala/io/delta/storage/commit/CoordinatedCommitsUtils.java rename to storage/src/main/java/io/delta/storage/commit/CoordinatedCommitsUtils.java index 144b6acec23..cdd03d5ed75 100644 --- a/storage/src/test/scala/io/delta/storage/commit/CoordinatedCommitsUtils.java +++ b/storage/src/main/java/io/delta/storage/commit/CoordinatedCommitsUtils.java @@ -33,12 +33,24 @@ public class CoordinatedCommitsUtils { private CoordinatedCommitsUtils() {} /** The subdirectory in which to store the unbackfilled commit files. */ - final static String COMMIT_SUBDIR = "_commits"; + private static final String COMMIT_SUBDIR = "_commits"; - /** The configuration key for the coordinated commits owner. */ - private static final String COORDINATED_COMMITS_COORDINATOR_CONF_KEY = + /** The configuration key for the coordinated commits owner name. */ + private static final String COORDINATED_COMMITS_COORDINATOR_NAME_KEY = "delta.coordinatedCommits.commitCoordinator-preview"; + /** + * Creates a new unbackfilled delta file path for the given commit version. + * The path is of the form `tablePath/_delta_log/_commits/00000000000000000001.uuid.json`. + */ + public static Path generateUnbackfilledDeltaFilePath( + Path logPath, + long version) { + String uuid = UUID.randomUUID().toString(); + Path basePath = new Path(logPath, COMMIT_SUBDIR); + return new Path(basePath, String.format("%020d.%s.json", version, uuid)); + } + /** * Returns the path to the backfilled delta file for the given commit version. * The path is of the form `tablePath/_delta_log/00000000000000000001.json`. @@ -56,9 +68,9 @@ public static boolean isCoordinatedCommitsToFSConversion( Long commitVersion, UpdatedActions updatedActions) { boolean oldMetadataHasCoordinatedCommits = - !getCoordinator(updatedActions.getOldMetadata()).isEmpty(); + getCoordinatorName(updatedActions.getOldMetadata()).isPresent(); boolean newMetadataHasCoordinatedCommits = - !getCoordinator(updatedActions.getNewMetadata()).isEmpty(); + getCoordinatorName(updatedActions.getNewMetadata()).isPresent(); return oldMetadataHasCoordinatedCommits && !newMetadataHasCoordinatedCommits && commitVersion > 0; } @@ -108,10 +120,17 @@ public static Path commitDirPath(Path logPath) { return new Path(logPath, COMMIT_SUBDIR); } - private static String getCoordinator(AbstractMetadata metadata) { + /** + * Retrieves the coordinator name from the provided abstract metadata. + * If no coordinator is set, an empty optional is returned. + * + * @param metadata The abstract metadata from which to retrieve the coordinator name. + * @return The coordinator name if set, otherwise an empty optional. + */ + public static Optional getCoordinatorName(AbstractMetadata metadata) { String coordinator = metadata .getConfiguration() - .get(COORDINATED_COMMITS_COORDINATOR_CONF_KEY); - return coordinator != null ? coordinator : ""; + .get(COORDINATED_COMMITS_COORDINATOR_NAME_KEY); + return Optional.ofNullable(coordinator); } } diff --git a/storage/src/test/scala/io/delta/storage/commit/InMemoryCommitCoordinator.scala b/storage/src/test/scala/io/delta/storage/commit/InMemoryCommitCoordinator.scala index 5032756bc92..551969a8d4e 100644 --- a/storage/src/test/scala/io/delta/storage/commit/InMemoryCommitCoordinator.scala +++ b/storage/src/test/scala/io/delta/storage/commit/InMemoryCommitCoordinator.scala @@ -22,6 +22,7 @@ import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.locks.ReentrantReadWriteLock import io.delta.storage.LogStore +import io.delta.storage.commit.CoordinatedCommitsUtils import io.delta.storage.commit.actions.AbstractMetadata import io.delta.storage.commit.actions.AbstractProtocol import org.apache.hadoop.conf.Configuration