From 36d1dae453178716823f71ea991d518f548f7ada Mon Sep 17 00:00:00 2001 From: Fred Storage Liu Date: Wed, 19 Feb 2025 18:07:06 -0800 Subject: [PATCH 1/9] Refactor for Delta Uniform IcebergConversionTransaction to use current iceberg PartitionSpec (#4174) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description Refactor for Delta Uniform IcebergConversionTransaction to use current iceberg PartitionSpec. ## How was this patch tested? UT ## Does this PR introduce _any_ user-facing changes? --- .../IcebergConversionTransaction.scala | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConversionTransaction.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConversionTransaction.scala index 81d8513243f..bebdcc72cab 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConversionTransaction.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConversionTransaction.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.conf.Configuration -import shadedForDelta.org.apache.iceberg.{AppendFiles, DeleteFiles, OverwriteFiles, PendingUpdate, RewriteFiles, Transaction => IcebergTransaction} +import shadedForDelta.org.apache.iceberg.{AppendFiles, DeleteFiles, OverwriteFiles, PartitionSpec, PendingUpdate, RewriteFiles, Transaction => IcebergTransaction} import shadedForDelta.org.apache.iceberg.ExpireSnapshots import shadedForDelta.org.apache.iceberg.mapping.MappingUtil import shadedForDelta.org.apache.iceberg.mapping.NameMappingParser @@ -64,7 +64,8 @@ class IcebergConversionTransaction( protected val postCommitSnapshot: Snapshot, protected val tableOp: IcebergTableOp = WRITE_TABLE, protected val lastConvertedIcebergSnapshotId: Option[Long] = None, - protected val lastConvertedDeltaVersion: Option[Long] = None) extends DeltaLogging { + protected val lastConvertedDeltaVersion: Option[Long] = None + ) extends DeltaLogging { /////////////////////////// // Nested Helper Classes // @@ -100,7 +101,7 @@ class IcebergConversionTransaction( convertDeltaAddFileToIcebergDataFile( add, tablePath, - partitionSpec, + currentPartitionSpec, logicalToPhysicalPartitionNames, statsParser, postCommitSnapshot @@ -141,7 +142,7 @@ class IcebergConversionTransaction( convertDeltaAddFileToIcebergDataFile( add, tablePath, - partitionSpec, + currentPartitionSpec, logicalToPhysicalPartitionNames, statsParser, postCommitSnapshot @@ -154,7 +155,7 @@ class IcebergConversionTransaction( convertDeltaRemoveFileToIcebergDataFile( remove, tablePath, - partitionSpec, + currentPartitionSpec, logicalToPhysicalPartitionNames, postCommitSnapshot) ) @@ -178,7 +179,7 @@ class IcebergConversionTransaction( convertDeltaRemoveFileToIcebergDataFile( f, tablePath, - partitionSpec, + currentPartitionSpec, logicalToPhysicalPartitionNames, postCommitSnapshot) }.toSet.asJava @@ -188,7 +189,7 @@ class IcebergConversionTransaction( convertDeltaAddFileToIcebergDataFile( f, tablePath, - partitionSpec, + currentPartitionSpec, logicalToPhysicalPartitionNames, statsParser, postCommitSnapshot @@ -212,8 +213,15 @@ class IcebergConversionTransaction( protected val tablePath = postCommitSnapshot.deltaLog.dataPath protected val icebergSchema = convertDeltaSchemaToIcebergSchema(postCommitSnapshot.metadata.schema) + // Initial partition spec converted from Delta protected val partitionSpec = createPartitionSpec(icebergSchema, postCommitSnapshot.metadata.partitionColumns) + + // Current partition spec from iceberg table + def currentPartitionSpec: PartitionSpec = { + Some(txn.table()).map(_.spec()).getOrElse(partitionSpec) + } + private val logicalToPhysicalPartitionNames = getPartitionPhysicalNameMapping(postCommitSnapshot.metadata.partitionSchema) From ac54ef77c230d282e0d3fbf54d9501948d500df3 Mon Sep 17 00:00:00 2001 From: Johan Lasperas Date: Thu, 20 Feb 2025 17:49:54 +0100 Subject: [PATCH 2/9] [Spark] Add order-agnostic type widening modes (#4164) ## Description All code paths implementing type widening so far had a clear 'before' and 'after' schema and checked that 'before' could be widened to 'after'. An additional use case is, given two schemas from two separate flows, to compute wider schema from the two. To support this, 'bidirectional' widening modes are added to `SchemaMergingUtils.mergeSchemas()`. These allow picking the wider of the two input types, or, for decimals, picking a decimal type that is wider than each input. ## How was this patch tested? Added unit tests for `SchemaMergingUtils.mergeSchemas()`. --- .../DecimalPrecisionTypeCoercionShims.scala | 26 ++ .../DecimalPrecisionTypeCoercionShims.scala | 26 ++ .../spark/sql/delta/DeltaAnalysis.scala | 4 +- .../spark/sql/delta/TypeWideningMode.scala | 84 +++++-- .../sql/delta/schema/SchemaMergingUtils.scala | 3 +- .../spark/sql/delta/schema/SchemaUtils.scala | 2 +- .../sql/delta/schema/SchemaUtilsSuite.scala | 233 +++++++++++++++++- 7 files changed, 355 insertions(+), 23 deletions(-) create mode 100644 spark/src/main/scala-spark-3.5/shims/DecimalPrecisionTypeCoercionShims.scala create mode 100644 spark/src/main/scala-spark-master/shims/DecimalPrecisionTypeCoercionShims.scala diff --git a/spark/src/main/scala-spark-3.5/shims/DecimalPrecisionTypeCoercionShims.scala b/spark/src/main/scala-spark-3.5/shims/DecimalPrecisionTypeCoercionShims.scala new file mode 100644 index 00000000000..41f050828c1 --- /dev/null +++ b/spark/src/main/scala-spark-3.5/shims/DecimalPrecisionTypeCoercionShims.scala @@ -0,0 +1,26 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta + +import org.apache.spark.sql.catalyst.analysis.DecimalPrecision +import org.apache.spark.sql.types.DecimalType + +object DecimalPrecisionTypeCoercionShims { + // Returns the wider decimal type that's wider than both of them + def widerDecimalType(d1: DecimalType, d2: DecimalType): DecimalType = + DecimalPrecision.widerDecimalType(d1, d2) +} diff --git a/spark/src/main/scala-spark-master/shims/DecimalPrecisionTypeCoercionShims.scala b/spark/src/main/scala-spark-master/shims/DecimalPrecisionTypeCoercionShims.scala new file mode 100644 index 00000000000..a907179fdd2 --- /dev/null +++ b/spark/src/main/scala-spark-master/shims/DecimalPrecisionTypeCoercionShims.scala @@ -0,0 +1,26 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta + +import org.apache.spark.sql.catalyst.analysis.DecimalPrecisionTypeCoercion +import org.apache.spark.sql.types.DecimalType + +object DecimalPrecisionTypeCoercionShims { + // Returns the wider decimal type that's wider than both of them + def widerDecimalType(d1: DecimalType, d2: DecimalType): DecimalType = + DecimalPrecisionTypeCoercion.widerDecimalType(d1, d2) +} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala index 10d3b6b0e9f..69142e22277 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala @@ -925,7 +925,7 @@ class DeltaAnalysis(session: SparkSession) if s != t && sNull == tNull => addCastsToArrayStructs(tblName, attr, s, t, sNull, typeWideningMode) case (s: AtomicType, t: AtomicType) - if typeWideningMode.shouldWidenType(fromType = t, toType = s) => + if typeWideningMode.shouldWidenTo(fromType = t, toType = s) => // Keep the type from the query, the target schema will be updated to widen the existing // type to match it. attr @@ -1097,7 +1097,7 @@ class DeltaAnalysis(session: SparkSession) case (StructField(name, sourceType: AtomicType, _, _), i @ TargetIndex(StructField(targetName, targetType: AtomicType, _, targetMetadata))) - if typeWideningMode.shouldWidenType(fromType = targetType, toType = sourceType) => + if typeWideningMode.shouldWidenTo(fromType = targetType, toType = sourceType) => Alias( GetStructField(parent, i, Option(name)), targetName)(explicitMetadata = Option(targetMetadata)) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/TypeWideningMode.scala b/spark/src/main/scala/org/apache/spark/sql/delta/TypeWideningMode.scala index eaeefc9358d..88234bda5fd 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/TypeWideningMode.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/TypeWideningMode.scala @@ -16,21 +16,33 @@ package org.apache.spark.sql.delta -import org.apache.spark.sql.types.AtomicType +import org.apache.spark.sql.delta.DecimalPrecisionTypeCoercionShims +import org.apache.spark.sql.util.ScalaExtensions._ + +import org.apache.spark.sql.types.{AtomicType, DecimalType} /** * A type widening mode captures a specific set of type changes that are allowed to be applied. * Currently: * - NoTypeWidening: No type change is allowed. - * - AllTypeWidening: All supported type widening changes are allowed. - * - TypeEvolution(uniformIcebergCompatibleOnly = true): Type changes that are eligible to be - * applied automatically during schema evolution and that are supported by Iceberg are allowed. - * - TypeEvolution(uniformIcebergCompatibleOnly = false): Type changes that are eligible to be - * applied automatically during schema evolution are allowed, even if they are not supported by - * Iceberg. + * - AllTypeWidening: Allows widening to the target type using any supported type change. + * - TypeEvolution: Only allows widening to the target type if the type change is eligible to be + * applied automatically during schema evolution. + * - AllTypeWideningToCommonWiderType: Allows widening to a common (possibly different) wider type + * using any supported type change. + * - TypeEvolutionToCommonWiderType: Allows widening to a common (possibly different) wider type + * using only type changes that are eligible to be applied automatically during schema + * evolution. + * + * TypeEvolution modes can be restricted to only type changes supported by Iceberg by passing + * `uniformIcebergCompatibleOnly = truet`, to ensure that we don't automatically apply a type change + * that would break Iceberg compatibility. */ sealed trait TypeWideningMode { - def shouldWidenType(fromType: AtomicType, toType: AtomicType): Boolean + def getWidenedType(fromType: AtomicType, toType: AtomicType): Option[AtomicType] + + def shouldWidenTo(fromType: AtomicType, toType: AtomicType): Boolean = + getWidenedType(fromType, toType).contains(toType) } object TypeWideningMode { @@ -38,13 +50,13 @@ object TypeWideningMode { * No type change allowed. Typically because type widening and/or schema evolution isn't enabled. */ case object NoTypeWidening extends TypeWideningMode { - override def shouldWidenType(fromType: AtomicType, toType: AtomicType): Boolean = false + override def getWidenedType(fromType: AtomicType, toType: AtomicType): Option[AtomicType] = None } /** All supported type widening changes are allowed. */ case object AllTypeWidening extends TypeWideningMode { - override def shouldWidenType(fromType: AtomicType, toType: AtomicType): Boolean = - TypeWidening.isTypeChangeSupported(fromType = fromType, toType = toType) + override def getWidenedType(fromType: AtomicType, toType: AtomicType): Option[AtomicType] = + Option.when(TypeWidening.isTypeChangeSupported(fromType = fromType, toType = toType))(toType) } /** @@ -52,8 +64,52 @@ object TypeWideningMode { * Can be restricted to only type changes supported by Iceberg. */ case class TypeEvolution(uniformIcebergCompatibleOnly: Boolean) extends TypeWideningMode { - override def shouldWidenType(fromType: AtomicType, toType: AtomicType): Boolean = - TypeWidening.isTypeChangeSupportedForSchemaEvolution( - fromType = fromType, toType = toType, uniformIcebergCompatibleOnly) + override def getWidenedType(fromType: AtomicType, toType: AtomicType): Option[AtomicType] = + Option.when(TypeWidening.isTypeChangeSupportedForSchemaEvolution( + fromType = fromType, toType = toType, uniformIcebergCompatibleOnly))(toType) + } + + /** + * All supported type widening changes are allowed. Unlike [[AllTypeWidening]], this also allows + * widening `to` to `from`, and for decimals, widening to a different decimal type that is wider + * than both input types. Use for example when merging two unrelated schemas and we want just want + * to find a wider schema to use. + */ + case object AllTypeWideningToCommonWiderType extends TypeWideningMode { + override def getWidenedType(left: AtomicType, right: AtomicType): Option[AtomicType] = + (left, right) match { + case (l, r) if TypeWidening.isTypeChangeSupported(l, r) => Some(r) + case (l, r) if TypeWidening.isTypeChangeSupported(r, l) => Some(l) + case (l: DecimalType, r: DecimalType) => + val wider = DecimalPrecisionTypeCoercionShims.widerDecimalType(l, r) + Option.when( + TypeWidening.isTypeChangeSupported(l, wider) && + TypeWidening.isTypeChangeSupported(r, wider))(wider) + case _ => None + } + } + + /** + * Type changes that are eligible to be applied automatically during schema evolution are allowed. + * Can be restricted to only type changes supported by Iceberg. Unlike [[TypeEvolution]], this + * also allows widening `to` to `from`, and for decimals, widening to a different decimal type + * that is wider han both input types. Use for example when merging two unrelated schemas and we + * want just want to find a wider schema to use. + */ + case class TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly: Boolean) + extends TypeWideningMode { + override def getWidenedType(left: AtomicType, right: AtomicType): Option[AtomicType] = { + def typeChangeSupported: (AtomicType, AtomicType) => Boolean = + TypeWidening.isTypeChangeSupportedForSchemaEvolution(_, _, uniformIcebergCompatibleOnly) + + (left, right) match { + case (l, r) if typeChangeSupported(l, r) => Some(r) + case (l, r) if typeChangeSupported(r, l) => Some(l) + case (l: DecimalType, r: DecimalType) => + val wider = DecimalPrecisionTypeCoercionShims.widerDecimalType(l, r) + Option.when(typeChangeSupported(l, wider) && typeChangeSupported(r, wider))(wider) + case _ => None + } + } } } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala index ada30d2e8b9..200ac9b9519 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala @@ -242,7 +242,8 @@ object SchemaMergingUtils { // If type widening is enabled and the type can be widened, it takes precedence over // keepExistingType. case (current: AtomicType, update: AtomicType) - if typeWideningMode.shouldWidenType(fromType = current, toType = update) => update + if typeWideningMode.getWidenedType(fromType = current, toType = update).isDefined => + typeWideningMode.getWidenedType(fromType = current, toType = update).get // Simply keeps the existing type for primitive types case (current, _) if keepExistingType => current diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala index 1bfc74ee1bc..75506d02253 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 @@ -443,7 +443,7 @@ def normalizeColumnNamesInDataType( isDatatypeReadCompatible(e.keyType, n.keyType) && isDatatypeReadCompatible(e.valueType, n.valueType) case (e: AtomicType, n: AtomicType) - if typeWideningMode.shouldWidenType(fromType = e, toType = n) => true + if typeWideningMode.shouldWidenTo(fromType = e, toType = n) => true case (a, b) => a == b } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala index 5bd6a73c924..a35424c0f47 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala @@ -22,7 +22,7 @@ import java.util.regex.Pattern import scala.annotation.tailrec -import org.apache.spark.sql.delta.{DeltaAnalysisException, DeltaLog, DeltaTestUtils, TypeWideningMode} +import org.apache.spark.sql.delta.{DeltaAnalysisException, DeltaExcludedBySparkVersionTestMixinShims, DeltaLog, DeltaTestUtils, TypeWideningMode} import org.apache.spark.sql.delta.RowCommitVersion import org.apache.spark.sql.delta.RowId import org.apache.spark.sql.delta.commands.cdc.CDCReader @@ -48,8 +48,10 @@ class SchemaUtilsSuite extends QueryTest with SharedSparkSession with GivenWhenThen with DeltaSQLTestUtils - with DeltaSQLCommandTest { + with DeltaSQLCommandTest + with DeltaExcludedBySparkVersionTestMixinShims { import SchemaUtils._ + import TypeWideningMode._ import testImplicits._ private def expectFailure(shouldContain: String*)(f: => Unit): Unit = { @@ -71,8 +73,11 @@ class SchemaUtilsSuite extends QueryTest s"Error message '${e.getMessage}' didn't contain the patterns: $shouldContainPatterns") } - private def expectAnalysisErrorClass(errorClass: String, params: Map[String, String]) - (f: => Unit): Unit = { + private def expectAnalysisErrorClass( + errorClass: String, + params: Map[String, String], + matchPVals: Boolean = true)( + f: => Unit): Unit = { val e = intercept[AnalysisException] { f } @@ -90,7 +95,7 @@ class SchemaUtilsSuite extends QueryTest err.get, errorClass, parameters = params, - matchPVals = true) + matchPVals = matchPVals) } ///////////////////////////// @@ -2569,6 +2574,224 @@ class SchemaUtilsSuite extends QueryTest assert(mergedSchema === expected) } + private val allTypeWideningModes = Set( + NoTypeWidening, + AllTypeWidening, + TypeEvolution(uniformIcebergCompatibleOnly = false), + TypeEvolution(uniformIcebergCompatibleOnly = true), + AllTypeWideningToCommonWiderType, + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = true) + ) + + test("typeWideningMode - byte->short->int is always allowed") { + val narrow = new StructType() + .add("a", ByteType) + .add("b", ByteType) + .add("c", ShortType) + .add("s", new StructType().add("x", ByteType)) + .add("m", MapType(ByteType, ShortType)) + .add("ar", ArrayType(ByteType)) + + val wide = new StructType() + .add("a", ShortType) + .add("b", IntegerType) + .add("c", IntegerType) + .add("s", new StructType().add("x", IntegerType)) + .add("m", MapType(ShortType, IntegerType)) + .add("ar", ArrayType(IntegerType)) + + for (typeWideningMode <- allTypeWideningModes) { + // byte, short, int are all stored as INT64 in parquet, [[mergeSchemas]] always allows + // widening between them. This was already the case before typeWideningMode was introduced. + val merged1 = mergeSchemas(narrow, wide, typeWideningMode = typeWideningMode) + assert(merged1 === wide) + val merged2 = mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + assert(merged2 === wide) + } + } + + // These type changes will only be available once Delta uses Spark 4.0. + for ((fromType, toType) <- Seq( + IntegerType -> LongType, + new StructType().add("x", IntegerType) -> new StructType().add("x", LongType), + MapType(IntegerType, IntegerType) -> MapType(LongType, LongType), + ArrayType(IntegerType) -> ArrayType(LongType) + )) + testSparkMasterOnly(s"typeWideningMode ${fromType.sql} -> ${toType.sql}") { + val narrow = new StructType().add("a", fromType) + val wide = new StructType().add("a", toType) + + for (typeWideningMode <- Seq( + NoTypeWidening, + AllTypeWidening, + TypeEvolution(uniformIcebergCompatibleOnly = false), + TypeEvolution(uniformIcebergCompatibleOnly = true))) { + // Narrowing is not allowed. + expectAnalysisErrorClass("DELTA_MERGE_INCOMPATIBLE_DATATYPE", + Map("currentDataType" -> "LongType", "updateDataType" -> "IntegerType")) { + mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + } + } + + for (typeWideningMode <- Seq( + AllTypeWideningToCommonWiderType, + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = true))) { + // These modes don't enforce an order on the inputs, widening from second schema to first + // is allowed. + val merged = mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + assert(merged === wide) + } + + for (typeWideningMode <- allTypeWideningModes -- Set(NoTypeWidening)) { + // Widening is allowed, unless mode is NoTypeWidening. + val merged = mergeSchemas(narrow, wide, typeWideningMode = typeWideningMode) + assert(merged === wide) + } + expectAnalysisErrorClass("DELTA_MERGE_INCOMPATIBLE_DATATYPE", + Map("currentDataType" -> "LongType", "updateDataType" -> "IntegerType")) { + mergeSchemas(wide, narrow, typeWideningMode = NoTypeWidening) + } + } + + for ((fromType, toType) <- Seq( + ShortType -> DoubleType, + IntegerType -> DecimalType(10, 0) + )) + testSparkMasterOnly( + s"typeWideningMode - blocked type evolution ${fromType.sql} -> ${toType.sql}") { + val narrow = new StructType().add("a", fromType) + val wide = new StructType().add("a", toType) + + for (typeWideningMode <- Seq( + TypeEvolution(uniformIcebergCompatibleOnly = false), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false), + TypeEvolution(uniformIcebergCompatibleOnly = true), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = true))) { + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DATATYPE", + Map("currentDataType" -> fromType.toString, "updateDataType" -> toType.toString), + matchPVals = false) { + mergeSchemas(narrow, wide, typeWideningMode = typeWideningMode) + } + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DATATYPE", + Map("currentDataType" -> toType.toString, "updateDataType" -> fromType.toString), + matchPVals = false) { + mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + } + } + } + + for ((fromType, toType) <- Seq( + DateType -> TimestampNTZType, + DecimalType(10, 2) -> DecimalType(12, 4) + )) + testSparkMasterOnly( + s"typeWideningMode - Uniform Iceberg compatibility ${fromType.sql} -> ${toType.sql}") { + val narrow = new StructType().add("a", fromType) + val wide = new StructType().add("a", toType) + + def checkAnalysisException(f: => Unit): Unit = { + val ex = intercept[DeltaAnalysisException](f).getCause.asInstanceOf[AnalysisException] + // Decimal scale increase return a slightly different error class. + assert(ex.errorClass.contains("DELTA_MERGE_INCOMPATIBLE_DATATYPE") || + ex.errorClass.contains("DELTA_MERGE_INCOMPATIBLE_DECIMAL_TYPE")) + } + + for (typeWideningMode <- Seq( + TypeEvolution(uniformIcebergCompatibleOnly = false), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false))) { + // Unsupported type changes by Iceberg are allowed without Iceberg compatibility. + val merged = mergeSchemas(narrow, wide, typeWideningMode = typeWideningMode) + assert(merged === wide) + } + + for (typeWideningMode <- Seq( + TypeEvolution(uniformIcebergCompatibleOnly = true), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = true))) { + // Widening is blocked for unsupported type changes with Iceberg compatibility. + checkAnalysisException { + mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + } + } + + // These modes don't enforce an order on the inputs, widening from second schema to first + // is allowed without Iceberg compatibility. + val merged = mergeSchemas(wide, narrow, + typeWideningMode = TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false)) + assert(merged === wide) + + for (typeWideningMode <- Seq( + TypeEvolution(uniformIcebergCompatibleOnly = true), + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = true), + TypeEvolution(uniformIcebergCompatibleOnly = true))) { + // Rejected either because this is a narrowing type change, or for the bidirectional mode, + // because it is not supported by Iceberg. + checkAnalysisException { + mergeSchemas(wide, narrow, typeWideningMode = typeWideningMode) + } + } + } + + testSparkMasterOnly( + s"typeWideningMode - widen to common wider decimal") { + val left = new StructType().add("a", DecimalType(10, 2)) + val right = new StructType().add("a", DecimalType(5, 4)) + val wider = new StructType().add("a", DecimalType(12, 4)) + + val modesCanWidenToCommonWiderDecimal = Set( + // Increasing decimal scale isn't supported by Iceberg, so only possible when we don't enforce + // Iceberg compatibility. + TypeEvolutionToCommonWiderType(uniformIcebergCompatibleOnly = false), + AllTypeWideningToCommonWiderType + ) + + for (typeWideningMode <- modesCanWidenToCommonWiderDecimal) { + assert(mergeSchemas(left, right, typeWideningMode = typeWideningMode) === wider) + assert(mergeSchemas(right, left, typeWideningMode = typeWideningMode) === wider) + } + + for (typeWideningMode <- allTypeWideningModes -- modesCanWidenToCommonWiderDecimal) { + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DECIMAL_TYPE", + Map("decimalRanges" -> "precision 10 and 5 & scale 2 and 4"), + matchPVals = false) { + mergeSchemas(left, right, typeWideningMode = typeWideningMode) + } + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DECIMAL_TYPE", + Map("decimalRanges" -> "precision 5 and 10 & scale 4 and 2"), + matchPVals = false) { + mergeSchemas(right, left, typeWideningMode = typeWideningMode) + } + } + + } + + testSparkMasterOnly( + s"typeWideningMode - widen to common wider decimal exceeds max decimal precision") { + // We'd need a DecimalType(40, 19) to fit both types, which exceeds max decimal precision of 38. + val left = new StructType().add("a", DecimalType(20, 19)) + val right = new StructType().add("a", DecimalType(21, 0)) + + for (typeWideningMode <- allTypeWideningModes) { + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DECIMAL_TYPE", + Map("decimalRanges" -> "precision 20 and 21 & scale 19 and 0"), + matchPVals = false) { + mergeSchemas(left, right, typeWideningMode = typeWideningMode) + } + expectAnalysisErrorClass( + "DELTA_MERGE_INCOMPATIBLE_DECIMAL_TYPE", + Map("decimalRanges" -> "precision 21 and 20 & scale 0 and 19"), + matchPVals = false) { + mergeSchemas(right, left, typeWideningMode = typeWideningMode) + } + } + } + //////////////////////////// // transformColumns //////////////////////////// From 71cf788135a8d3e996702a94f9334a477175acfa Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Thu, 20 Feb 2025 08:50:42 -0800 Subject: [PATCH 3/9] [Kernel] Fix typo in TableFeaturesSuite (#4170) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description Fix a minor typo in TableFeaturesSuite ## How was this patch tested? $ build/sbt kernelGroup/test ## Does this PR introduce _any_ user-facing changes? No --- .../tablefeatures/TableFeaturesSuite.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala index 2c9881a4127..1421c340889 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala @@ -101,8 +101,8 @@ class TableFeaturesSuite extends AnyFunSuite { // Test feature, metadata, expected result ("appendOnly", testMetadata(tblProps = Map("delta.appendOnly" -> "true")), true), ("appendOnly", testMetadata(tblProps = Map("delta.appendOnly" -> "false")), false), - ("invariants", testMetadata(includeInvaraint = true), true), - ("invariants", testMetadata(includeInvaraint = false), false), + ("invariants", testMetadata(includeInvariant = true), true), + ("invariants", testMetadata(includeInvariant = false), false), ("checkConstraints", testMetadata(tblProps = Map("delta.constraints.a" -> "a = b")), true), ("checkConstraints", testMetadata(), false), ("generatedColumns", testMetadata(includeGeneratedColumn = true), true), @@ -218,7 +218,7 @@ class TableFeaturesSuite extends AnyFunSuite { Seq( // Test format: feature, metadata, expected value - ("invariants", testMetadata(includeInvaraint = true), false), + ("invariants", testMetadata(includeInvariant = true), false), ("checkConstraints", testMetadata(tblProps = Map("delta.constraints.a" -> "a = b")), false), ("generatedColumns", testMetadata(includeGeneratedColumn = true), false), ("identityColumns", testMetadata(includeIdentityColumn = true), false)).foreach({ @@ -249,13 +249,13 @@ class TableFeaturesSuite extends AnyFunSuite { test("validateWriteSupported: protocol 2 with invariants") { checkUnsupported( createTestProtocol(minWriterVersion = 2), - metadata = testMetadata(includeInvaraint = true)) + metadata = testMetadata(includeInvariant = true)) } test("validateWriteSupported: protocol 2, with appendOnly and invariants") { checkUnsupported( createTestProtocol(minWriterVersion = 2), - metadata = testMetadata(includeInvaraint = true)) + metadata = testMetadata(includeInvariant = true)) } Seq(3, 4, 5, 6).foreach { minWriterVersion => @@ -312,7 +312,7 @@ class TableFeaturesSuite extends AnyFunSuite { test("validateWriteSupported: protocol 7 with invariants, schema contains invariants") { checkUnsupported( createTestProtocol(minWriterVersion = 7, "invariants"), - metadata = testMetadata(includeInvaraint = true)) + metadata = testMetadata(includeInvariant = true)) } def checkSupported( @@ -340,14 +340,14 @@ class TableFeaturesSuite extends AnyFunSuite { } def testMetadata( - includeInvaraint: Boolean = false, + includeInvariant: Boolean = false, includeTimestampNtzTypeCol: Boolean = false, includeVariantTypeCol: Boolean = false, includeGeneratedColumn: Boolean = false, includeIdentityColumn: Boolean = false, tblProps: Map[String, String] = Map.empty): Metadata = { val testSchema = createTestSchema( - includeInvaraint, + includeInvariant, includeTimestampNtzTypeCol, includeVariantTypeCol, includeGeneratedColumn, From e621c5dfc2225c6352a06cb4a0da950c87a12313 Mon Sep 17 00:00:00 2001 From: Stefan Kandic <154237371+stefankandic@users.noreply.github.com> Date: Thu, 20 Feb 2025 20:12:53 +0100 Subject: [PATCH 4/9] [SPARK] Add common sql test utility methods to `DeltaSQLTestUtils` (#4131) #### Which Delta project/connector is this regarding? - [X] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description Adding the most commonly used methods in testing; such as getting the delta log, snapshot, protocol, stats, columns etc in `DeltaSQLTestUtils` so that we don't have to keep reimplementing them over and over again and just use this trait instead. ## How was this patch tested? No testing required. ## Does this PR introduce _any_ user-facing changes? No. --- .../spark/sql/delta/DeltaVariantSuite.scala | 5 - .../sql/delta/test/DeltaSQLTestUtils.scala | 230 ++++++++++++++++++ 2 files changed, 230 insertions(+), 5 deletions(-) diff --git a/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala b/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala index 9cba024352f..399b542ef53 100644 --- a/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala +++ b/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala @@ -43,11 +43,6 @@ class DeltaVariantSuite import testImplicits._ - private def getProtocolForTable(table: String): Protocol = { - val deltaLog = DeltaLog.forTable(spark, TableIdentifier(table)) - deltaLog.unsafeVolatileSnapshot.protocol - } - private def assertVariantTypeTableFeatures( tableName: String, expectPreviewFeature: Boolean, diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLTestUtils.scala b/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLTestUtils.scala index e16667e886d..6baa4ca12cd 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLTestUtils.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLTestUtils.scala @@ -18,7 +18,17 @@ package org.apache.spark.sql.delta.test import java.io.File +import scala.util.Random + +import org.apache.spark.sql.delta.{DeltaColumnMappingTestUtilsBase, DeltaLog, DeltaTable, Snapshot, TableFeature} +import org.apache.spark.sql.delta.actions.Protocol +import org.apache.spark.sql.delta.stats.{DeltaStatistics, PreparedDeltaFileIndex} +import com.fasterxml.jackson.databind.{JsonNode, ObjectMapper} + +import org.apache.spark.sql.{AnalysisException, DataFrame} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils trait DeltaSQLTestUtils extends SQLTestUtils { @@ -74,4 +84,224 @@ trait DeltaSQLTestUtils extends SQLTestUtils { files.foreach(Utils.deleteRecursively) } } + + /** Returns random alphanumberic string to be used as a unique table name. */ + def uniqueTableName: String = Random.alphanumeric.take(10).mkString + + /** Gets the latest snapshot of the table. */ + def getSnapshot(tableName: String): Snapshot = { + DeltaLog.forTable(spark, TableIdentifier(tableName)).update() + } + + /** Gets the table protocol of the latest snapshot. */ + def getProtocolForTable(tableName: String): Protocol = { + getSnapshot(tableName).protocol + } + /** Gets the `StructField` of `columnPath`. */ + final def getColumnField(schema: StructType, columnPath: Seq[String]): StructField = { + schema.findNestedField(columnPath, includeCollections = true).get._2 + } + + /** Gets the `StructField` of `columnName`. */ + def getColumnField(tableName: String, columnName: String): StructField = { + val deltaLog = DeltaLog.forTable(spark, TableIdentifier(tableName)) + getColumnField(deltaLog.update().schema, columnName.split("\\.")) + } + + /** Gets the `DataType` of `columnPath`. */ + def getColumnType(schema: StructType, columnPath: Seq[String]): DataType = { + getColumnField(schema, columnPath).dataType + } + + /** Gets the `DataType` of `columnName`. */ + def getColumnType(tableName: String, columnName: String): DataType = { + getColumnField(tableName, columnName).dataType + } + + /** + * Gets the stats fields from the AddFiles of `snapshot`. The stats are ordered by the + * modification time of the files they are associated with. + */ + def getUnvalidatedStatsOrderByFileModTime(snapshot: Snapshot): Array[JsonNode] = { + snapshot.allFiles + .orderBy("modificationTime") + .collect() + .map(file => new ObjectMapper().readTree(file.stats)) + } + + /** + * Gets the stats fields from the AddFiles of `tableName`. The stats are ordered by the + * modification time of the files they are associated with. + */ + def getUnvalidatedStatsOrderByFileModTime(tableName: String): Array[JsonNode] = + getUnvalidatedStatsOrderByFileModTime(getSnapshot(tableName)) + + /** Gets the physical column path if there is column mapping metadata in the schema. */ + def getPhysicalColumnPath(tableSchema: StructType, columnName: String): Seq[String] = { + new DeltaColumnMappingTestUtilsBase {}.getPhysicalPathForStats( + columnName.split("\\."), tableSchema + ).get + } + + /** Gets the value of a specified field from `stats` JSON node if it exists. */ + def getStatFieldOpt(stats: JsonNode, path: Seq[String]): Option[JsonNode] = + path.foldLeft(Option(stats)) { + case (Some(node), key) if node.has(key) => Option(node.get(key)) + case _ => None + } + + /** Gets the min/max stats of `columnName` from `stats` if they exist. */ + private def getMinMaxStatsOpt( + tableName: String, + stats: JsonNode, + columnName: String): (Option[String], Option[String]) = { + val columnPath = columnName.split('.') + val schema = getSnapshot(tableName).schema + val colType = getColumnType(schema, columnPath) + assert(colType.isInstanceOf[StringType], s"Expected StringType, got $colType") + + val physicalColumnPath = getPhysicalColumnPath(schema, columnName) + val minStatsPath = DeltaStatistics.MIN +: physicalColumnPath + val maxStatsPath = DeltaStatistics.MAX +: physicalColumnPath + ( + getStatFieldOpt(stats, minStatsPath).map(_.asText()), + getStatFieldOpt(stats, maxStatsPath).map(_.asText())) + } + + /** Gets the min/max stats of `columnName` from `stats`. */ + def getMinMaxStats( + tableName: String, + stats: JsonNode, + columnName: String): (String, String) = { + val (minOpt, maxOpt) = getMinMaxStatsOpt(tableName, stats, columnName) + (minOpt.get, maxOpt.get) + } + + /** Verifies whether there are min/max stats of `columnName` in `stats`. */ + def assertMinMaxStatsPresence( + tableName: String, + stats: JsonNode, + columnName: String, + expectStats: Boolean): Unit = { + val (minStats, maxStats) = getMinMaxStatsOpt(tableName, stats, columnName) + assert(minStats.isDefined === expectStats) + assert(maxStats.isDefined === expectStats) + } + + /** Verifies min/max stats values of `columnName` in `stats`. */ + def assertMinMaxStats( + tableName: String, + stats: JsonNode, + columnName: String, + expectedMin: String, + expectedMax: String): Unit = { + val (min, max) = + getMinMaxStats(tableName, stats, columnName) + assert(min === expectedMin, s"Expected $expectedMin, got $min") + assert(max === expectedMax, s"Expected $expectedMax, got $max") + } + + /** Verifies minReaderVersion and minWriterVersion of the protocol. */ + def assertProtocolVersion( + protocol: Protocol, + minReaderVersion: Int, + minWriterVersion: Int): Unit = { + assert(protocol.minReaderVersion === minReaderVersion) + assert(protocol.minWriterVersion === minWriterVersion) + } + + /** Verifies column is of expected data type. */ + def assertColumnDataType( + tableName: String, + columnName: String, + expectedDataType: DataType): Unit = { + assert(getColumnType(tableName, columnName) === expectedDataType) + } + + /** Verifies `columnName` does not exist in `tableName`. */ + def assertColumnNotExist(tableName: String, columnName: String): Unit = { + val e = intercept[AnalysisException] { + sql(s"SELECT $columnName FROM $tableName") + } + assert(e.getMessage.contains(s"`$columnName` cannot be resolved")) + } + + /** + * Runs `select` query on `tableName` with `predicate` and verifies the number of rows returned + * and files read. + */ + def assertSelectQueryResults( + tableName: String, + predicate: String, + numRows: Int, + numFilesRead: Int): Unit = { + val query = sql(s"SELECT * FROM $tableName WHERE $predicate") + assertSelectQueryResults(query, numRows, numFilesRead) + } + + /** + * Runs `query` and verifies the number of rows returned + * and files read. + */ + def assertSelectQueryResults( + query: DataFrame, + numRows: Int, + numFilesRead: Int): Unit = { + assert(query.count() === numRows, s"Expected $numRows rows, got ${query.count()}") + val filesRead = getNumReadFiles(query) + assert(filesRead === numFilesRead, s"Expected $numFilesRead files read, got $filesRead") + } + + /** Returns the number of read files by the query with given query text. */ + def getNumReadFiles(queryText: String): Int = { + getNumReadFiles(sql(queryText)) + } + + /** Returns the number of read files by the given data frame query. */ + def getNumReadFiles(df: DataFrame): Int = { + val deltaScans = df.queryExecution.optimizedPlan.collect { + case DeltaTable(prepared: PreparedDeltaFileIndex) => prepared.preparedScan + } + assert(deltaScans.size == 1) + deltaScans.head.files.length + } + + /** Drops `columnName` from `tableName`. */ + def dropColumn(tableName: String, columnName: String): Unit = { + sql(s"ALTER TABLE $tableName DROP COLUMN $columnName") + assertColumnNotExist(tableName, columnName) + } + + /** Changes `columnName` to `newType` */ + def alterColumnType(tableName: String, columnName: String, newType: String): Unit = { + sql(s"ALTER TABLE $tableName ALTER COLUMN $columnName TYPE $newType") + } + + /** Whether the table protocol supports the given table feature. */ + def isFeatureSupported(tableName: String, tableFeature: TableFeature): Boolean = { + val protocol = getProtocolForTable(tableName) + protocol.isFeatureSupported(tableFeature) + } + + /** Whether the table protocol supports the given table feature. */ + def isFeatureSupported(tableName: String, featureName: String): Boolean = { + val protocol = getProtocolForTable(tableName) + protocol.readerFeatureNames.contains(featureName) || + protocol.writerFeatureNames.contains(featureName) + } + + /** Enables table feature for `tableName` and given `featureName`. */ + def enableTableFeature(tableName: String, featureName: String): Unit = { + sql(s""" + |ALTER TABLE $tableName + |SET TBLPROPERTIES('delta.feature.$featureName' = 'supported') + |""".stripMargin) + assert(isFeatureSupported(tableName, featureName)) + } + + /** Drops table feature for `tableName` and `featureName`. */ + def dropTableFeature(tableName: String, featureName: String): Unit = { + sql(s"ALTER TABLE $tableName DROP FEATURE `$featureName`") + assert(!isFeatureSupported(tableName, featureName)) + } } From c6120062ccef86369c2621d791bc4b62ffcda755 Mon Sep 17 00:00:00 2001 From: Venki Korukanti Date: Thu, 20 Feb 2025 12:21:01 -0800 Subject: [PATCH 5/9] [Kernel] Add table feature manipulation utilities in the Protocol (#4157) ## Description Utilities for * finding the implicitly and explicitly enabled table features * adding a feature to the protocol and creating an upgraded protocol ## How was this patch tested? Unittests. --- .../internal/TransactionBuilderImpl.java | 10 +- .../kernel/internal/actions/Protocol.java | 432 +++++++++++++++++- .../internal/tablefeatures/TableFeatures.java | 39 +- .../internal/actions/ProtocolSuite.scala | 372 +++++++++++++++ .../tablefeatures/TableFeaturesSuite.scala | 6 +- .../kernel/defaults/DomainMetadataSuite.scala | 4 +- .../kernel/defaults/RowTrackingSuite.scala | 4 +- 7 files changed, 830 insertions(+), 37 deletions(-) create mode 100644 kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/ProtocolSuite.scala diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java index 626d7aad109..4a32bb8b26e 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java @@ -155,9 +155,9 @@ public Transaction build(Engine engine) { if (!newWriterFeatures.isEmpty()) { logger.info("Automatically enabling writer features: {}", newWriterFeatures); shouldUpdateProtocol = true; - List oldWriterFeatures = protocol.getWriterFeatures(); + Set oldWriterFeatures = protocol.getWriterFeatures(); protocol = protocol.withNewWriterFeatures(newWriterFeatures); - List curWriterFeatures = protocol.getWriterFeatures(); + Set curWriterFeatures = protocol.getWriterFeatures(); checkArgument(!Objects.equals(oldWriterFeatures, curWriterFeatures)); TableFeatures.validateWriteSupportedTable(protocol, metadata, table.getPath(engine)); } @@ -287,10 +287,6 @@ private Metadata getInitialMetadata() { } private Protocol getInitialProtocol() { - return new Protocol( - DEFAULT_READ_VERSION, - DEFAULT_WRITE_VERSION, - null /* readerFeatures */, - null /* writerFeatures */); + return new Protocol(DEFAULT_READ_VERSION, DEFAULT_WRITE_VERSION); } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java index fa35d020c4b..fc649d075dc 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java @@ -15,10 +15,17 @@ */ package io.delta.kernel.internal.actions; +import static io.delta.kernel.internal.tablefeatures.TableFeatures.TABLE_FEATURES_MIN_WRITER_VERSION; +import static io.delta.kernel.internal.util.Preconditions.checkArgument; import static io.delta.kernel.internal.util.VectorUtils.stringArrayValue; +import static java.lang.String.format; +import static java.util.Collections.emptySet; +import static java.util.Collections.unmodifiableSet; +import static java.util.Objects.requireNonNull; import io.delta.kernel.data.*; import io.delta.kernel.internal.data.GenericRow; +import io.delta.kernel.internal.tablefeatures.TableFeature; import io.delta.kernel.internal.tablefeatures.TableFeatures; import io.delta.kernel.internal.util.Tuple2; import io.delta.kernel.internal.util.VectorUtils; @@ -27,9 +34,15 @@ import io.delta.kernel.types.StringType; import io.delta.kernel.types.StructType; import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class Protocol { + ///////////////////////////////////////////////////////////////////////////////////////////////// + /// Public static variables and methods /// + ///////////////////////////////////////////////////////////////////////////////////////////////// + public static Protocol fromColumnVector(ColumnVector vector, int rowId) { if (vector.isNullAt(rowId)) { return null; @@ -39,11 +52,11 @@ public static Protocol fromColumnVector(ColumnVector vector, int rowId) { vector.getChild(0).getInt(rowId), vector.getChild(1).getInt(rowId), vector.getChild(2).isNullAt(rowId) - ? Collections.emptyList() - : VectorUtils.toJavaList(vector.getChild(2).getArray(rowId)), + ? emptySet() + : new HashSet<>(VectorUtils.toJavaList(vector.getChild(2).getArray(rowId))), vector.getChild(3).isNullAt(rowId) - ? Collections.emptyList() - : VectorUtils.toJavaList(vector.getChild(3).getArray(rowId))); + ? emptySet() + : new HashSet<>(VectorUtils.toJavaList(vector.getChild(3).getArray(rowId)))); } public static final StructType FULL_SCHEMA = @@ -55,18 +68,30 @@ public static Protocol fromColumnVector(ColumnVector vector, int rowId) { private final int minReaderVersion; private final int minWriterVersion; - private final List readerFeatures; - private final List writerFeatures; + private final Set readerFeatures; + private final Set writerFeatures; + + // These are derived fields from minReaderVersion and minWriterVersion + private final boolean supportsReaderFeatures; + private final boolean supportsWriterFeatures; + + public Protocol(int minReaderVersion, int minWriterVersion) { + this(minReaderVersion, minWriterVersion, emptySet(), emptySet()); + } public Protocol( int minReaderVersion, int minWriterVersion, - List readerFeatures, - List writerFeatures) { + Set readerFeatures, + Set writerFeatures) { this.minReaderVersion = minReaderVersion; this.minWriterVersion = minWriterVersion; - this.readerFeatures = readerFeatures; - this.writerFeatures = writerFeatures; + this.readerFeatures = + unmodifiableSet(requireNonNull(readerFeatures, "readerFeatures cannot be null")); + this.writerFeatures = + unmodifiableSet(requireNonNull(writerFeatures, "writerFeatures cannot be null")); + this.supportsReaderFeatures = TableFeatures.supportsReaderFeatures(minReaderVersion); + this.supportsWriterFeatures = TableFeatures.supportsWriterFeatures(minWriterVersion); } public int getMinReaderVersion() { @@ -77,11 +102,11 @@ public int getMinWriterVersion() { return minWriterVersion; } - public List getReaderFeatures() { + public Set getReaderFeatures() { return readerFeatures; } - public List getWriterFeatures() { + public Set getWriterFeatures() { return writerFeatures; } @@ -96,8 +121,26 @@ public String toString() { return sb.toString(); } + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + Protocol protocol = (Protocol) o; + return minReaderVersion == protocol.minReaderVersion + && minWriterVersion == protocol.minWriterVersion + && Objects.equals(readerFeatures, protocol.readerFeatures) + && Objects.equals(writerFeatures, protocol.writerFeatures); + } + + @Override + public int hashCode() { + return Objects.hash(minReaderVersion, minWriterVersion, readerFeatures, writerFeatures); + } + /** - * Encode as a {@link Row} object with the schema {@link Protocol#FULL_SCHEMA}. + * Encode as a {@link Row} object with the schema {@link Protocol#FULL_SCHEMA}. Write any empty + * `readerFeatures` and `writerFeatures` as null. * * @return {@link Row} object with the schema {@link Protocol#FULL_SCHEMA} */ @@ -105,23 +148,378 @@ public Row toRow() { Map protocolMap = new HashMap<>(); protocolMap.put(0, minReaderVersion); protocolMap.put(1, minWriterVersion); - protocolMap.put(2, stringArrayValue(readerFeatures)); - protocolMap.put(3, stringArrayValue(writerFeatures)); + if (supportsReaderFeatures) { + protocolMap.put(2, stringArrayValue(new ArrayList<>(readerFeatures))); + } + if (supportsWriterFeatures) { + protocolMap.put(3, stringArrayValue(new ArrayList<>(writerFeatures))); + } return new GenericRow(Protocol.FULL_SCHEMA, protocolMap); } + ///////////////////////////////////////////////////////////////////////////////////////////////// + /// Public methods related to table features interaction with the protocol /// + ///////////////////////////////////////////////////////////////////////////////////////////////// + /** + * Get the set of features that are implicitly supported by the protocol. Features are implicitly + * supported if the reader and/or writer version is less than the versions that supports the + * explicit features specified in `readerFeatures` and `writerFeatures` sets. Examples: + * + *

+ * + *

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

+ * + *

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

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

Examples: + * + *

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

Examples regarding feature status: + * + *

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

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

For example: + * + *

    + *
  • (1, 7, AppendOnly, Invariants, CheckConstraints) results in (1, 3) + *
  • (3, 7, RowTracking) results in (1, 7, RowTracking) + *
+ */ + public Protocol normalized() { + // Normalization can only be applied to table feature protocols. + if (!isFeatureProtocol()) { + return this; + } + + Tuple2 versions = + TableFeatures.minimumRequiredVersions(getExplicitlySupportedFeatures()); + int minReaderVersion = versions._1; + int minWriterVersion = versions._2; + Protocol newProtocol = new Protocol(minReaderVersion, minWriterVersion); + + if (this.getImplicitlyAndExplicitlySupportedFeatures() + .equals(newProtocol.getImplicitlyAndExplicitlySupportedFeatures())) { + return newProtocol; + } else { + // means we have some feature that is added after table feature support. + // Whatever the feature (reader or readerWriter), it is always going to + // have minWriterVersion as 7. Required minReaderVersion + // should be based on the supported features. + return new Protocol(minReaderVersion, TABLE_FEATURES_MIN_WRITER_VERSION) + .withFeatures(getExplicitlySupportedFeatures()); + } + } + + /** + * Protocol denormalization is the process of converting a legacy protocol to the equivalent table + * features protocol. This is the inverse of protocol normalization. It can be used to allow + * operations on legacy protocols that yield results which cannot be represented anymore by a + * legacy protocol. For example + * + *
    + *
  • (1, 3) results in (1, 7, readerFeatures=[], writerFeatures=[appendOnly, invariants, + * checkConstraints]) + *
  • (2, 5) results in (2, 7, readerFeatures=[], writerFeatures=[appendOnly, invariants, + * checkConstraints, changeDataFeed, generatedColumns, columnMapping]) + *
+ */ + public Protocol denormalized() { + // Denormalization can only be applied to legacy protocols. + if (!isLegacyProtocol()) { + return this; + } + + Tuple2 versions = + TableFeatures.minimumRequiredVersions(getImplicitlySupportedFeatures()); + int minReaderVersion = versions._1; + + return new Protocol(minReaderVersion, TABLE_FEATURES_MIN_WRITER_VERSION) + .withFeatures(getImplicitlySupportedFeatures()); + } + + /** + * Helper method that applies both denormalization and normalization. This can be used to + * normalize invalid legacy protocols such as (2, 3), (1, 5). A legacy protocol is invalid when + * the version numbers are higher than required to support the implied feature set. + */ + public Protocol denormalizedNormalized() { + return this.denormalized().normalized(); + } + + /** + * Merge this protocol with multiple `protocols` to have the highest reader and writer versions + * plus all explicitly and implicitly supported features. + */ + public Protocol merge(Protocol... others) { + List protocols = new ArrayList<>(); + protocols.add(this); + protocols.addAll(Arrays.asList(others)); + + int mergedReaderVersion = + protocols.stream().mapToInt(Protocol::getMinReaderVersion).max().orElse(0); + + int mergedWriterVersion = + protocols.stream().mapToInt(Protocol::getMinWriterVersion).max().orElse(0); + + Set mergedReaderFeatures = + protocols.stream().flatMap(p -> p.readerFeatures.stream()).collect(Collectors.toSet()); + + Set mergedWriterFeatures = + protocols.stream().flatMap(p -> p.writerFeatures.stream()).collect(Collectors.toSet()); + + Set mergedImplicitFeatures = + protocols.stream() + .flatMap(p -> p.getImplicitlySupportedFeatures().stream()) + .collect(Collectors.toSet()); + + Protocol mergedProtocol = + new Protocol( + mergedReaderVersion, + mergedWriterVersion, + mergedReaderFeatures, + mergedWriterFeatures) + .withFeatures(mergedImplicitFeatures); + + // The merged protocol is always normalized in order to represent the protocol + // with the weakest possible form. This enables backward compatibility. + // This is preceded by a denormalization step. This allows to fix invalid legacy Protocols. + // For example, (2, 3) is normalized to (1, 3). This is because there is no legacy feature + // in the set with reader version 2 unless the writer version is at least 5. + return mergedProtocol.denormalizedNormalized(); + } + + ///////////////////////////////////////////////////////////////////////////////////////////////// + /// Legacy method which will be removed after the table feature integration is done /// + ///////////////////////////////////////////////////////////////////////////////////////////////// public Protocol withNewWriterFeatures(Set writerFeatures) { Tuple2 newProtocolVersions = TableFeatures.minProtocolVersionFromAutomaticallyEnabledFeatures(writerFeatures); - List newWriterFeatures = new ArrayList<>(writerFeatures); + Set newWriterFeatures = new HashSet<>(writerFeatures); if (this.writerFeatures != null) { newWriterFeatures.addAll(this.writerFeatures); } return new Protocol( newProtocolVersions._1, newProtocolVersions._2, - this.readerFeatures == null ? null : new ArrayList<>(this.readerFeatures), + this.readerFeatures == null ? null : new HashSet<>(this.readerFeatures), newWriterFeatures); } + + /** Validate the protocol contents represents a valid state */ + protected void validate() { + checkArgument(minReaderVersion >= 1, "minReaderVersion should be at least 1"); + checkArgument(minWriterVersion >= 1, "minWriterVersion should be at least 1"); + + // expect the reader and writer features to be empty if the protocol version does not support + checkArgument( + readerFeatures.isEmpty() || supportsReaderFeatures, + "Reader features are not supported for the reader version: " + minReaderVersion); + checkArgument( + writerFeatures.isEmpty() || supportsWriterFeatures, + "Writer features are not supported for the writer version: " + minWriterVersion); + + // If reader versions are supported, expect the writer versions to be supported as well + // We don't have any reader only features. + if (supportsReaderFeatures) { + checkArgument( + supportsWriterFeatures, + "writer version doesn't support writer features: " + minWriterVersion); + } + + if (supportsWriterFeatures) { + // ensure that the reader version supports all the readerWriter features + Set supportedFeatures = getExplicitlySupportedFeatures(); + supportedFeatures.stream() + .filter(TableFeature::isReaderWriterFeature) + .forEach( + feature -> { + checkArgument( + feature.minReaderVersion() <= minReaderVersion, + format( + "Reader version %d does not support readerWriter feature %s", + minReaderVersion, feature.featureName())); + + if (supportsReaderFeatures) { + // if the protocol supports reader features, then it should be part of the + // readerFeatures + checkArgument( + readerFeatures.contains(feature.featureName()), + format( + "ReaderWriter feature %s is not present in readerFeatures", + feature.featureName())); + } + }); + } else { + // ensure we don't get (minReaderVersion, minWriterVersion) that satisfy the readerWriter + // feature version requirements. E.g. (1, 5) is invalid as writer version indicates + // columnMapping supported but reader version does not support it (requires 2). + TableFeatures.TABLE_FEATURES.stream() + .filter(TableFeature::isReaderWriterFeature) + .forEach( + f -> { + if (f.minWriterVersion() <= minWriterVersion) { + checkArgument( + f.minReaderVersion() <= minReaderVersion, + format( + "Reader version %d does not support readerWriter feature %s", + minReaderVersion, f.featureName())); + } + }); + } + } + + /** is the protocol a legacy protocol, i.e before (3, 7) */ + private boolean isLegacyProtocol() { + return !supportsReaderFeatures && !supportsWriterFeatures; + } + + /** is the protocol a table feature protocol, i.e after (3, 7) */ + private boolean isFeatureProtocol() { + // checking for writer feature support is enough as we have + // writerOnly or readerWriter features, but no readerOnly features. + return supportsWriterFeatures; + } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java index 314df4d7dca..8f5e39832b0 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java @@ -342,6 +342,15 @@ private static class VacuumProtocolCheckTableFeature extends TableFeature.Reader /// END: Define the {@link TableFeature}s /// ///////////////////////////////////////////////////////////////////////////////// + ///////////////////////////////////////////////////////////////////////////////// + /// Public static variables and methods /// + ///////////////////////////////////////////////////////////////////////////////// + /** Min reader version that supports reader features. */ + public static final int TABLE_FEATURES_MIN_READER_VERSION = 3; + + /** Min reader version that supports writer features. */ + public static final int TABLE_FEATURES_MIN_WRITER_VERSION = 7; + public static final List TABLE_FEATURES = Collections.unmodifiableList( Arrays.asList( @@ -384,8 +393,29 @@ public static TableFeature getTableFeature(String featureName) { return tableFeature; } + /** Does reader version supports explicitly specifying reader feature set in protocol? */ + public static boolean supportsReaderFeatures(int minReaderVersion) { + return minReaderVersion >= TABLE_FEATURES_MIN_READER_VERSION; + } + + /** Does writer version supports explicitly specifying writer feature set in protocol? */ + public static boolean supportsWriterFeatures(int minWriterVersion) { + return minWriterVersion >= TABLE_FEATURES_MIN_WRITER_VERSION; + } + + /** Returns the minimum reader/writer versions required to support all provided features. */ + public static Tuple2 minimumRequiredVersions(Set features) { + int minReaderVersion = + features.stream().mapToInt(TableFeature::minReaderVersion).max().orElse(0); + + int minWriterVersion = + features.stream().mapToInt(TableFeature::minWriterVersion).max().orElse(0); + + return new Tuple2<>(Math.max(minReaderVersion, 1), Math.max(minWriterVersion, 1)); + } + ///////////////////////////////////////////////////////////////////////////////// - /// Everything below will be removed once the Kernel upgrades to use the /// + /// Everything below will be removed once the Kernel upgrades to use the /// /// above interfaces. /// ///////////////////////////////////////////////////////////////////////////////// private static final Set SUPPORTED_WRITER_FEATURES = @@ -424,9 +454,6 @@ public static TableFeature getTableFeature(String featureName) { public static final String INVARIANTS_FEATURE_NAME = "invariants"; - /** The minimum writer version required to support table features. */ - public static final int TABLE_FEATURES_MIN_WRITER_VERSION = 7; - //////////////////// // Helper Methods // //////////////////// @@ -438,7 +465,7 @@ public static void validateReadSupportedTable(Protocol protocol, String tablePat case 2: break; case 3: - List readerFeatures = protocol.getReaderFeatures(); + Set readerFeatures = protocol.getReaderFeatures(); if (!SUPPORTED_READER_FEATURES.containsAll(readerFeatures)) { Set unsupportedFeatures = new HashSet<>(readerFeatures); unsupportedFeatures.removeAll(SUPPORTED_READER_FEATURES); @@ -620,7 +647,7 @@ private static boolean metadataRequiresWriterFeatureToBeEnabled( } private static boolean isWriterFeatureSupported(Protocol protocol, String featureName) { - List writerFeatures = protocol.getWriterFeatures(); + Set writerFeatures = protocol.getWriterFeatures(); if (writerFeatures == null) { return false; } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/ProtocolSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/ProtocolSuite.scala new file mode 100644 index 00000000000..3c9768f091b --- /dev/null +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/actions/ProtocolSuite.scala @@ -0,0 +1,372 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.actions + +import scala.collection.JavaConverters._ + +import io.delta.kernel.internal.tablefeatures.TableFeatures + +import org.scalatest.funsuite.AnyFunSuite + +class ProtocolSuite extends AnyFunSuite { + ///////////////////////////////////////////////////////////////////////////////////////////////// + // Tests for TableFeature related methods on Protocol // + ///////////////////////////////////////////////////////////////////////////////////////////////// + + // Invalid protocol versions/features throw validation errors + Seq( + // Test format: + // minReaderVersion, minWriterVersion, readerFeatures, writerFeatures, expectedErrorMsg + (0, 1, Set(), Set(), "minReaderVersion should be at least 1"), + (1, 0, Set(), Set(), "minWriterVersion should be at least 1"), + ( + // writer version doesn't support writer features + 1, + 2, + Set("columnMapping"), + Set(), + "Reader features are not supported for the reader version: 1"), + ( + // writer version doesn't support writer features + 1, + 2, + Set(), + Set("columnMapping"), + "Writer features are not supported for the writer version: 2"), + // you can't have reader version with feature support, but not the writer version + (3, 5, Set(), Set(), "writer version doesn't support writer features: 5"), + // columnMapping feature is not supported for reader version 1 + (1, 5, Set(), Set(), "Reader version 1 does not support readerWriter feature columnMapping"), + ( + // readerWriter feature columnMapping is missing from the readerFeatures set + 3, + 7, + Set(), + Set("columnMapping"), + "ReaderWriter feature columnMapping is not present in readerFeatures"), + // minReaderVersion doesn't support readerWriter feature columnMapping requirement + ( + 1, + 7, + Set(), + Set("columnMapping"), + "Reader version 1 does not support readerWriter feature columnMapping")).foreach { + case ( + readerVersion, + writerVersion, + readerFeatures: Set[String], + writerFeatures: Set[String], + expectedError) => + test(s"Invalid protocol versions " + + s"($readerVersion, $writerVersion, $readerFeatures, $writerFeatures)") { + val protocol = + new Protocol(readerVersion, writerVersion, readerFeatures.asJava, writerFeatures.asJava) + val e = intercept[IllegalArgumentException] { + protocol.validate() + } + assert(e.getMessage === expectedError) + } + } + + // Tests for getImplicitlySupportedFeatures, getExplicitlySupportedFeatures and + // getImplicitlyAndExplicitlySupportedFeatures + Seq( + // Test format: + // (minReaderVersion, minWriterVersion, expected features) + (1, 1, Set()), + (1, 2, Set("appendOnly", "invariants")), + (1, 3, Set("appendOnly", "invariants", "checkConstraints")), + ( + 1, + 4, + Set("appendOnly", "invariants", "checkConstraints", "changeDataFeed", "generatedColumns")), + ( + 2, + 5, + Set( + "appendOnly", + "invariants", + "checkConstraints", + "changeDataFeed", + "generatedColumns", + "columnMapping")), + ( + 2, + 6, + Set( + "appendOnly", + "invariants", + "checkConstraints", + "changeDataFeed", + "generatedColumns", + "columnMapping", + "identityColumns"))).foreach { + case (minReaderVersion, minWriterVersion, expectedFeatures) => + test(s"getImplicitlySupportedFeatures with minReaderVersion $minReaderVersion and " + + s"minWriterVersion $minWriterVersion") { + val protocol = new Protocol(minReaderVersion, minWriterVersion) + assert( + protocol.getImplicitlySupportedFeatures.asScala.map(_.featureName()) === expectedFeatures) + + assert( + protocol.getImplicitlyAndExplicitlySupportedFeatures.asScala.map(_.featureName()) === + expectedFeatures) + + assert( + protocol.getExplicitlySupportedFeatures.asScala.map(_.featureName()) === Set()) + } + } + + Seq( + // Test format: readerFeatures, writerFeatures, expected set + (Set(), Set(), Set()), + (Set(), Set("rowTracking"), Set("rowTracking")), + (Set(), Set("checkConstraints", "rowTracking"), Set("checkConstraints", "rowTracking")), + ( + Set("columnMapping"), + Set("columnMapping", "domainMetadata"), + Set("columnMapping", "domainMetadata"))).foreach { + case ( + readerFeatures: Set[String], + writerFeatures: Set[String], + expectedFeatureSet: Set[String]) => + test(s"getExplicitlySupportedFeatures $readerFeatures $writerFeatures") { + val protocol = new Protocol(3, 7, readerFeatures.asJava, writerFeatures.asJava) + assert( + protocol.getExplicitlySupportedFeatures.asScala.map(_.featureName()) === + expectedFeatureSet) + + assert( + protocol.getImplicitlyAndExplicitlySupportedFeatures.asScala.map(_.featureName()) === + expectedFeatureSet) + + assert(protocol.getImplicitlySupportedFeatures.asScala.map(_.featureName()) === Set()) + } + } + + // Tests for `normalized + Seq( + // Test format: input, expected output out of the `normalized` + + // If the protocol has no table features, then the normalized shouldn't change + (1, 1, Set[String](), Set[String]()) -> (1, 1, Set[String](), Set[String]()), + (1, 2, Set[String](), Set[String]()) -> (1, 2, Set[String](), Set[String]()), + (2, 5, Set[String](), Set[String]()) -> (2, 5, Set[String](), Set[String]()), + + // If the protocol has table features, then the normalized may or + // may not have the table features + (3, 7, Set[String](), Set("appendOnly", "invariants")) -> + (1, 2, Set[String](), Set[String]()), + (3, 7, Set[String](), Set("appendOnly", "invariants", "checkConstraints")) -> + (1, 3, Set[String](), Set[String]()), + ( + 3, + 7, + Set[String](), + Set("appendOnly", "invariants", "checkConstraints", "changeDataFeed", "generatedColumns")) -> + (1, 4, Set[String](), Set[String]()), + ( + 3, + 7, + Set("columnMapping"), + Set( + "appendOnly", + "invariants", + "checkConstraints", + "changeDataFeed", + "generatedColumns", + "columnMapping")) -> + (2, 5, Set[String](), Set[String]()), + + // reader version is downgraded + // can't downgrade the writer version, because version 2 (appendOnly) also has support for + // invariants which is not supported in the writer features in the input + (1, 7, Set[String](), Set("appendOnly")) -> (1, 7, Set[String](), Set[String]("appendOnly")), + (3, 7, Set("columnMapping"), Set("columnMapping")) -> + (2, 7, Set[String](), Set("columnMapping")), + (3, 7, Set("columnMapping"), Set("columnMapping", "domainMetadata")) -> + (2, 7, Set[String](), Set("columnMapping", "domainMetadata"))).foreach { + case ( + (readerVersion, writerVersion, readerFeatures, writerFeatures), + ( + expReaderVersion, + expWriterVersion, + expReaderFeatures, + expWriterFeatures)) => + test(s"normalized $readerVersion $writerVersion $readerFeatures $writerFeatures") { + val protocol = + new Protocol(readerVersion, writerVersion, readerFeatures.asJava, writerFeatures.asJava) + val normalized = protocol.normalized() + assert(normalized.getMinReaderVersion === expReaderVersion) + assert(normalized.getMinWriterVersion === expWriterVersion) + assert(normalized.getReaderFeatures.asScala === expReaderFeatures) + assert(normalized.getWriterFeatures.asScala === expWriterFeatures) + } + } + + // Tests for `denormalized` + Seq( + // Test format: input, expected output out of the `denormalized` + (1, 1, Set[String](), Set[String]()) -> (1, 7, Set[String](), Set[String]()), + (1, 2, Set[String](), Set[String]()) -> (1, 7, Set[String](), Set("appendOnly", "invariants")), + (2, 5, Set[String](), Set[String]()) -> ( + 2, + 7, + Set[String](), + Set( + "appendOnly", + "invariants", + "checkConstraints", + "changeDataFeed", + "generatedColumns", + "columnMapping")), + + // invalid protocol versions (2, 3) + (2, 3, Set[String](), Set[String]()) -> ( + 1, + 7, + Set[String](), + Set("appendOnly", "invariants", "checkConstraints")), + + // shouldn't change the protocol already has the table feature set support + (3, 7, Set[String](), Set("appendOnly", "invariants")) -> + (3, 7, Set[String](), Set("appendOnly", "invariants")), + (3, 7, Set[String](), Set("appendOnly", "invariants", "checkConstraints")) -> + (3, 7, Set[String](), Set("appendOnly", "invariants", "checkConstraints"))).foreach { + case ( + (readerVersion, writerVersion, readerFeatures, writerFeatures), + ( + expReaderVersion, + expWriterVersion, + expReaderFeatures, + expWriterFeatures)) => + test(s"denormalized $readerVersion $writerVersion $readerFeatures $writerFeatures") { + val protocol = + new Protocol(readerVersion, writerVersion, readerFeatures.asJava, writerFeatures.asJava) + val denormalized = protocol.denormalized() + assert(denormalized.getMinReaderVersion === expReaderVersion) + assert(denormalized.getMinWriterVersion === expWriterVersion) + assert(denormalized.getReaderFeatures.asScala === expReaderFeatures) + assert(denormalized.getWriterFeatures.asScala === expWriterFeatures) + } + } + + // Tests for `withFeature` and `normalized` + Seq( + // can't downgrade the writer version, because version 2 (appendOnly) also has support for + // invariants which is not supported in the writer features in the input + Set("appendOnly") -> (1, 7, Set[String](), Set("appendOnly")), + Set("invariants") -> (1, 7, Set[String](), Set[String]("invariants")), + Set("appendOnly", "invariants") -> (1, 2, Set[String](), Set[String]()), + Set("checkConstraints") -> (1, 7, Set[String](), Set("checkConstraints")), + Set("changeDataFeed") -> (1, 7, Set[String](), Set("changeDataFeed")), + Set("appendOnly", "invariants", "checkConstraints") -> (1, 3, Set[String](), Set[String]()), + Set("generatedColumns") -> (1, 7, Set[String](), Set("generatedColumns")), + Set("columnMapping") -> (2, 7, Set(), Set("columnMapping")), + Set("identityColumns") -> (1, 7, Set[String](), Set[String]("identityColumns")), + + // expect the dependency features also to be supported + Set("icebergCompatV2") -> + (2, 7, Set[String](), Set[String]("icebergCompatV2", "columnMapping")), + Set("rowTracking") -> ( + 1, + 7, + Set[String](), + Set[String]("rowTracking", "domainMetadata"))).foreach { + case (features, (expReaderVersion, expWriterVersion, expReaderFeatures, expWriterFeatures)) => + test(s"withFeature $features") { + val protocol = new Protocol(3, 7) + val updated = protocol + .withFeatures(features.map(TableFeatures.getTableFeature).asJava) + .normalized() + assert(updated.getMinReaderVersion === expReaderVersion) + assert(updated.getMinWriterVersion === expWriterVersion) + assert(updated.getReaderFeatures.asScala === expReaderFeatures) + assert(updated.getWriterFeatures.asScala === expWriterFeatures) + } + } + + test("withFeature - can't add a feature at the current version") { + val protocol = new Protocol(1, 2) + val e = intercept[UnsupportedOperationException] { + protocol.withFeatures(Set(TableFeatures.getTableFeature("columnMapping")).asJava) + } + assert(e.getMessage === "TableFeature requires higher reader protocol version") + } + + // Tests for `merge` (also tests denormalized and normalized) + Seq( + // Test format: (protocol1, protocol2) -> expected merged protocol + ( + (1, 1, Set[String](), Set[String]()), + (1, 2, Set[String](), Set[String]())) -> + (1, 2, Set[String](), Set[String]()), + ((1, 2, Set[String](), Set[String]()), (1, 3, Set[String](), Set[String]())) -> + (1, 3, Set[String](), Set[String]()), + ((1, 4, Set[String](), Set[String]()), (2, 5, Set[String](), Set[String]())) -> + (2, 5, Set[String](), Set[String]()), + ((1, 4, Set[String](), Set[String]()), (2, 6, Set[String](), Set[String]())) -> + (2, 6, Set[String](), Set[String]()), + ((1, 2, Set[String](), Set[String]()), (1, 7, Set[String](), Set("invariants"))) -> + (1, 2, Set[String](), Set[String]()), + ((1, 2, Set[String](), Set[String]()), (3, 7, Set("columnMapping"), Set("columnMapping"))) -> + (2, 7, Set[String](), Set("columnMapping", "invariants", "appendOnly")), + ( + (1, 2, Set[String](), Set[String]()), + (3, 7, Set("columnMapping"), Set("columnMapping", "domainMetadata"))) -> + (2, 7, Set[String](), Set("domainMetadata", "columnMapping", "invariants", "appendOnly")), + ( + (2, 5, Set[String](), Set[String]()), + (3, 7, Set("v2Checkpoint"), Set("v2Checkpoint", "domainMetadata"))) -> + ( + 3, + 7, + Set("columnMapping", "v2Checkpoint"), + Set( + "domainMetadata", + "columnMapping", + "v2Checkpoint", + "invariants", + "appendOnly", + "checkConstraints", + "changeDataFeed", + "generatedColumns"))).foreach({ + case ( + ( + (readerVersion1, writerVersion1, readerFeatures1, writerFeatures1), + (readerVersion2, writerVersion2, readerFeatures2, writerFeatures2)), + (expReaderVersion, expWriterVersion, expReaderFeatures, expWriterFeatures)) => + test(s"merge $readerVersion1 $writerVersion1 $readerFeatures1 $writerFeatures1 " + + s"$readerVersion2 $writerVersion2 $readerFeatures2 $writerFeatures2") { + val protocol1 = + new Protocol( + readerVersion1, + writerVersion1, + readerFeatures1.asJava, + writerFeatures1.asJava) + val protocol2 = new Protocol( + readerVersion2, + writerVersion2, + readerFeatures2.asJava, + writerFeatures2.asJava) + val merged = protocol1.merge(protocol2) + assert(merged.getMinReaderVersion === expReaderVersion) + assert(merged.getMinWriterVersion === expWriterVersion) + assert(merged.getReaderFeatures.asScala === expReaderFeatures) + assert(merged.getWriterFeatures.asScala === expWriterFeatures) + } + }) +} diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala index 1421c340889..5e7a88a2dbd 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/tablefeatures/TableFeaturesSuite.scala @@ -96,7 +96,7 @@ class TableFeaturesSuite extends AnyFunSuite { TableFeatures.TABLE_FEATURES.size() == readerWriterFeatures.size + writerOnlyFeatures.size) } - val testProtocol = new Protocol(1, 2, Collections.emptyList(), Collections.emptyList()) + val testProtocol = new Protocol(1, 2, Collections.emptySet(), Collections.emptySet()) Seq( // Test feature, metadata, expected result ("appendOnly", testMetadata(tblProps = Map("delta.appendOnly" -> "true")), true), @@ -335,8 +335,8 @@ class TableFeaturesSuite extends AnyFunSuite { 0, minWriterVersion, // reader features - it doesn't matter as the read fails anyway before the writer check - Collections.emptyList(), - writerFeatures.toSeq.asJava) + Collections.emptySet(), + writerFeatures.toSet.asJava) } def testMetadata( diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala index e6f84bc0f1b..b54eb11e95e 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala @@ -86,8 +86,8 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase val protocol = new Protocol( 3, // minReaderVersion 7, // minWriterVersion - Collections.emptyList(), // readerFeatures - Seq("domainMetadata").asJava // writerFeatures + Collections.emptySet(), // readerFeatures + Set("domainMetadata").asJava // writerFeatures ) val protocolAction = SingleAction.createProtocolSingleAction(protocol.toRow) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala index de81f7935fd..fc669338fcc 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala @@ -54,8 +54,8 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val protocol = new Protocol( 3, // minReaderVersion 7, // minWriterVersion - Collections.emptyList(), // readerFeatures - writerFeatures.asJava // writerFeatures + Collections.emptySet(), // readerFeatures + writerFeatures.toSet.asJava // writerFeatures ) val protocolAction = SingleAction.createProtocolSingleAction(protocol.toRow) val txn = createTxn(engine, tablePath, isNewTable = false, schema, Seq.empty) From 8045052d2aed4b2f6aeb1627bd40de90c0dca386 Mon Sep 17 00:00:00 2001 From: Fred Storage Liu Date: Thu, 20 Feb 2025 13:42:16 -0800 Subject: [PATCH 6/9] Refactor remove file handling in InMemoryLogReplay (#4180) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description Refactor remove file hashset in InMemoryLogReplay into activeRemoveFiles and cancelledRemoveFiles, because they are semantically different that the later had cancelled an AddFile, while the former did not. ## How was this patch tested? existing UT. ## Does this PR introduce _any_ user-facing changes? --- .../sql/delta/actions/InMemoryLogReplay.scala | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/actions/InMemoryLogReplay.scala b/spark/src/main/scala/org/apache/spark/sql/delta/actions/InMemoryLogReplay.scala index 99d51324b5f..0352272290a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/actions/InMemoryLogReplay.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/actions/InMemoryLogReplay.scala @@ -47,7 +47,12 @@ class InMemoryLogReplay( private val transactions = new scala.collection.mutable.HashMap[String, SetTransaction]() private val domainMetadatas = collection.mutable.Map.empty[String, DomainMetadata] private val activeFiles = new scala.collection.mutable.HashMap[UniqueFileActionTuple, AddFile]() - private val tombstones = new scala.collection.mutable.HashMap[UniqueFileActionTuple, RemoveFile]() + // RemoveFiles that had cancelled AddFile during replay + private val cancelledRemoveFiles = + new scala.collection.mutable.HashMap[UniqueFileActionTuple, RemoveFile]() + // RemoveFiles that had NOT cancelled any AddFile during replay + private val activeRemoveFiles = + new scala.collection.mutable.HashMap[UniqueFileActionTuple, RemoveFile]() override def append(version: Long, actions: Iterator[Action]): Unit = { assert(currentVersion == -1 || version == currentVersion + 1, @@ -69,11 +74,15 @@ class InMemoryLogReplay( val uniquePath = UniqueFileActionTuple(add.pathAsUri, add.getDeletionVectorUniqueId) activeFiles(uniquePath) = add.copy(dataChange = false) // Remove the tombstone to make sure we only output one `FileAction`. - tombstones.remove(uniquePath) + cancelledRemoveFiles.remove(uniquePath) + // Remove from activeRemoveFiles to handle commits that add a previously-removed file + activeRemoveFiles.remove(uniquePath) case remove: RemoveFile => val uniquePath = UniqueFileActionTuple(remove.pathAsUri, remove.getDeletionVectorUniqueId) - activeFiles.remove(uniquePath) - tombstones(uniquePath) = remove.copy(dataChange = false) + activeFiles.remove(uniquePath) match { + case Some(_) => cancelledRemoveFiles(uniquePath) = remove + case None => activeRemoveFiles(uniquePath) = remove + } case _: CommitInfo => // do nothing case _: AddCDCFile => // do nothing case null => // Some crazy future feature. Ignore @@ -81,7 +90,9 @@ class InMemoryLogReplay( } private def getTombstones: Iterable[FileAction] = { - tombstones.values.filter(_.delTimestamp > minFileRetentionTimestamp) + (cancelledRemoveFiles.values ++ activeRemoveFiles.values) + .filter(_.delTimestamp > minFileRetentionTimestamp) + .map(_.copy(dataChange = false)) } private[delta] def getTransactions: Iterable[SetTransaction] = { From 7e85686e79ed519beb494b55de0a9bb982d32a8e Mon Sep 17 00:00:00 2001 From: Xin Huang <42597328+huan233usc@users.noreply.github.com> Date: Thu, 20 Feb 2025 15:06:40 -0800 Subject: [PATCH 7/9] [Kernel]Add simple crc post commit for incremental crc writing. (#4134) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description This PR introduces a new post commit hook - ChecksumSimple, for writing CRC file after txn commit. CRC file will only be written only commit version - 1's snapshot reads CRC during state construction Other case will be handled in a separate PR ## How was this patch tested? E2e test ## Does this PR introduce _any_ user-facing changes? No --- .../io/delta/kernel/hook/PostCommitHook.java | 9 +- .../delta/kernel/internal/SnapshotImpl.java | 2 +- .../kernel/internal/TransactionImpl.java | 38 ++++ .../kernel/internal/checksum/CRCInfo.java | 164 +++++++++++++++ .../{replay => checksum}/ChecksumReader.java | 4 +- .../internal/checksum/ChecksumWriter.java | 58 ++++++ .../internal/hook/ChecksumSimpleHook.java | 53 +++++ .../delta/kernel/internal/replay/CRCInfo.java | 74 ------- .../kernel/internal/replay/LogReplay.java | 2 + .../internal/snapshot/SnapshotHint.java | 2 +- .../checksum/ChecksumWriterSuite.scala | 197 ++++++++++++++++++ .../ChecksumSimpleComparisonSuite.scala | 173 +++++++++++++++ .../defaults/DeltaTableWriteSuiteBase.scala | 21 +- .../DeltaTableWriteWithCrcSuite.scala | 67 ++++++ .../defaults/DeltaTableWritesSuite.scala | 26 ++- 15 files changed, 799 insertions(+), 91 deletions(-) create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/CRCInfo.java rename kernel/kernel-api/src/main/java/io/delta/kernel/internal/{replay => checksum}/ChecksumReader.java (98%) create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumWriter.java create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/internal/hook/ChecksumSimpleHook.java delete mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CRCInfo.java create mode 100644 kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala create mode 100644 kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala create mode 100644 kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteWithCrcSuite.scala diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/hook/PostCommitHook.java b/kernel/kernel-api/src/main/java/io/delta/kernel/hook/PostCommitHook.java index 173168ff1a0..50162b065fd 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/hook/PostCommitHook.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/hook/PostCommitHook.java @@ -33,7 +33,14 @@ enum PostCommitHookType { * perform this operation, reading previous checkpoint + logs is required to construct a new * checkpoint, with latency scaling based on log size (typically seconds to minutes). */ - CHECKPOINT + CHECKPOINT, + /** + * Writes a checksum file at the version committed by the transaction. This hook is present when + * all required table statistics (e.g. table size) for checksum file are known when a + * transaction commits. This operation has a minimal latency with no requirement of reading + * previous checkpoint or logs. + */ + CHECKSUM_SIMPLE } /** Invokes the post commit operation whose implementation must be thread safe. */ diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java index be0497d3e41..77b51352b29 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java @@ -25,10 +25,10 @@ import io.delta.kernel.internal.actions.DomainMetadata; import io.delta.kernel.internal.actions.Metadata; import io.delta.kernel.internal.actions.Protocol; +import io.delta.kernel.internal.checksum.CRCInfo; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.metrics.SnapshotQueryContext; import io.delta.kernel.internal.metrics.SnapshotReportImpl; -import io.delta.kernel.internal.replay.CRCInfo; import io.delta.kernel.internal.replay.CreateCheckpointIterator; import io.delta.kernel.internal.replay.LogReplay; import io.delta.kernel.internal.snapshot.LogSegment; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java index 1dba7458002..e950126acb6 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java @@ -29,9 +29,11 @@ import io.delta.kernel.expressions.Column; import io.delta.kernel.hook.PostCommitHook; import io.delta.kernel.internal.actions.*; +import io.delta.kernel.internal.checksum.CRCInfo; import io.delta.kernel.internal.data.TransactionStateRow; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.hook.CheckpointHook; +import io.delta.kernel.internal.hook.ChecksumSimpleHook; import io.delta.kernel.internal.metrics.TransactionMetrics; import io.delta.kernel.internal.metrics.TransactionReportImpl; import io.delta.kernel.internal.replay.ConflictChecker; @@ -39,6 +41,7 @@ import io.delta.kernel.internal.rowtracking.RowTracking; import io.delta.kernel.internal.tablefeatures.TableFeatures; import io.delta.kernel.internal.util.*; +import io.delta.kernel.metrics.TransactionMetricsResult; import io.delta.kernel.metrics.TransactionReport; import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterable; @@ -342,6 +345,7 @@ private TransactionCommitResult doCommit( dataAndMetadataActions.map( action -> { transactionMetrics.totalActionsCounter.increment(); + // TODO: handle RemoveFiles. if (!action.isNullAt(ADD_FILE_ORDINAL)) { transactionMetrics.addFilesCounter.increment(); transactionMetrics.addFilesSizeInBytesCounter.increment( @@ -362,6 +366,10 @@ private TransactionCommitResult doCommit( postCommitHooks.add(new CheckpointHook(dataPath, commitAsVersion)); } + buildPostCommitCrcInfoIfCurrentCrcAvailable( + commitAsVersion, transactionMetrics.captureTransactionMetricsResult()) + .ifPresent(crcInfo -> postCommitHooks.add(new ChecksumSimpleHook(crcInfo, logPath))); + return new TransactionCommitResult(commitAsVersion, postCommitHooks); } catch (FileAlreadyExistsException e) { throw e; @@ -437,6 +445,36 @@ private void recordTransactionReport( engine.getMetricsReporters().forEach(reporter -> reporter.report(transactionReport)); } + private Optional buildPostCommitCrcInfoIfCurrentCrcAvailable( + long commitAtVersion, TransactionMetricsResult metricsResult) { + if (isNewTable) { + return Optional.of( + new CRCInfo( + commitAtVersion, + metadata, + protocol, + metricsResult.getTotalAddFilesSizeInBytes(), + metricsResult.getNumAddFiles(), + Optional.of(txnId.toString()))); + } + + return readSnapshot + .getCurrentCrcInfo() + // in the case of a conflicting txn and successful retry the readSnapshot may not be + // commitVersion - 1 + .filter(lastCrcInfo -> commitAtVersion == lastCrcInfo.getVersion() + 1) + .map( + lastCrcInfo -> + new CRCInfo( + commitAtVersion, + metadata, + protocol, + // TODO: handle RemoveFiles for calculating table size and num of files. + lastCrcInfo.getTableSizeBytes() + metricsResult.getTotalAddFilesSizeInBytes(), + lastCrcInfo.getNumFiles() + metricsResult.getNumAddFiles(), + Optional.of(txnId.toString()))); + } + /** * Get the part of the schema of the table that needs the statistics to be collected per file. * diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/CRCInfo.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/CRCInfo.java new file mode 100644 index 00000000000..b4d72dc89ad --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/CRCInfo.java @@ -0,0 +1,164 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.checksum; + +import static io.delta.kernel.internal.util.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +import io.delta.kernel.data.ColumnVector; +import io.delta.kernel.data.ColumnarBatch; +import io.delta.kernel.data.Row; +import io.delta.kernel.internal.actions.Metadata; +import io.delta.kernel.internal.actions.Protocol; +import io.delta.kernel.internal.data.GenericRow; +import io.delta.kernel.internal.util.InternalUtils; +import io.delta.kernel.types.LongType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructType; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CRCInfo { + private static final Logger logger = LoggerFactory.getLogger(CRCInfo.class); + + // Constants for schema field names + private static final String TABLE_SIZE_BYTES = "tableSizeBytes"; + private static final String NUM_FILES = "numFiles"; + private static final String NUM_METADATA = "numMetadata"; + private static final String NUM_PROTOCOL = "numProtocol"; + private static final String METADATA = "metadata"; + private static final String PROTOCOL = "protocol"; + private static final String TXN_ID = "txnId"; + + public static final StructType CRC_FILE_SCHEMA = + new StructType() + .add(TABLE_SIZE_BYTES, LongType.LONG) + .add(NUM_FILES, LongType.LONG) + .add(NUM_METADATA, LongType.LONG) + .add(NUM_PROTOCOL, LongType.LONG) + .add(METADATA, Metadata.FULL_SCHEMA) + .add(PROTOCOL, Protocol.FULL_SCHEMA) + .add(TXN_ID, StringType.STRING, /*nullable*/ true); + + public static Optional fromColumnarBatch( + long version, ColumnarBatch batch, int rowId, String crcFilePath) { + // Read required fields. + Protocol protocol = + Protocol.fromColumnVector(batch.getColumnVector(getSchemaIndex(PROTOCOL)), rowId); + Metadata metadata = + Metadata.fromColumnVector(batch.getColumnVector(getSchemaIndex(METADATA)), rowId); + long tableSizeBytes = + InternalUtils.requireNonNull( + batch.getColumnVector(getSchemaIndex(TABLE_SIZE_BYTES)), rowId, TABLE_SIZE_BYTES) + .getLong(rowId); + long numFiles = + InternalUtils.requireNonNull( + batch.getColumnVector(getSchemaIndex(NUM_FILES)), rowId, NUM_FILES) + .getLong(rowId); + + // Read optional fields + ColumnVector txnIdColumnVector = batch.getColumnVector(getSchemaIndex(TXN_ID)); + Optional txnId = + txnIdColumnVector.isNullAt(rowId) + ? Optional.empty() + : Optional.of(txnIdColumnVector.getString(rowId)); + + // protocol and metadata are nullable per fromColumnVector's implementation. + if (protocol == null || metadata == null) { + logger.warn("Invalid checksum file missing protocol and/or metadata: {}", crcFilePath); + return Optional.empty(); + } + return Optional.of(new CRCInfo(version, metadata, protocol, tableSizeBytes, numFiles, txnId)); + } + + private final long version; + private final Metadata metadata; + private final Protocol protocol; + private final long tableSizeBytes; + private final long numFiles; + private final Optional txnId; + + public CRCInfo( + long version, + Metadata metadata, + Protocol protocol, + long tableSizeBytes, + long numFiles, + Optional txnId) { + checkArgument(tableSizeBytes >= 0); + checkArgument(numFiles >= 0); + this.version = version; + this.metadata = requireNonNull(metadata); + this.protocol = requireNonNull(protocol); + this.tableSizeBytes = tableSizeBytes; + this.numFiles = numFiles; + this.txnId = requireNonNull(txnId); + } + + /** The version of the Delta table that this CRCInfo represents. */ + public long getVersion() { + return version; + } + + /** The {@link Metadata} stored in this CRCInfo. */ + public Metadata getMetadata() { + return metadata; + } + + /** The {@link Protocol} stored in this CRCInfo. */ + public Protocol getProtocol() { + return protocol; + } + + public long getNumFiles() { + return numFiles; + } + + public long getTableSizeBytes() { + return tableSizeBytes; + } + + public Optional getTxnId() { + return txnId; + } + + /** + * Encode as a {@link Row} object with the schema {@link CRCInfo#CRC_FILE_SCHEMA}. + * + * @return {@link Row} object with the schema {@link CRCInfo#CRC_FILE_SCHEMA} + */ + public Row toRow() { + Map values = new HashMap<>(); + // Add required fields + values.put(getSchemaIndex(TABLE_SIZE_BYTES), tableSizeBytes); + values.put(getSchemaIndex(NUM_FILES), numFiles); + values.put(getSchemaIndex(NUM_METADATA), 1L); + values.put(getSchemaIndex(NUM_PROTOCOL), 1L); + values.put(getSchemaIndex(METADATA), metadata.toRow()); + values.put(getSchemaIndex(PROTOCOL), protocol.toRow()); + + // Add optional fields + txnId.ifPresent(txn -> values.put(getSchemaIndex(TXN_ID), txn)); + return new GenericRow(CRC_FILE_SCHEMA, values); + } + + private static int getSchemaIndex(String fieldName) { + return CRC_FILE_SCHEMA.indexOf(fieldName); + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ChecksumReader.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumReader.java similarity index 98% rename from kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ChecksumReader.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumReader.java index 314c456afe8..eeb651a34f5 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ChecksumReader.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumReader.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.internal.replay; +package io.delta.kernel.internal.checksum; import static io.delta.kernel.internal.util.FileNames.*; import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; @@ -96,7 +96,7 @@ private static Optional readChecksumFile(Engine engine, Path filePath) .getJsonHandler() .readJsonFiles( singletonCloseableIterator(FileStatus.of(filePath.toString())), - CRCInfo.FULL_SCHEMA, + CRCInfo.CRC_FILE_SCHEMA, Optional.empty())) { // We do this instead of iterating through the rows or using `getSingularRow` so we // can use the existing fromColumnVector methods in Protocol, Metadata, Format etc diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumWriter.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumWriter.java new file mode 100644 index 00000000000..caec865694a --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumWriter.java @@ -0,0 +1,58 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.checksum; + +import static io.delta.kernel.internal.DeltaErrors.wrapEngineExceptionThrowsIO; +import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; +import static java.util.Objects.requireNonNull; + +import io.delta.kernel.engine.Engine; +import io.delta.kernel.internal.fs.Path; +import io.delta.kernel.internal.util.FileNames; +import java.io.IOException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Writers for writing checksum files from a snapshot */ +public class ChecksumWriter { + + private static final Logger logger = LoggerFactory.getLogger(ChecksumWriter.class); + + private final Path logPath; + + public ChecksumWriter(Path logPath) { + this.logPath = requireNonNull(logPath); + } + + /** Writes a checksum file */ + public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { + Path newChecksumPath = FileNames.checksumFile(logPath, crcInfo.getVersion()); + logger.info("Writing checksum file to path: {}", newChecksumPath); + wrapEngineExceptionThrowsIO( + () -> { + engine + .getJsonHandler() + .writeJsonFileAtomically( + newChecksumPath.toString(), + singletonCloseableIterator(crcInfo.toRow()), + false /* overwrite */); + logger.info("Write checksum file `{}` succeeds", newChecksumPath); + return null; + }, + "Write checksum file `%s`", + newChecksumPath); + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/hook/ChecksumSimpleHook.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/hook/ChecksumSimpleHook.java new file mode 100644 index 00000000000..bb4d0e8c19f --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/hook/ChecksumSimpleHook.java @@ -0,0 +1,53 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.hook; + +import static io.delta.kernel.internal.util.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +import io.delta.kernel.engine.Engine; +import io.delta.kernel.hook.PostCommitHook; +import io.delta.kernel.internal.checksum.CRCInfo; +import io.delta.kernel.internal.checksum.ChecksumWriter; +import io.delta.kernel.internal.fs.Path; +import java.io.IOException; + +/** + * A post-commit hook that writes a new checksum file at the version committed by the transaction. + * This hook performs a simple checksum operation without requiring previous checkpoint or log + * reading. + */ +public class ChecksumSimpleHook implements PostCommitHook { + + private final CRCInfo crcInfo; + private final Path logPath; + + public ChecksumSimpleHook(CRCInfo crcInfo, Path logPath) { + this.crcInfo = requireNonNull(crcInfo); + this.logPath = requireNonNull(logPath); + } + + @Override + public void threadSafeInvoke(Engine engine) throws IOException { + checkArgument(engine != null); + new ChecksumWriter(logPath).writeCheckSum(engine, crcInfo); + } + + @Override + public PostCommitHookType getType() { + return PostCommitHookType.CHECKSUM_SIMPLE; + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CRCInfo.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CRCInfo.java deleted file mode 100644 index 85b1094f477..00000000000 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CRCInfo.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Copyright (2025) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.delta.kernel.internal.replay; - -import static java.util.Objects.requireNonNull; - -import io.delta.kernel.data.ColumnarBatch; -import io.delta.kernel.internal.actions.Metadata; -import io.delta.kernel.internal.actions.Protocol; -import io.delta.kernel.types.StructType; -import java.util.Optional; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class CRCInfo { - private static final Logger logger = LoggerFactory.getLogger(CRCInfo.class); - - public static Optional fromColumnarBatch( - long version, ColumnarBatch batch, int rowId, String crcFilePath) { - Protocol protocol = Protocol.fromColumnVector(batch.getColumnVector(PROTOCOL_ORDINAL), rowId); - Metadata metadata = Metadata.fromColumnVector(batch.getColumnVector(METADATA_ORDINAL), rowId); - // protocol and metadata are nullable per fromColumnVector's implementation. - if (protocol == null || metadata == null) { - logger.warn("Invalid checksum file missing protocol and/or metadata: {}", crcFilePath); - return Optional.empty(); - } - return Optional.of(new CRCInfo(version, metadata, protocol)); - } - - // We can add additional fields later - public static final StructType FULL_SCHEMA = - new StructType().add("protocol", Protocol.FULL_SCHEMA).add("metadata", Metadata.FULL_SCHEMA); - - private static final int PROTOCOL_ORDINAL = 0; - private static final int METADATA_ORDINAL = 1; - - private final long version; - private final Metadata metadata; - private final Protocol protocol; - - protected CRCInfo(long version, Metadata metadata, Protocol protocol) { - this.version = version; - this.metadata = requireNonNull(metadata); - this.protocol = requireNonNull(protocol); - } - - /** The version of the Delta table that this CRCInfo represents. */ - public long getVersion() { - return version; - } - - /** The {@link Metadata} stored in this CRCInfo. */ - public Metadata getMetadata() { - return metadata; - } - - /** The {@link Protocol} stored in this CRCInfo. */ - public Protocol getProtocol() { - return protocol; - } -} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/LogReplay.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/LogReplay.java index 97cbf3ad637..ab4942185d8 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/LogReplay.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/LogReplay.java @@ -28,6 +28,8 @@ import io.delta.kernel.expressions.Predicate; import io.delta.kernel.internal.actions.*; import io.delta.kernel.internal.checkpoints.SidecarFile; +import io.delta.kernel.internal.checksum.CRCInfo; +import io.delta.kernel.internal.checksum.ChecksumReader; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.lang.Lazy; import io.delta.kernel.internal.metrics.ScanMetrics; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotHint.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotHint.java index 5423d786b3f..76b9f653e21 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotHint.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotHint.java @@ -18,7 +18,7 @@ import io.delta.kernel.internal.actions.Metadata; import io.delta.kernel.internal.actions.Protocol; -import io.delta.kernel.internal.replay.CRCInfo; +import io.delta.kernel.internal.checksum.CRCInfo; /** Contains summary information of a {@link io.delta.kernel.Snapshot}. */ public class SnapshotHint { diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala new file mode 100644 index 00000000000..1e73f99de19 --- /dev/null +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala @@ -0,0 +1,197 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.checksum + +import java.util +import java.util.{Collections, Optional} + +import io.delta.kernel.data.Row +import io.delta.kernel.internal.actions.{Format, Metadata, Protocol} +import io.delta.kernel.internal.checksum.CRCInfo.CRC_FILE_SCHEMA +import io.delta.kernel.internal.data.GenericRow +import io.delta.kernel.internal.fs.Path +import io.delta.kernel.internal.util.VectorUtils +import io.delta.kernel.internal.util.VectorUtils.{stringArrayValue, stringStringMapValue} +import io.delta.kernel.test.{BaseMockJsonHandler, MockEngineUtils} +import io.delta.kernel.types.StructType +import io.delta.kernel.utils.CloseableIterator + +import org.scalatest.funsuite.AnyFunSuite + +/** + * Test suite for ChecksumWriter functionality. + */ +class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { + + private val FAKE_DELTA_LOG_PATH = new Path("/path/to/delta/log") + + // Schema field indices in crc file + private val TABLE_SIZE_BYTES_IDX = CRC_FILE_SCHEMA.indexOf("tableSizeBytes") + private val NUM_FILES_IDX = CRC_FILE_SCHEMA.indexOf("numFiles") + private val NUM_METADATA_IDX = CRC_FILE_SCHEMA.indexOf("numMetadata") + private val NUM_PROTOCOL_IDX = CRC_FILE_SCHEMA.indexOf("numProtocol") + private val TXN_ID_IDX = CRC_FILE_SCHEMA.indexOf("txnId") + private val METADATA_IDX = CRC_FILE_SCHEMA.indexOf("metadata") + private val PROTOCOL_IDX = CRC_FILE_SCHEMA.indexOf("protocol") + + test("write checksum") { + val jsonHandler = new MockCheckSumFileJsonWriter() + val checksumWriter = new ChecksumWriter(FAKE_DELTA_LOG_PATH) + val protocol = createTestProtocol() + val metadata = createTestMetadata() + + def testChecksumWrite(txn: Optional[String]): Unit = { + val version = 1L + val tableSizeBytes = 100L + val numFiles = 1L + + checksumWriter.writeCheckSum( + mockEngine(jsonHandler = jsonHandler), + new CRCInfo(version, metadata, protocol, tableSizeBytes, numFiles, txn)) + + verifyChecksumFile(jsonHandler, version) + verifyChecksumContent(jsonHandler.capturedCrcRow.get, tableSizeBytes, numFiles, txn) + verifyMetadataAndProtocol(jsonHandler.capturedCrcRow.get, metadata, protocol) + } + + // Test with and without transaction ID + testChecksumWrite(Optional.of("txn")) + testChecksumWrite(Optional.empty()) + } + + private def verifyChecksumFile(jsonHandler: MockCheckSumFileJsonWriter, version: Long): Unit = { + assert(jsonHandler.checksumFilePath == s"$FAKE_DELTA_LOG_PATH/${"%020d".format(version)}.crc") + assert(jsonHandler.capturedCrcRow.isDefined) + assert(jsonHandler.capturedCrcRow.get.getSchema == CRC_FILE_SCHEMA) + } + + private def verifyChecksumContent( + actualCheckSumRow: Row, + expectedTableSizeBytes: Long, + expectedNumFiles: Long, + expectedTxnId: Optional[String]): Unit = { + assert(!actualCheckSumRow.isNullAt(TABLE_SIZE_BYTES_IDX) && actualCheckSumRow.getLong( + TABLE_SIZE_BYTES_IDX) == expectedTableSizeBytes) + assert(!actualCheckSumRow.isNullAt( + NUM_FILES_IDX) && actualCheckSumRow.getLong(NUM_FILES_IDX) == expectedNumFiles) + assert(!actualCheckSumRow.isNullAt( + NUM_METADATA_IDX) && actualCheckSumRow.getLong(NUM_METADATA_IDX) == 1L) + assert(!actualCheckSumRow.isNullAt( + NUM_PROTOCOL_IDX) && actualCheckSumRow.getLong(NUM_PROTOCOL_IDX) == 1L) + + if (expectedTxnId.isPresent) { + assert(actualCheckSumRow.getString(TXN_ID_IDX) == expectedTxnId.get()) + } else { + assert(actualCheckSumRow.isNullAt(TXN_ID_IDX)) + } + } + + private def verifyMetadataAndProtocol( + actualRow: Row, + expectedMetadata: Metadata, + expectedProtocol: Protocol): Unit = { + checkMetadata(expectedMetadata, actualRow.getStruct(METADATA_IDX)) + checkProtocol(expectedProtocol, actualRow.getStruct(PROTOCOL_IDX)) + } + + // TODO: implement compare in Metadata and remove this method + private def checkMetadata(expectedMetadata: Metadata, actualMetadataRow: Row): Unit = { + assert(actualMetadataRow.getSchema == Metadata.FULL_SCHEMA) + + def getOptionalString(field: String): Optional[String] = + Optional.ofNullable(actualMetadataRow.getString(Metadata.FULL_SCHEMA.indexOf(field))) + + assert( + actualMetadataRow.getString(Metadata.FULL_SCHEMA.indexOf("id")) == expectedMetadata.getId) + assert(getOptionalString("name") == expectedMetadata.getName) + assert(getOptionalString("description") == expectedMetadata.getDescription) + + val formatRow = actualMetadataRow.getStruct(Metadata.FULL_SCHEMA.indexOf("format")) + assert( + formatRow + .getString( + Format.FULL_SCHEMA.indexOf("provider")) == expectedMetadata.getFormat.getProvider) + + assert( + actualMetadataRow + .getString( + Metadata.FULL_SCHEMA.indexOf("schemaString")) == expectedMetadata.getSchemaString) + assert( + actualMetadataRow + .getArray(Metadata.FULL_SCHEMA.indexOf("partitionColumns")) + == expectedMetadata.getPartitionColumns) + assert( + Optional + .ofNullable(actualMetadataRow.getLong(Metadata.FULL_SCHEMA.indexOf("createdTime"))) + == expectedMetadata.getCreatedTime) + assert( + VectorUtils + .toJavaMap(actualMetadataRow.getMap(Metadata.FULL_SCHEMA.indexOf("configuration"))) + == expectedMetadata.getConfiguration) + } + + // TODO: implement compare in Protocol and remove this method + private def checkProtocol(expectedProtocol: Protocol, actualProtocolRow: Row): Unit = { + assert(actualProtocolRow.getSchema == Protocol.FULL_SCHEMA) + assert( + expectedProtocol.getMinReaderVersion == actualProtocolRow + .getInt(Protocol.FULL_SCHEMA.indexOf("minReaderVersion"))) + assert( + expectedProtocol.getMinWriterVersion == actualProtocolRow + .getInt(Protocol.FULL_SCHEMA.indexOf("minWriterVersion"))) + } + + private def createTestMetadata(): Metadata = { + new Metadata( + "id", + Optional.of("name"), + Optional.of("description"), + new Format("parquet", Collections.emptyMap()), + "schemaString", + new StructType(), + stringArrayValue(util.Arrays.asList("c3")), + Optional.of(123), + stringStringMapValue(new util.HashMap[String, String]() { + put("delta.appendOnly", "true") + })) + } + + private def createTestProtocol(): Protocol = { + new Protocol( + /* minReaderVersion= */ 1, + /* minWriterVersion= */ 2, + Collections.emptySet(), + Collections.emptySet()) + } +} + +/** + * Mock implementation of JsonHandler for testing checksum file writing. + */ +class MockCheckSumFileJsonWriter extends BaseMockJsonHandler { + var capturedCrcRow: Option[Row] = None + var checksumFilePath: String = "" + + override def writeJsonFileAtomically( + filePath: String, + data: CloseableIterator[Row], + overwrite: Boolean): Unit = { + checksumFilePath = filePath + assert(data.hasNext, "Expected data iterator to contain exactly one row") + capturedCrcRow = Some(data.next()) + assert(!data.hasNext, "Expected data iterator to contain exactly one row") + } +} diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala new file mode 100644 index 00000000000..754a1572b00 --- /dev/null +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala @@ -0,0 +1,173 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.defaults + +import java.io.File +import java.nio.file.Files +import java.util + +import scala.collection.immutable.Seq +import scala.jdk.CollectionConverters.setAsJavaSetConverter + +import io.delta.kernel.{Operation, Table} +import io.delta.kernel.data.Row +import io.delta.kernel.defaults.utils.TestUtils +import io.delta.kernel.engine.Engine +import io.delta.kernel.internal.DeltaLogActionUtils.DeltaAction +import io.delta.kernel.internal.TableImpl +import io.delta.kernel.internal.actions.{AddFile, SingleAction} +import io.delta.kernel.internal.checksum.{ChecksumReader, CRCInfo} +import io.delta.kernel.internal.fs.Path +import io.delta.kernel.internal.util.FileNames +import io.delta.kernel.internal.util.Utils.toCloseableIterator +import io.delta.kernel.types.LongType.LONG +import io.delta.kernel.types.StructType +import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} + +import org.apache.spark.sql.functions.col + +/** + * Test suite to verify checksum file correctness by comparing + * Delta Spark and Delta Kernel generated checksum files. + * This suite ensures that both implementations generate consistent checksums + * for various table operations. + */ +class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUtils { + + private val PARTITION_COLUMN = "part" + + test("create table, insert data and verify checksum") { + withTempDirAndEngine { (tablePath, engine) => + val sparkTablePath = tablePath + "spark" + val kernelTablePath = tablePath + "kernel" + + createTxn( + engine, + kernelTablePath, + isNewTable = true, + schema = new StructType().add("id", LONG), + partCols = Seq.empty).commit(engine, emptyIterable()) + .getPostCommitHooks + .forEach(hook => hook.threadSafeInvoke(engine)) + spark.sql(s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` (id LONG) USING DELTA") + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + + (1 to 10).foreach { version => + spark.range(0, version).write.format("delta").mode("append").save(sparkTablePath) + commitSparkChangeToKernel(kernelTablePath, engine, sparkTablePath, version) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, version) + } + } + } + + test("create partitioned table, insert and verify checksum") { + withTempDirAndEngine { (tablePath, engine) => + val sparkTablePath = tablePath + "spark" + val kernelTablePath = tablePath + "kernel" + + createTxn( + engine, + kernelTablePath, + isNewTable = true, + schema = new StructType().add("id", LONG).add(PARTITION_COLUMN, LONG), + partCols = Seq(PARTITION_COLUMN)).commit(engine, emptyIterable()) + .getPostCommitHooks + .forEach(hook => hook.threadSafeInvoke(engine)) + spark.sql( + s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` " + + s"(id LONG, part LONG) USING DELTA PARTITIONED BY (part)") + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + + (1 to 10).foreach { version => + spark.range(0, version).withColumn(PARTITION_COLUMN, col("id") % 2) + .write.format("delta").mode("append").save(sparkTablePath) + commitSparkChangeToKernel(kernelTablePath, engine, sparkTablePath, version) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, version) + } + } + } + + private def assertChecksumEquals( + engine: Engine, + sparkTablePath: String, + kernelTablePath: String, + version: Long): Unit = { + val sparkCrcPath = buildCrcPath(sparkTablePath, version) + val kernelCrcPath = buildCrcPath(kernelTablePath, version) + + assert( + Files.exists(sparkCrcPath) && Files.exists(kernelCrcPath), + s"CRC files not found for version $version") + + val sparkCrc = readCrcInfo(engine, sparkTablePath, version) + val kernelCrc = readCrcInfo(engine, kernelTablePath, version) + + assertCrcInfoEquals(sparkCrc, kernelCrc) + } + + private def readCrcInfo(engine: Engine, path: String, version: Long): CRCInfo = { + ChecksumReader + .getCRCInfo(engine, new Path(s"$path/_delta_log"), version, version) + .orElseThrow(() => new IllegalStateException(s"CRC info not found for version $version")) + } + + private def buildCrcPath(basePath: String, version: Long): java.nio.file.Path = { + new File(FileNames.checksumFile(new Path(f"$basePath/_delta_log"), version).toString).toPath + } + + // TODO: Add equals/hashCode to metadata, protocol then CRCInfo. + private def assertCrcInfoEquals(crc1: CRCInfo, crc2: CRCInfo): Unit = { + assert(crc1.getVersion === crc2.getVersion) + assert(crc1.getNumFiles === crc2.getNumFiles) + assert(crc1.getTableSizeBytes === crc2.getTableSizeBytes) + assert(crc1.getMetadata.getSchema === crc2.getMetadata.getSchema) + assert(crc1.getMetadata.getPartitionColNames === crc2.getMetadata.getPartitionColNames) + } + + // TODO docs + private def commitSparkChangeToKernel( + path: String, + engine: Engine, + sparkTablePath: String, + versionToConvert: Long): Unit = { + + val txn = Table.forPath(engine, path) + .createTransactionBuilder(engine, "test-engine", Operation.WRITE) + .build(engine) + + val tableChange = Table.forPath(engine, sparkTablePath).asInstanceOf[TableImpl].getChanges( + engine, + versionToConvert, + versionToConvert, + // TODO include REMOVE action as well once we support it + Set(DeltaAction.ADD).asJava) + + val addFilesRows = new util.ArrayList[Row]() + tableChange.forEach(batch => + batch.getRows.forEach(row => { + val addIndex = row.getSchema.indexOf("add") + if (!row.isNullAt(addIndex)) { + addFilesRows.add( + SingleAction.createAddFileSingleAction(new AddFile(row.getStruct(addIndex)).toRow)) + } + })) + + txn + .commit(engine, inMemoryIterable(toCloseableIterator(addFilesRows.iterator()))) + .getPostCommitHooks + .forEach(_.threadSafeInvoke(engine)) + } +} diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala index b39c20d3c87..7b581beaeb1 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala @@ -43,9 +43,8 @@ import io.delta.kernel.internal.util.Utils.singletonCloseableIterator import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types.StructType +import io.delta.kernel.utils.{CloseableIterable, CloseableIterator, FileStatus} import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} -import io.delta.kernel.utils.CloseableIterator -import io.delta.kernel.utils.FileStatus import org.apache.spark.sql.delta.VersionNotFoundException @@ -320,7 +319,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } val combineActions = inMemoryIterable(actions.reduceLeft(_ combine _)) - txn.commit(engine, combineActions) + commitTransaction(txn, engine, combineActions) } def appendData( @@ -381,7 +380,10 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { assertMetadataProp(snapshot, key, expectedValue) } - def verifyWrittenContent(path: String, expSchema: StructType, expData: Seq[TestRow]): Unit = { + protected def verifyWrittenContent( + path: String, + expSchema: StructType, + expData: Seq[TestRow]): Unit = { val actSchema = tableSchema(path) assert(actSchema === expSchema) @@ -460,4 +462,15 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { .stream() .anyMatch(hook => hook.getType == PostCommitHookType.CHECKPOINT) === isReadyForCheckpoint) } + + /** + * Commit transaction, all child suites should use this instead of txn.commit + * directly and could override it for specific test cases (e.g. commit and write CRC). + */ + protected def commitTransaction( + txn: Transaction, + engine: Engine, + dataActions: CloseableIterable[Row]): TransactionCommitResult = { + txn.commit(engine, dataActions) + } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteWithCrcSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteWithCrcSuite.scala new file mode 100644 index 00000000000..53750cbdbbb --- /dev/null +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteWithCrcSuite.scala @@ -0,0 +1,67 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.defaults +import scala.collection.immutable.Seq +import scala.language.implicitConversions + +import io.delta.kernel.{Transaction, TransactionCommitResult} +import io.delta.kernel.data.Row +import io.delta.kernel.defaults.utils.TestRow +import io.delta.kernel.engine.Engine +import io.delta.kernel.hook.PostCommitHook.PostCommitHookType +import io.delta.kernel.internal.checksum.ChecksumReader +import io.delta.kernel.internal.fs.Path +import io.delta.kernel.types.StructType +import io.delta.kernel.utils.CloseableIterable + +/** + * Test suite that run all tests in DeltaTableWritesSuite with CRC file written + * after each delta commit. This test suite will verify that the written CRC files are valid. + */ +class DeltaTableWriteWithCrcSuite extends DeltaTableWritesSuite { + + override def commitTransaction( + txn: Transaction, + engine: Engine, + dataActions: CloseableIterable[Row]): TransactionCommitResult = { + val result = txn.commit(engine, dataActions) + result.getPostCommitHooks + .stream() + .filter(hook => hook.getType == PostCommitHookType.CHECKSUM_SIMPLE) + .forEach(hook => hook.threadSafeInvoke(engine)) + result + } + + override def verifyWrittenContent( + path: String, + expSchema: StructType, + expData: Seq[TestRow]): Unit = { + super.verifyWrittenContent(path, expSchema, expData) + verifyChecksumValid(path) + } + + /** Ensure checksum is readable by CRC reader. */ + def verifyChecksumValid( + tablePath: String): Unit = { + val checksumVersion = latestSnapshot(tablePath, defaultEngine).getVersion + val crcInfo = ChecksumReader.getCRCInfo( + defaultEngine, + new Path(f"$tablePath/_delta_log/"), + checksumVersion, + checksumVersion) + assert(crcInfo.isPresent) + } +} diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala index a72b6f6598e..a4c59f6b511 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala @@ -45,6 +45,7 @@ import io.delta.kernel.types.TimestampType.TIMESTAMP import io.delta.kernel.utils.CloseableIterable import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} +/** Transaction commit in this suite IS REQUIRED TO use commitTransaction than .commit */ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { /////////////////////////////////////////////////////////////////////////// @@ -96,7 +97,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) val txn = txnBuilder.withSchema(engine, testSchema).build(engine) - txn.commit(engine, emptyIterable()) + commitTransaction(txn, engine, emptyIterable()) { val ex = intercept[TableAlreadyExistsException] { @@ -130,7 +131,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa assert(txn.getSchema(engine) === testSchema) assert(txn.getPartitionColumns(engine) === Seq.empty.asJava) - val txnResult = txn.commit(engine, emptyIterable()) + val txnResult = commitTransaction(txn, engine, emptyIterable()) assert(txnResult.getVersion === 0) assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) @@ -346,7 +347,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa assert(txn.getSchema(engine) === schema) // Expect the partition column name is exactly same as the one in the schema assert(txn.getPartitionColumns(engine) === Seq("Part1", "part2").asJava) - val txnResult = txn.commit(engine, emptyIterable()) + val txnResult = commitTransaction(txn, engine, emptyIterable()) assert(txnResult.getVersion === 0) assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) @@ -364,7 +365,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val table = Table.forPath(engine, tablePath) val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) val txn = txnBuilder.withSchema(engine, schema).build(engine) - val txnResult = txn.commit(engine, emptyIterable()) + val txnResult = commitTransaction(txn, engine, emptyIterable()) assert(txnResult.getVersion === 0) assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) @@ -434,12 +435,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val stagedFiles = stageData(txnState, Map.empty, dataBatches1) val stagedActionsIterable = inMemoryIterable(stagedFiles) - val commitResult = txn.commit(engine, stagedActionsIterable) + val commitResult = commitTransaction(txn, engine, stagedActionsIterable) assert(commitResult.getVersion == 0) // try to commit the same transaction and expect failure val ex = intercept[IllegalStateException] { - txn.commit(engine, stagedActionsIterable) + commitTransaction(txn, engine, stagedActionsIterable) } assert(ex.getMessage.contains( "Transaction is already attempted to commit. Create a new transaction.")) @@ -561,7 +562,10 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa // partition cols are not written in the commit info for inserts val partitionBy = if (i == 0) expPartCols else null verifyCommitInfo(tblPath, version = i, partitionBy, operation = WRITE) - verifyWrittenContent(tblPath, schema, if (i == 0) expV0Data else expV0Data ++ expV1Data) + verifyWrittenContent( + tblPath, + schema, + if (i == 0) expV0Data else expV0Data ++ expV1Data) } } } @@ -794,6 +798,8 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa test("insert into table - idempotent writes") { withTempDirAndEngine { (tblPath, engine) => + // TODO: re-enable when CRC_FULL post commit hook is added, txn2 requires CRC_FULL + assume(this.suiteName != ("DeltaTableWriteWithCrcSuite")) val data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) var expData = Seq.empty[TestRow] // as the data in inserted, update this. @@ -822,7 +828,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa txn: Transaction, actions: CloseableIterable[Row], expTblVer: Long): Unit = { - val commitResult = txn.commit(engine, actions) + val commitResult = commitTransaction(txn, engine, actions) expData = expData ++ data.flatMap(_._2).flatMap(_.toTestRows) @@ -891,6 +897,8 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa test("conflicts - creating new table - table created by other txn after current txn start") { withTempDirAndEngine { (tablePath, engine) => + // TODO: re-enable when CRC_FULL post commit hook is added + assume(this.suiteName != ("DeltaTableWriteWithCrcSuite")) val losingTx = createTestTxn(engine, tablePath, Some(testSchema)) // don't commit losingTxn, instead create a new txn and commit it @@ -939,6 +947,8 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa // Different scenarios that have multiple winning txns and with a checkpoint in between. Seq(1, 5, 12).foreach { numWinningTxs => test(s"conflicts - concurrent data append ($numWinningTxs) after the losing txn has started") { + // TODO: re-enable when CRC_FULL post commit hook is added + assume(this.suiteName != ("DeltaTableWriteWithCrcSuite")) withTempDirAndEngine { (tablePath, engine) => val testData = Seq(Map.empty[String, Literal] -> dataBatches1) var expData = Seq.empty[TestRow] From 1e3076a529d1a844e87b6a96f168355b1eaf1197 Mon Sep 17 00:00:00 2001 From: Lukas Rupprecht Date: Mon, 24 Feb 2025 08:45:14 -0800 Subject: [PATCH 8/9] [Spark] Honor codegen configs in DataSkippingStatsTracker (#4120) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description This PR uses MutableProjection.create instead of directly generating the MutableProjection via GeneratMutableProjection.generate. As the latter forces codegen, it can fail if the generated code is too large, which, e.g., can happen for very wide schemas. The new code uses the correct Spark API to automatically fall back to an InterpretedMutableProjection if codegen fails for any reason. ## How was this patch tested? This PR just uses the correct, higher-level API call to create a MutableProjection so existing tests are sufficient. ## Does this PR introduce _any_ user-facing changes? No --- .../sql/delta/sources/DeltaSQLConf.scala | 10 +++++++ .../stats/DataSkippingStatsTracker.scala | 27 +++++++++++++------ 2 files changed, 29 insertions(+), 8 deletions(-) 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 25b5eac3866..cdb46f48692 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 @@ -2273,6 +2273,16 @@ trait DeltaSQLConfBase { .checkValue(v => v >= 1, "Must be at least 1.") .createWithDefault(100) + val DELTA_STATS_COLLECTION_FALLBACK_TO_INTERPRETED_PROJECTION = + buildConf("collectStats.fallbackToInterpretedProjection") + .internal() + .doc("When enabled, the updateStats expression will use the standard code path" + + " that falls back to an interpreted expression if codegen fails. This should" + + " always be true. The config only exists to force the old behavior, which was" + + " to always use codegen.") + .booleanConf + .createWithDefault(true) + val DELTA_CONVERT_ICEBERG_STATS = buildConf("collectStats.convertIceberg") .internal() .doc("When enabled, attempts to convert Iceberg stats to Delta stats when cloning from " + diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingStatsTracker.scala b/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingStatsTracker.scala index eaff49795ba..407c6bd2fca 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingStatsTracker.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingStatsTracker.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.delta.stats import scala.collection.mutable import org.apache.spark.sql.delta.expressions.JoinedProjection +import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} @@ -27,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration @@ -87,14 +89,23 @@ class DeltaTaskStatisticsTracker( // This projection combines the intermediate results stored by aggBuffer with the values of the // currently processed row and updates aggBuffer in place. - private val updateStats: MutableProjection = GenerateMutableProjection.generate( - expressions = JoinedProjection.bind( - aggBufferAttrs, - dataCols, - aggregates.flatMap(_.updateExpressions)), - inputSchema = Nil, - useSubexprElimination = true - ) + private val updateStats: MutableProjection = { + val aggs = aggregates.flatMap(_.updateExpressions) + val expressions = JoinedProjection.bind(aggBufferAttrs, dataCols, aggs) + if (SQLConf.get.getConf( + DeltaSQLConf.DELTA_STATS_COLLECTION_FALLBACK_TO_INTERPRETED_PROJECTION)) { + MutableProjection.create( + exprs = expressions, + inputSchema = Nil + ) + } else { + GenerateMutableProjection.generate( + expressions = expressions, + inputSchema = Nil, + useSubexprElimination = true + ) + } + } // This executes the whole statsColExpr in order to compute the final stats value for the file. // In order to evaluate it, we have to replace its aggregate functions with the corresponding From e628ff95c2d91d1a0abd815bcf34124670538bcf Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Mon, 24 Feb 2025 11:27:19 -0800 Subject: [PATCH 9/9] [Kernel] Public API for setting and getting domain metadatas (#4181) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description Adds support for setting and getting domain metadatas using public Kernel APIs. The protocol outlines "user-controlled" metadata domains, see https://github.com/delta-io/delta/blob/master/PROTOCOL.md#domain-metadata, we should support setting and getting these as part of the Kernel transaction/snapshot APIs. ## How was this patch tested? Adds new tests as well as refactors existing test suite to use the new APIs. NOTE: I think the way the current suite is structured could be improved, especially with these API changes. This PR does not attempt to fix that and just adds tests + adjusts current tests to use the new APIs. We can refactor this as a follow up task (lower priority). ## Does this PR introduce _any_ user-facing changes? Yes, adds the following public APIs - `txnBuilder.withDomainMetadata` - `txnBuilder.withDomainMetadataRemoved` - `snapshot.getDomainMetadataConfiguration` --- .../main/java/io/delta/kernel/Snapshot.java | 9 + .../io/delta/kernel/TransactionBuilder.java | 37 ++ .../DomainDoesNotExistException.java | 30 ++ .../delta/kernel/internal/SnapshotImpl.java | 7 + .../internal/TransactionBuilderImpl.java | 88 ++++- .../kernel/internal/TransactionImpl.java | 8 +- .../internal/actions/DomainMetadata.java | 14 + .../internal/tablefeatures/TableFeatures.java | 1 + .../kernel/defaults/DomainMetadataSuite.scala | 316 +++++++++++++++++- 9 files changed, 493 insertions(+), 17 deletions(-) create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/DomainDoesNotExistException.java diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java index 0eb816e61d1..8d1a6d96965 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java @@ -20,6 +20,7 @@ import io.delta.kernel.engine.Engine; import io.delta.kernel.types.StructType; import java.util.List; +import java.util.Optional; /** * Represents the snapshot of a Delta table. @@ -61,6 +62,14 @@ public interface Snapshot { */ StructType getSchema(); + /** + * Returns the configuration for the provided {@code domain} if it exists in the snapshot. Returns + * empty if the {@code domain} is not present in the snapshot. + * + * @return the configuration for the provided domain if it exists + */ + Optional getDomainMetadata(String domain); + /** * Create a scan builder to construct a {@link Scan} to read data from this snapshot. * diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java b/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java index d1f7447d87c..2e907178519 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java @@ -18,6 +18,7 @@ import io.delta.kernel.annotation.Evolving; import io.delta.kernel.engine.Engine; import io.delta.kernel.exceptions.ConcurrentTransactionException; +import io.delta.kernel.exceptions.DomainDoesNotExistException; import io.delta.kernel.exceptions.InvalidConfigurationValueException; import io.delta.kernel.exceptions.UnknownConfigurationException; import io.delta.kernel.internal.TableConfig; @@ -91,6 +92,40 @@ TransactionBuilder withTransactionId( */ TransactionBuilder withMaxRetries(int maxRetries); + /** + * Commit the provided domain metadata as part of this transaction. If this is called more than + * once with the same {@code domain} the latest provided {@code config} will be committed in the + * transaction. Only user-controlled domains are allowed (aka. domains with a `delta.` prefix are + * not allowed). Adding and removing a domain with the same identifier in the same txn is not + * allowed. + * + *

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

Please note using this API will automatically upgrade the protocol of the table to support + * Domain Metadata if it is not already supported. See + * How does Delta Lake manage feature compatibility? for more details. This may break existing + * writers that do not support the Domain Metadata feature; readers will be unaffected. + * + * @param domain the domain identifier + * @param config configuration string for this domain + * @return updated {@link TransactionBuilder} instance + */ + TransactionBuilder withDomainMetadata(String domain, String config); + + /** + * Mark the domain metadata with identifier {@code domain} as removed in this transaction. If this + * domain does not exist in the latest version of the table will throw a {@link + * DomainDoesNotExistException} upon calling {@link TransactionBuilder#build(Engine)}. Adding and + * removing a domain with the same identifier in one txn is not allowed. + * + * @param domain the domain identifier for the domain to remove + * @return updated {@link TransactionBuilder} instance + */ + TransactionBuilder withDomainMetadataRemoved(String domain); + /** * Build the transaction. Also validates the given info to ensure that a valid transaction can be * created. @@ -101,6 +136,8 @@ TransactionBuilder withTransactionId( * @throws InvalidConfigurationValueException if the value of the property is invalid. * @throws UnknownConfigurationException if any of the properties are unknown to {@link * TableConfig}. + * @throws DomainDoesNotExistException if removing a domain that does not exist in the latest + * version of the table */ Transaction build(Engine engine); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/DomainDoesNotExistException.java b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/DomainDoesNotExistException.java new file mode 100644 index 00000000000..31821f9b519 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/DomainDoesNotExistException.java @@ -0,0 +1,30 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.exceptions; + +import io.delta.kernel.annotation.Evolving; + +/** Thrown when attempting to remove a domain metadata that does not exist in the read snapshot. */ +@Evolving +public class DomainDoesNotExistException extends KernelException { + public DomainDoesNotExistException(String tablePath, String domain, long snapshotVersion) { + super( + String.format( + "%s: Cannot remove domain metadata with identifier %s because it does not exist in the " + + "read snapshot at version %s", + tablePath, domain, snapshotVersion)); + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java index 77b51352b29..f5ea016c37d 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java @@ -117,6 +117,13 @@ public StructType getSchema() { return getMetadata().getSchema(); } + @Override + public Optional getDomainMetadata(String domain) { + return Optional.ofNullable(getDomainMetadataMap().get(domain)) + .filter(dm -> !dm.isRemoved()) // only consider active domain metadatas (not tombstones) + .map(DomainMetadata::getConfiguration); + } + @Override public ScanBuilder getScanBuilder() { return new ScanBuilderImpl( diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java index 4a32bb8b26e..4b3129a6b1d 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java @@ -19,6 +19,7 @@ import static io.delta.kernel.internal.DeltaErrors.tableAlreadyExists; import static io.delta.kernel.internal.TransactionImpl.DEFAULT_READ_VERSION; import static io.delta.kernel.internal.TransactionImpl.DEFAULT_WRITE_VERSION; +import static io.delta.kernel.internal.tablefeatures.TableFeatures.DOMAIN_METADATA_W_FEATURE; import static io.delta.kernel.internal.util.ColumnMapping.isColumnMappingModeEnabled; import static io.delta.kernel.internal.util.Preconditions.checkArgument; import static io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames; @@ -28,6 +29,7 @@ import io.delta.kernel.*; import io.delta.kernel.engine.Engine; +import io.delta.kernel.exceptions.DomainDoesNotExistException; import io.delta.kernel.exceptions.TableNotFoundException; import io.delta.kernel.internal.actions.*; import io.delta.kernel.internal.fs.Path; @@ -53,6 +55,8 @@ public class TransactionBuilderImpl implements TransactionBuilder { private final TableImpl table; private final String engineInfo; private final Operation operation; + private final Map domainMetadatasAdded = new HashMap<>(); + private final Set domainMetadatasRemoved = new HashSet<>(); private Optional schema = Optional.empty(); private Optional> partitionColumns = Optional.empty(); private Optional setTxnOpt = Optional.empty(); @@ -110,6 +114,31 @@ public TransactionBuilder withMaxRetries(int maxRetries) { return this; } + @Override + public TransactionBuilder withDomainMetadata(String domain, String config) { + checkArgument( + DomainMetadata.isUserControlledDomain(domain), + "Setting a system-controlled domain is not allowed: " + domain); + checkArgument( + !domainMetadatasRemoved.contains(domain), + "Cannot add a domain that is removed in this transaction"); + // we override any existing value + domainMetadatasAdded.put(domain, new DomainMetadata(domain, config, false /* removed */)); + return this; + } + + @Override + public TransactionBuilder withDomainMetadataRemoved(String domain) { + checkArgument( + DomainMetadata.isUserControlledDomain(domain), + "Removing a system-controlled domain is not allowed: " + domain); + checkArgument( + !domainMetadatasAdded.containsKey(domain), + "Cannot remove a domain that is added in this transaction"); + domainMetadatasRemoved.add(domain); + return this; + } + @Override public Transaction build(Engine engine) { SnapshotImpl snapshot; @@ -163,6 +192,21 @@ public Transaction build(Engine engine) { } } + /* --------------- Domain Metadata Protocol upgrade if necessary------------ */ + if (!TableFeatures.isDomainMetadataSupported(protocol)) { + if (!domainMetadatasAdded.isEmpty()) { + // This txn is setting a domain metadata, enable the feature in the protocol + logger.info( + "Automatically enabling writer feature: {}", DOMAIN_METADATA_W_FEATURE.featureName()); + protocol = + protocol.withNewWriterFeatures( + Collections.singleton(DOMAIN_METADATA_W_FEATURE.featureName())); + shouldUpdateProtocol = true; + } + // If domainMetadatasRemoved is non-empty we do nothing. A DomainDoesNotExistException will be + // thrown in `getDomainMetadatasToCommit` since the domain cannot exist in the readSnapshot. + } + return new TransactionImpl( isNewTable, table.getDataPath(), @@ -176,7 +220,8 @@ public Transaction build(Engine engine) { shouldUpdateMetadata, shouldUpdateProtocol, maxRetries, - table.getClock()); + table.getClock(), + getDomainMetadatasToCommit(snapshot)); } /** Validate the given parameters for the transaction. */ @@ -289,4 +334,45 @@ private Metadata getInitialMetadata() { private Protocol getInitialProtocol() { return new Protocol(DEFAULT_READ_VERSION, DEFAULT_WRITE_VERSION); } + + /** + * Returns a list of the domain metadatas to commit. This consists of the domain metadatas added + * in the transaction using {@link TransactionBuilder#withDomainMetadata(String, String)} and the + * tombstones for the domain metadatas removed in the transaction using {@link + * TransactionBuilder#withDomainMetadataRemoved(String)}. + */ + private List getDomainMetadatasToCommit(SnapshotImpl snapshot) { + // Add all domain metadatas added in the transaction + List finalDomainMetadatas = new ArrayList<>(domainMetadatasAdded.values()); + + // Generate the tombstones for the removed domain metadatas + Map snapshotDomainMetadataMap = snapshot.getDomainMetadataMap(); + for (String domainName : domainMetadatasRemoved) { + // Note: we know domainName is not already in finalDomainMetadatas because we do not allow + // removing and adding a domain with the same identifier in a single txn! + if (snapshotDomainMetadataMap.containsKey(domainName)) { + DomainMetadata domainToRemove = snapshotDomainMetadataMap.get(domainName); + if (domainToRemove.isRemoved()) { + // If the domain is already removed we throw an error to avoid any inconsistencies or + // ambiguity. The snapshot read by the connector is inconsistent with the snapshot + // loaded here as the domain to remove no longer exists. + throw new DomainDoesNotExistException( + table.getDataPath().toString(), domainName, snapshot.getVersion()); + } + finalDomainMetadatas.add(domainToRemove.removed()); + } else { + // We must throw an error if the domain does not exist. Otherwise, there could be unexpected + // behavior within conflict resolution. For example, consider the following + // 1. Table has no domains set in V0 + // 2. txnA is started and wants to remove domain "foo" + // 3. txnB is started and adds domain "foo" and commits V1 before txnA + // 4. txnA needs to perform conflict resolution against the V1 commit from txnB + // Conflict resolution should fail but since the domain does not exist we cannot create + // a tombstone to mark it as removed and correctly perform conflict resolution. + throw new DomainDoesNotExistException( + table.getDataPath().toString(), domainName, snapshot.getVersion()); + } + } + return finalDomainMetadatas; + } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java index e950126acb6..7e0919783df 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java @@ -29,6 +29,7 @@ import io.delta.kernel.expressions.Column; import io.delta.kernel.hook.PostCommitHook; import io.delta.kernel.internal.actions.*; +import io.delta.kernel.internal.annotation.VisibleForTesting; import io.delta.kernel.internal.checksum.CRCInfo; import io.delta.kernel.internal.data.TransactionStateRow; import io.delta.kernel.internal.fs.Path; @@ -72,7 +73,7 @@ public class TransactionImpl implements Transaction { private final Optional setTxnOpt; private final boolean shouldUpdateProtocol; private final Clock clock; - private List domainMetadatas = new ArrayList<>(); + private List domainMetadatas; private Metadata metadata; private boolean shouldUpdateMetadata; private int maxRetries; @@ -92,7 +93,8 @@ public TransactionImpl( boolean shouldUpdateMetadata, boolean shouldUpdateProtocol, int maxRetries, - Clock clock) { + Clock clock, + List domainMetadatas) { this.isNewTable = isNewTable; this.dataPath = dataPath; this.logPath = logPath; @@ -106,6 +108,7 @@ public TransactionImpl( this.shouldUpdateProtocol = shouldUpdateProtocol; this.maxRetries = maxRetries; this.clock = clock; + this.domainMetadatas = domainMetadatas; } @Override @@ -132,6 +135,7 @@ public Optional getSetTxnOpt() { * * @param domainMetadatas List of domain metadata to be added to the transaction. */ + @VisibleForTesting public void addDomainMetadatas(List domainMetadatas) { this.domainMetadatas.addAll(domainMetadatas); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/DomainMetadata.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/DomainMetadata.java index d3972267c90..8585b62bf5b 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/DomainMetadata.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/DomainMetadata.java @@ -26,10 +26,19 @@ import io.delta.kernel.types.StringType; import io.delta.kernel.types.StructType; import java.util.HashMap; +import java.util.Locale; import java.util.Map; /** Delta log action representing an `DomainMetadata` action */ public class DomainMetadata { + + /** Whether the provided {@code domain} is a user-controlled domain */ + public static boolean isUserControlledDomain(String domain) { + // Domain identifiers are case-sensitive, but we don't want to allow users to set domains + // with prefixes like `DELTA.` either, so perform case-insensitive check for this purpose + return !domain.toLowerCase(Locale.ROOT).startsWith("delta."); + } + /** Full schema of the {@link DomainMetadata} action in the Delta Log. */ public static final StructType FULL_SCHEMA = new StructType() @@ -117,6 +126,11 @@ public Row toRow() { return new GenericRow(DomainMetadata.FULL_SCHEMA, domainMetadataMap); } + public DomainMetadata removed() { + checkArgument(!removed, "Cannot remove a domain metadata tombstone (already removed)"); + return new DomainMetadata(domain, configuration, true /* removed */); + } + @Override public String toString() { return String.format( diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java index 8f5e39832b0..145af202ad1 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/tablefeatures/TableFeatures.java @@ -621,6 +621,7 @@ private static int getMinReaderVersion(String feature) { */ private static int getMinWriterVersion(String feature) { switch (feature) { + case "domainMetadata": // fall through case "inCommitTimestamp": return 7; default: diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala index b54eb11e95e..cf84816ed3d 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala @@ -25,9 +25,10 @@ import io.delta.kernel.defaults.internal.parquet.ParquetSuiteBase import io.delta.kernel.engine.Engine import io.delta.kernel.exceptions._ import io.delta.kernel.expressions.Literal -import io.delta.kernel.internal.{SnapshotImpl, TableImpl, TransactionBuilderImpl, TransactionImpl} +import io.delta.kernel.internal.{SnapshotImpl, TableConfig, TableImpl, TransactionBuilderImpl, TransactionImpl} import io.delta.kernel.internal.actions.{DomainMetadata, Protocol, SingleAction} import io.delta.kernel.internal.rowtracking.RowTrackingMetadataDomain +import io.delta.kernel.internal.tablefeatures.TableFeatures import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} @@ -43,7 +44,17 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase private def assertDomainMetadata( snapshot: SnapshotImpl, expectedValue: Map[String, DomainMetadata]): Unit = { + // Check using internal API assert(expectedValue === snapshot.getDomainMetadataMap.asScala) + // Verify public API + expectedValue.foreach { case (key, domainMetadata) => + snapshot.getDomainMetadata(key).toScala match { + case Some(config) => + assert(!domainMetadata.isRemoved && config == domainMetadata.getConfiguration) + case None => + assert(domainMetadata.isRemoved) + } + } } private def assertDomainMetadata( @@ -58,23 +69,35 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase private def createTxnWithDomainMetadatas( engine: Engine, tablePath: String, - domainMetadatas: Seq[DomainMetadata]): Transaction = { + domainMetadatas: Seq[DomainMetadata], + useInternalApi: Boolean = false): Transaction = { - val txnBuilder = createWriteTxnBuilder(TableImpl.forPath(engine, tablePath)) - .asInstanceOf[TransactionBuilderImpl] + var txnBuilder = createWriteTxnBuilder(TableImpl.forPath(engine, tablePath)) - val txn = txnBuilder.build(engine).asInstanceOf[TransactionImpl] - txn.addDomainMetadatas(domainMetadatas.asJava) - txn + if (useInternalApi) { + val txn = txnBuilder.build(engine).asInstanceOf[TransactionImpl] + txn.addDomainMetadatas(domainMetadatas.asJava) + txn + } else { + domainMetadatas.foreach { dm => + if (dm.isRemoved) { + txnBuilder = txnBuilder.withDomainMetadataRemoved(dm.getDomain()) + } else { + txnBuilder = txnBuilder.withDomainMetadata(dm.getDomain(), dm.getConfiguration()) + } + } + txnBuilder.build(engine) + } } private def commitDomainMetadataAndVerify( engine: Engine, tablePath: String, domainMetadatas: Seq[DomainMetadata], - expectedValue: Map[String, DomainMetadata]): Unit = { + expectedValue: Map[String, DomainMetadata], + useInternalApi: Boolean = false): Unit = { // Create the transaction with domain metadata and commit - val txn = createTxnWithDomainMetadatas(engine, tablePath, domainMetadatas) + val txn = createTxnWithDomainMetadatas(engine, tablePath, domainMetadatas, useInternalApi) txn.commit(engine, emptyIterable()) // Verify the final state includes the expected domain metadata @@ -82,6 +105,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase assertDomainMetadata(table, engine, expectedValue) } + // TODO we probably don't always need this since domain metadata is now automatically enabled private def setDomainMetadataSupport(engine: Engine, tablePath: String): Unit = { val protocol = new Protocol( 3, // minReaderVersion @@ -178,7 +202,8 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase .commit(engine, emptyIterable()) val dm1 = new DomainMetadata("domain1", "", false) - val txn1 = createTxnWithDomainMetadatas(engine, tablePath, List(dm1)) + // We use the internal API because our public API will automatically upgrade the protocol + val txn1 = createTxnWithDomainMetadatas(engine, tablePath, List(dm1), useInternalApi = true) // We expect the commit to fail because the table doesn't support domain metadata val e = intercept[KernelException] { @@ -210,7 +235,12 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase val dm2 = new DomainMetadata("domain2", "", false) val dm1_2 = new DomainMetadata("domain1", """{"key1":"10"}"""", false) - val txn = createTxnWithDomainMetadatas(engine, tablePath, List(dm1_1, dm2, dm1_2)) + // use internal API because public API overrides multiple domains with the same identifier + val txn = createTxnWithDomainMetadatas( + engine, + tablePath, + List(dm1_1, dm2, dm1_2), + useInternalApi = true) val e = intercept[IllegalArgumentException] { txn.commit(engine, emptyIterable()) @@ -270,7 +300,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase val dm2 = new DomainMetadata("domain2", "", false) val dm3 = new DomainMetadata("domain3", """{"key3":"3"}""", false) val dm1_2 = new DomainMetadata("domain1", """{"key1":"10"}""", false) - val dm3_2 = new DomainMetadata("domain3", """{"key3":"30"}""", true) + val dm3_2 = new DomainMetadata("domain3", """{"key3":"3"}""", true) Seq( (Seq(dm1), Map("domain1" -> dm1)), @@ -532,7 +562,9 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase engine, tablePath, domainMetadatas = Seq(dmAction), - expectedValue = Map(rowTrackingMetadataDomain.getDomainName -> dmAction)) + expectedValue = Map(rowTrackingMetadataDomain.getDomainName -> dmAction), + useInternalApi = true // cannot commit system-controlled domains through public API + ) // Read the RowTrackingMetadataDomain from the table and verify val table = Table.forPath(engine, tablePath) @@ -583,7 +615,9 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase engine, tablePath, domainMetadatas = Seq(dmAction), - expectedValue = Map(dmAction.getDomain -> dmAction)) + expectedValue = Map(dmAction.getDomain -> dmAction), + useInternalApi = true // cannot commit system-controlled domains through public API + ) // Use Spark to read the table's row tracking metadata domain and verify the result val deltaLog = DeltaLog.forTable(spark, new Path(tablePath)) @@ -593,4 +627,258 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase } } } + + test("basic txnBuilder.withDomainMetadata API tests") { + // withDomainMetadata is tested thoroughly elsewhere in this suite, here we just test API + // specific behaviors + + // Cannot set system-controlled domain metadata + Seq("delta.foo", "DELTA.foo").foreach { domain => + val e = intercept[IllegalArgumentException] { + createWriteTxnBuilder(Table.forPath(defaultEngine, "fake/path")) + .withDomainMetadata(domain, "misc config") + } + assert(e.getMessage.contains("Setting a system-controlled domain is not allowed")) + } + + // Setting the same domain more than once uses the latest pair + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + + val dm1_1 = new DomainMetadata("domain1", """{"key1":"1"}""", false) + val dm1_2 = new DomainMetadata("domain1", """{"key1":"10"}"""", false) + + commitDomainMetadataAndVerify(engine, tablePath, List(dm1_1, dm1_2), Map("domain1" -> dm1_2)) + } + } + + test("basic txnBuilder.withDomainMetadataRemoved API tests") { + // withDomainMetadataRemoved is tested thoroughly elsewhere in this suite, here we just test API + // specific behaviors + + // Cannot remove system-controlled domain metadata + Seq("delta.foo", "DELTA.foo").foreach { domain => + val e = intercept[IllegalArgumentException] { + createWriteTxnBuilder(Table.forPath(defaultEngine, "fake/path")) + .withDomainMetadataRemoved(domain) + } + + assert(e.getMessage.contains("Removing a system-controlled domain is not allowed")) + } + + // Can remove same domain more than once in same txn + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + + // Set up table with domain "domain1 + val dm1 = new DomainMetadata("domain1", """{"key1":"1"}""", false) + commitDomainMetadataAndVerify(engine, tablePath, List(dm1), Map("domain1" -> dm1)) + + val dm1_removed = dm1.removed() + commitDomainMetadataAndVerify( + engine, + tablePath, + List(dm1_removed, dm1_removed, dm1_removed), + Map("domain1" -> dm1_removed)) + } + } + + test("txnBuilder.withDomainMetadataRemoved removing a non-existent domain") { + // Remove domain that does not exist and has never existed + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + + intercept[DomainDoesNotExistException] { + createWriteTxnBuilder(Table.forPath(defaultEngine, tablePath)) + .withDomainMetadataRemoved("foo") + .build(defaultEngine) + } + } + + // Remove domain that exists as a tombstone + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + + // Set up table with domain "domain1" + val dm1 = new DomainMetadata("domain1", """{"key1":"1"}""", false) + commitDomainMetadataAndVerify(engine, tablePath, List(dm1), Map("domain1" -> dm1)) + + // Remove domain1 so it exists as a tombstone + val dm1_removed = dm1.removed() + commitDomainMetadataAndVerify( + engine, + tablePath, + List(dm1_removed), + Map("domain1" -> dm1_removed)) + + // Removing it again should fail since it doesn't exist + intercept[DomainDoesNotExistException] { + commitDomainMetadataAndVerify( + engine, + tablePath, + List(dm1_removed), + Map("domain1" -> dm1_removed)) + } + } + } + + test("Using add and remove with the same domain in the same txn") { + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + // We forbid adding + removing a domain with the same identifier in a transaction to avoid + // any ambiguous behavior + // For example, is the expected behavior + // a) we don't write any domain metadata, and it's a no-op (remove cancels out the add) + // b) we remove the previous domain from the read snapshot, and add the new one as the current + // domain metadata + + { + val e = intercept[IllegalArgumentException] { + createWriteTxnBuilder(Table.forPath(defaultEngine, tablePath)) + .withDomainMetadata("foo", "fake config") + .withDomainMetadataRemoved("foo") + } + assert(e.getMessage.contains("Cannot remove a domain that is added in this transaction")) + } + { + val e = intercept[IllegalArgumentException] { + createWriteTxnBuilder(Table.forPath(defaultEngine, tablePath)) + .withDomainMetadataRemoved("foo") + .withDomainMetadata("foo", "fake config") + } + assert(e.getMessage.contains("Cannot add a domain that is removed in this transaction")) + } + } + } + + test("basic snapshot.getDomainMetadataConfiguration API tests") { + // getDomainMetadataConfiguration is tested thoroughly elsewhere in this suite, here we just + // test the API directly to be safe + + withTempDirAndEngine { (tablePath, engine) => + createTableWithDomainMetadataSupported(engine, tablePath) + + // Non-existent domain is not returned + assert(!latestSnapshot(tablePath).getDomainMetadata("foo").isPresent) + + // Commit domain foo + val fooDm = new DomainMetadata("foo", "foo!", false) + commitDomainMetadataAndVerify(engine, tablePath, List(fooDm), Map("foo" -> fooDm)) + assert( // Check here even though already verified in commitDomainMetadataAndVerify + latestSnapshot(tablePath).getDomainMetadata("foo").toScala.contains("foo!")) + + // Remove domain foo (so tombstone exists but should not be returned) + val fooDm_removed = fooDm.removed() + commitDomainMetadataAndVerify( + engine, + tablePath, + List(fooDm_removed), + Map("foo" -> fooDm_removed)) + // Already checked in commitDomainMetadataAndVerify but check again + assert(!latestSnapshot(tablePath).getDomainMetadata("foo").isPresent) + } + } + + /* --------------- Automatic upgrade table feature tests -------------- */ + + private def verifyDomainMetadataFeatureSupport( + engine: Engine, + tablePath: String, + isSupported: Boolean = true): Unit = { + val snapshotImpl = Table.forPath(engine, tablePath).getLatestSnapshot(engine) + .asInstanceOf[SnapshotImpl] + assert(TableFeatures.isDomainMetadataSupported(snapshotImpl.getProtocol) == isSupported) + } + + /** + * For the given tablePath, commit a transaction that sets a domain metadata using + * withDomainMetadata when building the transaction (which should automatically upgrade the table + * protocol). Verifies that the domain metadata table feature is unsupported before committing + * (for an existing table) and is supported after committing. + * + * @param isNewTable if true, sets a schema during transaction building to create a new table + * @param tableProperties if provided sets the table properties during transaction building + */ + private def verifyAutomaticUpgrade( + engine: Engine, + tablePath: String, + isNewTable: Boolean = false, + tableProperties: Option[Map[String, String]] = None): Unit = { + if (!isNewTable) { + // Verify it's not supported yet + verifyDomainMetadataFeatureSupport(engine, tablePath, isSupported = false) + } + // Commit to table using withDomainMetadata and upgrade protocol + var txnBuilder = createWriteTxnBuilder(Table.forPath(engine, tablePath)) + .withDomainMetadata("foo", "configuration string") + if (isNewTable) { + txnBuilder = txnBuilder.withSchema(engine, testSchema) + } + if (tableProperties.nonEmpty) { + txnBuilder = txnBuilder.withTableProperties(engine, tableProperties.get.asJava) + } + txnBuilder.build(engine).commit(engine, emptyIterable()) + // Check the feature is now supported + verifyDomainMetadataFeatureSupport(engine, tablePath) + } + + test("automatically enable DomainMetadata when using withDomainMetadata - new table") { + // New table using the withDomainMetadata API + withTempDirAndEngine { (tablePath, engine) => + verifyAutomaticUpgrade(engine, tablePath, isNewTable = true) + } + } + + test("automatically enable DomainMetadata when using withDomainMetadata - existing " + + "table with legacy protocol") { + withTempDirAndEngine { (tablePath, engine) => + // Create table with legacy protocol + createTxn(tablePath = tablePath, isNewTable = true, schema = testSchema, partCols = Seq()) + .commit(engine, emptyIterable()) + verifyAutomaticUpgrade(engine, tablePath) + } + } + + test("automatically enable DomainMetadata when using withDomainMetadata - existing " + + "table with TF protocol") { + withTempDirAndEngine { (tablePath, engine) => + // Create table with table feature protocol + createTxn( + tablePath = tablePath, + isNewTable = true, + schema = testSchema, + partCols = Seq(), + // Enable inCommitTimestamps to bump the protocol + tableProperties = Map(TableConfig.IN_COMMIT_TIMESTAMPS_ENABLED.getKey -> "true")) + .commit(engine, emptyIterable()) + verifyAutomaticUpgrade(engine, tablePath) + } + } + + test("automatically enable DomainMetadata when using withDomainMetadata - existing " + + "table, upgrade two TF in same txn") { + withTempDirAndEngine { (tablePath, engine) => + // Create table with legacy protocol + createTxn(tablePath = tablePath, isNewTable = true, schema = testSchema, partCols = Seq()) + .commit(engine, emptyIterable()) + verifyAutomaticUpgrade( + engine, + tablePath, + // Enable inCommitTimestamps as well + tableProperties = Some(Map(TableConfig.IN_COMMIT_TIMESTAMPS_ENABLED.getKey -> "true"))) + } + } + + test("removing a domain on a table without DomainMetadata support") { + withTempDirAndEngine { (tablePath, engine) => + // Create table with legacy protocol + createTxn(tablePath = tablePath, isNewTable = true, schema = testSchema, partCols = Seq()) + .commit(engine, emptyIterable()) + intercept[DomainDoesNotExistException] { + createWriteTxnBuilder(Table.forPath(engine, tablePath)) + .withDomainMetadataRemoved("foo") + .build(engine) + } + } + } }