From 1c0521332999dc5e072d46c30c436837b1f8d252 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Thu, 6 Feb 2025 17:43:15 -0800 Subject: [PATCH 01/73] basic crc simple writer --- .../io/delta/kernel/hook/PostCommitHook.java | 3 +- .../delta/kernel/internal/SnapshotImpl.java | 2 +- .../kernel/internal/TransactionImpl.java | 36 +++++++++ .../{replay => checksum}/CRCInfo.java | 53 ++++++++++---- .../{replay => checksum}/ChecksumReader.java | 4 +- .../internal/checksum/ChecksumUtils.java | 38 ++++++++++ .../internal/checksum/ChecksumWriter.java | 73 +++++++++++++++++++ .../internal/hook/ChecksumSimpleHook.java | 45 ++++++++++++ .../kernel/internal/replay/LogReplay.java | 2 + .../internal/snapshot/SnapshotHint.java | 2 +- 10 files changed, 240 insertions(+), 18 deletions(-) rename kernel/kernel-api/src/main/java/io/delta/kernel/internal/{replay => checksum}/CRCInfo.java (58%) 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/ChecksumUtils.java 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 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..741d3a3676b 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,8 @@ 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, + 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 b2bef46757a..ee6407eddb0 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,15 +29,18 @@ 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; import io.delta.kernel.internal.replay.ConflictChecker.TransactionRebaseState; import io.delta.kernel.internal.rowtracking.RowTracking; 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; @@ -361,6 +364,9 @@ private TransactionCommitResult doCommit( postCommitHooks.add(new CheckpointHook(dataPath, commitAsVersion)); } + buildPostCommitCrcInfo(commitAsVersion, transactionMetrics.captureTransactionMetricsResult()) + .ifPresent(crcInfo -> postCommitHooks.add(new ChecksumSimpleHook(crcInfo, logPath))); + return new TransactionCommitResult(commitAsVersion, postCommitHooks); } catch (FileAlreadyExistsException e) { throw e; @@ -436,6 +442,36 @@ private void recordTransactionReport( engine.getMetricsReporters().forEach(reporter -> reporter.report(transactionReport)); } + private Optional buildPostCommitCrcInfo( + long commitAtVersion, TransactionMetricsResult metricsResult) { + // Create table + if (commitAtVersion == 0) { + return Optional.of( + new CRCInfo( + commitAtVersion, + metadata, + protocol, + metricsResult.getTotalAddFilesSizeInBytes(), + metricsResult.getNumAddFiles(), + Optional.of(txnId.toString()))); + } + // Retry or CRC is read for old version + if (!readSnapshot.getCurrentCrcInfo().isPresent() + || commitAtVersion != readSnapshot.getCurrentCrcInfo().get().getVersion() + 1) { + return Optional.empty(); + } + + CRCInfo lastCrcInfo = readSnapshot.getCurrentCrcInfo().get(); + return Optional.of( + new CRCInfo( + commitAtVersion, + metadata, + protocol, + 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/replay/CRCInfo.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/CRCInfo.java similarity index 58% rename from kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CRCInfo.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/CRCInfo.java index 85b1094f477..fa23ab35463 100644 --- 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/checksum/CRCInfo.java @@ -13,14 +13,14 @@ * 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.checksum.ChecksumUtils.CRC_FILE_SCHEMA; 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; @@ -30,31 +30,46 @@ public class CRCInfo { 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 protocol = + Protocol.fromColumnVector( + batch.getColumnVector(CRC_FILE_SCHEMA.indexOf("protocol")), rowId); + Metadata metadata = + Metadata.fromColumnVector( + batch.getColumnVector(CRC_FILE_SCHEMA.indexOf("metadata")), rowId); + long tableSizeBytes = + batch.getColumnVector(CRC_FILE_SCHEMA.indexOf("tableSizeBytes")).getLong(rowId); + long numFiles = batch.getColumnVector(CRC_FILE_SCHEMA.indexOf("numFiles")).getLong(rowId); + Optional txnId = + Optional.ofNullable( + batch.getColumnVector(CRC_FILE_SCHEMA.indexOf("txnId")).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)); + return Optional.of(new CRCInfo(version, metadata, protocol, tableSizeBytes, numFiles, txnId)); } - // 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; + private final long tableSizeBytes; + private final long numFiles; + private final Optional txnId; - protected CRCInfo(long version, Metadata metadata, Protocol protocol) { + public CRCInfo( + long version, + Metadata metadata, + Protocol protocol, + long tableSizeBytes, + long numFiles, + Optional txnId) { this.version = version; this.metadata = requireNonNull(metadata); this.protocol = requireNonNull(protocol); + this.tableSizeBytes = tableSizeBytes; + this.numFiles = numFiles; + this.txnId = txnId; } /** The version of the Delta table that this CRCInfo represents. */ @@ -71,4 +86,16 @@ public Metadata getMetadata() { public Protocol getProtocol() { return protocol; } + + public long getNumFiles() { + return numFiles; + } + + public long getTableSizeBytes() { + return tableSizeBytes; + } + + public Optional getTxnId() { + return txnId; + } } 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..da9f20b7b0e 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, + ChecksumUtils.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/ChecksumUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumUtils.java new file mode 100644 index 00000000000..13117da767d --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumUtils.java @@ -0,0 +1,38 @@ +/* + * 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 io.delta.kernel.internal.actions.Metadata; +import io.delta.kernel.internal.actions.Protocol; +import io.delta.kernel.types.LongType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructType; + +/** Helper class for shared components in checksum file read and write */ +public final class ChecksumUtils { + + public static StructType CRC_FILE_SCHEMA = + new StructType() + .add("tableSizeBytes", LongType.LONG) + .add("numFiles", LongType.LONG) + .add("numMetadata", LongType.LONG) + .add("numProtocol", LongType.LONG) + .add("metadata", Metadata.FULL_SCHEMA) + .add("protocol", Protocol.FULL_SCHEMA) + .add("txnId", StringType.STRING, /*nullable*/ true); + + private ChecksumUtils() {} +} 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..0f125acafb2 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumWriter.java @@ -0,0 +1,73 @@ +/* + * 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.checksum.ChecksumUtils.CRC_FILE_SCHEMA; +import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; + +import io.delta.kernel.data.Row; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.internal.data.GenericRow; +import io.delta.kernel.internal.fs.Path; +import io.delta.kernel.internal.util.FileNames; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +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 = logPath; + } + + /** Writes a checksum file */ + public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { + // No sufficient information to write checksum file. + Path newChecksumPath = FileNames.checksumFile(logPath, crcInfo.getVersion()); + wrapEngineExceptionThrowsIO( + () -> { + engine + .getJsonHandler() + .writeJsonFileAtomically( + newChecksumPath.toString(), + singletonCloseableIterator(buildCheckSumRow(crcInfo)), + false /* overwrite */); + return null; + }, + "Write checksum file `%s`", + newChecksumPath); + } + + private Row buildCheckSumRow(CRCInfo crcInfo) { + Map value = new HashMap<>(); + value.put(CRC_FILE_SCHEMA.indexOf("tableSizeBytes"), crcInfo.getTableSizeBytes()); + value.put(CRC_FILE_SCHEMA.indexOf("numFiles"), crcInfo.getNumFiles()); + value.put(CRC_FILE_SCHEMA.indexOf("numMetadata"), 1L); + value.put(CRC_FILE_SCHEMA.indexOf("numProtocol"), 1L); + value.put(CRC_FILE_SCHEMA.indexOf("metadata"), crcInfo.getMetadata().toRow()); + value.put(CRC_FILE_SCHEMA.indexOf("protocol"), crcInfo.getProtocol().toRow()); + crcInfo.getTxnId().ifPresent(txn -> value.put(CRC_FILE_SCHEMA.indexOf("txnId"), txn)); + return new GenericRow(CRC_FILE_SCHEMA, value); + } +} 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..0b6fd68aaab --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/hook/ChecksumSimpleHook.java @@ -0,0 +1,45 @@ +/* + * 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 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; + +/** Write a new checkpoint at the version committed by the txn. */ +public class ChecksumSimpleHook implements PostCommitHook { + + private final CRCInfo crcInfo; + private final Path logPath; + + public ChecksumSimpleHook(CRCInfo crcInfo, Path logPath) { + this.crcInfo = crcInfo; + this.logPath = logPath; + } + + @Override + public void threadSafeInvoke(Engine engine) throws IOException { + 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/LogReplay.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/LogReplay.java index f51d210ca98..e71cb232d18 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 @@ -29,6 +29,8 @@ import io.delta.kernel.internal.TableFeatures; 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 { From ca06a90cf9738a29ce7bfd326bc8d0ed6e8f30a2 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Thu, 6 Feb 2025 18:14:45 -0800 Subject: [PATCH 02/73] first test --- .../kernel/defaults/ChecksumSimpleSuite.scala | 5 ++ .../defaults/DeltaTableWriteSuiteBase.scala | 10 +++ .../defaults/DeltaTableWritesSuite.scala | 64 +++++++++++-------- 3 files changed, 52 insertions(+), 27 deletions(-) create mode 100644 kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleSuite.scala diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleSuite.scala new file mode 100644 index 00000000000..8752d02e2c2 --- /dev/null +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleSuite.scala @@ -0,0 +1,5 @@ +package io.delta.kernel.defaults +import io.delta.kernel.defaults.utils.TestUtils +import org.scalatest.funsuite.AnyFunSuite + +class ChecksumSimpleSuite extends AnyFunSuite with TestUtils {} diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala index 6f24cca7aa1..66e27ab1b4b 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 @@ -445,4 +445,14 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { ) === isReadyForCheckpoint ) } + + def assertChecksumSimpleReadiness(txnResult: TransactionCommitResult): Unit = { + assert( + txnResult.getPostCommitHooks + .stream() + .anyMatch( + hook => hook.getType == PostCommitHookType.CHECKSUM_SIMPLE + ) + ) + } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala index 22f3d07fcd0..8f26a2d0afb 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 @@ -329,34 +329,44 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } } - test("create a partitioned table") { - withTempDirAndEngine { (tablePath, engine) => - val table = Table.forPath(engine, tablePath) - val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) - - val schema = new StructType() - .add("id", INTEGER) - .add("Part1", INTEGER) // partition column - .add("part2", INTEGER) // partition column - - val txn = txnBuilder - .withSchema(engine, schema) - // partition columns should preserve the same case the one in the schema - .withPartitionColumns(engine, Seq("part1", "PART2").asJava) - .build(engine) - - 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()) - - assert(txnResult.getVersion === 0) - assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) + Seq(true, false).foreach( + writeCheckSum => + test("create a partitioned table" + { + if (writeCheckSum) "and execute post commit hook" else "" + }) { + withTempDirAndEngine { + (tablePath, engine) => + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + + val schema = new StructType() + .add("id", INTEGER) + .add("Part1", INTEGER) // partition column + .add("part2", INTEGER) // partition column + + val txn = txnBuilder + .withSchema(engine, schema) + // partition columns should preserve the same case the one in the schema + .withPartitionColumns(engine, Seq("part1", "PART2").asJava) + .build(engine) + + 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()) + + assert(txnResult.getVersion === 0) + assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) + assertChecksumSimpleReadiness(txnResult) + if (writeCheckSum) { + txnResult.getPostCommitHooks.forEach(hook => hook.threadSafeInvoke(engine)) + } - verifyCommitInfo(tablePath, version = 0, Seq("Part1", "part2")) - verifyWrittenContent(tablePath, schema, Seq.empty) - } - } + verifyCommitInfo(tablePath, version = 0, Seq("Part1", "part2")) + verifyWrittenContent(tablePath, schema, Seq.empty) + } + } + ) test("create table with all supported types") { withTempDirAndEngine { (tablePath, engine) => From 3caed702b264785cf7712a3e05ef4192880cfad9 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Fri, 7 Feb 2025 15:59:32 -0800 Subject: [PATCH 03/73] add tests --- .../defaults/DeltaTableWriteSuiteBase.scala | 112 +- .../defaults/DeltaTableWritesSuite.scala | 976 +++++++++++------- 2 files changed, 684 insertions(+), 404 deletions(-) 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 66e27ab1b4b..107cb837152 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 @@ -22,7 +22,7 @@ import io.delta.kernel.defaults.utils.{TestRow, TestUtils} import io.delta.kernel.engine.Engine import io.delta.kernel.internal.actions.{Metadata, Protocol, SingleAction} import io.delta.kernel.internal.fs.{Path => DeltaPath} -import io.delta.kernel.internal.util.FileNames +import io.delta.kernel.internal.util.{Clock, FileNames, VectorUtils} import io.delta.kernel.internal.util.Utils.singletonCloseableIterator import io.delta.kernel.internal.{SnapshotImpl, TableConfig, TableImpl} import io.delta.kernel.utils.FileStatus @@ -38,7 +38,6 @@ import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, import io.delta.kernel.defaults.internal.data.DefaultColumnarBatch import io.delta.kernel.expressions.Literal import io.delta.kernel.expressions.Literal.ofInt -import io.delta.kernel.internal.util.Clock import io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.types.IntegerType.INTEGER @@ -47,15 +46,17 @@ import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} import io.delta.kernel.utils.CloseableIterator import io.delta.kernel.Operation.CREATE_TABLE import io.delta.kernel.hook.PostCommitHook.PostCommitHookType +import io.delta.kernel.internal.checksum.{CRCInfo, ChecksumUtils} import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.delta.VersionNotFoundException import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.matchers.should.Matchers.convertToAnyShouldWrapper import java.io.File import java.nio.file.{Files, Paths} -import java.util.Optional +import java.util.{Locale, Optional} import scala.collection.JavaConverters._ import scala.collection.immutable.{ListMap, Seq} @@ -314,23 +315,27 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def appendData( - engine: Engine = defaultEngine, - tablePath: String, - isNewTable: Boolean = false, - schema: StructType = null, - partCols: Seq[String] = null, - data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])], - clock: Clock = () => System.currentTimeMillis, - tableProperties: Map[String, String] = null): TransactionCommitResult = { - + engine: Engine = defaultEngine, + tablePath: String, + isNewTable: Boolean = false, + schema: StructType = null, + partCols: Seq[String] = null, + data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])], + clock: Clock = () => System.currentTimeMillis, + tableProperties: Map[String, String] = null, + executePostCommitHook: Boolean = false): TransactionCommitResult = { val txn = createTxn(engine, tablePath, isNewTable, schema, partCols, tableProperties, clock) - commitAppendData(engine, txn, data) + val commitResult = commitAppendData(engine, txn, data) + if (executePostCommitHook) { + commitResult.getPostCommitHooks.forEach(hook => hook.threadSafeInvoke(engine)) + } + commitResult } def assertMetadataProp( - snapshot: SnapshotImpl, - key: TableConfig[_ <: Any], - expectedValue: Any): Unit = { + snapshot: SnapshotImpl, + key: TableConfig[_ <: Any], + expectedValue: Any): Unit = { assert(key.fromMetadata(snapshot.getMetadata) == expectedValue) } @@ -365,7 +370,13 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { assertMetadataProp(snapshot, key, expectedValue) } - def verifyWrittenContent(path: String, expSchema: StructType, expData: Seq[TestRow]): Unit = { + def verifyWrittenContent( + path: String, + expSchema: StructType, + expData: Seq[TestRow], + expPartitionColumns: Seq[String] = Seq(), + version: Option[Long] = Option.empty, + checksumWritten: Boolean = false): Unit = { val actSchema = tableSchema(path) assert(actSchema === expSchema) @@ -376,26 +387,38 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { // Spark reads the timestamp partition columns in local timezone vs. Kernel reads in UTC. We // need to set the timezone to UTC before reading the data using Spark to make the tests pass withSparkTimeZone("UTC") { - val resultSpark = spark.sql(s"SELECT * FROM delta.`$path`").collect().map(TestRow(_)) + val resultSpark = spark + .sql(s"SELECT * FROM delta.`$path`" + { + if (version.isDefined) s" VERSION AS OF ${version.get}" else "" + }) + .collect() + .map(TestRow(_)) checkAnswer(resultSpark, expData) } + + if (checksumWritten) { + checkChecksumContent(path, version, expSchema, expPartitionColumns) + } } def verifyCommitInfo( - tablePath: String, - version: Long, - partitionCols: Seq[String] = Seq.empty, - isBlindAppend: Boolean = true, - operation: Operation = CREATE_TABLE): Unit = { - val row = spark.sql(s"DESCRIBE HISTORY delta.`$tablePath`") + tablePath: String, + version: Long, + partitionCols: Seq[String] = Seq.empty, + isBlindAppend: Boolean = true, + operation: Operation = CREATE_TABLE): Unit = { + val row = spark + .sql(s"DESCRIBE HISTORY delta.`$tablePath`") .filter(s"version = $version") .select( "version", "operationParameters.partitionBy", "isBlindAppend", "engineInfo", - "operation") - .collect().last + "operation" + ) + .collect() + .last assert(row.getAs[Long]("version") === version) assert(row.getAs[Long]("partitionBy") === @@ -455,4 +478,41 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { ) ) } + + def checkChecksumContent( + tablePath: String, + version: Option[Long], + expSchema: StructType, + expPartitionColumns: Seq[String]): Unit = { + val checksumVersion = + if (version.isDefined) version.get else latestSnapshot(tablePath, defaultEngine).getVersion + assert( + Files.exists( + new File(f"$tablePath/_delta_log/$checksumVersion%020d.crc").toPath + ) + ) + val columnarBatches = defaultEngine + .getJsonHandler() + .readJsonFiles( + singletonCloseableIterator( + FileStatus.of(f"$tablePath/_delta_log/$checksumVersion%020d.crc") + ), + ChecksumUtils.CRC_FILE_SCHEMA, + Optional.empty() + ) + assert(columnarBatches.hasNext) + val crcRow = columnarBatches.next() + assert(crcRow.getSize === 1) + val metadataField = Metadata.fromColumnVector( + crcRow.getColumnVector(ChecksumUtils.CRC_FILE_SCHEMA.indexOf("metadata")), + 0 + ) + assert(metadataField.getSchema === expSchema) + assert( + metadataField.getPartitionColNames.asScala === expPartitionColumns + .map(s => s.toLowerCase(Locale.ROOT)) + .toSet + ) + assert(!columnarBatches.hasNext) + } } 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 8f26a2d0afb..dbc38a5f3e9 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 @@ -330,9 +330,9 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } Seq(true, false).foreach( - writeCheckSum => + executePostCommitHook => test("create a partitioned table" + { - if (writeCheckSum) "and execute post commit hook" else "" + if (executePostCommitHook) " and execute post commit hook" else "" }) { withTempDirAndEngine { (tablePath, engine) => @@ -358,84 +358,143 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa assert(txnResult.getVersion === 0) assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) assertChecksumSimpleReadiness(txnResult) - if (writeCheckSum) { + if (executePostCommitHook) { txnResult.getPostCommitHooks.forEach(hook => hook.threadSafeInvoke(engine)) } verifyCommitInfo(tablePath, version = 0, Seq("Part1", "part2")) - verifyWrittenContent(tablePath, schema, Seq.empty) + verifyWrittenContent( + tablePath, + schema, + Seq.empty, + Seq("Part1", "part2"), + version = Some(0), + checksumWritten = executePostCommitHook + ) } } ) - test("create table with all supported types") { - withTempDirAndEngine { (tablePath, engine) => - val parquetAllTypes = goldenTablePath("parquet-all-types") - val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) + Seq(true, false).foreach( + executePostCommitHook => + test("create table with all supported types" + { + if (executePostCommitHook) " and execute post commit hook" else "" + }) { + withTempDirAndEngine { + (tablePath, engine) => + val parquetAllTypes = goldenTablePath("parquet-all-types") + val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) - val table = Table.forPath(engine, tablePath) - val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) - val txn = txnBuilder.withSchema(engine, schema).build(engine) - val txnResult = txn.commit(engine, emptyIterable()) + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + val txn = txnBuilder.withSchema(engine, schema).build(engine) + val txnResult = txn.commit(engine, emptyIterable()) - assert(txnResult.getVersion === 0) - assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) + assert(txnResult.getVersion === 0) + assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) + assertChecksumSimpleReadiness(txnResult) + if (executePostCommitHook) { + txnResult.getPostCommitHooks.forEach(hook => hook.threadSafeInvoke(engine)) + } - verifyCommitInfo(tablePath, version = 0) - verifyWrittenContent(tablePath, schema, Seq.empty) - } - } + verifyCommitInfo(tablePath, version = 0) + verifyWrittenContent( + tablePath, + schema, + Seq.empty, + version = Some(0), + checksumWritten = executePostCommitHook + ) + } + } + ) /////////////////////////////////////////////////////////////////////////// // Create table and insert data tests (CTAS & INSERT) /////////////////////////////////////////////////////////////////////////// - test("insert into table - table created from scratch") { - withTempDirAndEngine { (tblPath, engine) => - val commitResult0 = appendData( - engine, - tblPath, - isNewTable = true, - testSchema, - partCols = Seq.empty, - data = Seq(Map.empty[String, Literal] -> (dataBatches1 ++ dataBatches2)) - ) - - val expectedAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap(_.toTestRows) - - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) - verifyWrittenContent(tblPath, testSchema, expectedAnswer) - } - } - - test("insert into table - already existing table") { - withTempDirAndEngine { (tblPath, engine) => - val commitResult0 = appendData( - engine, - tblPath, - isNewTable = true, - testSchema, - partCols = Seq.empty, - data = Seq(Map.empty[String, Literal] -> dataBatches1) - ) - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) - verifyWrittenContent(tblPath, testSchema, dataBatches1.flatMap(_.toTestRows)) - - val commitResult1 = appendData( - engine, - tblPath, - data = Seq(Map.empty[String, Literal] -> dataBatches2) - ) - - val expAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap(_.toTestRows) + Seq(true, false).foreach( + executePostCommitHook => + test("insert into table - table created from scratch" + { + if (executePostCommitHook) " and execute post commit hook" else "" + }) { + withTempDirAndEngine { + (tblPath, engine) => + val commitResult0 = appendData( + engine, + tblPath, + isNewTable = true, + testSchema, + partCols = Seq.empty, + data = Seq(Map.empty[String, Literal] -> (dataBatches1 ++ dataBatches2)), + executePostCommitHook = executePostCommitHook + ) + + val expectedAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap( + _.toTestRows + ) + + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) + verifyWrittenContent( + tblPath, + testSchema, + expectedAnswer, + version = Some(0), + checksumWritten = executePostCommitHook + ) + } + } + ) - verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) - verifyWrittenContent(tblPath, testSchema, expAnswer) + Seq(true, false).foreach( + executePostCommitHook => + test("insert into table - already existing table" + { + if (executePostCommitHook) " and execute post commit hook" else "" + }) { + withTempDirAndEngine { + (tblPath, engine) => + val commitResult0 = appendData( + engine, + tblPath, + isNewTable = true, + testSchema, + partCols = Seq.empty, + data = Seq(Map.empty[String, Literal] -> dataBatches1), + executePostCommitHook = executePostCommitHook + ) + + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) + verifyWrittenContent( + tblPath, + testSchema, + dataBatches1.flatMap(_.toTestRows), + version = Some(0), + checksumWritten = executePostCommitHook + ) + + val commitResult1 = appendData( + engine, + tblPath, + data = Seq(Map.empty[String, Literal] -> dataBatches2), + executePostCommitHook = executePostCommitHook + ) + + val expAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap(_.toTestRows) + + verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) + verifyWrittenContent( + tblPath, + testSchema, + expAnswer, + version = Some(1), + checksumWritten = executePostCommitHook + ) } - } + } + ) test("insert into table - fails when committing the same txn twice") { withTempDirAndEngine { (tblPath, engine) => @@ -461,130 +520,174 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } } - test("insert into partitioned table - table created from scratch") { - withTempDirAndEngine { (tblPath, engine) => - val commitResult0 = appendData( - engine, - tblPath, - isNewTable = true, - testPartitionSchema, - testPartitionColumns, - Seq( - Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1, - Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2 - ) - ) - - val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ - dataPartitionBatches2.flatMap(_.toTestRows) - - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, testPartitionColumns, operation = WRITE) - verifyWrittenContent(tblPath, testPartitionSchema, expData) - } - } - - test("insert into partitioned table - already existing table") { - withTempDirAndEngine { (tempTblPath, engine) => - val tblPath = tempTblPath + "/table+ with special chars" - val partitionCols = Seq("part1", "part2") - - { - val commitResult0 = appendData( - engine, - tblPath, - isNewTable = true, - testPartitionSchema, - testPartitionColumns, - data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) - ) - - val expData = dataPartitionBatches1.flatMap(_.toTestRows) - - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, partitionCols, operation = WRITE) - verifyWrittenContent(tblPath, testPartitionSchema, expData) + Seq(true, false).foreach( + executePostCommitHook => + test("insert into partitioned table - table created from scratch" + { + if (executePostCommitHook) " and execute post commit hook" else "" + }) { + withTempDirAndEngine { + (tblPath, engine) => + val commitResult0 = appendData( + engine, + tblPath, + isNewTable = true, + testPartitionSchema, + testPartitionColumns, + Seq( + Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1, + Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2 + ), + executePostCommitHook = executePostCommitHook + ) + + val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ + dataPartitionBatches2.flatMap(_.toTestRows) + + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, testPartitionColumns, operation = WRITE) + verifyWrittenContent( + tblPath, + testPartitionSchema, + expData, + testPartitionColumns, + version = Some(0), + checksumWritten = executePostCommitHook + ) + } } - { - val commitResult1 = appendData( - engine, - tblPath, - data = Seq(Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2) - ) - - val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ - dataPartitionBatches2.flatMap(_.toTestRows) + ) - verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) - verifyWrittenContent(tblPath, testPartitionSchema, expData) + Seq(true, false).foreach( + executePostCommitHook => + test("insert into partitioned table - already existing table" + { + if (executePostCommitHook) " and execute post commit hook" else "" + }) { + withTempDirAndEngine { + (tempTblPath, engine) => + val tblPath = tempTblPath + "/table+ with special chars" + val partitionCols = Seq("part1", "part2") + + { + val commitResult0 = appendData( + engine, + tblPath, + isNewTable = true, + testPartitionSchema, + testPartitionColumns, + data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1), + executePostCommitHook = executePostCommitHook + ) + + val expData = dataPartitionBatches1.flatMap(_.toTestRows) + + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, partitionCols, operation = WRITE) + verifyWrittenContent( + tblPath, + testPartitionSchema, + expData, + testPartitionColumns, + version = Some(0), + checksumWritten = executePostCommitHook + ) + } + { + val commitResult1 = appendData( + engine, + tblPath, + data = Seq(Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2), + executePostCommitHook = executePostCommitHook + ) + + val expData = + dataPartitionBatches1.flatMap(_.toTestRows) ++ + dataPartitionBatches2.flatMap(_.toTestRows) + + verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) + verifyWrittenContent( + tblPath, + testPartitionSchema, + expData, + testPartitionColumns, + version = Some(1), + checksumWritten = executePostCommitHook + ) + } + } } - } - } - - test("insert into partitioned table - handling case sensitivity of partition columns") { - withTempDirAndEngine { (tblPath, engine) => - val schema = new StructType() - .add("id", INTEGER) - .add("Name", STRING) - .add("Part1", DOUBLE) // partition column - .add("parT2", TIMESTAMP) // partition column - - val partCols = Seq("part1", "Part2") // given as input to the txn builder - - // expected partition cols in the commit info or elsewhere in the Delta log. - // it is expected to contain the partition columns in the same case as the schema - val expPartCols = Seq("Part1", "parT2") - - val v0Part0Values = Map( - "PART1" -> ofDouble(1.0), - "pART2" -> ofTimestamp(1231212L)) - val v0Part0Data = - generateData(schema, expPartCols, v0Part0Values, batchSize = 200, numBatches = 3) - - val v0Part1Values = Map( - "Part1" -> ofDouble(7), - "PARt2" -> ofTimestamp(123112L)) - val v0Part1Data = - generateData(schema, expPartCols, v0Part1Values, batchSize = 100, numBatches = 7) - - val v1Part0Values = Map( - "PART1" -> ofNull(DOUBLE), - "pART2" -> ofTimestamp(1231212L)) - val v1Part0Data = - generateData(schema, expPartCols, v1Part0Values, batchSize = 200, numBatches = 3) - - val v1Part1Values = Map( - "Part1" -> ofDouble(7), - "PARt2" -> ofNull(TIMESTAMP)) - val v1Part1Data = - generateData(schema, expPartCols, v1Part1Values, batchSize = 100, numBatches = 7) - - val dataPerVersion = Map( - 0 -> Seq(v0Part0Values -> v0Part0Data, v0Part1Values -> v0Part1Data), - 1 -> Seq(v1Part0Values -> v1Part0Data, v1Part1Values -> v1Part1Data) - ) - - val expV0Data = v0Part0Data.flatMap(_.toTestRows) ++ v0Part1Data.flatMap(_.toTestRows) - val expV1Data = v1Part0Data.flatMap(_.toTestRows) ++ v1Part1Data.flatMap(_.toTestRows) + ) - for (i <- 0 until 2) { - val commitResult = appendData( - engine, - tblPath, - isNewTable = i == 0, - schema, - partCols, - dataPerVersion(i)) - - verifyCommitResult(commitResult, expVersion = i, expIsReadyForCheckpoint = false) - // 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) + Seq(true, false).foreach( + executePostCommitHook => + test("insert into partitioned table - handling case sensitivity of partition columns" + { + if (executePostCommitHook) " and execute post commit hook" else "" + }) { + withTempDirAndEngine { + (tblPath, engine) => + val schema = new StructType() + .add("id", INTEGER) + .add("Name", STRING) + .add("Part1", DOUBLE) // partition column + .add("parT2", TIMESTAMP) // partition column + + val partCols = Seq("part1", "Part2") // given as input to the txn builder + + // expected partition cols in the commit info or elsewhere in the Delta log. + // it is expected to contain the partition columns in the same case as the schema + val expPartCols = Seq("Part1", "parT2") + + val v0Part0Values = Map("PART1" -> ofDouble(1.0), "pART2" -> ofTimestamp(1231212L)) + val v0Part0Data = + generateData(schema, expPartCols, v0Part0Values, batchSize = 200, numBatches = 3) + + val v0Part1Values = Map("Part1" -> ofDouble(7), "PARt2" -> ofTimestamp(123112L)) + val v0Part1Data = + generateData(schema, expPartCols, v0Part1Values, batchSize = 100, numBatches = 7) + + val v1Part0Values = Map("PART1" -> ofNull(DOUBLE), "pART2" -> ofTimestamp(1231212L)) + val v1Part0Data = + generateData(schema, expPartCols, v1Part0Values, batchSize = 200, numBatches = 3) + + val v1Part1Values = Map("Part1" -> ofDouble(7), "PARt2" -> ofNull(TIMESTAMP)) + val v1Part1Data = + generateData(schema, expPartCols, v1Part1Values, batchSize = 100, numBatches = 7) + + val dataPerVersion = Map( + 0 -> Seq(v0Part0Values -> v0Part0Data, v0Part1Values -> v0Part1Data), + 1 -> Seq(v1Part0Values -> v1Part0Data, v1Part1Values -> v1Part1Data) + ) + + val expV0Data = v0Part0Data.flatMap(_.toTestRows) ++ v0Part1Data.flatMap(_.toTestRows) + val expV1Data = v1Part0Data.flatMap(_.toTestRows) ++ v1Part1Data.flatMap(_.toTestRows) + + for (i <- 0 until 2) { + val commitResult = appendData( + engine, + tblPath, + isNewTable = i == 0, + schema, + partCols, + dataPerVersion(i), + executePostCommitHook = executePostCommitHook + ) + + verifyCommitResult(commitResult, expVersion = i, expIsReadyForCheckpoint = false) + // 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, + partCols, + version = Some(i), + checksumWritten = executePostCommitHook + ) + } + } } - } - } + ) Seq(10, 2).foreach { checkpointInterval => test(s"insert into partitioned table - isReadyForCheckpoint(interval=$checkpointInterval)") { @@ -641,130 +744,196 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } } - test("insert into table - all supported types data") { - withTempDirAndEngine { (tblPath, engine) => - val parquetAllTypes = goldenTablePath("parquet-all-types") - val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) - - val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] - val dataWithPartInfo = Seq(Map.empty[String, Literal] -> data) - - appendData(engine, tblPath, isNewTable = true, schema, Seq.empty, dataWithPartInfo) - var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - - val checkpointInterval = 4 - setCheckpointInterval(tblPath, checkpointInterval) - - for (i <- 2 until 5) { - // insert until a checkpoint is required - val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) - - expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - checkpointIfReady(engine, tblPath, commitResult, expSize = i /* one file per version */) - - verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) - verifyCommitInfo(tblPath, version = i, null, operation = WRITE) - verifyWrittenContent(tblPath, schema, expData) - } - assertCheckpointExists(tblPath, atVersion = checkpointInterval) - } - } - - test("insert into partitioned table - all supported partition column types data") { - withTempDirAndEngine { (tblPath, engine) => - val parquetAllTypes = goldenTablePath("parquet-all-types") - val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) - val partCols = Seq( - "byteType", - "shortType", - "integerType", - "longType", - "floatType", - "doubleType", - "decimal", - "booleanType", - "stringType", - "binaryType", - "dateType", - "timestampType" - ) - val casePreservingPartCols = - casePreservingPartitionColNames(schema, partCols.asJava).asScala.to[Seq] - - // get the partition values from the data batch at the given rowId - def getPartitionValues(batch: ColumnarBatch, rowId: Int): Map[String, Literal] = { - casePreservingPartCols.map { partCol => - val colIndex = schema.indexOf(partCol) - val vector = batch.getColumnVector(colIndex) - - val literal = if (vector.isNullAt(rowId)) { - Literal.ofNull(vector.getDataType) - } else { - vector.getDataType match { - case _: ByteType => Literal.ofByte(vector.getByte(rowId)) - case _: ShortType => Literal.ofShort(vector.getShort(rowId)) - case _: IntegerType => Literal.ofInt(vector.getInt(rowId)) - case _: LongType => Literal.ofLong(vector.getLong(rowId)) - case _: FloatType => Literal.ofFloat(vector.getFloat(rowId)) - case _: DoubleType => Literal.ofDouble(vector.getDouble(rowId)) - case dt: DecimalType => - Literal.ofDecimal(vector.getDecimal(rowId), dt.getPrecision, dt.getScale) - case _: BooleanType => Literal.ofBoolean(vector.getBoolean(rowId)) - case _: StringType => Literal.ofString(vector.getString(rowId)) - case _: BinaryType => Literal.ofBinary(vector.getBinary(rowId)) - case _: DateType => Literal.ofDate(vector.getInt(rowId)) - case _: TimestampType => Literal.ofTimestamp(vector.getLong(rowId)) - case _ => - throw new IllegalArgumentException(s"Unsupported type: ${vector.getDataType}") + Seq(true, false).foreach( + executePostCommitHook => + test("insert into table - all supported types data" + { + if (executePostCommitHook) " and execute post commit hook" else "" + }) { + withTempDirAndEngine { + (tblPath, engine) => + val parquetAllTypes = goldenTablePath("parquet-all-types") + val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) + + val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] + val dataWithPartInfo = Seq(Map.empty[String, Literal] -> data) + + appendData( + engine, + tblPath, + isNewTable = true, + schema, + Seq.empty, + dataWithPartInfo, + executePostCommitHook = executePostCommitHook + ) + var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + + val checkpointInterval = 4 + setCheckpointInterval(tblPath, checkpointInterval) + + for (i <- 2 until 5) { + // insert until a checkpoint is required + val commitResult = appendData( + engine, + tblPath, + data = dataWithPartInfo, + executePostCommitHook = executePostCommitHook + ) + + expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + // post commit hooks contains check point + if (!executePostCommitHook) { + checkpointIfReady( + engine, + tblPath, + commitResult, + expSize = i /* one file per version */ + ) + } + + verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) + verifyCommitInfo(tblPath, version = i, null, operation = WRITE) + verifyWrittenContent( + tblPath, + schema, + expData, + version = Some(i), + checksumWritten = executePostCommitHook + ) } - } - (partCol, literal) - }.toMap - } - - val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] - - // From the above table read data, convert each row as a new batch with partition info - // Take the values of the partitionCols from the data and create a new batch with the - // selection vector to just select a single row. - var dataWithPartInfo = Seq.empty[(Map[String, Literal], Seq[FilteredColumnarBatch])] - - data.foreach { filteredBatch => - val batch = filteredBatch.getData - Seq.range(0, batch.getSize).foreach { rowId => - val partValues = getPartitionValues(batch, rowId) - val filteredBatch = new FilteredColumnarBatch( - batch, - Optional.of(selectSingleElement(batch.getSize, rowId))) - dataWithPartInfo = dataWithPartInfo :+ (partValues, Seq(filteredBatch)) + assertCheckpointExists(tblPath, atVersion = checkpointInterval) } } + ) - appendData(engine, tblPath, isNewTable = true, schema, partCols, dataWithPartInfo) - verifyCommitInfo(tblPath, version = 0, casePreservingPartCols, operation = WRITE) - - var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - - val checkpointInterval = 2 - setCheckpointInterval(tblPath, checkpointInterval) // version 1 - - for (i <- 2 until 4) { - // insert until a checkpoint is required - val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) + Seq(true, false).foreach( + executePostCommitHook => + test("insert into partitioned table - all supported partition column types data" + { + if (executePostCommitHook) " and execute post commit hook" else "" + }) { + withTempDirAndEngine { + (tblPath, engine) => + val parquetAllTypes = goldenTablePath("parquet-all-types") + val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) + val partCols = Seq( + "byteType", + "shortType", + "integerType", + "longType", + "floatType", + "doubleType", + "decimal", + "booleanType", + "stringType", + "binaryType", + "dateType", + "timestampType" + ) + val casePreservingPartCols = + casePreservingPartitionColNames(schema, partCols.asJava).asScala.to[Seq] + + // get the partition values from the data batch at the given rowId + def getPartitionValues(batch: ColumnarBatch, rowId: Int): Map[String, Literal] = { + casePreservingPartCols.map { + partCol => + val colIndex = schema.indexOf(partCol) + val vector = batch.getColumnVector(colIndex) + + val literal = if (vector.isNullAt(rowId)) { + Literal.ofNull(vector.getDataType) + } else { + vector.getDataType match { + case _: ByteType => Literal.ofByte(vector.getByte(rowId)) + case _: ShortType => Literal.ofShort(vector.getShort(rowId)) + case _: IntegerType => Literal.ofInt(vector.getInt(rowId)) + case _: LongType => Literal.ofLong(vector.getLong(rowId)) + case _: FloatType => Literal.ofFloat(vector.getFloat(rowId)) + case _: DoubleType => Literal.ofDouble(vector.getDouble(rowId)) + case dt: DecimalType => + Literal.ofDecimal(vector.getDecimal(rowId), dt.getPrecision, dt.getScale) + case _: BooleanType => Literal.ofBoolean(vector.getBoolean(rowId)) + case _: StringType => Literal.ofString(vector.getString(rowId)) + case _: BinaryType => Literal.ofBinary(vector.getBinary(rowId)) + case _: DateType => Literal.ofDate(vector.getInt(rowId)) + case _: TimestampType => Literal.ofTimestamp(vector.getLong(rowId)) + case _ => + throw new IllegalArgumentException( + s"Unsupported type: ${vector.getDataType}" + ) + } + } + (partCol, literal) + }.toMap + } - expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] + + // From the above table read data, convert each row as a new batch with partition info + // Take the values of the partitionCols from the data and create a new batch with the + // selection vector to just select a single row. + var dataWithPartInfo = Seq.empty[(Map[String, Literal], Seq[FilteredColumnarBatch])] + + data.foreach { + filteredBatch => + val batch = filteredBatch.getData + Seq.range(0, batch.getSize).foreach { rowId => + val partValues = getPartitionValues(batch, rowId) + val filteredBatch = new FilteredColumnarBatch( + batch, + Optional.of(selectSingleElement(batch.getSize, rowId)) + ) + dataWithPartInfo = dataWithPartInfo :+ (partValues, Seq(filteredBatch)) + } + } - val fileCount = dataFileCount(tblPath) - checkpointIfReady(engine, tblPath, commitResult, expSize = fileCount) + appendData( + engine, + tblPath, + isNewTable = true, + schema, + partCols, + dataWithPartInfo, + executePostCommitHook = executePostCommitHook + ) + verifyCommitInfo(tblPath, version = 0, casePreservingPartCols, operation = WRITE) + + var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + + val checkpointInterval = 2 + setCheckpointInterval(tblPath, checkpointInterval) // version 1 + + for (i <- 2 until 4) { + // insert until a checkpoint is required + val commitResult = appendData( + engine, + tblPath, + data = dataWithPartInfo, + executePostCommitHook = executePostCommitHook + ) + + expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + + val fileCount = dataFileCount(tblPath) + if (!executePostCommitHook) { + checkpointIfReady(engine, tblPath, commitResult, expSize = fileCount) + } + + verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) + verifyCommitInfo(tblPath, version = i, partitionCols = null, operation = WRITE) + verifyWrittenContent( + tblPath, + schema, + expData, + partCols, + version = Some(i), + checksumWritten = executePostCommitHook + ) + } - verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) - verifyCommitInfo(tblPath, version = i, partitionCols = null, operation = WRITE) - verifyWrittenContent(tblPath, schema, expData) + assertCheckpointExists(tblPath, atVersion = checkpointInterval) + } } - - assertCheckpointExists(tblPath, atVersion = checkpointInterval) - } - } + ) test("insert into table - given data schema mismatch") { withTempDirAndEngine { (tblPath, engine) => @@ -811,99 +980,150 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } } - test("insert into table - idempotent writes") { - withTempDirAndEngine { (tblPath, engine) => - val data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) - var expData = Seq.empty[TestRow] // as the data in inserted, update this. - - def prepTxnAndActions(newTbl: Boolean, appId: String, txnVer: Long) - : (Transaction, CloseableIterable[Row]) = { - var txnBuilder = createWriteTxnBuilder(Table.forPath(engine, tblPath)) - - if (appId != null) txnBuilder = txnBuilder.withTransactionId(engine, appId, txnVer) - - if (newTbl) { - txnBuilder = txnBuilder.withSchema(engine, testPartitionSchema) - .withPartitionColumns(engine, testPartitionColumns.asJava) - } - val txn = txnBuilder.build(engine) - - val combinedActions = inMemoryIterable( - data.map { case (partValues, partData) => - stageData(txn.getTransactionState(engine), partValues, partData) - }.reduceLeft(_ combine _)) - - (txn, combinedActions) - } - - def commitAndVerify(newTbl: Boolean, txn: Transaction, - actions: CloseableIterable[Row], expTblVer: Long): Unit = { - val commitResult = txn.commit(engine, actions) + Seq(true, false).foreach( + executePostCommitHook => + test("insert into table - idempotent writes" + { + if (executePostCommitHook) " and execute post commit hook" else "" + }) { + withTempDirAndEngine { + (tblPath, engine) => + val data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) + var expData = Seq.empty[TestRow] // as the data in inserted, update this. + + def prepTxnAndActions( + newTbl: Boolean, + appId: String, + txnVer: Long): (Transaction, CloseableIterable[Row]) = { + var txnBuilder = createWriteTxnBuilder(Table.forPath(engine, tblPath)) + + if (appId != null) txnBuilder = txnBuilder.withTransactionId(engine, appId, txnVer) + + if (newTbl) { + txnBuilder = txnBuilder. + withSchema(engine, testPartitionSchema). + withPartitionColumns(engine, testPartitionColumns.asJava) + } + val txn = txnBuilder.build(engine) + val combinedActions = inMemoryIterable( + data.map { case (partValues, partData) => + stageData(txn.getTransactionState(engine), partValues, partData) + }.reduceLeft(_ combine _)) + (txn, combinedActions) + } - expData = expData ++ data.flatMap(_._2).flatMap(_.toTestRows) + def commitAndVerify( + newTbl: Boolean, + txn: Transaction, + actions: CloseableIterable[Row], + expTblVer: Long, + checksumWritten: Boolean): Unit = { + val commitResult = txn.commit(engine, actions) + if (executePostCommitHook) { + commitResult.getPostCommitHooks.forEach( + hook => hook.threadSafeInvoke(defaultEngine) + ) + } + + expData = expData ++ data.flatMap(_._2).flatMap(_.toTestRows) + + verifyCommitResult( + commitResult, + expVersion = expTblVer, + expIsReadyForCheckpoint = false + ) + val expPartCols = if (newTbl) testPartitionColumns else null + verifyCommitInfo(tblPath, version = expTblVer, expPartCols, operation = WRITE) + verifyWrittenContent( + tblPath, + testPartitionSchema, + expData, + testPartitionColumns, + version = Some(expTblVer), + checksumWritten + ) + } - verifyCommitResult(commitResult, expVersion = expTblVer, expIsReadyForCheckpoint = false) - val expPartCols = if (newTbl) testPartitionColumns else null - verifyCommitInfo(tblPath, version = expTblVer, expPartCols, operation = WRITE) - verifyWrittenContent(tblPath, testPartitionSchema, expData) - } + def addDataWithTxnId( + newTbl: Boolean, + appId: String, + txnVer: Long, + expTblVer: Long): Unit = { + val (txn, combinedActions) = prepTxnAndActions(newTbl, appId, txnVer) + commitAndVerify(newTbl, txn, combinedActions, expTblVer, executePostCommitHook) + } - def addDataWithTxnId(newTbl: Boolean, appId: String, txnVer: Long, expTblVer: Long): Unit = { - val (txn, combinedActions) = prepTxnAndActions(newTbl, appId, txnVer) - commitAndVerify(newTbl, txn, combinedActions, expTblVer) - } + def expFailure(appId: String, txnVer: Long, latestTxnVer: Long)(fn: => Any): Unit = { + val ex = intercept[ConcurrentTransactionException] { + fn + } + assert( + ex.getMessage.contains( + s"This error occurs when multiple updates are using the " + + s"same transaction identifier to write into " + + s"this table.\nApplication ID: $appId, " + + s"Attempted version: $txnVer, Latest version in table: $latestTxnVer" + ) + ) + } - def expFailure(appId: String, txnVer: Long, latestTxnVer: Long)(fn: => Any): Unit = { - val ex = intercept[ConcurrentTransactionException] { - fn - } - assert(ex.getMessage.contains(s"This error occurs when multiple updates are using the " + - s"same transaction identifier to write into this table.\nApplication ID: $appId, " + - s"Attempted version: $txnVer, Latest version in table: $latestTxnVer")) - } + // Create a transaction with id (txnAppId1, 0) and commit it + addDataWithTxnId(newTbl = true, appId = "txnAppId1", txnVer = 0, expTblVer = 0) - // Create a transaction with id (txnAppId1, 0) and commit it - addDataWithTxnId(newTbl = true, appId = "txnAppId1", txnVer = 0, expTblVer = 0) + // Try to create a transaction with id (txnAppId1, 0) and commit it - should be valid + addDataWithTxnId(newTbl = false, appId = "txnAppId1", txnVer = 1, expTblVer = 1) - // Try to create a transaction with id (txnAppId1, 0) and commit it - should be valid - addDataWithTxnId(newTbl = false, appId = "txnAppId1", txnVer = 1, expTblVer = 1) + // Try to create a transaction with id (txnAppId1, 1) and try to commit it + // Should fail the it is already committed above. + expFailure("txnAppId1", txnVer = 1, latestTxnVer = 1) { + addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 1, expTblVer = 2) + } - // Try to create a transaction with id (txnAppId1, 1) and try to commit it - // Should fail the it is already committed above. - expFailure("txnAppId1", txnVer = 1, latestTxnVer = 1) { - addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 1, expTblVer = 2) - } + // append with no txn id + addDataWithTxnId(newTbl = false, appId = null, txnVer = 0, expTblVer = 2) - // append with no txn id - addDataWithTxnId(newTbl = false, appId = null, txnVer = 0, expTblVer = 2) + // Try to create a transaction with id (txnAppId2, 1) and commit it + // Should be successful as the transaction app id is different + addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 1, expTblVer = 3) - // Try to create a transaction with id (txnAppId2, 1) and commit it - // Should be successful as the transaction app id is different - addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 1, expTblVer = 3) + // Try to create a transaction with id (txnAppId2, 0) and commit it + // Should fail as the transaction app id is same + // but the version is less than the committed + expFailure("txnAppId2", txnVer = 0, latestTxnVer = 1) { + addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 0, expTblVer = 4) + } - // Try to create a transaction with id (txnAppId2, 0) and commit it - // Should fail as the transaction app id is same but the version is less than the committed - expFailure("txnAppId2", txnVer = 0, latestTxnVer = 1) { - addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 0, expTblVer = 4) - } + // Start a transaction (txnAppId2, 2), but don't commit it yet + val (txn, combinedActions) = prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 2) + // Now start a new transaction with the same id (txnAppId2, 2) and commit it + addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 2, expTblVer = 4) + // Now try to commit the previous transaction (txnAppId2, 2) - should fail + expFailure("txnAppId2", txnVer = 2, latestTxnVer = 2) { + commitAndVerify( + newTbl = false, + txn, + combinedActions, + expTblVer = 5, + checksumWritten = false + ) + } - // Start a transaction (txnAppId2, 2), but don't commit it yet - val (txn, combinedActions) = prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 2) - // Now start a new transaction with the same id (txnAppId2, 2) and commit it - addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 2, expTblVer = 4) - // Now try to commit the previous transaction (txnAppId2, 2) - should fail - expFailure("txnAppId2", txnVer = 2, latestTxnVer = 2) { - commitAndVerify(newTbl = false, txn, combinedActions, expTblVer = 5) + // Start a transaction (txnAppId2, 3), but don't commit it yet + val (txn2, combinedActions2) = + prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 3) + // Now start a new transaction with the different id (txnAppId1, 10) and commit it + addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 10, expTblVer = 5) + // Now try to commit the previous transaction (txnAppId2, 3) - should pass + // checksum file is not written + commitAndVerify( + newTbl = false, + txn2, + combinedActions2, + expTblVer = 6, + checksumWritten = false + ) + } } - - // Start a transaction (txnAppId2, 3), but don't commit it yet - val (txn2, combinedActions2) = prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 3) - // Now start a new transaction with the different id (txnAppId1, 10) and commit it - addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 10, expTblVer = 5) - // Now try to commit the previous transaction (txnAppId2, 3) - should pass - commitAndVerify(newTbl = false, txn2, combinedActions2, expTblVer = 6) - } - } + ) test("conflicts - creating new table - table created by other txn after current txn start") { withTempDirAndEngine { (tablePath, engine) => From 3b7dc2c25c6ffefc5ff17c8238bf76efd09d8cb8 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Sun, 9 Feb 2025 22:39:52 -0800 Subject: [PATCH 04/73] add tests --- .../ChecksumSimpleComparisonSuite.scala | 343 ++++++++++++++++++ .../kernel/defaults/ChecksumSimpleSuite.scala | 5 - 2 files changed, 343 insertions(+), 5 deletions(-) create mode 100644 kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala delete mode 100644 kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleSuite.scala 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..f0a03a6cf77 --- /dev/null +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala @@ -0,0 +1,343 @@ +/* + * 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 io.delta.kernel.Operation.CREATE_TABLE +import io.delta.kernel.{Operation, Table, Transaction} +import io.delta.kernel.data.Row +import io.delta.kernel.defaults.utils.TestUtils +import io.delta.kernel.engine.Engine +import io.delta.kernel.expressions.Literal +import io.delta.kernel.internal.actions.{AddFile, SingleAction} +import io.delta.kernel.internal.checksum.{CRCInfo, ChecksumReader} +import io.delta.kernel.internal.fs.Path +import io.delta.kernel.internal.util.{FileNames, VectorUtils} +import io.delta.kernel.internal.util.Utils.{singletonCloseableIterator, toCloseableIterator} +import io.delta.kernel.types.IntegerType.INTEGER +import io.delta.kernel.types.StructType +import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} +import io.delta.kernel.utils.{CloseableIterator, DataFileStatus, FileStatus} +import org.scalatest.funsuite.AnyFunSuite + +import java.io.File +import java.nio.file.Files +import java.util +import java.util.Collections.{emptyMap, singletonMap} +import java.util.Optional +import scala.collection.JavaConverters._ + +/** + * Test suites to copy delta-spark's log to delta kernel and verify correctness of written checksum file. + */ +class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { + + test("create table, insert data and verify checksum") { + withTempDirAndEngine { (tablePath, engine) => + val sparkTablePath = tablePath + "spark" + val kernelTablePath = tablePath + "kernel" + Table + .forPath(engine, kernelTablePath) + .createTransactionBuilder(engine, "test-engine", CREATE_TABLE) + .withSchema(engine, new StructType().add("id", INTEGER)) + .build(engine) + .commit(engine, emptyIterable()) + .getPostCommitHooks + .forEach( + hook => hook.threadSafeInvoke(engine) + ) + spark.sql(s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` (id Integer) USING DELTA") + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + + (1 to 10).foreach( + version => insertIntoUnpartitionedTable(engine, sparkTablePath, kernelTablePath, version) + ) + + } + } + + test("create table as select and verify checksum") { + withTempDirAndEngine { (tablePath, engine) => + val sparkTablePath = tablePath + "spark" + val kernelTablePath = tablePath + "kernel" + spark.sql( + s"CREATE OR REPLACE TABLE delta.`${tablePath + "spark"}` USING DELTA AS SELECT 1 as id" + ) + val txn = Table + .forPath(engine, kernelTablePath) + .createTransactionBuilder(engine, "test-engine", CREATE_TABLE) + .withSchema(engine, new StructType().add("id", INTEGER)) + .build(engine) + + copyCommitAndCommitTxnForUnpartitionedTable(txn, engine, sparkTablePath, versionAtCommit = 0) + + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + + (1 to 10).foreach( + version => insertIntoUnpartitionedTable(engine, sparkTablePath, kernelTablePath, version) + ) + } + } + + test("create partitioned table and verify checksum") { + withTempDirAndEngine { (tablePath, engine) => + val sparkTablePath = tablePath + "spark" + val kernelTablePath = tablePath + "kernel" + Table + .forPath(engine, kernelTablePath) + .createTransactionBuilder(engine, "test-engine", CREATE_TABLE) + .withSchema(engine, new StructType().add("id", INTEGER).add("part", INTEGER)) + .withPartitionColumns(engine, Seq("part").asJava) + .build(engine) + .commit(engine, emptyIterable()) + .getPostCommitHooks + .forEach( + hook => hook.threadSafeInvoke(engine) + ) + spark.sql( + s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` " + + s"(id Integer, part Integer) USING DELTA PARTITIONED BY (part)" + ) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + + (1 to 10).foreach( + version => insertIntoPartitionedTable(engine, sparkTablePath, kernelTablePath, version) + ) + } + } + + test("create partition table as select and verify checksum") { + withTempDirAndEngine { (tablePath, engine) => + val sparkTablePath = tablePath + "spark" + val kernelTablePath = tablePath + "kernel" + spark.sql( + s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` USING DELTA " + + s"PARTITIONED BY (part) AS " + + s"SELECT 1 as id, 1 as part union all select 2 as id, 2 as part" + ) + val txn = Table + .forPath(engine, kernelTablePath) + .createTransactionBuilder(engine, "test-engine", CREATE_TABLE) + .withSchema(engine, new StructType().add("id", INTEGER).add("part", INTEGER)) + .withPartitionColumns(engine, Seq("part").asJava) + .build(engine) + + copyCommitAndCommitTxnForPartitionedTable( + txn, + engine, + sparkTablePath, + Seq(1, 2).toSet, + versionAtCommit = 0 + ) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + + (1 to 10).foreach( + version => insertIntoPartitionedTable(engine, sparkTablePath, kernelTablePath, version) + ) + } + } + + def assertChecksumEquals( + engine: Engine, + sparkTablePath: String, + kernelTablePath: String, + version: Long): Unit = { + assert( + Files.exists( + new File(f"$sparkTablePath/_delta_log/$version%020d.crc").toPath + ) && + Files.exists( + new File(f"$kernelTablePath/_delta_log/$version%020d.crc").toPath + ) + ) + assertCrcInfoEquals( + ChecksumReader + .getCRCInfo(engine, new Path(f"$sparkTablePath/_delta_log"), version, version) + .get(), + ChecksumReader + .getCRCInfo(engine, new Path(f"$kernelTablePath/_delta_log"), version, version) + .get() + ) + } + + 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) + } + + def insertIntoPartitionedTable( + engine: Engine, + sparkTablePath: String, + kernelTablePath: String, + versionAtCommit: Long): Unit = { + var valueToAppend = "(0, 0)" + var addedPartition = Set(0) + (0L to versionAtCommit).foreach(i => { + val partitionValue = 2 * i + addedPartition = addedPartition + partitionValue.toInt + valueToAppend = valueToAppend + s",($i, $partitionValue)" + }) + spark.sql( + s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend" + ) + + val txn = Table + .forPath(engine, kernelTablePath) + .createTransactionBuilder(engine, "test-engine", Operation.WRITE) + .build(engine) + + copyCommitAndCommitTxnForPartitionedTable( + txn, + engine, + sparkTablePath, + addedPartition, + versionAtCommit + ) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) + } + + def insertIntoUnpartitionedTable( + engine: Engine, + sparkTablePath: String, + kernelTablePath: String, + versionAtCommit: Long): Unit = { + var valueToAppend = "(0)" + (0L to versionAtCommit).foreach(i => valueToAppend = valueToAppend + s",($i)") + spark.sql( + s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend" + ) + + val txn = Table + .forPath(engine, kernelTablePath) + .createTransactionBuilder(engine, "test-engine", Operation.WRITE) + .build(engine) + + copyCommitAndCommitTxnForUnpartitionedTable(txn, engine, sparkTablePath, versionAtCommit) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) + } + + def copyCommitAndCommitTxnForUnpartitionedTable( + txn: Transaction, + engine: Engine, + sparkTablePath: String, + versionAtCommit: Long): Unit = { + val txnState = txn.getTransactionState(engine); + + val writeContext = Transaction + .getWriteContext(engine, txnState, emptyMap()) + + val dataActions = Transaction + .generateAppendActions( + engine, + txnState, + convertDeltaLogToAppendActions(engine, sparkTablePath, versionAtCommit, Option.empty), + writeContext + ) + + txn + .commit(engine, inMemoryIterable(dataActions)) + .getPostCommitHooks + .forEach( + hook => hook.threadSafeInvoke(engine) + ) + + } + + def copyCommitAndCommitTxnForPartitionedTable( + txn: Transaction, + engine: Engine, + sparkTablePath: String, + addedPartition: Set[Int], + versionAtCommit: Long): Unit = { + val txnState = txn.getTransactionState(engine); + + val dataActions = new util.ArrayList[Row]() + + addedPartition.foreach({ partition => + val writeContext = Transaction + .getWriteContext( + engine, + txnState, + singletonMap("part", Literal.ofInt(partition)) + ) + + Transaction + .generateAppendActions( + engine, + txnState, + convertDeltaLogToAppendActions( + engine, + sparkTablePath, + versionAtCommit, + Some(partition.toString) + ), + writeContext + ) + .forEach( + action => dataActions.add(action) + ) + }) + + txn + .commit(engine, inMemoryIterable(toCloseableIterator(dataActions.iterator()))) + .getPostCommitHooks + .forEach( + hook => hook.threadSafeInvoke(engine) + ) + + } + + def convertDeltaLogToAppendActions( + engine: Engine, + tablePath: String, + version: Long, + partition: Option[String]): CloseableIterator[DataFileStatus] = { + val logPath = new Path(tablePath, "_delta_log") + val file = FileStatus.of(FileNames.deltaFile(logPath, version), 0, 0) + val columnarBatches = engine.getJsonHandler.readJsonFiles( + singletonCloseableIterator(file), + SingleAction.FULL_SCHEMA, + Optional.empty() + ) + val addFiles = new util.ArrayList[DataFileStatus]() + while (columnarBatches.hasNext) { + val batch = columnarBatches.next + val rows = batch.getRows + while (rows.hasNext) { + val row = rows.next() + if (!row.isNullAt(row.getSchema.indexOf("add"))) { + val addFile = new AddFile(row.getStruct(row.getSchema.indexOf("add"))) + if (partition.isEmpty || partition.get == VectorUtils + .toJavaMap(addFile.getPartitionValues) + .get("part")) { + addFiles.add( + new DataFileStatus( + addFile.getPath, + addFile.getSize, + addFile.getModificationTime, + // TODO: populate the stats once https://github.com/delta-io/delta/issues/4139 fixed + Optional.empty() + ) + ) + } + } + } + } + toCloseableIterator(addFiles.iterator()) + } +} diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleSuite.scala deleted file mode 100644 index 8752d02e2c2..00000000000 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ChecksumSimpleSuite.scala +++ /dev/null @@ -1,5 +0,0 @@ -package io.delta.kernel.defaults -import io.delta.kernel.defaults.utils.TestUtils -import org.scalatest.funsuite.AnyFunSuite - -class ChecksumSimpleSuite extends AnyFunSuite with TestUtils {} From 2f58dfaa4b53cebb1e276a4ce65f9bc3b25fa957 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Sun, 9 Feb 2025 22:42:55 -0800 Subject: [PATCH 05/73] fix scala format --- .../kernel/defaults/ChecksumSimpleComparisonSuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) 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 index f0a03a6cf77..2c86009657d 100644 --- 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 @@ -39,9 +39,10 @@ import java.util.Collections.{emptyMap, singletonMap} import java.util.Optional import scala.collection.JavaConverters._ -/** - * Test suites to copy delta-spark's log to delta kernel and verify correctness of written checksum file. - */ +/* * + * Test suites to copy delta-spark's log to delta kernel and verify + * the correctness of written checksum file. + * */ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { test("create table, insert data and verify checksum") { From e7f4beb7f4ca197e9203aa44e2645826fe2ec28d Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Sun, 9 Feb 2025 22:59:31 -0800 Subject: [PATCH 06/73] add unit test --- .../checksum/ChecksumWriterSuite.scala | 179 ++++++++++++++++++ 1 file changed, 179 insertions(+) create mode 100644 kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala 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..16113ec784c --- /dev/null +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala @@ -0,0 +1,179 @@ +/* + * 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 io.delta.kernel.data.Row +import io.delta.kernel.internal.actions.{Format, Metadata, Protocol} +import io.delta.kernel.internal.checksum.ChecksumUtils.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 + +import java.util +import java.util.{Collections, Optional, OptionalLong} + +class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { + + private val FAKE_DELTA_LOG_PATH = new Path("/path/to/delta/log") + + 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 = { + checksumWriter.writeCheckSum( + mockEngine(jsonHandler = jsonHandler), + new CRCInfo(1L, metadata, protocol, 100L, 1L, txn) + ) + + assert(jsonHandler.checksumFilePath == "/path/to/delta/log/00000000000000000001.crc") + assert(jsonHandler.capturedCrcRow.getSchema == CRC_FILE_SCHEMA) + assert( + jsonHandler.capturedCrcRow + .getLong(CRC_FILE_SCHEMA.indexOf("tableSizeBytes")) == 100L + ) + assert( + jsonHandler.capturedCrcRow.getLong(CRC_FILE_SCHEMA.indexOf("numFiles")) == 1L + ) + assert( + jsonHandler.capturedCrcRow + .getLong(CRC_FILE_SCHEMA.indexOf("numMetadata")) == 1L + ) + assert( + jsonHandler.capturedCrcRow + .getLong(CRC_FILE_SCHEMA.indexOf("numProtocol")) == 1L + ) + + if (txn.isPresent) { + assert( + jsonHandler.capturedCrcRow.getString( + CRC_FILE_SCHEMA.indexOf("txnId") + ) == txn.get() + ) + } else { + assert(jsonHandler.capturedCrcRow.isNullAt(CRC_FILE_SCHEMA.indexOf("txnId"))) + } + + checkMetadata( + metadata, + jsonHandler.capturedCrcRow.getStruct(CRC_FILE_SCHEMA.indexOf("metadata")) + ) + checkProtocol( + protocol, + jsonHandler.capturedCrcRow.getStruct(CRC_FILE_SCHEMA.indexOf("protocol")) + ) + + } + testChecksumWrite(Optional.of("txn")); + testChecksumWrite(Optional.empty()); + } + + 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") + }) + ) + } + + def createTestProtocol(): Protocol = { + new Protocol( + /* minReaderVersion= */ 1, + /* minWriterVersion= */ 2, + Collections.emptyList(), + Collections.emptyList() + ) + } + + def checkMetadata(metadata: Metadata, metadataRow: Row): Unit = { + assert(metadataRow.getSchema == Metadata.FULL_SCHEMA) + assert(metadataRow.getString(Metadata.FULL_SCHEMA.indexOf("id")) == metadata.getId) + assert( + Optional + .ofNullable(metadataRow.getString(Metadata.FULL_SCHEMA.indexOf("name"))) == metadata.getName + ) + assert( + Optional.ofNullable(metadataRow.getString(Metadata.FULL_SCHEMA.indexOf("description"))) + == metadata.getDescription + ) + assert( + metadataRow + .getStruct( + Metadata.FULL_SCHEMA.indexOf("format") + ) + .getString(Format.FULL_SCHEMA.indexOf("provider")) == metadata.getFormat.getProvider + ) + assert( + metadataRow + .getString(Metadata.FULL_SCHEMA.indexOf("schemaString")) == metadata.getSchemaString + ) + assert( + metadataRow + .getArray(Metadata.FULL_SCHEMA.indexOf("partitionColumns")) == metadata.getPartitionColumns + ) + assert( + Optional + .ofNullable(metadataRow.getLong(Metadata.FULL_SCHEMA.indexOf("createdTime"))) == metadata.getCreatedTime + ) + assert( + VectorUtils + .toJavaMap(metadataRow.getMap(Metadata.FULL_SCHEMA.indexOf("configuration"))) == metadata.getConfiguration + ) + } + + def checkProtocol(protocol: Protocol, protocolRow: Row): Unit = { + assert(protocolRow.getSchema == Protocol.FULL_SCHEMA) + assert( + protocol.getMinReaderVersion == + protocolRow.getInt(Protocol.FULL_SCHEMA.indexOf("minReaderVersion")) + ) + assert( + protocol.getMinWriterVersion == + protocolRow.getInt(Protocol.FULL_SCHEMA.indexOf("minWriterVersion")) + ) + } +} + +class MockCheckSumFileJsonWriter extends BaseMockJsonHandler { + var capturedCrcRow: Row = new GenericRow(new StructType(), new util.HashMap[Integer, AnyRef]); + var checksumFilePath = "" + + override def writeJsonFileAtomically( + filePath: String, + data: CloseableIterator[Row], + overwrite: Boolean): Unit = { + checksumFilePath = filePath + assert(data.hasNext) + capturedCrcRow = data.next() + assert(!data.hasNext) + } + +} From 52bda11f377b8722ffc084d8d5c7bdd7082ff389 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Mon, 10 Feb 2025 10:19:18 -0800 Subject: [PATCH 07/73] add doc --- .../src/main/java/io/delta/kernel/hook/PostCommitHook.java | 5 +++++ 1 file changed, 5 insertions(+) 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 741d3a3676b..0ba4e89801e 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 @@ -34,6 +34,11 @@ enum PostCommitHookType { * checkpoint, with latency scaling based on log size (typically seconds to minutes). */ CHECKPOINT, + /** + * Writes a checksum file at the version committed by the transaction. This hook is present + * when the read snapshot is constructed based on previous checksum file. + * To perform this operation, no extra reading previous checkpoint or logs required. + */ CHECKSUM_SIMPLE } From d1626dc0fbc193ab69a90484f9f4d855de4e9d12 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Mon, 10 Feb 2025 10:33:16 -0800 Subject: [PATCH 08/73] format java --- .../src/main/java/io/delta/kernel/hook/PostCommitHook.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 0ba4e89801e..c3dbc26c058 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 @@ -35,9 +35,9 @@ enum PostCommitHookType { */ CHECKPOINT, /** - * Writes a checksum file at the version committed by the transaction. This hook is present - * when the read snapshot is constructed based on previous checksum file. - * To perform this operation, no extra reading previous checkpoint or logs required. + * Writes a checksum file at the version committed by the transaction. This hook is present when + * the read snapshot is constructed based on previous checksum file. To perform this operation, + * no extra reading previous checkpoint or logs required. */ CHECKSUM_SIMPLE } From 2a1092d489ffbfb2ae682788a8c0052ee9a8ce53 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Mon, 10 Feb 2025 10:51:25 -0800 Subject: [PATCH 09/73] format scala --- .../kernel/internal/checksum/ChecksumWriterSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 index 16113ec784c..52be49e4bf0 100644 --- 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 @@ -140,12 +140,12 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { .getArray(Metadata.FULL_SCHEMA.indexOf("partitionColumns")) == metadata.getPartitionColumns ) assert( - Optional - .ofNullable(metadataRow.getLong(Metadata.FULL_SCHEMA.indexOf("createdTime"))) == metadata.getCreatedTime + Optional.ofNullable(metadataRow.getLong( + Metadata.FULL_SCHEMA.indexOf("createdTime"))) == metadata.getCreatedTime ) assert( - VectorUtils - .toJavaMap(metadataRow.getMap(Metadata.FULL_SCHEMA.indexOf("configuration"))) == metadata.getConfiguration + VectorUtils.toJavaMap(metadataRow.getMap( + Metadata.FULL_SCHEMA.indexOf("configuration"))) == metadata.getConfiguration ) } From 6c8069ff65c50e0b8ea3dc51873dec8c073e9c57 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Mon, 10 Feb 2025 12:48:23 -0800 Subject: [PATCH 10/73] fix test --- .../io/delta/kernel/defaults/DeltaTableWritesSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 dbc38a5f3e9..458ef64b252 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 @@ -771,7 +771,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val checkpointInterval = 4 setCheckpointInterval(tblPath, checkpointInterval) - for (i <- 2 until 5) { + for (i <- 1 until 5) { // insert until a checkpoint is required val commitResult = appendData( engine, @@ -902,7 +902,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val checkpointInterval = 2 setCheckpointInterval(tblPath, checkpointInterval) // version 1 - for (i <- 2 until 4) { + for (i <- 1 until 4) { // insert until a checkpoint is required val commitResult = appendData( engine, From 471e17215553c597b81ee543b77c8556c5c1b1bb Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Mon, 10 Feb 2025 13:45:49 -0800 Subject: [PATCH 11/73] fix test --- .../kernel/defaults/DeltaTableWritesSuite.scala | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) 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 458ef64b252..6e06e331c29 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 @@ -39,6 +39,7 @@ import io.delta.kernel.types.TimestampType.TIMESTAMP import io.delta.kernel.types._ import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} import io.delta.kernel.utils.CloseableIterable +import org.apache.spark.sql.delta.sources.DeltaSQLConf import java.util.{Locale, Optional} import scala.collection.JavaConverters._ @@ -769,9 +770,13 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) val checkpointInterval = 4 - setCheckpointInterval(tblPath, checkpointInterval) + withSQLConf( + DeltaSQLConf.DELTA_WRITE_CHECKSUM_ENABLED.key -> executePostCommitHook.toString + ) { + setCheckpointInterval(tblPath, checkpointInterval) + } - for (i <- 1 until 5) { + for (i <- 2 until 5) { // insert until a checkpoint is required val commitResult = appendData( engine, @@ -900,9 +905,13 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) val checkpointInterval = 2 - setCheckpointInterval(tblPath, checkpointInterval) // version 1 + withSQLConf( + DeltaSQLConf.DELTA_WRITE_CHECKSUM_ENABLED.key -> executePostCommitHook.toString + ) { + setCheckpointInterval(tblPath, checkpointInterval) // version 1 + } - for (i <- 1 until 4) { + for (i <- 2 until 4) { // insert until a checkpoint is required val commitResult = appendData( engine, From 8c3c33be026d94e67bea3966156f518ae9cd3a0b Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Mon, 10 Feb 2025 14:57:46 -0800 Subject: [PATCH 12/73] refactor 1 --- .../io/delta/kernel/hook/PostCommitHook.java | 5 +- .../kernel/internal/TransactionImpl.java | 27 ++- .../internal/checksum/ChecksumWriter.java | 37 ++-- .../internal/hook/ChecksumSimpleHook.java | 14 +- .../checksum/ChecksumWriterSuite.scala | 161 ++++++++++-------- .../ChecksumSimpleComparisonSuite.scala | 156 ++++++++++------- .../defaults/DeltaTableWriteSuiteBase.scala | 38 ++--- 7 files changed, 263 insertions(+), 175 deletions(-) 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 c3dbc26c058..8373181d80f 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 @@ -36,8 +36,9 @@ enum PostCommitHookType { CHECKPOINT, /** * Writes a checksum file at the version committed by the transaction. This hook is present when - * the read snapshot is constructed based on previous checksum file. To perform this operation, - * no extra reading previous checkpoint or logs required. + * 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 } 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 ee6407eddb0..aa108a87639 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 @@ -445,7 +445,7 @@ private void recordTransactionReport( private Optional buildPostCommitCrcInfo( long commitAtVersion, TransactionMetricsResult metricsResult) { // Create table - if (commitAtVersion == 0) { + if (isNewTable) { return Optional.of( new CRCInfo( commitAtVersion, @@ -455,23 +455,40 @@ private Optional buildPostCommitCrcInfo( metricsResult.getNumAddFiles(), Optional.of(txnId.toString()))); } - // Retry or CRC is read for old version + // We cannot compute the table statistic if the crc info of commitAtVersion is missing if (!readSnapshot.getCurrentCrcInfo().isPresent() || commitAtVersion != readSnapshot.getCurrentCrcInfo().get().getVersion() + 1) { return Optional.empty(); } - CRCInfo lastCrcInfo = readSnapshot.getCurrentCrcInfo().get(); + return calculateNewCrcInfo(commitAtVersion, metricsResult); + } + + private Optional calculateNewCrcInfo( + long commitAtVersion, TransactionMetricsResult metricsResult) { + CRCInfo lastCrcInfo = + readSnapshot + .getCurrentCrcInfo() + .orElseThrow(() -> new IllegalStateException("CRC info must be present at this point")); + return Optional.of( new CRCInfo( commitAtVersion, metadata, protocol, - lastCrcInfo.getTableSizeBytes() + metricsResult.getTotalAddFilesSizeInBytes(), - lastCrcInfo.getNumFiles() + metricsResult.getNumAddFiles(), + calculateNewTableSize(lastCrcInfo, metricsResult), + calculateNewFileCount(lastCrcInfo, metricsResult), Optional.of(txnId.toString()))); } + private long calculateNewTableSize(CRCInfo lastCrcInfo, TransactionMetricsResult metricsResult) { + return lastCrcInfo.getTableSizeBytes() + metricsResult.getTotalAddFilesSizeInBytes(); + } + + private long calculateNewFileCount(CRCInfo lastCrcInfo, TransactionMetricsResult metricsResult) { + return lastCrcInfo.getNumFiles() + metricsResult.getNumAddFiles(); + } + /** * 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/ChecksumWriter.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumWriter.java index 0f125acafb2..0e4f2c17e53 100644 --- 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 @@ -18,6 +18,7 @@ import static io.delta.kernel.internal.DeltaErrors.wrapEngineExceptionThrowsIO; import static io.delta.kernel.internal.checksum.ChecksumUtils.CRC_FILE_SCHEMA; import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; +import static java.util.Objects.requireNonNull; import io.delta.kernel.data.Row; import io.delta.kernel.engine.Engine; @@ -36,15 +37,24 @@ public class ChecksumWriter { private static final Logger logger = LoggerFactory.getLogger(ChecksumWriter.class); private final Path logPath; + // 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 ChecksumWriter(Path logPath) { - this.logPath = logPath; + this.logPath = requireNonNull(logPath); } /** Writes a checksum file */ public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { // No sufficient information to write checksum file. Path newChecksumPath = FileNames.checksumFile(logPath, crcInfo.getVersion()); + logger.debug("Writing checksum file to path: {}", newChecksumPath); wrapEngineExceptionThrowsIO( () -> { engine @@ -60,14 +70,21 @@ public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { } private Row buildCheckSumRow(CRCInfo crcInfo) { - Map value = new HashMap<>(); - value.put(CRC_FILE_SCHEMA.indexOf("tableSizeBytes"), crcInfo.getTableSizeBytes()); - value.put(CRC_FILE_SCHEMA.indexOf("numFiles"), crcInfo.getNumFiles()); - value.put(CRC_FILE_SCHEMA.indexOf("numMetadata"), 1L); - value.put(CRC_FILE_SCHEMA.indexOf("numProtocol"), 1L); - value.put(CRC_FILE_SCHEMA.indexOf("metadata"), crcInfo.getMetadata().toRow()); - value.put(CRC_FILE_SCHEMA.indexOf("protocol"), crcInfo.getProtocol().toRow()); - crcInfo.getTxnId().ifPresent(txn -> value.put(CRC_FILE_SCHEMA.indexOf("txnId"), txn)); - return new GenericRow(CRC_FILE_SCHEMA, value); + Map values = new HashMap<>(); + // Add required fields + values.put(getSchemaIndex(TABLE_SIZE_BYTES), crcInfo.getTableSizeBytes()); + values.put(getSchemaIndex(NUM_FILES), crcInfo.getNumFiles()); + values.put(getSchemaIndex(NUM_METADATA), 1L); + values.put(getSchemaIndex(NUM_PROTOCOL), 1L); + values.put(getSchemaIndex(METADATA), crcInfo.getMetadata().toRow()); + values.put(getSchemaIndex(PROTOCOL), crcInfo.getProtocol().toRow()); + + // Add optional fields + crcInfo.getTxnId().ifPresent(txn -> values.put(getSchemaIndex(TXN_ID), txn)); + return new GenericRow(CRC_FILE_SCHEMA, values); + } + + private int getSchemaIndex(String fieldName) { + return CRC_FILE_SCHEMA.indexOf(fieldName); } } 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 index 0b6fd68aaab..bb4d0e8c19f 100644 --- 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 @@ -15,6 +15,9 @@ */ 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; @@ -22,19 +25,24 @@ import io.delta.kernel.internal.fs.Path; import java.io.IOException; -/** Write a new checkpoint at the version committed by the txn. */ +/** + * 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 = crcInfo; - this.logPath = 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); } 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 index 52be49e4bf0..fdb4ab59c22 100644 --- 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 @@ -25,15 +25,27 @@ import io.delta.kernel.internal.util.VectorUtils.{stringArrayValue, stringString import io.delta.kernel.test.{BaseMockJsonHandler, MockEngineUtils} import io.delta.kernel.types.StructType import io.delta.kernel.utils.CloseableIterator -import org.scalatest.funsuite.AnyFunSuite import java.util import java.util.{Collections, Optional, OptionalLong} +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 = CRC_FILE_SCHEMA.indexOf("tableSizeBytes") + private val NUM_FILES = CRC_FILE_SCHEMA.indexOf("numFiles") + private val NUM_METADATA = CRC_FILE_SCHEMA.indexOf("numMetadata") + private val NUM_PROTOCOL = CRC_FILE_SCHEMA.indexOf("numProtocol") + private val TXN_ID = CRC_FILE_SCHEMA.indexOf("txnId") + private val METADATA = CRC_FILE_SCHEMA.indexOf("metadata") + private val PROTOCOL = CRC_FILE_SCHEMA.indexOf("protocol") + test("write checksum") { val jsonHandler = new MockCheckSumFileJsonWriter() val checksumWriter = new ChecksumWriter(FAKE_DELTA_LOG_PATH) @@ -41,54 +53,56 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { val metadata = createTestMetadata() def testChecksumWrite(txn: Optional[String]): Unit = { + // Given + val version = 1L + val tableSizeBytes = 100L + val numFiles = 1L + + // When checksumWriter.writeCheckSum( mockEngine(jsonHandler = jsonHandler), - new CRCInfo(1L, metadata, protocol, 100L, 1L, txn) + new CRCInfo(version, metadata, protocol, tableSizeBytes, numFiles, txn) ) - assert(jsonHandler.checksumFilePath == "/path/to/delta/log/00000000000000000001.crc") - assert(jsonHandler.capturedCrcRow.getSchema == CRC_FILE_SCHEMA) - assert( - jsonHandler.capturedCrcRow - .getLong(CRC_FILE_SCHEMA.indexOf("tableSizeBytes")) == 100L - ) - assert( - jsonHandler.capturedCrcRow.getLong(CRC_FILE_SCHEMA.indexOf("numFiles")) == 1L - ) - assert( - jsonHandler.capturedCrcRow - .getLong(CRC_FILE_SCHEMA.indexOf("numMetadata")) == 1L - ) - assert( - jsonHandler.capturedCrcRow - .getLong(CRC_FILE_SCHEMA.indexOf("numProtocol")) == 1L - ) + // Then + verifyChecksumFile(jsonHandler, version) + verifyChecksumContent(jsonHandler.capturedCrcRow, tableSizeBytes, numFiles, txn) + verifyMetadataAndProtocol(jsonHandler.capturedCrcRow, metadata, protocol) + } - if (txn.isPresent) { - assert( - jsonHandler.capturedCrcRow.getString( - CRC_FILE_SCHEMA.indexOf("txnId") - ) == txn.get() - ) - } else { - assert(jsonHandler.capturedCrcRow.isNullAt(CRC_FILE_SCHEMA.indexOf("txnId"))) - } - - checkMetadata( - metadata, - jsonHandler.capturedCrcRow.getStruct(CRC_FILE_SCHEMA.indexOf("metadata")) - ) - checkProtocol( - protocol, - jsonHandler.capturedCrcRow.getStruct(CRC_FILE_SCHEMA.indexOf("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.getSchema == CRC_FILE_SCHEMA) + } + private def verifyChecksumContent( + row: Row, + tableSizeBytes: Long, + numFiles: Long, + txn: Optional[String]): Unit = { + assert(row.getLong(TABLE_SIZE_BYTES) == tableSizeBytes) + assert(row.getLong(NUM_FILES) == numFiles) + assert(row.getLong(NUM_METADATA) == 1L) + assert(row.getLong(NUM_PROTOCOL) == 1L) + + if (txn.isPresent) { + assert(row.getString(TXN_ID) == txn.get()) + } else { + assert(row.isNullAt(TXN_ID)) } - testChecksumWrite(Optional.of("txn")); - testChecksumWrite(Optional.empty()); } - def createTestMetadata(): Metadata = { + private def verifyMetadataAndProtocol(row: Row, metadata: Metadata, protocol: Protocol): Unit = { + checkMetadata(metadata, row.getStruct(METADATA)) + checkProtocol(protocol, row.getStruct(PROTOCOL)) + } + + private def createTestMetadata(): Metadata = { new Metadata( "id", Optional.of("name"), @@ -104,7 +118,7 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { ) } - def createTestProtocol(): Protocol = { + private def createTestProtocol(): Protocol = { new Protocol( /* minReaderVersion= */ 1, /* minWriterVersion= */ 2, @@ -113,24 +127,21 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { ) } - def checkMetadata(metadata: Metadata, metadataRow: Row): Unit = { + private def checkMetadata(metadata: Metadata, metadataRow: Row): Unit = { assert(metadataRow.getSchema == Metadata.FULL_SCHEMA) + + def getOptionalString(field: String): Optional[String] = + Optional.ofNullable(metadataRow.getString(Metadata.FULL_SCHEMA.indexOf(field))) + assert(metadataRow.getString(Metadata.FULL_SCHEMA.indexOf("id")) == metadata.getId) + assert(getOptionalString("name") == metadata.getName) + assert(getOptionalString("description") == metadata.getDescription) + + val formatRow = metadataRow.getStruct(Metadata.FULL_SCHEMA.indexOf("format")) assert( - Optional - .ofNullable(metadataRow.getString(Metadata.FULL_SCHEMA.indexOf("name"))) == metadata.getName - ) - assert( - Optional.ofNullable(metadataRow.getString(Metadata.FULL_SCHEMA.indexOf("description"))) - == metadata.getDescription - ) - assert( - metadataRow - .getStruct( - Metadata.FULL_SCHEMA.indexOf("format") - ) - .getString(Format.FULL_SCHEMA.indexOf("provider")) == metadata.getFormat.getProvider + formatRow.getString(Format.FULL_SCHEMA.indexOf("provider")) == metadata.getFormat.getProvider ) + assert( metadataRow .getString(Metadata.FULL_SCHEMA.indexOf("schemaString")) == metadata.getSchemaString @@ -140,40 +151,44 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { .getArray(Metadata.FULL_SCHEMA.indexOf("partitionColumns")) == metadata.getPartitionColumns ) assert( - Optional.ofNullable(metadataRow.getLong( - Metadata.FULL_SCHEMA.indexOf("createdTime"))) == metadata.getCreatedTime + Optional + .ofNullable(metadataRow.getLong(Metadata.FULL_SCHEMA.indexOf("createdTime"))) + == metadata.getCreatedTime ) assert( - VectorUtils.toJavaMap(metadataRow.getMap( - Metadata.FULL_SCHEMA.indexOf("configuration"))) == metadata.getConfiguration + VectorUtils + .toJavaMap(metadataRow.getMap(Metadata.FULL_SCHEMA.indexOf("configuration"))) + == metadata.getConfiguration ) } - def checkProtocol(protocol: Protocol, protocolRow: Row): Unit = { + private def checkProtocol(protocol: Protocol, protocolRow: Row): Unit = { assert(protocolRow.getSchema == Protocol.FULL_SCHEMA) assert( - protocol.getMinReaderVersion == - protocolRow.getInt(Protocol.FULL_SCHEMA.indexOf("minReaderVersion")) + protocol.getMinReaderVersion == protocolRow + .getInt(Protocol.FULL_SCHEMA.indexOf("minReaderVersion")) ) assert( - protocol.getMinWriterVersion == - protocolRow.getInt(Protocol.FULL_SCHEMA.indexOf("minWriterVersion")) + protocol.getMinWriterVersion == protocolRow + .getInt(Protocol.FULL_SCHEMA.indexOf("minWriterVersion")) ) } } +/** + * Mock implementation of JsonHandler for testing checksum file writing. + */ class MockCheckSumFileJsonWriter extends BaseMockJsonHandler { - var capturedCrcRow: Row = new GenericRow(new StructType(), new util.HashMap[Integer, AnyRef]); - var checksumFilePath = "" + var capturedCrcRow: Row = new GenericRow(new StructType(), new util.HashMap[Integer, AnyRef]) + var checksumFilePath: String = "" override def writeJsonFileAtomically( - filePath: String, - data: CloseableIterator[Row], - overwrite: Boolean): Unit = { + filePath: String, + data: CloseableIterator[Row], + overwrite: Boolean): Unit = { checksumFilePath = filePath - assert(data.hasNext) + assert(data.hasNext, "Expected data iterator to contain exactly one row") capturedCrcRow = data.next() - assert(!data.hasNext) + assert(!data.hasNext, "Expected data iterator to contain exactly one row") } - -} +} \ No newline at end of file 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 index 2c86009657d..ddfa2699426 100644 --- 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 @@ -17,7 +17,7 @@ package io.delta.kernel.defaults import io.delta.kernel.Operation.CREATE_TABLE import io.delta.kernel.{Operation, Table, Transaction} -import io.delta.kernel.data.Row +import io.delta.kernel.data.{ColumnarBatch, Row} import io.delta.kernel.defaults.utils.TestUtils import io.delta.kernel.engine.Engine import io.delta.kernel.expressions.Literal @@ -39,12 +39,14 @@ import java.util.Collections.{emptyMap, singletonMap} import java.util.Optional import scala.collection.JavaConverters._ -/* * - * Test suites to copy delta-spark's log to delta kernel and verify - * the correctness of written checksum file. - * */ +/** + * Test suite to verify checksum file correctness by comparing + * Delta Spark and Delta Kernel generated checksum files. + */ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { + private val PARTITION_COLUMN = "part" + test("create table, insert data and verify checksum") { withTempDirAndEngine { (tablePath, engine) => val sparkTablePath = tablePath + "spark" @@ -82,9 +84,9 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { .withSchema(engine, new StructType().add("id", INTEGER)) .build(engine) - copyCommitAndCommitTxnForUnpartitionedTable(txn, engine, sparkTablePath, versionAtCommit = 0) + copyAndCommitTxnForUnpartitionedTable(txn, engine, sparkTablePath, versionAtCommit = 0) - assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( version => insertIntoUnpartitionedTable(engine, sparkTablePath, kernelTablePath, version) @@ -99,8 +101,8 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { Table .forPath(engine, kernelTablePath) .createTransactionBuilder(engine, "test-engine", CREATE_TABLE) - .withSchema(engine, new StructType().add("id", INTEGER).add("part", INTEGER)) - .withPartitionColumns(engine, Seq("part").asJava) + .withSchema(engine, new StructType().add("id", INTEGER).add(PARTITION_COLUMN, INTEGER)) + .withPartitionColumns(engine, Seq(PARTITION_COLUMN).asJava) .build(engine) .commit(engine, emptyIterable()) .getPostCommitHooks @@ -131,11 +133,11 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { val txn = Table .forPath(engine, kernelTablePath) .createTransactionBuilder(engine, "test-engine", CREATE_TABLE) - .withSchema(engine, new StructType().add("id", INTEGER).add("part", INTEGER)) - .withPartitionColumns(engine, Seq("part").asJava) + .withSchema(engine, new StructType().add("id", INTEGER).add(PARTITION_COLUMN, INTEGER)) + .withPartitionColumns(engine, Seq(PARTITION_COLUMN).asJava) .build(engine) - copyCommitAndCommitTxnForPartitionedTable( + copyAndCommitTxnForPartitionedTable( txn, engine, sparkTablePath, @@ -150,30 +152,36 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { } } - def assertChecksumEquals( + 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( - new File(f"$sparkTablePath/_delta_log/$version%020d.crc").toPath - ) && - Files.exists( - new File(f"$kernelTablePath/_delta_log/$version%020d.crc").toPath - ) - ) - assertCrcInfoEquals( - ChecksumReader - .getCRCInfo(engine, new Path(f"$sparkTablePath/_delta_log"), version, version) - .get(), - ChecksumReader - .getCRCInfo(engine, new Path(f"$kernelTablePath/_delta_log"), version, version) - .get() + 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(f"$basePath/_delta_log/$version%020d.crc").toPath } - def assertCrcInfoEquals(crc1: CRCInfo, crc2: CRCInfo): Unit = { + private def assertCrcInfoEquals(crc1: CRCInfo, crc2: CRCInfo): Unit = { assert(crc1.getVersion === crc2.getVersion) assert(crc1.getNumFiles === crc2.getNumFiles) assert(crc1.getTableSizeBytes === crc2.getTableSizeBytes) @@ -181,7 +189,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assert(crc1.getMetadata.getPartitionColNames === crc2.getMetadata.getPartitionColNames) } - def insertIntoPartitionedTable( + private def insertIntoPartitionedTable( engine: Engine, sparkTablePath: String, kernelTablePath: String, @@ -202,7 +210,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { .createTransactionBuilder(engine, "test-engine", Operation.WRITE) .build(engine) - copyCommitAndCommitTxnForPartitionedTable( + copyAndCommitTxnForPartitionedTable( txn, engine, sparkTablePath, @@ -212,7 +220,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } - def insertIntoUnpartitionedTable( + private def insertIntoUnpartitionedTable( engine: Engine, sparkTablePath: String, kernelTablePath: String, @@ -228,11 +236,11 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { .createTransactionBuilder(engine, "test-engine", Operation.WRITE) .build(engine) - copyCommitAndCommitTxnForUnpartitionedTable(txn, engine, sparkTablePath, versionAtCommit) + copyAndCommitTxnForUnpartitionedTable(txn, engine, sparkTablePath, versionAtCommit) assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } - def copyCommitAndCommitTxnForUnpartitionedTable( + private def copyAndCommitTxnForUnpartitionedTable( txn: Transaction, engine: Engine, sparkTablePath: String, @@ -259,7 +267,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { } - def copyCommitAndCommitTxnForPartitionedTable( + private def copyAndCommitTxnForPartitionedTable( txn: Transaction, engine: Engine, sparkTablePath: String, @@ -274,7 +282,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { .getWriteContext( engine, txnState, - singletonMap("part", Literal.ofInt(partition)) + singletonMap(PARTITION_COLUMN, Literal.ofInt(partition)) ) Transaction @@ -303,42 +311,66 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { } - def convertDeltaLogToAppendActions( + private def convertDeltaLogToAppendActions( engine: Engine, tablePath: String, version: Long, partition: Option[String]): CloseableIterator[DataFileStatus] = { + val logPath = new Path(tablePath, "_delta_log") - val file = FileStatus.of(FileNames.deltaFile(logPath, version), 0, 0) + val deltaFile = FileStatus.of(FileNames.deltaFile(logPath, version), 0, 0) + + val addFiles = new util.ArrayList[DataFileStatus]() + val columnarBatches = engine.getJsonHandler.readJsonFiles( - singletonCloseableIterator(file), + singletonCloseableIterator(deltaFile), SingleAction.FULL_SCHEMA, - Optional.empty() - ) - val addFiles = new util.ArrayList[DataFileStatus]() + Optional.empty()) + while (columnarBatches.hasNext) { - val batch = columnarBatches.next - val rows = batch.getRows - while (rows.hasNext) { - val row = rows.next() - if (!row.isNullAt(row.getSchema.indexOf("add"))) { - val addFile = new AddFile(row.getStruct(row.getSchema.indexOf("add"))) - if (partition.isEmpty || partition.get == VectorUtils - .toJavaMap(addFile.getPartitionValues) - .get("part")) { - addFiles.add( - new DataFileStatus( - addFile.getPath, - addFile.getSize, - addFile.getModificationTime, - // TODO: populate the stats once https://github.com/delta-io/delta/issues/4139 fixed - Optional.empty() - ) - ) - } - } - } + processColumnarBatch(columnarBatches.next(), partition, addFiles) } + toCloseableIterator(addFiles.iterator()) } + + private def processColumnarBatch( + batch: ColumnarBatch, + partition: Option[String], + addFiles: util.ArrayList[DataFileStatus]): Unit = { + + val rows = batch.getRows + while (rows.hasNext) { + val row = rows.next() + val addIndex = row.getSchema.indexOf("add") + + if (!row.isNullAt(addIndex)) { + processAddFile(row.getStruct(addIndex), partition, addFiles) + } + } + } + + private def processAddFile( + addFileRow: Row, + partition: Option[String], + addFiles: util.ArrayList[DataFileStatus]): Unit = { + + val addFile = new AddFile(addFileRow) + + if (shouldIncludeFile(addFile, partition)) { + addFiles.add( + new DataFileStatus( + addFile.getPath, + addFile.getSize, + addFile.getModificationTime, + Optional.empty() // TODO: populate stats once #4139 is fixed + ) + ) + } + } + + private def shouldIncludeFile(addFile: AddFile, partition: Option[String]): Boolean = { + partition.isEmpty || + partition.get == VectorUtils.toJavaMap(addFile.getPartitionValues).get(PARTITION_COLUMN) + } } 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 107cb837152..84a5441cf1b 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 @@ -315,15 +315,15 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def appendData( - engine: Engine = defaultEngine, - tablePath: String, - isNewTable: Boolean = false, - schema: StructType = null, - partCols: Seq[String] = null, - data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])], - clock: Clock = () => System.currentTimeMillis, - tableProperties: Map[String, String] = null, - executePostCommitHook: Boolean = false): TransactionCommitResult = { + engine: Engine = defaultEngine, + tablePath: String, + isNewTable: Boolean = false, + schema: StructType = null, + partCols: Seq[String] = null, + data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])], + clock: Clock = () => System.currentTimeMillis, + tableProperties: Map[String, String] = null, + executePostCommitHook: Boolean = false): TransactionCommitResult = { val txn = createTxn(engine, tablePath, isNewTable, schema, partCols, tableProperties, clock) val commitResult = commitAppendData(engine, txn, data) if (executePostCommitHook) { @@ -333,9 +333,9 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def assertMetadataProp( - snapshot: SnapshotImpl, - key: TableConfig[_ <: Any], - expectedValue: Any): Unit = { + snapshot: SnapshotImpl, + key: TableConfig[_ <: Any], + expectedValue: Any): Unit = { assert(key.fromMetadata(snapshot.getMetadata) == expectedValue) } @@ -402,11 +402,11 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } def verifyCommitInfo( - tablePath: String, - version: Long, - partitionCols: Seq[String] = Seq.empty, - isBlindAppend: Boolean = true, - operation: Operation = CREATE_TABLE): Unit = { + tablePath: String, + version: Long, + partitionCols: Seq[String] = Seq.empty, + isBlindAppend: Boolean = true, + operation: Operation = CREATE_TABLE): Unit = { val row = spark .sql(s"DESCRIBE HISTORY delta.`$tablePath`") .filter(s"version = $version") @@ -416,9 +416,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { "isBlindAppend", "engineInfo", "operation" - ) - .collect() - .last + ).collect().last assert(row.getAs[Long]("version") === version) assert(row.getAs[Long]("partitionBy") === From 5d0b5a76331c8b03e45138c7d33020ca76ca7b55 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Mon, 10 Feb 2025 15:04:16 -0800 Subject: [PATCH 13/73] refactor 2 --- .../checksum/ChecksumWriterSuite.scala | 2 +- .../defaults/DeltaTableWriteSuiteBase.scala | 46 +++++++++++-------- 2 files changed, 27 insertions(+), 21 deletions(-) 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 index fdb4ab59c22..e04e8e2e3e2 100644 --- 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 @@ -191,4 +191,4 @@ class MockCheckSumFileJsonWriter extends BaseMockJsonHandler { capturedCrcRow = data.next() assert(!data.hasNext, "Expected data iterator to contain exactly one row") } -} \ No newline at end of file +} 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 84a5441cf1b..c4653814700 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 @@ -415,8 +415,8 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { "operationParameters.partitionBy", "isBlindAppend", "engineInfo", - "operation" - ).collect().last + "operation") + .collect().last assert(row.getAs[Long]("version") === version) assert(row.getAs[Long]("partitionBy") === @@ -482,35 +482,41 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { version: Option[Long], expSchema: StructType, expPartitionColumns: Seq[String]): Unit = { - val checksumVersion = - if (version.isDefined) version.get else latestSnapshot(tablePath, defaultEngine).getVersion - assert( - Files.exists( - new File(f"$tablePath/_delta_log/$checksumVersion%020d.crc").toPath - ) - ) + val checksumVersion = version.getOrElse(latestSnapshot(tablePath, defaultEngine).getVersion) + val checksumFile = new File(f"$tablePath/_delta_log/$checksumVersion%020d.crc") + + assert(Files.exists(checksumFile.toPath), s"Checksum file not found: ${checksumFile.getPath}") + val columnarBatches = defaultEngine .getJsonHandler() .readJsonFiles( - singletonCloseableIterator( - FileStatus.of(f"$tablePath/_delta_log/$checksumVersion%020d.crc") - ), + singletonCloseableIterator(FileStatus.of(checksumFile.getPath)), ChecksumUtils.CRC_FILE_SCHEMA, Optional.empty() ) - assert(columnarBatches.hasNext) + + assert(columnarBatches.hasNext, "Empty checksum file") val crcRow = columnarBatches.next() - assert(crcRow.getSize === 1) - val metadataField = Metadata.fromColumnVector( + assert(crcRow.getSize === 1, s"Expected single row, found ${crcRow.getSize}") + + val metadata = Metadata.fromColumnVector( crcRow.getColumnVector(ChecksumUtils.CRC_FILE_SCHEMA.indexOf("metadata")), 0 ) - assert(metadataField.getSchema === expSchema) + + assert( + metadata.getSchema === expSchema, + s"Schema mismatch.\nExpected: $expSchema\nActual: ${metadata.getSchema}" + ) + + val normalizedPartitions = expPartitionColumns.map(_.toLowerCase(Locale.ROOT)).toSet assert( - metadataField.getPartitionColNames.asScala === expPartitionColumns - .map(s => s.toLowerCase(Locale.ROOT)) - .toSet + metadata.getPartitionColNames.asScala === normalizedPartitions, + s"Partition columns mismatch.\n" + + s"Expected: $normalizedPartitions\n" + + s"Actual: ${metadata.getPartitionColNames.asScala}" ) - assert(!columnarBatches.hasNext) + + assert(!columnarBatches.hasNext, "Unexpected additional data in checksum file") } } From 6796fa5d3b1af9c5150f33f172d5ba8f34a11125 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Mon, 10 Feb 2025 15:05:54 -0800 Subject: [PATCH 14/73] remove redundant doc --- .../delta/kernel/internal/checksum/ChecksumWriterSuite.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) 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 index e04e8e2e3e2..571e0f876c1 100644 --- 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 @@ -53,18 +53,15 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { val metadata = createTestMetadata() def testChecksumWrite(txn: Optional[String]): Unit = { - // Given val version = 1L val tableSizeBytes = 100L val numFiles = 1L - // When checksumWriter.writeCheckSum( mockEngine(jsonHandler = jsonHandler), new CRCInfo(version, metadata, protocol, tableSizeBytes, numFiles, txn) ) - - // Then + verifyChecksumFile(jsonHandler, version) verifyChecksumContent(jsonHandler.capturedCrcRow, tableSizeBytes, numFiles, txn) verifyMetadataAndProtocol(jsonHandler.capturedCrcRow, metadata, protocol) From 2a93c498362b47f21dd9f657de4b0e159cc3b1da Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Mon, 10 Feb 2025 15:10:04 -0800 Subject: [PATCH 15/73] fix doc --- .../src/main/java/io/delta/kernel/internal/TransactionImpl.java | 2 +- .../io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 aa108a87639..a3ed3b17da1 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 @@ -455,7 +455,7 @@ private Optional buildPostCommitCrcInfo( metricsResult.getNumAddFiles(), Optional.of(txnId.toString()))); } - // We cannot compute the table statistic if the crc info of commitAtVersion is missing + // We cannot compute the table statistic if the crc info of commitAtVersion - 1 is missing if (!readSnapshot.getCurrentCrcInfo().isPresent() || commitAtVersion != readSnapshot.getCurrentCrcInfo().get().getVersion() + 1) { return Optional.empty(); 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 index 571e0f876c1..1644d027b94 100644 --- 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 @@ -61,7 +61,7 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { mockEngine(jsonHandler = jsonHandler), new CRCInfo(version, metadata, protocol, tableSizeBytes, numFiles, txn) ) - + verifyChecksumFile(jsonHandler, version) verifyChecksumContent(jsonHandler.capturedCrcRow, tableSizeBytes, numFiles, txn) verifyMetadataAndProtocol(jsonHandler.capturedCrcRow, metadata, protocol) From 826f623deadae022ca12bbe0f97d4d7148711ba5 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Mon, 10 Feb 2025 15:12:05 -0800 Subject: [PATCH 16/73] fix scala --- .../kernel/internal/checksum/ChecksumWriterSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 index 1644d027b94..a4f31cbd6d0 100644 --- 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 @@ -180,9 +180,9 @@ class MockCheckSumFileJsonWriter extends BaseMockJsonHandler { var checksumFilePath: String = "" override def writeJsonFileAtomically( - filePath: String, - data: CloseableIterator[Row], - overwrite: Boolean): Unit = { + filePath: String, + data: CloseableIterator[Row], + overwrite: Boolean): Unit = { checksumFilePath = filePath assert(data.hasNext, "Expected data iterator to contain exactly one row") capturedCrcRow = data.next() From ebaf2c8288be5b019d98bd1d7e9c7b1123ca874c Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 12 Feb 2025 14:19:13 -0800 Subject: [PATCH 17/73] resolve comment --- .../io/delta/kernel/hook/PostCommitHook.java | 6 +-- .../kernel/internal/TransactionImpl.java | 43 ++++++------------- .../kernel/internal/checksum/CRCInfo.java | 23 +++++++++- .../internal/checksum/ChecksumReader.java | 2 +- .../internal/checksum/ChecksumUtils.java | 38 ---------------- .../internal/checksum/ChecksumWriter.java | 13 +----- .../checksum/ChecksumWriterSuite.scala | 8 ++-- .../ChecksumSimpleComparisonSuite.scala | 8 ++-- 8 files changed, 47 insertions(+), 94 deletions(-) delete mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumUtils.java 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 8373181d80f..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 @@ -36,9 +36,9 @@ enum PostCommitHookType { 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. + * 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 } 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 a3ed3b17da1..57972f5cb3a 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 @@ -455,38 +455,19 @@ private Optional buildPostCommitCrcInfo( metricsResult.getNumAddFiles(), Optional.of(txnId.toString()))); } - // We cannot compute the table statistic if the crc info of commitAtVersion - 1 is missing - if (!readSnapshot.getCurrentCrcInfo().isPresent() - || commitAtVersion != readSnapshot.getCurrentCrcInfo().get().getVersion() + 1) { - return Optional.empty(); - } - - return calculateNewCrcInfo(commitAtVersion, metricsResult); - } - - private Optional calculateNewCrcInfo( - long commitAtVersion, TransactionMetricsResult metricsResult) { - CRCInfo lastCrcInfo = - readSnapshot - .getCurrentCrcInfo() - .orElseThrow(() -> new IllegalStateException("CRC info must be present at this point")); - - return Optional.of( - new CRCInfo( - commitAtVersion, - metadata, - protocol, - calculateNewTableSize(lastCrcInfo, metricsResult), - calculateNewFileCount(lastCrcInfo, metricsResult), - Optional.of(txnId.toString()))); - } - - private long calculateNewTableSize(CRCInfo lastCrcInfo, TransactionMetricsResult metricsResult) { - return lastCrcInfo.getTableSizeBytes() + metricsResult.getTotalAddFilesSizeInBytes(); - } - private long calculateNewFileCount(CRCInfo lastCrcInfo, TransactionMetricsResult metricsResult) { - return lastCrcInfo.getNumFiles() + metricsResult.getNumAddFiles(); + return readSnapshot + .getCurrentCrcInfo() + .filter(lastCrcInfo -> commitAtVersion == lastCrcInfo.getVersion() + 1) + .map( + lastCrcInfo -> + new CRCInfo( + commitAtVersion, + metadata, + protocol, + lastCrcInfo.getTableSizeBytes() + metricsResult.getTotalAddFilesSizeInBytes(), + lastCrcInfo.getNumFiles() + metricsResult.getNumAddFiles(), + Optional.of(txnId.toString()))); } /** 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 index fa23ab35463..8df8adfe1b4 100644 --- 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 @@ -15,12 +15,14 @@ */ package io.delta.kernel.internal.checksum; -import static io.delta.kernel.internal.checksum.ChecksumUtils.CRC_FILE_SCHEMA; 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.LongType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructType; import java.util.Optional; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,6 +30,25 @@ public class CRCInfo { private static final Logger logger = LoggerFactory.getLogger(CRCInfo.class); + // Constants for schema field names + public static final String TABLE_SIZE_BYTES = "tableSizeBytes"; + public static final String NUM_FILES = "numFiles"; + public static final String NUM_METADATA = "numMetadata"; + public static final String NUM_PROTOCOL = "numProtocol"; + public static final String METADATA = "metadata"; + public static final String PROTOCOL = "protocol"; + public static final String TXN_ID = "txnId"; + + public static 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) { Protocol protocol = diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumReader.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumReader.java index da9f20b7b0e..eeb651a34f5 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumReader.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumReader.java @@ -96,7 +96,7 @@ private static Optional readChecksumFile(Engine engine, Path filePath) .getJsonHandler() .readJsonFiles( singletonCloseableIterator(FileStatus.of(filePath.toString())), - ChecksumUtils.CRC_FILE_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/ChecksumUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumUtils.java deleted file mode 100644 index 13117da767d..00000000000 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checksum/ChecksumUtils.java +++ /dev/null @@ -1,38 +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.checksum; - -import io.delta.kernel.internal.actions.Metadata; -import io.delta.kernel.internal.actions.Protocol; -import io.delta.kernel.types.LongType; -import io.delta.kernel.types.StringType; -import io.delta.kernel.types.StructType; - -/** Helper class for shared components in checksum file read and write */ -public final class ChecksumUtils { - - public static StructType CRC_FILE_SCHEMA = - new StructType() - .add("tableSizeBytes", LongType.LONG) - .add("numFiles", LongType.LONG) - .add("numMetadata", LongType.LONG) - .add("numProtocol", LongType.LONG) - .add("metadata", Metadata.FULL_SCHEMA) - .add("protocol", Protocol.FULL_SCHEMA) - .add("txnId", StringType.STRING, /*nullable*/ true); - - private ChecksumUtils() {} -} 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 index 0e4f2c17e53..2a5358cafab 100644 --- 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 @@ -16,7 +16,7 @@ package io.delta.kernel.internal.checksum; import static io.delta.kernel.internal.DeltaErrors.wrapEngineExceptionThrowsIO; -import static io.delta.kernel.internal.checksum.ChecksumUtils.CRC_FILE_SCHEMA; +import static io.delta.kernel.internal.checksum.CRCInfo.*; import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; import static java.util.Objects.requireNonNull; @@ -37,14 +37,6 @@ public class ChecksumWriter { private static final Logger logger = LoggerFactory.getLogger(ChecksumWriter.class); private final Path logPath; - // 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 ChecksumWriter(Path logPath) { this.logPath = requireNonNull(logPath); @@ -52,9 +44,8 @@ public ChecksumWriter(Path logPath) { /** Writes a checksum file */ public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { - // No sufficient information to write checksum file. Path newChecksumPath = FileNames.checksumFile(logPath, crcInfo.getVersion()); - logger.debug("Writing checksum file to path: {}", newChecksumPath); + logger.info("Writing checksum file to path: {}", newChecksumPath); wrapEngineExceptionThrowsIO( () -> { engine 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 index a4f31cbd6d0..b259b6f8518 100644 --- 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 @@ -183,9 +183,9 @@ class MockCheckSumFileJsonWriter extends BaseMockJsonHandler { filePath: String, data: CloseableIterator[Row], overwrite: Boolean): Unit = { - checksumFilePath = filePath - assert(data.hasNext, "Expected data iterator to contain exactly one row") - capturedCrcRow = data.next() - assert(!data.hasNext, "Expected data iterator to contain exactly one row") + checksumFilePath = filePath + assert(data.hasNext, "Expected data iterator to contain exactly one row") + capturedCrcRow = 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 index ddfa2699426..3af9e4a952f 100644 --- 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 @@ -89,7 +89,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( - version => insertIntoUnpartitionedTable(engine, sparkTablePath, kernelTablePath, version) + version => insertIntoTable(engine, sparkTablePath, kernelTablePath, version) ) } } @@ -106,9 +106,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { .build(engine) .commit(engine, emptyIterable()) .getPostCommitHooks - .forEach( - hook => hook.threadSafeInvoke(engine) - ) + .forEach(hook => hook.threadSafeInvoke(engine)) spark.sql( s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` " + s"(id Integer, part Integer) USING DELTA PARTITIONED BY (part)" @@ -220,7 +218,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } - private def insertIntoUnpartitionedTable( + private def insertIntoTable( engine: Engine, sparkTablePath: String, kernelTablePath: String, From 000a36dbf0a7b5ac186257d7ab94146190552ede Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 12 Feb 2025 14:25:05 -0800 Subject: [PATCH 18/73] fix indent --- .../delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 3af9e4a952f..5c12a61a199 100644 --- 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 @@ -86,7 +86,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { copyAndCommitTxnForUnpartitionedTable(txn, engine, sparkTablePath, versionAtCommit = 0) - assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( version => insertIntoTable(engine, sparkTablePath, kernelTablePath, version) From 0c425be5e4653f900b0a1192784270a58c49ac18 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 12 Feb 2025 14:29:27 -0800 Subject: [PATCH 19/73] fix test --- .../io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index b259b6f8518..cb8cd4be6ef 100644 --- 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 @@ -17,7 +17,7 @@ package io.delta.kernel.internal.checksum import io.delta.kernel.data.Row import io.delta.kernel.internal.actions.{Format, Metadata, Protocol} -import io.delta.kernel.internal.checksum.ChecksumUtils.CRC_FILE_SCHEMA +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 From e972c31a1d47d7ee29a6d3d9f3c081828ca503ad Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 12 Feb 2025 14:41:33 -0800 Subject: [PATCH 20/73] fix indent --- .../kernel/defaults/ChecksumSimpleComparisonSuite.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 index 5c12a61a199..7d491dec30b 100644 --- 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 @@ -65,7 +65,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( - version => insertIntoUnpartitionedTable(engine, sparkTablePath, kernelTablePath, version) + version => insertIntoTable(engine, sparkTablePath, kernelTablePath, version) ) } @@ -259,9 +259,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { txn .commit(engine, inMemoryIterable(dataActions)) .getPostCommitHooks - .forEach( - hook => hook.threadSafeInvoke(engine) - ) + .forEach(hook => hook.threadSafeInvoke(engine)) } From 73df6276b3ead6905f94965eda414e40585b6d9e Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 12 Feb 2025 14:45:19 -0800 Subject: [PATCH 21/73] fix indent --- .../delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) 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 index 7d491dec30b..9a9ccc3a8b2 100644 --- 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 @@ -58,9 +58,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { .build(engine) .commit(engine, emptyIterable()) .getPostCommitHooks - .forEach( - hook => hook.threadSafeInvoke(engine) - ) + .forEach(hook => hook.threadSafeInvoke(engine)) spark.sql(s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` (id Integer) USING DELTA") assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) From 1df8af5765094271eb97d6e6f853764ab8e0b743 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 12 Feb 2025 14:50:49 -0800 Subject: [PATCH 22/73] rename method --- .../defaults/ChecksumSimpleComparisonSuite.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) 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 index 9a9ccc3a8b2..5f61fe72f9d 100644 --- 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 @@ -63,7 +63,8 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( - version => insertIntoTable(engine, sparkTablePath, kernelTablePath, version) + version => insertIntoUnpartitionedTableAndCheckCrc( + engine, sparkTablePath, kernelTablePath, version) ) } @@ -87,7 +88,8 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( - version => insertIntoTable(engine, sparkTablePath, kernelTablePath, version) + version => insertIntoUnpartitionedTableAndCheckCrc( + engine, sparkTablePath, kernelTablePath, version) ) } } @@ -112,7 +114,8 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( - version => insertIntoPartitionedTable(engine, sparkTablePath, kernelTablePath, version) + version => insertIntoUnpartitionedTableAndCheckCrc( + engine, sparkTablePath, kernelTablePath, version) ) } } @@ -143,7 +146,8 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( - version => insertIntoPartitionedTable(engine, sparkTablePath, kernelTablePath, version) + version => insertIntoPartitionedTableAndCheckCrc( + engine, sparkTablePath, kernelTablePath, version) ) } } @@ -185,7 +189,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assert(crc1.getMetadata.getPartitionColNames === crc2.getMetadata.getPartitionColNames) } - private def insertIntoPartitionedTable( + private def insertIntoPartitionedTableAndCheckCrc( engine: Engine, sparkTablePath: String, kernelTablePath: String, @@ -216,7 +220,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } - private def insertIntoTable( + private def insertIntoUnpartitionedTableAndCheckCrc( engine: Engine, sparkTablePath: String, kernelTablePath: String, From a3ba086380ad02d474ca78177d109fd40ce80aef Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 12 Feb 2025 14:54:40 -0800 Subject: [PATCH 23/73] fix indent --- .../internal/checksum/ChecksumWriterSuite.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) 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 index cb8cd4be6ef..7818c844578 100644 --- 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 @@ -180,12 +180,12 @@ class MockCheckSumFileJsonWriter extends BaseMockJsonHandler { 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 = data.next() - assert(!data.hasNext, "Expected data iterator to contain exactly one row") + filePath: String, + data: CloseableIterator[Row], + overwrite: Boolean): Unit = { + checksumFilePath = filePath + assert(data.hasNext, "Expected data iterator to contain exactly one row") + capturedCrcRow = data.next() + assert(!data.hasNext, "Expected data iterator to contain exactly one row") } } From ee72c537246a8a16ed178abe91947b5a730a3da8 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 12 Feb 2025 15:01:18 -0800 Subject: [PATCH 24/73] fix build --- .../io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 c4653814700..ad7ee6cfb57 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 @@ -46,7 +46,7 @@ import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} import io.delta.kernel.utils.CloseableIterator import io.delta.kernel.Operation.CREATE_TABLE import io.delta.kernel.hook.PostCommitHook.PostCommitHookType -import io.delta.kernel.internal.checksum.{CRCInfo, ChecksumUtils} +import io.delta.kernel.internal.checksum.CRCInfo import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -491,7 +491,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { .getJsonHandler() .readJsonFiles( singletonCloseableIterator(FileStatus.of(checksumFile.getPath)), - ChecksumUtils.CRC_FILE_SCHEMA, + CRCInfo.CRC_FILE_SCHEMA, Optional.empty() ) @@ -500,7 +500,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { assert(crcRow.getSize === 1, s"Expected single row, found ${crcRow.getSize}") val metadata = Metadata.fromColumnVector( - crcRow.getColumnVector(ChecksumUtils.CRC_FILE_SCHEMA.indexOf("metadata")), + crcRow.getColumnVector(CRCInfo.CRC_FILE_SCHEMA.indexOf("metadata")), 0 ) From c8cf5b17b3ca4254cc9c43dea8c14f8da7fa6d82 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 12 Feb 2025 15:02:59 -0800 Subject: [PATCH 25/73] fix test --- .../delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 5f61fe72f9d..8481fb76077 100644 --- 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 @@ -114,7 +114,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( - version => insertIntoUnpartitionedTableAndCheckCrc( + version => insertIntoPartitionedTableAndCheckCrc( engine, sparkTablePath, kernelTablePath, version) ) } From d9225248ce601964c084f799674061819bac7855 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Thu, 13 Feb 2025 22:28:52 -0800 Subject: [PATCH 26/73] fix test naming --- .../kernel/internal/checksum/CRCInfo.java | 14 +++--- .../internal/checksum/ChecksumWriter.java | 5 ++- .../checksum/ChecksumWriterSuite.scala | 44 ++++++++++--------- 3 files changed, 33 insertions(+), 30 deletions(-) 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 index 8df8adfe1b4..9211f432d5a 100644 --- 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 @@ -39,7 +39,7 @@ public class CRCInfo { public static final String PROTOCOL = "protocol"; public static final String TXN_ID = "txnId"; - public static StructType CRC_FILE_SCHEMA = + public static final StructType CRC_FILE_SCHEMA = new StructType() .add(TABLE_SIZE_BYTES, LongType.LONG) .add(NUM_FILES, LongType.LONG) @@ -52,17 +52,15 @@ public class CRCInfo { public static Optional fromColumnarBatch( long version, ColumnarBatch batch, int rowId, String crcFilePath) { Protocol protocol = - Protocol.fromColumnVector( - batch.getColumnVector(CRC_FILE_SCHEMA.indexOf("protocol")), rowId); + Protocol.fromColumnVector(batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(PROTOCOL)), rowId); Metadata metadata = - Metadata.fromColumnVector( - batch.getColumnVector(CRC_FILE_SCHEMA.indexOf("metadata")), rowId); + Metadata.fromColumnVector(batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(METADATA)), rowId); long tableSizeBytes = - batch.getColumnVector(CRC_FILE_SCHEMA.indexOf("tableSizeBytes")).getLong(rowId); - long numFiles = batch.getColumnVector(CRC_FILE_SCHEMA.indexOf("numFiles")).getLong(rowId); + batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TABLE_SIZE_BYTES)).getLong(rowId); + long numFiles = batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(NUM_FILES)).getLong(rowId); Optional txnId = Optional.ofNullable( - batch.getColumnVector(CRC_FILE_SCHEMA.indexOf("txnId")).getString(rowId)); + batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TXN_ID)).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); 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 index 2a5358cafab..684ec4c0819 100644 --- 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 @@ -52,15 +52,16 @@ public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { .getJsonHandler() .writeJsonFileAtomically( newChecksumPath.toString(), - singletonCloseableIterator(buildCheckSumRow(crcInfo)), + singletonCloseableIterator(toRow(crcInfo)), false /* overwrite */); + logger.info("Write checksum file `{}` succeeds", newChecksumPath); return null; }, "Write checksum file `%s`", newChecksumPath); } - private Row buildCheckSumRow(CRCInfo crcInfo) { + private Row toRow(CRCInfo crcInfo) { Map values = new HashMap<>(); // Add required fields values.put(getSchemaIndex(TABLE_SIZE_BYTES), crcInfo.getTableSizeBytes()); 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 index 7818c844578..97a94c89253 100644 --- 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 @@ -124,49 +124,53 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { ) } - private def checkMetadata(metadata: Metadata, metadataRow: Row): Unit = { - assert(metadataRow.getSchema == Metadata.FULL_SCHEMA) + private def checkMetadata(expectedMetadata: Metadata, actualMetadataRow: Row): Unit = { + assert(actualMetadataRow.getSchema == Metadata.FULL_SCHEMA) def getOptionalString(field: String): Optional[String] = - Optional.ofNullable(metadataRow.getString(Metadata.FULL_SCHEMA.indexOf(field))) + Optional.ofNullable(actualMetadataRow.getString(Metadata.FULL_SCHEMA.indexOf(field))) - assert(metadataRow.getString(Metadata.FULL_SCHEMA.indexOf("id")) == metadata.getId) - assert(getOptionalString("name") == metadata.getName) - assert(getOptionalString("description") == metadata.getDescription) + assert( + actualMetadataRow.getString(Metadata.FULL_SCHEMA.indexOf("id")) == expectedMetadata.getId + ) + assert(getOptionalString("name") == expectedMetadata.getName) + assert(getOptionalString("description") == expectedMetadata.getDescription) - val formatRow = metadataRow.getStruct(Metadata.FULL_SCHEMA.indexOf("format")) + val formatRow = actualMetadataRow.getStruct(Metadata.FULL_SCHEMA.indexOf("format")) assert( - formatRow.getString(Format.FULL_SCHEMA.indexOf("provider")) == metadata.getFormat.getProvider + formatRow + .getString(Format.FULL_SCHEMA.indexOf("provider")) == expectedMetadata.getFormat.getProvider ) assert( - metadataRow - .getString(Metadata.FULL_SCHEMA.indexOf("schemaString")) == metadata.getSchemaString + actualMetadataRow + .getString(Metadata.FULL_SCHEMA.indexOf("schemaString")) == expectedMetadata.getSchemaString ) assert( - metadataRow - .getArray(Metadata.FULL_SCHEMA.indexOf("partitionColumns")) == metadata.getPartitionColumns + actualMetadataRow + .getArray(Metadata.FULL_SCHEMA.indexOf("partitionColumns")) + == expectedMetadata.getPartitionColumns ) assert( Optional - .ofNullable(metadataRow.getLong(Metadata.FULL_SCHEMA.indexOf("createdTime"))) - == metadata.getCreatedTime + .ofNullable(actualMetadataRow.getLong(Metadata.FULL_SCHEMA.indexOf("createdTime"))) + == expectedMetadata.getCreatedTime ) assert( VectorUtils - .toJavaMap(metadataRow.getMap(Metadata.FULL_SCHEMA.indexOf("configuration"))) - == metadata.getConfiguration + .toJavaMap(actualMetadataRow.getMap(Metadata.FULL_SCHEMA.indexOf("configuration"))) + == expectedMetadata.getConfiguration ) } - private def checkProtocol(protocol: Protocol, protocolRow: Row): Unit = { - assert(protocolRow.getSchema == Protocol.FULL_SCHEMA) + private def checkProtocol(expectedProtocol: Protocol, actualProtocolRow: Row): Unit = { + assert(actualProtocolRow.getSchema == Protocol.FULL_SCHEMA) assert( - protocol.getMinReaderVersion == protocolRow + expectedProtocol.getMinReaderVersion == actualProtocolRow .getInt(Protocol.FULL_SCHEMA.indexOf("minReaderVersion")) ) assert( - protocol.getMinWriterVersion == protocolRow + expectedProtocol.getMinWriterVersion == actualProtocolRow .getInt(Protocol.FULL_SCHEMA.indexOf("minWriterVersion")) ) } From d2fc81501bfa30d369624485146c497510055251 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Thu, 13 Feb 2025 23:08:10 -0800 Subject: [PATCH 27/73] add doc to tests, simply the test code --- .../ChecksumSimpleComparisonSuite.scala | 166 +++++++----------- 1 file changed, 61 insertions(+), 105 deletions(-) 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 index 8481fb76077..4a42c563d62 100644 --- 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 @@ -42,6 +42,8 @@ import scala.collection.JavaConverters._ /** * 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 AnyFunSuite with TestUtils { @@ -70,31 +72,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { } } - test("create table as select and verify checksum") { - withTempDirAndEngine { (tablePath, engine) => - val sparkTablePath = tablePath + "spark" - val kernelTablePath = tablePath + "kernel" - spark.sql( - s"CREATE OR REPLACE TABLE delta.`${tablePath + "spark"}` USING DELTA AS SELECT 1 as id" - ) - val txn = Table - .forPath(engine, kernelTablePath) - .createTransactionBuilder(engine, "test-engine", CREATE_TABLE) - .withSchema(engine, new StructType().add("id", INTEGER)) - .build(engine) - - copyAndCommitTxnForUnpartitionedTable(txn, engine, sparkTablePath, versionAtCommit = 0) - - assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) - - (1 to 10).foreach( - version => insertIntoUnpartitionedTableAndCheckCrc( - engine, sparkTablePath, kernelTablePath, version) - ) - } - } - - test("create partitioned table and verify checksum") { + test("create partitioned table, insert and verify checksum") { withTempDirAndEngine { (tablePath, engine) => val sparkTablePath = tablePath + "spark" val kernelTablePath = tablePath + "kernel" @@ -120,38 +98,6 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { } } - test("create partition table as select and verify checksum") { - withTempDirAndEngine { (tablePath, engine) => - val sparkTablePath = tablePath + "spark" - val kernelTablePath = tablePath + "kernel" - spark.sql( - s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` USING DELTA " + - s"PARTITIONED BY (part) AS " + - s"SELECT 1 as id, 1 as part union all select 2 as id, 2 as part" - ) - val txn = Table - .forPath(engine, kernelTablePath) - .createTransactionBuilder(engine, "test-engine", CREATE_TABLE) - .withSchema(engine, new StructType().add("id", INTEGER).add(PARTITION_COLUMN, INTEGER)) - .withPartitionColumns(engine, Seq(PARTITION_COLUMN).asJava) - .build(engine) - - copyAndCommitTxnForPartitionedTable( - txn, - engine, - sparkTablePath, - Seq(1, 2).toSet, - versionAtCommit = 0 - ) - assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) - - (1 to 10).foreach( - version => insertIntoPartitionedTableAndCheckCrc( - engine, sparkTablePath, kernelTablePath, version) - ) - } - } - private def assertChecksumEquals( engine: Engine, sparkTablePath: String, @@ -189,6 +135,10 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assert(crc1.getMetadata.getPartitionColNames === crc2.getMetadata.getPartitionColNames) } + /** + * Insert into partition spark table, copy the commit log to kernel table + * and verify the checksum files for are consistent between spark and kernel + * */ private def insertIntoPartitionedTableAndCheckCrc( engine: Engine, sparkTablePath: String, @@ -210,7 +160,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { .createTransactionBuilder(engine, "test-engine", Operation.WRITE) .build(engine) - copyAndCommitTxnForPartitionedTable( + convertSparkDeltaLogToKernelCommitForPartitionedTable( txn, engine, sparkTablePath, @@ -220,6 +170,10 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } + /** + * Insert into unpartitioned spark table, copy the commit log to kernel table + * and verify the checksum files for are consistent between spark and kernel + * */ private def insertIntoUnpartitionedTableAndCheckCrc( engine: Engine, sparkTablePath: String, @@ -236,46 +190,26 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { .createTransactionBuilder(engine, "test-engine", Operation.WRITE) .build(engine) - copyAndCommitTxnForUnpartitionedTable(txn, engine, sparkTablePath, versionAtCommit) + convertSparkDeltaLogToKernelCommitForUnpartitionedTable( + txn, + engine, + sparkTablePath, + versionAtCommit + ) assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } - private def copyAndCommitTxnForUnpartitionedTable( - txn: Transaction, - engine: Engine, - sparkTablePath: String, - versionAtCommit: Long): Unit = { - val txnState = txn.getTransactionState(engine); - - val writeContext = Transaction - .getWriteContext(engine, txnState, emptyMap()) - - val dataActions = Transaction - .generateAppendActions( - engine, - txnState, - convertDeltaLogToAppendActions(engine, sparkTablePath, versionAtCommit, Option.empty), - writeContext - ) - - txn - .commit(engine, inMemoryIterable(dataActions)) - .getPostCommitHooks - .forEach(hook => hook.threadSafeInvoke(engine)) - - } - - private def copyAndCommitTxnForPartitionedTable( + private def convertSparkDeltaLogToKernelCommitForPartitionedTable( txn: Transaction, engine: Engine, sparkTablePath: String, - addedPartition: Set[Int], - versionAtCommit: Long): Unit = { + updatedPartition: Set[Int], + versionToCovert: Long): Unit = { val txnState = txn.getTransactionState(engine); val dataActions = new util.ArrayList[Row]() - addedPartition.foreach({ partition => + updatedPartition.foreach({ partition => val writeContext = Transaction .getWriteContext( engine, @@ -283,21 +217,15 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { singletonMap(PARTITION_COLUMN, Literal.ofInt(partition)) ) - Transaction - .generateAppendActions( - engine, - txnState, - convertDeltaLogToAppendActions( + Transaction.generateAppendActions( + engine, + txnState, + convertSparkTableDeltaLogToKernelAppendActions( engine, sparkTablePath, - versionAtCommit, - Some(partition.toString) - ), - writeContext - ) - .forEach( - action => dataActions.add(action) - ) + versionToCovert, + Some(partition.toString)), + writeContext).forEach(action => dataActions.add(action)) }) txn @@ -309,13 +237,42 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { } - private def convertDeltaLogToAppendActions( + private def convertSparkDeltaLogToKernelCommitForUnpartitionedTable( + txn: Transaction, engine: Engine, - tablePath: String, + sparkTablePath: String, + versionToConvert: Long): Unit = { + val txnState = txn.getTransactionState(engine); + + val writeContext = Transaction + .getWriteContext(engine, txnState, emptyMap()) + + val dataActions = Transaction + .generateAppendActions( + engine, + txnState, + convertSparkTableDeltaLogToKernelAppendActions( + engine, + sparkTablePath, + versionToConvert, + Option.empty + ), + writeContext + ) + + txn + .commit(engine, inMemoryIterable(dataActions)) + .getPostCommitHooks + .forEach(hook => hook.threadSafeInvoke(engine)) + } + + private def convertSparkTableDeltaLogToKernelAppendActions( + engine: Engine, + sparkTablePath: String, version: Long, partition: Option[String]): CloseableIterator[DataFileStatus] = { - val logPath = new Path(tablePath, "_delta_log") + val logPath = new Path(sparkTablePath, "_delta_log") val deltaFile = FileStatus.of(FileNames.deltaFile(logPath, version), 0, 0) val addFiles = new util.ArrayList[DataFileStatus]() @@ -336,7 +293,6 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { batch: ColumnarBatch, partition: Option[String], addFiles: util.ArrayList[DataFileStatus]): Unit = { - val rows = batch.getRows while (rows.hasNext) { val row = rows.next() From d1504f0a0714528abe4ebf73ada1b73910536eb3 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Thu, 13 Feb 2025 23:10:06 -0800 Subject: [PATCH 28/73] rename params --- .../kernel/internal/checksum/ChecksumWriterSuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) 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 index 97a94c89253..83dd5efc57a 100644 --- 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 @@ -94,9 +94,10 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { } } - private def verifyMetadataAndProtocol(row: Row, metadata: Metadata, protocol: Protocol): Unit = { - checkMetadata(metadata, row.getStruct(METADATA)) - checkProtocol(protocol, row.getStruct(PROTOCOL)) + private def verifyMetadataAndProtocol( + row: Row, expectedMetadata: Metadata, expectedProtocol: Protocol): Unit = { + checkMetadata(expectedMetadata, row.getStruct(METADATA)) + checkProtocol(expectedProtocol, row.getStruct(PROTOCOL)) } private def createTestMetadata(): Metadata = { From 78477d1201762fd53c71482a036c0e68ce37e61b Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Fri, 14 Feb 2025 09:56:51 -0800 Subject: [PATCH 29/73] fix comment --- .../io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 ad7ee6cfb57..553e74f2c84 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 @@ -501,7 +501,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { val metadata = Metadata.fromColumnVector( crcRow.getColumnVector(CRCInfo.CRC_FILE_SCHEMA.indexOf("metadata")), - 0 + /* rowId= */0 ) assert( From 7f7e9e704330b0f32eac23120c935f85f9e6bf84 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Fri, 14 Feb 2025 09:59:19 -0800 Subject: [PATCH 30/73] fix comment --- .../checksum/ChecksumWriterSuite.scala | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) 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 index 83dd5efc57a..42535eb0a3d 100644 --- 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 @@ -78,26 +78,28 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { } private def verifyChecksumContent( - row: Row, - tableSizeBytes: Long, - numFiles: Long, - txn: Optional[String]): Unit = { - assert(row.getLong(TABLE_SIZE_BYTES) == tableSizeBytes) - assert(row.getLong(NUM_FILES) == numFiles) - assert(row.getLong(NUM_METADATA) == 1L) - assert(row.getLong(NUM_PROTOCOL) == 1L) - - if (txn.isPresent) { - assert(row.getString(TXN_ID) == txn.get()) + actualCheckSumRow: Row, + expectedTableSizeBytes: Long, + expectedNumFiles: Long, + expectedTxnId: Optional[String]): Unit = { + assert(actualCheckSumRow.getLong(TABLE_SIZE_BYTES) == expectedTableSizeBytes) + assert(actualCheckSumRow.getLong(NUM_FILES) == expectedNumFiles) + assert(actualCheckSumRow.getLong(NUM_METADATA) == 1L) + assert(actualCheckSumRow.getLong(NUM_PROTOCOL) == 1L) + + if (expectedTxnId.isPresent) { + assert(actualCheckSumRow.getString(TXN_ID) == expectedTxnId.get()) } else { - assert(row.isNullAt(TXN_ID)) + assert(actualCheckSumRow.isNullAt(TXN_ID)) } } private def verifyMetadataAndProtocol( - row: Row, expectedMetadata: Metadata, expectedProtocol: Protocol): Unit = { - checkMetadata(expectedMetadata, row.getStruct(METADATA)) - checkProtocol(expectedProtocol, row.getStruct(PROTOCOL)) + actualRow: Row, + expectedMetadata: Metadata, + expectedProtocol: Protocol): Unit = { + checkMetadata(expectedMetadata, actualRow.getStruct(METADATA)) + checkProtocol(expectedProtocol, actualRow.getStruct(PROTOCOL)) } private def createTestMetadata(): Metadata = { From bbd9ea323cd3e45c3cd6ec09f844c76ea06e9f6d Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Fri, 14 Feb 2025 10:03:22 -0800 Subject: [PATCH 31/73] add doc --- .../kernel/defaults/ChecksumSimpleComparisonSuite.scala | 8 ++++++++ 1 file changed, 8 insertions(+) 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 index 4a42c563d62..9e3ef4c5a50 100644 --- 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 @@ -199,6 +199,10 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } + /** + * Read the spark table's commit log file, translate the added file for each partition + * in the log to kernel's append action, and commit to kernel table + * */ private def convertSparkDeltaLogToKernelCommitForPartitionedTable( txn: Transaction, engine: Engine, @@ -237,6 +241,10 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { } + /** + * Read the spark table's commit log file, translate the added file + * in the log to kernel's append action, and commit to kernel table + * */ private def convertSparkDeltaLogToKernelCommitForUnpartitionedTable( txn: Transaction, engine: Engine, From 76a5c385731dc740c98fe731bba32421614b78e4 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 11:37:26 -0800 Subject: [PATCH 32/73] move to row to crc --- .../kernel/internal/checksum/CRCInfo.java | 42 +++++++++++++++---- .../internal/checksum/ChecksumWriter.java | 28 ++----------- 2 files changed, 38 insertions(+), 32 deletions(-) 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 index 9211f432d5a..1b971506a5f 100644 --- 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 @@ -18,11 +18,15 @@ import static java.util.Objects.requireNonNull; 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.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; @@ -31,13 +35,13 @@ public class CRCInfo { private static final Logger logger = LoggerFactory.getLogger(CRCInfo.class); // Constants for schema field names - public static final String TABLE_SIZE_BYTES = "tableSizeBytes"; - public static final String NUM_FILES = "numFiles"; - public static final String NUM_METADATA = "numMetadata"; - public static final String NUM_PROTOCOL = "numProtocol"; - public static final String METADATA = "metadata"; - public static final String PROTOCOL = "protocol"; - public static final String TXN_ID = "txnId"; + 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() @@ -117,4 +121,28 @@ public long getTableSizeBytes() { 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 int getSchemaIndex(String fieldName) { + return CRC_FILE_SCHEMA.indexOf(fieldName); + } } 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 index 684ec4c0819..a09831b70c7 100644 --- 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 @@ -16,18 +16,14 @@ package io.delta.kernel.internal.checksum; import static io.delta.kernel.internal.DeltaErrors.wrapEngineExceptionThrowsIO; -import static io.delta.kernel.internal.checksum.CRCInfo.*; +import static io.delta.kernel.internal.util.Preconditions.checkArgument; import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; import static java.util.Objects.requireNonNull; -import io.delta.kernel.data.Row; import io.delta.kernel.engine.Engine; -import io.delta.kernel.internal.data.GenericRow; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.util.FileNames; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +40,7 @@ public ChecksumWriter(Path logPath) { /** Writes a checksum file */ public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { + checkArgument(crcInfo.getNumFiles() >= 0 && crcInfo.getTableSizeBytes() >=0); Path newChecksumPath = FileNames.checksumFile(logPath, crcInfo.getVersion()); logger.info("Writing checksum file to path: {}", newChecksumPath); wrapEngineExceptionThrowsIO( @@ -52,7 +49,7 @@ public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { .getJsonHandler() .writeJsonFileAtomically( newChecksumPath.toString(), - singletonCloseableIterator(toRow(crcInfo)), + singletonCloseableIterator(crcInfo.toRow()), false /* overwrite */); logger.info("Write checksum file `{}` succeeds", newChecksumPath); return null; @@ -60,23 +57,4 @@ public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { "Write checksum file `%s`", newChecksumPath); } - - private Row toRow(CRCInfo crcInfo) { - Map values = new HashMap<>(); - // Add required fields - values.put(getSchemaIndex(TABLE_SIZE_BYTES), crcInfo.getTableSizeBytes()); - values.put(getSchemaIndex(NUM_FILES), crcInfo.getNumFiles()); - values.put(getSchemaIndex(NUM_METADATA), 1L); - values.put(getSchemaIndex(NUM_PROTOCOL), 1L); - values.put(getSchemaIndex(METADATA), crcInfo.getMetadata().toRow()); - values.put(getSchemaIndex(PROTOCOL), crcInfo.getProtocol().toRow()); - - // Add optional fields - crcInfo.getTxnId().ifPresent(txn -> values.put(getSchemaIndex(TXN_ID), txn)); - return new GenericRow(CRC_FILE_SCHEMA, values); - } - - private int getSchemaIndex(String fieldName) { - return CRC_FILE_SCHEMA.indexOf(fieldName); - } } From f85c845dfcf880ac6297d6ad630774ab316a58a1 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 11:52:18 -0800 Subject: [PATCH 33/73] fix java --- .../java/io/delta/kernel/internal/checksum/ChecksumWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index a09831b70c7..1c84a0edb28 100644 --- 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 @@ -40,7 +40,7 @@ public ChecksumWriter(Path logPath) { /** Writes a checksum file */ public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { - checkArgument(crcInfo.getNumFiles() >= 0 && crcInfo.getTableSizeBytes() >=0); + checkArgument(crcInfo.getNumFiles() >= 0 && crcInfo.getTableSizeBytes() >= 0); Path newChecksumPath = FileNames.checksumFile(logPath, crcInfo.getVersion()); logger.info("Writing checksum file to path: {}", newChecksumPath); wrapEngineExceptionThrowsIO( From 97f36fc9c4e658a57459b83be0510be002bedc33 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 12:06:37 -0800 Subject: [PATCH 34/73] fix indent --- .../io/delta/kernel/internal/checksum/ChecksumWriterSuite.scala | 2 +- .../delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) 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 index 42535eb0a3d..8d2a509ed11 100644 --- 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 @@ -78,7 +78,7 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { } private def verifyChecksumContent( - actualCheckSumRow: Row, + actualCheckSumRow: Row, expectedTableSizeBytes: Long, expectedNumFiles: Long, expectedTxnId: Optional[String]): Unit = { 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 index 9e3ef4c5a50..9e1022ff1fd 100644 --- 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 @@ -127,6 +127,7 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { new File(f"$basePath/_delta_log/$version%020d.crc").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) From 56b2e7ee4c38a014369e956387670686e6ff834f Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 12:34:52 -0800 Subject: [PATCH 35/73] format scala --- .../checksum/ChecksumWriterSuite.scala | 54 +++++++++---------- 1 file changed, 27 insertions(+), 27 deletions(-) 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 index 8d2a509ed11..d571e1d1256 100644 --- 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 @@ -25,10 +25,10 @@ import io.delta.kernel.internal.util.VectorUtils.{stringArrayValue, stringString import io.delta.kernel.test.{BaseMockJsonHandler, MockEngineUtils} import io.delta.kernel.types.StructType import io.delta.kernel.utils.CloseableIterator +import org.scalatest.funsuite.AnyFunSuite import java.util -import java.util.{Collections, Optional, OptionalLong} -import org.scalatest.funsuite.AnyFunSuite +import java.util.{Collections, Optional} /** * Test suite for ChecksumWriter functionality. @@ -102,31 +102,6 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { checkProtocol(expectedProtocol, actualRow.getStruct(PROTOCOL)) } - 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.emptyList(), - Collections.emptyList() - ) - } - private def checkMetadata(expectedMetadata: Metadata, actualMetadataRow: Row): Unit = { assert(actualMetadataRow.getSchema == Metadata.FULL_SCHEMA) @@ -177,6 +152,31 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { .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.emptyList(), + Collections.emptyList() + ) + } } /** From a06bf987180574bcecc23a520a3c620e6d7a683c Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 12:35:56 -0800 Subject: [PATCH 36/73] format scala --- .../ChecksumSimpleComparisonSuite.scala | 215 +++++++++--------- 1 file changed, 110 insertions(+), 105 deletions(-) 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 index 9e1022ff1fd..f406fbad315 100644 --- 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 @@ -16,7 +16,6 @@ package io.delta.kernel.defaults import io.delta.kernel.Operation.CREATE_TABLE -import io.delta.kernel.{Operation, Table, Transaction} import io.delta.kernel.data.{ColumnarBatch, Row} import io.delta.kernel.defaults.utils.TestUtils import io.delta.kernel.engine.Engine @@ -24,12 +23,13 @@ import io.delta.kernel.expressions.Literal import io.delta.kernel.internal.actions.{AddFile, SingleAction} import io.delta.kernel.internal.checksum.{CRCInfo, ChecksumReader} import io.delta.kernel.internal.fs.Path -import io.delta.kernel.internal.util.{FileNames, VectorUtils} import io.delta.kernel.internal.util.Utils.{singletonCloseableIterator, toCloseableIterator} +import io.delta.kernel.internal.util.{FileNames, VectorUtils} import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types.StructType import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} import io.delta.kernel.utils.{CloseableIterator, DataFileStatus, FileStatus} +import io.delta.kernel.{Operation, Table, Transaction} import org.scalatest.funsuite.AnyFunSuite import java.io.File @@ -65,8 +65,8 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( - version => insertIntoUnpartitionedTableAndCheckCrc( - engine, sparkTablePath, kernelTablePath, version) + version => + insertIntoUnpartitionedTableAndCheckCrc(engine, sparkTablePath, kernelTablePath, version) ) } @@ -92,50 +92,12 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) (1 to 10).foreach( - version => insertIntoPartitionedTableAndCheckCrc( - engine, sparkTablePath, kernelTablePath, version) + version => + insertIntoPartitionedTableAndCheckCrc(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(f"$basePath/_delta_log/$version%020d.crc").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) - } - /** * Insert into partition spark table, copy the commit log to kernel table * and verify the checksum files for are consistent between spark and kernel @@ -171,33 +133,42 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } - /** - * Insert into unpartitioned spark table, copy the commit log to kernel table - * and verify the checksum files for are consistent between spark and kernel - * */ - private def insertIntoUnpartitionedTableAndCheckCrc( + private def assertChecksumEquals( engine: Engine, sparkTablePath: String, kernelTablePath: String, - versionAtCommit: Long): Unit = { - var valueToAppend = "(0)" - (0L to versionAtCommit).foreach(i => valueToAppend = valueToAppend + s",($i)") - spark.sql( - s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend" + 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 txn = Table - .forPath(engine, kernelTablePath) - .createTransactionBuilder(engine, "test-engine", Operation.WRITE) - .build(engine) + val sparkCrc = readCrcInfo(engine, sparkTablePath, version) + val kernelCrc = readCrcInfo(engine, kernelTablePath, version) - convertSparkDeltaLogToKernelCommitForUnpartitionedTable( - txn, - engine, - sparkTablePath, - versionAtCommit - ) - assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) + 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(f"$basePath/_delta_log/$version%020d.crc").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) } /** @@ -222,15 +193,19 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { singletonMap(PARTITION_COLUMN, Literal.ofInt(partition)) ) - Transaction.generateAppendActions( - engine, - txnState, - convertSparkTableDeltaLogToKernelAppendActions( + Transaction + .generateAppendActions( + engine, + txnState, + convertSparkTableDeltaLogToKernelAppendActions( engine, sparkTablePath, versionToCovert, - Some(partition.toString)), - writeContext).forEach(action => dataActions.add(action)) + Some(partition.toString) + ), + writeContext + ) + .forEach(action => dataActions.add(action)) }) txn @@ -242,39 +217,6 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { } - /** - * Read the spark table's commit log file, translate the added file - * in the log to kernel's append action, and commit to kernel table - * */ - private def convertSparkDeltaLogToKernelCommitForUnpartitionedTable( - txn: Transaction, - engine: Engine, - sparkTablePath: String, - versionToConvert: Long): Unit = { - val txnState = txn.getTransactionState(engine); - - val writeContext = Transaction - .getWriteContext(engine, txnState, emptyMap()) - - val dataActions = Transaction - .generateAppendActions( - engine, - txnState, - convertSparkTableDeltaLogToKernelAppendActions( - engine, - sparkTablePath, - versionToConvert, - Option.empty - ), - writeContext - ) - - txn - .commit(engine, inMemoryIterable(dataActions)) - .getPostCommitHooks - .forEach(hook => hook.threadSafeInvoke(engine)) - } - private def convertSparkTableDeltaLogToKernelAppendActions( engine: Engine, sparkTablePath: String, @@ -289,7 +231,8 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { val columnarBatches = engine.getJsonHandler.readJsonFiles( singletonCloseableIterator(deltaFile), SingleAction.FULL_SCHEMA, - Optional.empty()) + Optional.empty() + ) while (columnarBatches.hasNext) { processColumnarBatch(columnarBatches.next(), partition, addFiles) @@ -336,4 +279,66 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { partition.isEmpty || partition.get == VectorUtils.toJavaMap(addFile.getPartitionValues).get(PARTITION_COLUMN) } + + /** + * Insert into unpartitioned spark table, copy the commit log to kernel table + * and verify the checksum files for are consistent between spark and kernel + * */ + private def insertIntoUnpartitionedTableAndCheckCrc( + engine: Engine, + sparkTablePath: String, + kernelTablePath: String, + versionAtCommit: Long): Unit = { + var valueToAppend = "(0)" + (0L to versionAtCommit).foreach(i => valueToAppend = valueToAppend + s",($i)") + spark.sql( + s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend" + ) + + val txn = Table + .forPath(engine, kernelTablePath) + .createTransactionBuilder(engine, "test-engine", Operation.WRITE) + .build(engine) + + convertSparkDeltaLogToKernelCommitForUnpartitionedTable( + txn, + engine, + sparkTablePath, + versionAtCommit + ) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) + } + + /** + * Read the spark table's commit log file, translate the added file + * in the log to kernel's append action, and commit to kernel table + * */ + private def convertSparkDeltaLogToKernelCommitForUnpartitionedTable( + txn: Transaction, + engine: Engine, + sparkTablePath: String, + versionToConvert: Long): Unit = { + val txnState = txn.getTransactionState(engine); + + val writeContext = Transaction + .getWriteContext(engine, txnState, emptyMap()) + + val dataActions = Transaction + .generateAppendActions( + engine, + txnState, + convertSparkTableDeltaLogToKernelAppendActions( + engine, + sparkTablePath, + versionToConvert, + Option.empty + ), + writeContext + ) + + txn + .commit(engine, inMemoryIterable(dataActions)) + .getPostCommitHooks + .forEach(hook => hook.threadSafeInvoke(engine)) + } } From 1e7828be15a140886b590897ed862eeda9166ad5 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 12:46:29 -0800 Subject: [PATCH 37/73] share utils code --- .../ChecksumSimpleComparisonSuite.scala | 159 ++++++++---------- 1 file changed, 67 insertions(+), 92 deletions(-) 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 index f406fbad315..ca959894b7c 100644 --- 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 @@ -123,16 +123,40 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { .createTransactionBuilder(engine, "test-engine", Operation.WRITE) .build(engine) - convertSparkDeltaLogToKernelCommitForPartitionedTable( + convertSparkDeltaLogToKernelCommit( txn, engine, sparkTablePath, - addedPartition, - versionAtCommit + versionAtCommit, + Some(addedPartition) ) assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } + /** + * Insert into unpartitioned spark table, copy the commit log to kernel table + * and verify the checksum files for are consistent between spark and kernel + * */ + private def insertIntoUnpartitionedTableAndCheckCrc( + engine: Engine, + sparkTablePath: String, + kernelTablePath: String, + versionAtCommit: Long): Unit = { + var valueToAppend = "(0)" + (0L to versionAtCommit).foreach(i => valueToAppend = valueToAppend + s",($i)") + spark.sql( + s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend" + ) + + val txn = Table + .forPath(engine, kernelTablePath) + .createTransactionBuilder(engine, "test-engine", Operation.WRITE) + .build(engine) + + convertSparkDeltaLogToKernelCommit(txn, engine, sparkTablePath, versionAtCommit) + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) + } + private def assertChecksumEquals( engine: Engine, sparkTablePath: String, @@ -171,50 +195,62 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { assert(crc1.getMetadata.getPartitionColNames === crc2.getMetadata.getPartitionColNames) } - /** - * Read the spark table's commit log file, translate the added file for each partition - * in the log to kernel's append action, and commit to kernel table - * */ - private def convertSparkDeltaLogToKernelCommitForPartitionedTable( + private def convertSparkDeltaLogToKernelCommit( txn: Transaction, engine: Engine, sparkTablePath: String, - updatedPartition: Set[Int], - versionToCovert: Long): Unit = { - val txnState = txn.getTransactionState(engine); - - val dataActions = new util.ArrayList[Row]() + versionToConvert: Long, + addedPartition: Option[(Set[Int])] = None): Unit = { - updatedPartition.foreach({ partition => - val writeContext = Transaction - .getWriteContext( - engine, - txnState, - singletonMap(PARTITION_COLUMN, Literal.ofInt(partition)) - ) + val txnState = txn.getTransactionState(engine) - Transaction - .generateAppendActions( + val dataActionsIterator = addedPartition match { + case None => + // Unpartitioned table case + val writeContext = Transaction.getWriteContext(engine, txnState, emptyMap()) + Transaction.generateAppendActions( engine, txnState, convertSparkTableDeltaLogToKernelAppendActions( engine, sparkTablePath, - versionToCovert, - Some(partition.toString) + versionToConvert, + None ), writeContext ) - .forEach(action => dataActions.add(action)) - }) + + case Some(partitions) => + // Partitioned table case + val actions = new util.ArrayList[Row]() + partitions.foreach { partition => + val writeContext = Transaction.getWriteContext( + engine, + txnState, + singletonMap(PARTITION_COLUMN, Literal.ofInt(partition)) + ) + + Transaction + .generateAppendActions( + engine, + txnState, + convertSparkTableDeltaLogToKernelAppendActions( + engine, + sparkTablePath, + versionToConvert, + Some(partition.toString) + ), + writeContext + ) + .forEach(action => actions.add(action)) + } + actions.iterator() + } txn - .commit(engine, inMemoryIterable(toCloseableIterator(dataActions.iterator()))) + .commit(engine, inMemoryIterable(toCloseableIterator(dataActionsIterator))) .getPostCommitHooks - .forEach( - hook => hook.threadSafeInvoke(engine) - ) - + .forEach(_.threadSafeInvoke(engine)) } private def convertSparkTableDeltaLogToKernelAppendActions( @@ -280,65 +316,4 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { partition.get == VectorUtils.toJavaMap(addFile.getPartitionValues).get(PARTITION_COLUMN) } - /** - * Insert into unpartitioned spark table, copy the commit log to kernel table - * and verify the checksum files for are consistent between spark and kernel - * */ - private def insertIntoUnpartitionedTableAndCheckCrc( - engine: Engine, - sparkTablePath: String, - kernelTablePath: String, - versionAtCommit: Long): Unit = { - var valueToAppend = "(0)" - (0L to versionAtCommit).foreach(i => valueToAppend = valueToAppend + s",($i)") - spark.sql( - s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend" - ) - - val txn = Table - .forPath(engine, kernelTablePath) - .createTransactionBuilder(engine, "test-engine", Operation.WRITE) - .build(engine) - - convertSparkDeltaLogToKernelCommitForUnpartitionedTable( - txn, - engine, - sparkTablePath, - versionAtCommit - ) - assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) - } - - /** - * Read the spark table's commit log file, translate the added file - * in the log to kernel's append action, and commit to kernel table - * */ - private def convertSparkDeltaLogToKernelCommitForUnpartitionedTable( - txn: Transaction, - engine: Engine, - sparkTablePath: String, - versionToConvert: Long): Unit = { - val txnState = txn.getTransactionState(engine); - - val writeContext = Transaction - .getWriteContext(engine, txnState, emptyMap()) - - val dataActions = Transaction - .generateAppendActions( - engine, - txnState, - convertSparkTableDeltaLogToKernelAppendActions( - engine, - sparkTablePath, - versionToConvert, - Option.empty - ), - writeContext - ) - - txn - .commit(engine, inMemoryIterable(dataActions)) - .getPostCommitHooks - .forEach(hook => hook.threadSafeInvoke(engine)) - } } From 4cddb6774bec6bb71567fe6c9b2b8d8d07637198 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 13:33:32 -0800 Subject: [PATCH 38/73] share utils code --- .../ChecksumSimpleComparisonSuite.scala | 35 ++++++++++--------- 1 file changed, 18 insertions(+), 17 deletions(-) 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 index ca959894b7c..7c0f77fb659 100644 --- 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 @@ -15,7 +15,6 @@ */ package io.delta.kernel.defaults -import io.delta.kernel.Operation.CREATE_TABLE import io.delta.kernel.data.{ColumnarBatch, Row} import io.delta.kernel.defaults.utils.TestUtils import io.delta.kernel.engine.Engine @@ -30,14 +29,13 @@ import io.delta.kernel.types.StructType import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} import io.delta.kernel.utils.{CloseableIterator, DataFileStatus, FileStatus} import io.delta.kernel.{Operation, Table, Transaction} -import org.scalatest.funsuite.AnyFunSuite import java.io.File import java.nio.file.Files import java.util import java.util.Collections.{emptyMap, singletonMap} import java.util.Optional -import scala.collection.JavaConverters._ +import scala.collection.immutable.Seq /** * Test suite to verify checksum file correctness by comparing @@ -45,7 +43,7 @@ import scala.collection.JavaConverters._ * This suite ensures that both implementations generate consistent checksums * for various table operations. */ -class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { +class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUtils { private val PARTITION_COLUMN = "part" @@ -53,12 +51,14 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { withTempDirAndEngine { (tablePath, engine) => val sparkTablePath = tablePath + "spark" val kernelTablePath = tablePath + "kernel" - Table - .forPath(engine, kernelTablePath) - .createTransactionBuilder(engine, "test-engine", CREATE_TABLE) - .withSchema(engine, new StructType().add("id", INTEGER)) - .build(engine) - .commit(engine, emptyIterable()) + + createTxn( + engine, + kernelTablePath, + isNewTable = true, + schema = new StructType().add("id", INTEGER), + partCols = Seq.empty + ).commit(engine, emptyIterable()) .getPostCommitHooks .forEach(hook => hook.threadSafeInvoke(engine)) spark.sql(s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` (id Integer) USING DELTA") @@ -76,13 +76,14 @@ class ChecksumSimpleComparisonSuite extends AnyFunSuite with TestUtils { withTempDirAndEngine { (tablePath, engine) => val sparkTablePath = tablePath + "spark" val kernelTablePath = tablePath + "kernel" - Table - .forPath(engine, kernelTablePath) - .createTransactionBuilder(engine, "test-engine", CREATE_TABLE) - .withSchema(engine, new StructType().add("id", INTEGER).add(PARTITION_COLUMN, INTEGER)) - .withPartitionColumns(engine, Seq(PARTITION_COLUMN).asJava) - .build(engine) - .commit(engine, emptyIterable()) + + createTxn( + engine, + kernelTablePath, + isNewTable = true, + schema = new StructType().add("id", INTEGER).add(PARTITION_COLUMN, INTEGER), + partCols = Seq(PARTITION_COLUMN) + ).commit(engine, emptyIterable()) .getPostCommitHooks .forEach(hook => hook.threadSafeInvoke(engine)) spark.sql( From dc4c94c5d5ea9ad637271f1adb994d0b015ee95b Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 15:29:56 -0800 Subject: [PATCH 39/73] refactor test --- .../defaults/DeltaTableWriteSuiteBase.scala | 62 +- .../DeltaTableWriteWithCrcSuite.scala | 115 ++ .../defaults/DeltaTableWritesSuite.scala | 1100 +++++++---------- .../kernel/defaults/utils/TestUtils.scala | 6 +- 4 files changed, 572 insertions(+), 711 deletions(-) create mode 100644 kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteWithCrcSuite.scala 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 553e74f2c84..e25b8f3fb4a 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 @@ -24,8 +24,8 @@ import io.delta.kernel.internal.actions.{Metadata, Protocol, SingleAction} import io.delta.kernel.internal.fs.{Path => DeltaPath} import io.delta.kernel.internal.util.{Clock, FileNames, VectorUtils} import io.delta.kernel.internal.util.Utils.singletonCloseableIterator -import io.delta.kernel.internal.{SnapshotImpl, TableConfig, TableImpl} -import io.delta.kernel.utils.FileStatus +import io.delta.kernel.internal.{SnapshotImpl, TableConfig, TableImpl, TransactionImpl} +import io.delta.kernel.utils.{CloseableIterable, CloseableIterator, FileStatus} import io.delta.kernel.{ Meta, Operation, @@ -43,7 +43,6 @@ 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.{emptyIterable, inMemoryIterable} -import io.delta.kernel.utils.CloseableIterator import io.delta.kernel.Operation.CREATE_TABLE import io.delta.kernel.hook.PostCommitHook.PostCommitHookType import io.delta.kernel.internal.checksum.CRCInfo @@ -311,7 +310,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { } val combineActions = inMemoryIterable(actions.reduceLeft(_ combine _)) - txn.commit(engine, combineActions) + commitTransaction(txn, engine, combineActions) } def appendData( @@ -370,13 +369,12 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { assertMetadataProp(snapshot, key, expectedValue) } - def verifyWrittenContent( + protected def verifyWrittenContent( path: String, expSchema: StructType, expData: Seq[TestRow], expPartitionColumns: Seq[String] = Seq(), - version: Option[Long] = Option.empty, - checksumWritten: Boolean = false): Unit = { + version: Option[Long] = Option.empty): Unit = { val actSchema = tableSchema(path) assert(actSchema === expSchema) @@ -395,10 +393,6 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { .map(TestRow(_)) checkAnswer(resultSpark, expData) } - - if (checksumWritten) { - checkChecksumContent(path, version, expSchema, expPartitionColumns) - } } def verifyCommitInfo( @@ -477,46 +471,10 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { ) } - def checkChecksumContent( - tablePath: String, - version: Option[Long], - expSchema: StructType, - expPartitionColumns: Seq[String]): Unit = { - val checksumVersion = version.getOrElse(latestSnapshot(tablePath, defaultEngine).getVersion) - val checksumFile = new File(f"$tablePath/_delta_log/$checksumVersion%020d.crc") - - assert(Files.exists(checksumFile.toPath), s"Checksum file not found: ${checksumFile.getPath}") - - val columnarBatches = defaultEngine - .getJsonHandler() - .readJsonFiles( - singletonCloseableIterator(FileStatus.of(checksumFile.getPath)), - CRCInfo.CRC_FILE_SCHEMA, - Optional.empty() - ) - - assert(columnarBatches.hasNext, "Empty checksum file") - val crcRow = columnarBatches.next() - assert(crcRow.getSize === 1, s"Expected single row, found ${crcRow.getSize}") - - val metadata = Metadata.fromColumnVector( - crcRow.getColumnVector(CRCInfo.CRC_FILE_SCHEMA.indexOf("metadata")), - /* rowId= */0 - ) - - assert( - metadata.getSchema === expSchema, - s"Schema mismatch.\nExpected: $expSchema\nActual: ${metadata.getSchema}" - ) - - val normalizedPartitions = expPartitionColumns.map(_.toLowerCase(Locale.ROOT)).toSet - assert( - metadata.getPartitionColNames.asScala === normalizedPartitions, - s"Partition columns mismatch.\n" + - s"Expected: $normalizedPartitions\n" + - s"Actual: ${metadata.getPartitionColNames.asScala}" - ) - - assert(!columnarBatches.hasNext, "Unexpected additional data in checksum file") + 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..cfe13e80214 --- /dev/null +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteWithCrcSuite.scala @@ -0,0 +1,115 @@ +package io.delta.kernel.defaults +import io.delta.kernel.data.Row +import io.delta.kernel.{Transaction, TransactionCommitResult} +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.TransactionImpl +import io.delta.kernel.internal.actions.Metadata +import io.delta.kernel.internal.checksum.CRCInfo +import io.delta.kernel.internal.util.Utils.singletonCloseableIterator +import io.delta.kernel.types.StructType +import io.delta.kernel.utils.{CloseableIterable, FileStatus} + +import java.io.File +import java.nio.file.Files +import java.util.{Locale, Optional} +import scala.collection.immutable.Seq +import scala.jdk.CollectionConverters.{asScalaBufferConverter, asScalaSetConverter} +import scala.language.implicitConversions + +class DeltaTableWriteWithCrcSuite extends DeltaTableWritesSuite { + + implicit class TransactionOps(txn: Transaction) { + def commitAndGenerateCrc( + 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 commitTransaction( + txn: Transaction, + engine: Engine, + dataActions: CloseableIterable[Row]): TransactionCommitResult = { + txn.commitAndGenerateCrc(engine, dataActions) + } + + override def verifyWrittenContent( + path: String, + expSchema: StructType, + expData: Seq[TestRow], + expPartitionColumns: Seq[String] = Seq(), + version: Option[Long] = Option.empty): Unit = { + val actSchema = tableSchema(path) + assert(actSchema === expSchema) + + // verify data using Kernel reader + checkTable(path, expData) + + // verify data using Spark reader. + // Spark reads the timestamp partition columns in local timezone vs. Kernel reads in UTC. We + // need to set the timezone to UTC before reading the data using Spark to make the tests pass + withSparkTimeZone("UTC") { + val resultSpark = spark + .sql(s"SELECT * FROM delta.`$path`" + { + if (version.isDefined) s" VERSION AS OF ${version.get}" else "" + }) + .collect() + .map(TestRow(_)) + checkAnswer(resultSpark, expData) + } + + checkChecksumContent(path, version, expSchema, expPartitionColumns) + } + + def checkChecksumContent( + tablePath: String, + version: Option[Long], + expSchema: StructType, + expPartitionColumns: Seq[String]): Unit = { + val checksumVersion = version.getOrElse(latestSnapshot(tablePath, defaultEngine).getVersion) + val checksumFile = new File(f"$tablePath/_delta_log/$checksumVersion%020d.crc") + + assert(Files.exists(checksumFile.toPath), s"Checksum file not found: ${checksumFile.getPath}") + + val columnarBatches = defaultEngine + .getJsonHandler() + .readJsonFiles( + singletonCloseableIterator(FileStatus.of(checksumFile.getPath)), + CRCInfo.CRC_FILE_SCHEMA, + Optional.empty() + ) + + assert(columnarBatches.hasNext, "Empty checksum file") + val crcRow = columnarBatches.next() + assert(crcRow.getSize === 1, s"Expected single row, found ${crcRow.getSize}") + + val metadata = Metadata.fromColumnVector( + crcRow.getColumnVector(CRCInfo.CRC_FILE_SCHEMA.indexOf("metadata")), + /* rowId= */ 0 + ) + + assert( + metadata.getSchema === expSchema, + s"Schema mismatch.\nExpected: $expSchema\nActual: ${metadata.getSchema}" + ) + + val normalizedPartitions = expPartitionColumns.map(_.toLowerCase(Locale.ROOT)).toSet + assert( + metadata.getPartitionColNames.asScala === normalizedPartitions, + s"Partition columns mismatch.\n" + + s"Expected: $normalizedPartitions\n" + + s"Actual: ${metadata.getPartitionColNames.asScala}" + ) + + assert(!columnarBatches.hasNext, "Unexpected additional data in checksum file") + } +} 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 6e06e331c29..a07b5ea5951 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 @@ -25,7 +25,6 @@ import io.delta.kernel.engine.Engine import io.delta.kernel.exceptions._ import io.delta.kernel.expressions.Literal import io.delta.kernel.expressions.Literal._ -import io.delta.kernel.hook.PostCommitHook.PostCommitHookType import io.delta.kernel.internal.checkpoints.CheckpointerSuite.selectSingleElement import io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames import io.delta.kernel.internal.{SnapshotImpl, TableConfig} @@ -39,7 +38,6 @@ import io.delta.kernel.types.TimestampType.TIMESTAMP import io.delta.kernel.types._ import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} import io.delta.kernel.utils.CloseableIterable -import org.apache.spark.sql.delta.sources.DeltaSQLConf import java.util.{Locale, Optional} import scala.collection.JavaConverters._ @@ -96,7 +94,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 +128,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) @@ -330,172 +328,105 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } } - Seq(true, false).foreach( - executePostCommitHook => - test("create a partitioned table" + { - if (executePostCommitHook) " and execute post commit hook" else "" - }) { - withTempDirAndEngine { - (tablePath, engine) => - val table = Table.forPath(engine, tablePath) - val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) - - val schema = new StructType() - .add("id", INTEGER) - .add("Part1", INTEGER) // partition column - .add("part2", INTEGER) // partition column - - val txn = txnBuilder - .withSchema(engine, schema) - // partition columns should preserve the same case the one in the schema - .withPartitionColumns(engine, Seq("part1", "PART2").asJava) - .build(engine) - - 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()) - - assert(txnResult.getVersion === 0) - assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) - assertChecksumSimpleReadiness(txnResult) - if (executePostCommitHook) { - txnResult.getPostCommitHooks.forEach(hook => hook.threadSafeInvoke(engine)) - } + test("create a partitioned table") { + withTempDirAndEngine { (tablePath, engine) => + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) - verifyCommitInfo(tablePath, version = 0, Seq("Part1", "part2")) - verifyWrittenContent( - tablePath, - schema, - Seq.empty, - Seq("Part1", "part2"), - version = Some(0), - checksumWritten = executePostCommitHook - ) - } - } - ) - - Seq(true, false).foreach( - executePostCommitHook => - test("create table with all supported types" + { - if (executePostCommitHook) " and execute post commit hook" else "" - }) { - withTempDirAndEngine { - (tablePath, engine) => - val parquetAllTypes = goldenTablePath("parquet-all-types") - val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) - - 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()) - - assert(txnResult.getVersion === 0) - assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) - assertChecksumSimpleReadiness(txnResult) - if (executePostCommitHook) { - txnResult.getPostCommitHooks.forEach(hook => hook.threadSafeInvoke(engine)) - } + val schema = new StructType() + .add("id", INTEGER) + .add("Part1", INTEGER) // partition column + .add("part2", INTEGER) // partition column - verifyCommitInfo(tablePath, version = 0) - verifyWrittenContent( - tablePath, - schema, - Seq.empty, - version = Some(0), - checksumWritten = executePostCommitHook - ) - } - } - ) + val txn = txnBuilder + .withSchema(engine, schema) + // partition columns should preserve the same case the one in the schema + .withPartitionColumns(engine, Seq("part1", "PART2").asJava) + .build(engine) + + 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 = commitTransaction(txn, engine, emptyIterable()) + + assert(txnResult.getVersion === 0) + assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) + + verifyCommitInfo(tablePath, version = 0, Seq("Part1", "part2")) + verifyWrittenContent(tablePath, schema, Seq.empty, Seq("Part1", "part2")) + } + } + + test("create table with all supported types") { + withTempDirAndEngine { (tablePath, engine) => + val parquetAllTypes = goldenTablePath("parquet-all-types") + val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) + + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + val txn = txnBuilder.withSchema(engine, schema).build(engine) + val txnResult = commitTransaction(txn, engine, emptyIterable()) + + assert(txnResult.getVersion === 0) + assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) + + verifyCommitInfo(tablePath, version = 0) + verifyWrittenContent(tablePath, schema, Seq.empty) + } + } /////////////////////////////////////////////////////////////////////////// // Create table and insert data tests (CTAS & INSERT) /////////////////////////////////////////////////////////////////////////// + test("insert into table - table created from scratch") { + withTempDirAndEngine { (tblPath, engine) => + val commitResult0 = appendData( + engine, + tblPath, + isNewTable = true, + testSchema, + partCols = Seq.empty, + data = Seq(Map.empty[String, Literal] -> (dataBatches1 ++ dataBatches2)) + ) - Seq(true, false).foreach( - executePostCommitHook => - test("insert into table - table created from scratch" + { - if (executePostCommitHook) " and execute post commit hook" else "" - }) { - withTempDirAndEngine { - (tblPath, engine) => - val commitResult0 = appendData( - engine, - tblPath, - isNewTable = true, - testSchema, - partCols = Seq.empty, - data = Seq(Map.empty[String, Literal] -> (dataBatches1 ++ dataBatches2)), - executePostCommitHook = executePostCommitHook - ) + val expectedAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap( + _.toTestRows + ) - val expectedAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap( - _.toTestRows - ) - - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) - verifyWrittenContent( - tblPath, - testSchema, - expectedAnswer, - version = Some(0), - checksumWritten = executePostCommitHook - ) - } - } - ) - - Seq(true, false).foreach( - executePostCommitHook => - test("insert into table - already existing table" + { - if (executePostCommitHook) " and execute post commit hook" else "" - }) { - withTempDirAndEngine { - (tblPath, engine) => - val commitResult0 = appendData( - engine, - tblPath, - isNewTable = true, - testSchema, - partCols = Seq.empty, - data = Seq(Map.empty[String, Literal] -> dataBatches1), - executePostCommitHook = executePostCommitHook - ) + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) + verifyWrittenContent(tblPath, testSchema, expectedAnswer) + } + } - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) - verifyWrittenContent( - tblPath, - testSchema, - dataBatches1.flatMap(_.toTestRows), - version = Some(0), - checksumWritten = executePostCommitHook - ) + test("insert into table - already existing table") { + withTempDirAndEngine { (tblPath, engine) => + val commitResult0 = appendData( + engine, + tblPath, + isNewTable = true, + testSchema, + partCols = Seq.empty, + data = Seq(Map.empty[String, Literal] -> dataBatches1) + ) - val commitResult1 = appendData( - engine, - tblPath, - data = Seq(Map.empty[String, Literal] -> dataBatches2), - executePostCommitHook = executePostCommitHook - ) + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) + verifyWrittenContent(tblPath, testSchema, dataBatches1.flatMap(_.toTestRows)) + + val commitResult1 = appendData( + engine, + tblPath, + data = Seq(Map.empty[String, Literal] -> dataBatches2) + ) - val expAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap(_.toTestRows) + val expAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap(_.toTestRows) - verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) - verifyWrittenContent( - tblPath, - testSchema, - expAnswer, - version = Some(1), - checksumWritten = executePostCommitHook - ) + verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) + verifyWrittenContent(tblPath, testSchema, expAnswer) } - } - ) + } test("insert into table - fails when committing the same txn twice") { withTempDirAndEngine { (tblPath, engine) => @@ -509,186 +440,132 @@ 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) - } - assert(ex.getMessage.contains( - "Transaction is already attempted to commit. Create a new transaction.")) + commitTransaction(txn, engine, stagedActionsIterable) + } + assert( + ex.getMessage + .contains("Transaction is already attempted to commit. Create a new transaction.") + ) } } - Seq(true, false).foreach( - executePostCommitHook => - test("insert into partitioned table - table created from scratch" + { - if (executePostCommitHook) " and execute post commit hook" else "" - }) { - withTempDirAndEngine { - (tblPath, engine) => - val commitResult0 = appendData( - engine, - tblPath, - isNewTable = true, - testPartitionSchema, - testPartitionColumns, - Seq( - Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1, - Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2 - ), - executePostCommitHook = executePostCommitHook - ) + test("insert into partitioned table - table created from scratch") { + withTempDirAndEngine { (tblPath, engine) => + val commitResult0 = appendData( + engine, + tblPath, + isNewTable = true, + testPartitionSchema, + testPartitionColumns, + Seq( + Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1, + Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2 + ) + ) - val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ - dataPartitionBatches2.flatMap(_.toTestRows) - - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, testPartitionColumns, operation = WRITE) - verifyWrittenContent( - tblPath, - testPartitionSchema, - expData, - testPartitionColumns, - version = Some(0), - checksumWritten = executePostCommitHook - ) - } - } - ) - - Seq(true, false).foreach( - executePostCommitHook => - test("insert into partitioned table - already existing table" + { - if (executePostCommitHook) " and execute post commit hook" else "" - }) { - withTempDirAndEngine { - (tempTblPath, engine) => - val tblPath = tempTblPath + "/table+ with special chars" - val partitionCols = Seq("part1", "part2") - - { - val commitResult0 = appendData( - engine, - tblPath, - isNewTable = true, - testPartitionSchema, - testPartitionColumns, - data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1), - executePostCommitHook = executePostCommitHook - ) - - val expData = dataPartitionBatches1.flatMap(_.toTestRows) - - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, partitionCols, operation = WRITE) - verifyWrittenContent( - tblPath, - testPartitionSchema, - expData, - testPartitionColumns, - version = Some(0), - checksumWritten = executePostCommitHook - ) - } - { - val commitResult1 = appendData( - engine, - tblPath, - data = Seq(Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2), - executePostCommitHook = executePostCommitHook - ) - - val expData = - dataPartitionBatches1.flatMap(_.toTestRows) ++ - dataPartitionBatches2.flatMap(_.toTestRows) - - verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) - verifyWrittenContent( - tblPath, - testPartitionSchema, - expData, - testPartitionColumns, - version = Some(1), - checksumWritten = executePostCommitHook - ) - } + val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ + dataPartitionBatches2.flatMap(_.toTestRows) + + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, testPartitionColumns, operation = WRITE) + verifyWrittenContent(tblPath, testPartitionSchema, expData, testPartitionColumns) + } + } + + test("insert into partitioned table - already existing table") { + withTempDirAndEngine { (tempTblPath, engine) => + val tblPath = tempTblPath + "/table+ with special chars" + val partitionCols = Seq("part1", "part2") + + { + val commitResult0 = appendData( + engine, + tblPath, + isNewTable = true, + testPartitionSchema, + testPartitionColumns, + data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) + ) + + val expData = dataPartitionBatches1.flatMap(_.toTestRows) + + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, partitionCols, operation = WRITE) + verifyWrittenContent(tblPath, testPartitionSchema, expData, partitionCols) } + { + val commitResult1 = appendData( + engine, + tblPath, + data = Seq(Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2) + ) + + val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ + dataPartitionBatches2.flatMap(_.toTestRows) + + verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) + verifyWrittenContent(tblPath, testPartitionSchema, expData, partitionCols) } - ) - - Seq(true, false).foreach( - executePostCommitHook => - test("insert into partitioned table - handling case sensitivity of partition columns" + { - if (executePostCommitHook) " and execute post commit hook" else "" - }) { - withTempDirAndEngine { - (tblPath, engine) => - val schema = new StructType() - .add("id", INTEGER) - .add("Name", STRING) - .add("Part1", DOUBLE) // partition column - .add("parT2", TIMESTAMP) // partition column - - val partCols = Seq("part1", "Part2") // given as input to the txn builder - - // expected partition cols in the commit info or elsewhere in the Delta log. - // it is expected to contain the partition columns in the same case as the schema - val expPartCols = Seq("Part1", "parT2") - - val v0Part0Values = Map("PART1" -> ofDouble(1.0), "pART2" -> ofTimestamp(1231212L)) - val v0Part0Data = - generateData(schema, expPartCols, v0Part0Values, batchSize = 200, numBatches = 3) - - val v0Part1Values = Map("Part1" -> ofDouble(7), "PARt2" -> ofTimestamp(123112L)) - val v0Part1Data = - generateData(schema, expPartCols, v0Part1Values, batchSize = 100, numBatches = 7) - - val v1Part0Values = Map("PART1" -> ofNull(DOUBLE), "pART2" -> ofTimestamp(1231212L)) - val v1Part0Data = - generateData(schema, expPartCols, v1Part0Values, batchSize = 200, numBatches = 3) - - val v1Part1Values = Map("Part1" -> ofDouble(7), "PARt2" -> ofNull(TIMESTAMP)) - val v1Part1Data = - generateData(schema, expPartCols, v1Part1Values, batchSize = 100, numBatches = 7) - - val dataPerVersion = Map( - 0 -> Seq(v0Part0Values -> v0Part0Data, v0Part1Values -> v0Part1Data), - 1 -> Seq(v1Part0Values -> v1Part0Data, v1Part1Values -> v1Part1Data) - ) + } + } - val expV0Data = v0Part0Data.flatMap(_.toTestRows) ++ v0Part1Data.flatMap(_.toTestRows) - val expV1Data = v1Part0Data.flatMap(_.toTestRows) ++ v1Part1Data.flatMap(_.toTestRows) - - for (i <- 0 until 2) { - val commitResult = appendData( - engine, - tblPath, - isNewTable = i == 0, - schema, - partCols, - dataPerVersion(i), - executePostCommitHook = executePostCommitHook - ) - - verifyCommitResult(commitResult, expVersion = i, expIsReadyForCheckpoint = false) - // 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, - partCols, - version = Some(i), - checksumWritten = executePostCommitHook - ) - } - } + test("insert into partitioned table - handling case sensitivity of partition columns") { + withTempDirAndEngine { (tblPath, engine) => + val schema = new StructType() + .add("id", INTEGER) + .add("Name", STRING) + .add("Part1", DOUBLE) // partition column + .add("parT2", TIMESTAMP) // partition column + + val partCols = Seq("part1", "Part2") // given as input to the txn builder + + // expected partition cols in the commit info or elsewhere in the Delta log. + // it is expected to contain the partition columns in the same case as the schema + val expPartCols = Seq("Part1", "parT2") + + val v0Part0Values = Map("PART1" -> ofDouble(1.0), "pART2" -> ofTimestamp(1231212L)) + val v0Part0Data = + generateData(schema, expPartCols, v0Part0Values, batchSize = 200, numBatches = 3) + + val v0Part1Values = Map("Part1" -> ofDouble(7), "PARt2" -> ofTimestamp(123112L)) + val v0Part1Data = + generateData(schema, expPartCols, v0Part1Values, batchSize = 100, numBatches = 7) + + val v1Part0Values = Map("PART1" -> ofNull(DOUBLE), "pART2" -> ofTimestamp(1231212L)) + val v1Part0Data = + generateData(schema, expPartCols, v1Part0Values, batchSize = 200, numBatches = 3) + + val v1Part1Values = Map("Part1" -> ofDouble(7), "PARt2" -> ofNull(TIMESTAMP)) + val v1Part1Data = + generateData(schema, expPartCols, v1Part1Values, batchSize = 100, numBatches = 7) + + val dataPerVersion = Map( + 0 -> Seq(v0Part0Values -> v0Part0Data, v0Part1Values -> v0Part1Data), + 1 -> Seq(v1Part0Values -> v1Part0Data, v1Part1Values -> v1Part1Data) + ) + + val expV0Data = v0Part0Data.flatMap(_.toTestRows) ++ v0Part1Data.flatMap(_.toTestRows) + val expV1Data = v1Part0Data.flatMap(_.toTestRows) ++ v1Part1Data.flatMap(_.toTestRows) + + for (i <- 0 until 2) { + val commitResult = + appendData(engine, tblPath, isNewTable = i == 0, schema, partCols, dataPerVersion(i)) + + verifyCommitResult(commitResult, expVersion = i, expIsReadyForCheckpoint = false) + // 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, partCols) } - ) + } + } Seq(10, 2).foreach { checkpointInterval => test(s"insert into partitioned table - isReadyForCheckpoint(interval=$checkpointInterval)") { @@ -740,209 +617,136 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa // delete all commit files before version 30 in both cases and expect the read to pass as // there is a checkpoint at version 30 and should be used for state reconstruction. deleteDeltaFilesBefore(tblPath, beforeVersion = 30) - verifyWrittenContent(tblPath, schema, expData) + verifyWrittenContent(tblPath, schema, expData, partCols) } } } - Seq(true, false).foreach( - executePostCommitHook => - test("insert into table - all supported types data" + { - if (executePostCommitHook) " and execute post commit hook" else "" - }) { - withTempDirAndEngine { - (tblPath, engine) => - val parquetAllTypes = goldenTablePath("parquet-all-types") - val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) - - val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] - val dataWithPartInfo = Seq(Map.empty[String, Literal] -> data) - - appendData( - engine, - tblPath, - isNewTable = true, - schema, - Seq.empty, - dataWithPartInfo, - executePostCommitHook = executePostCommitHook - ) - var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + test("insert into table - all supported types data") { + withTempDirAndEngine { (tblPath, engine) => + val parquetAllTypes = goldenTablePath("parquet-all-types") + val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) - val checkpointInterval = 4 - withSQLConf( - DeltaSQLConf.DELTA_WRITE_CHECKSUM_ENABLED.key -> executePostCommitHook.toString - ) { - setCheckpointInterval(tblPath, checkpointInterval) - } + val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] + val dataWithPartInfo = Seq(Map.empty[String, Literal] -> data) - for (i <- 2 until 5) { - // insert until a checkpoint is required - val commitResult = appendData( - engine, - tblPath, - data = dataWithPartInfo, - executePostCommitHook = executePostCommitHook - ) - - expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - // post commit hooks contains check point - if (!executePostCommitHook) { - checkpointIfReady( - engine, - tblPath, - commitResult, - expSize = i /* one file per version */ - ) - } + appendData(engine, tblPath, isNewTable = true, schema, Seq.empty, dataWithPartInfo) + var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) - verifyCommitInfo(tblPath, version = i, null, operation = WRITE) - verifyWrittenContent( - tblPath, - schema, - expData, - version = Some(i), - checksumWritten = executePostCommitHook - ) - } - assertCheckpointExists(tblPath, atVersion = checkpointInterval) - } - } - ) - - Seq(true, false).foreach( - executePostCommitHook => - test("insert into partitioned table - all supported partition column types data" + { - if (executePostCommitHook) " and execute post commit hook" else "" - }) { - withTempDirAndEngine { - (tblPath, engine) => - val parquetAllTypes = goldenTablePath("parquet-all-types") - val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) - val partCols = Seq( - "byteType", - "shortType", - "integerType", - "longType", - "floatType", - "doubleType", - "decimal", - "booleanType", - "stringType", - "binaryType", - "dateType", - "timestampType" - ) - val casePreservingPartCols = - casePreservingPartitionColNames(schema, partCols.asJava).asScala.to[Seq] - - // get the partition values from the data batch at the given rowId - def getPartitionValues(batch: ColumnarBatch, rowId: Int): Map[String, Literal] = { - casePreservingPartCols.map { - partCol => - val colIndex = schema.indexOf(partCol) - val vector = batch.getColumnVector(colIndex) - - val literal = if (vector.isNullAt(rowId)) { - Literal.ofNull(vector.getDataType) - } else { - vector.getDataType match { - case _: ByteType => Literal.ofByte(vector.getByte(rowId)) - case _: ShortType => Literal.ofShort(vector.getShort(rowId)) - case _: IntegerType => Literal.ofInt(vector.getInt(rowId)) - case _: LongType => Literal.ofLong(vector.getLong(rowId)) - case _: FloatType => Literal.ofFloat(vector.getFloat(rowId)) - case _: DoubleType => Literal.ofDouble(vector.getDouble(rowId)) - case dt: DecimalType => - Literal.ofDecimal(vector.getDecimal(rowId), dt.getPrecision, dt.getScale) - case _: BooleanType => Literal.ofBoolean(vector.getBoolean(rowId)) - case _: StringType => Literal.ofString(vector.getString(rowId)) - case _: BinaryType => Literal.ofBinary(vector.getBinary(rowId)) - case _: DateType => Literal.ofDate(vector.getInt(rowId)) - case _: TimestampType => Literal.ofTimestamp(vector.getLong(rowId)) - case _ => - throw new IllegalArgumentException( - s"Unsupported type: ${vector.getDataType}" - ) - } - } - (partCol, literal) - }.toMap + val checkpointInterval = 4 + setCheckpointInterval(tblPath, checkpointInterval) + + for (i <- 2 until 5) { + // insert until a checkpoint is required + val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) + + expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + checkpointIfReady(engine, tblPath, commitResult, expSize = i /* one file per version */ ) + + verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) + verifyCommitInfo(tblPath, version = i, null, operation = WRITE) + verifyWrittenContent(tblPath, schema, expData) } + assertCheckpointExists(tblPath, atVersion = checkpointInterval) + } + } - val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] - - // From the above table read data, convert each row as a new batch with partition info - // Take the values of the partitionCols from the data and create a new batch with the - // selection vector to just select a single row. - var dataWithPartInfo = Seq.empty[(Map[String, Literal], Seq[FilteredColumnarBatch])] - - data.foreach { - filteredBatch => - val batch = filteredBatch.getData - Seq.range(0, batch.getSize).foreach { rowId => - val partValues = getPartitionValues(batch, rowId) - val filteredBatch = new FilteredColumnarBatch( - batch, - Optional.of(selectSingleElement(batch.getSize, rowId)) - ) - dataWithPartInfo = dataWithPartInfo :+ (partValues, Seq(filteredBatch)) - } + test("insert into partitioned table - all supported partition column types data") { + withTempDirAndEngine { (tblPath, engine) => + val parquetAllTypes = goldenTablePath("parquet-all-types") + val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) + val partCols = Seq( + "byteType", + "shortType", + "integerType", + "longType", + "floatType", + "doubleType", + "decimal", + "booleanType", + "stringType", + "binaryType", + "dateType", + "timestampType" + ) + val casePreservingPartCols = + casePreservingPartitionColNames(schema, partCols.asJava).asScala.to[Seq] + + // get the partition values from the data batch at the given rowId + def getPartitionValues(batch: ColumnarBatch, rowId: Int): Map[String, Literal] = { + casePreservingPartCols.map { partCol => + val colIndex = schema.indexOf(partCol) + val vector = batch.getColumnVector(colIndex) + + val literal = if (vector.isNullAt(rowId)) { + Literal.ofNull(vector.getDataType) + } else { + vector.getDataType match { + case _: ByteType => Literal.ofByte(vector.getByte(rowId)) + case _: ShortType => Literal.ofShort(vector.getShort(rowId)) + case _: IntegerType => Literal.ofInt(vector.getInt(rowId)) + case _: LongType => Literal.ofLong(vector.getLong(rowId)) + case _: FloatType => Literal.ofFloat(vector.getFloat(rowId)) + case _: DoubleType => Literal.ofDouble(vector.getDouble(rowId)) + case dt: DecimalType => + Literal.ofDecimal(vector.getDecimal(rowId), dt.getPrecision, dt.getScale) + case _: BooleanType => Literal.ofBoolean(vector.getBoolean(rowId)) + case _: StringType => Literal.ofString(vector.getString(rowId)) + case _: BinaryType => Literal.ofBinary(vector.getBinary(rowId)) + case _: DateType => Literal.ofDate(vector.getInt(rowId)) + case _: TimestampType => Literal.ofTimestamp(vector.getLong(rowId)) + case _ => + throw new IllegalArgumentException(s"Unsupported type: ${vector.getDataType}") } + } + (partCol, literal) + }.toMap + } + + val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] - appendData( - engine, - tblPath, - isNewTable = true, - schema, - partCols, - dataWithPartInfo, - executePostCommitHook = executePostCommitHook + // From the above table read data, convert each row as a new batch with partition info + // Take the values of the partitionCols from the data and create a new batch with the + // selection vector to just select a single row. + var dataWithPartInfo = Seq.empty[(Map[String, Literal], Seq[FilteredColumnarBatch])] + + data.foreach { filteredBatch => + val batch = filteredBatch.getData + Seq.range(0, batch.getSize).foreach { rowId => + val partValues = getPartitionValues(batch, rowId) + val filteredBatch = new FilteredColumnarBatch( + batch, + Optional.of(selectSingleElement(batch.getSize, rowId)) ) - verifyCommitInfo(tblPath, version = 0, casePreservingPartCols, operation = WRITE) + dataWithPartInfo = dataWithPartInfo :+ (partValues, Seq(filteredBatch)) + } + } - var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + appendData(engine, tblPath, isNewTable = true, schema, partCols, dataWithPartInfo) + verifyCommitInfo(tblPath, version = 0, casePreservingPartCols, operation = WRITE) - val checkpointInterval = 2 - withSQLConf( - DeltaSQLConf.DELTA_WRITE_CHECKSUM_ENABLED.key -> executePostCommitHook.toString - ) { - setCheckpointInterval(tblPath, checkpointInterval) // version 1 - } + var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - for (i <- 2 until 4) { - // insert until a checkpoint is required - val commitResult = appendData( - engine, - tblPath, - data = dataWithPartInfo, - executePostCommitHook = executePostCommitHook - ) + val checkpointInterval = 2 + setCheckpointInterval(tblPath, checkpointInterval) // version 1 - expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + for (i <- 2 until 4) { + // insert until a checkpoint is required + val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) - val fileCount = dataFileCount(tblPath) - if (!executePostCommitHook) { - checkpointIfReady(engine, tblPath, commitResult, expSize = fileCount) - } + expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) - verifyCommitInfo(tblPath, version = i, partitionCols = null, operation = WRITE) - verifyWrittenContent( - tblPath, - schema, - expData, - partCols, - version = Some(i), - checksumWritten = executePostCommitHook - ) - } + val fileCount = dataFileCount(tblPath) + checkpointIfReady(engine, tblPath, commitResult, expSize = fileCount) - assertCheckpointExists(tblPath, atVersion = checkpointInterval) + verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) + verifyCommitInfo(tblPath, version = i, partitionCols = null, operation = WRITE) + verifyWrittenContent(tblPath, schema, expData, partCols) } - } - ) + + assertCheckpointExists(tblPath, atVersion = checkpointInterval) + } + } test("insert into table - given data schema mismatch") { withTempDirAndEngine { (tblPath, engine) => @@ -982,160 +786,137 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa isNewTable = true, testPartitionSchema, testPartitionColumns, - data) - } - assert(ex.getMessage.contains( - "Partition column part2 is of type integer but the value provided is of type string")) + data + ) + } + assert( + ex.getMessage.contains( + "Partition column part2 is of type integer but the value provided is of type string" + ) + ) } } - Seq(true, false).foreach( - executePostCommitHook => - test("insert into table - idempotent writes" + { - if (executePostCommitHook) " and execute post commit hook" else "" - }) { - withTempDirAndEngine { - (tblPath, engine) => - val data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) - var expData = Seq.empty[TestRow] // as the data in inserted, update this. - - def prepTxnAndActions( - newTbl: Boolean, - appId: String, - txnVer: Long): (Transaction, CloseableIterable[Row]) = { - var txnBuilder = createWriteTxnBuilder(Table.forPath(engine, tblPath)) - - if (appId != null) txnBuilder = txnBuilder.withTransactionId(engine, appId, txnVer) - - if (newTbl) { - txnBuilder = txnBuilder. - withSchema(engine, testPartitionSchema). - withPartitionColumns(engine, testPartitionColumns.asJava) - } - val txn = txnBuilder.build(engine) - val combinedActions = inMemoryIterable( - data.map { case (partValues, partData) => - stageData(txn.getTransactionState(engine), partValues, partData) - }.reduceLeft(_ combine _)) - (txn, combinedActions) - } + test("insert into table - idempotent writes") { + withTempDirAndEngine { (tblPath, engine) => - def commitAndVerify( - newTbl: Boolean, - txn: Transaction, - actions: CloseableIterable[Row], - expTblVer: Long, - checksumWritten: Boolean): Unit = { - val commitResult = txn.commit(engine, actions) - if (executePostCommitHook) { - commitResult.getPostCommitHooks.forEach( - hook => hook.threadSafeInvoke(defaultEngine) - ) + // TODO: re-enable when CRC_FULL post commit hook is added, txn2 requires CRC_FULL + assume(this.suiteName != ("DeltaTableWriteWithCrcSuite")) + val data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) + var expData = Seq.empty[TestRow] // as the data in inserted, update this. + + def prepTxnAndActions( + newTbl: Boolean, + appId: String, + txnVer: Long): (Transaction, CloseableIterable[Row]) = { + var txnBuilder = createWriteTxnBuilder(Table.forPath(engine, tblPath)) + + if (appId != null) txnBuilder = txnBuilder.withTransactionId(engine, appId, txnVer) + + if (newTbl) { + txnBuilder = txnBuilder + .withSchema(engine, testPartitionSchema) + .withPartitionColumns(engine, testPartitionColumns.asJava) + } + val txn = txnBuilder.build(engine) + + val combinedActions = inMemoryIterable( + data + .map { + case (partValues, partData) => + stageData(txn.getTransactionState(engine), partValues, partData) } + .reduceLeft(_ combine _) + ) - expData = expData ++ data.flatMap(_._2).flatMap(_.toTestRows) - - verifyCommitResult( - commitResult, - expVersion = expTblVer, - expIsReadyForCheckpoint = false - ) - val expPartCols = if (newTbl) testPartitionColumns else null - verifyCommitInfo(tblPath, version = expTblVer, expPartCols, operation = WRITE) - verifyWrittenContent( - tblPath, - testPartitionSchema, - expData, - testPartitionColumns, - version = Some(expTblVer), - checksumWritten - ) - } + (txn, combinedActions) + } - def addDataWithTxnId( - newTbl: Boolean, - appId: String, - txnVer: Long, - expTblVer: Long): Unit = { - val (txn, combinedActions) = prepTxnAndActions(newTbl, appId, txnVer) - commitAndVerify(newTbl, txn, combinedActions, expTblVer, executePostCommitHook) - } + def commitAndVerify( + newTbl: Boolean, + txn: Transaction, + actions: CloseableIterable[Row], + expTblVer: Long): Unit = { + val commitResult = commitTransaction(txn, engine, actions) - def expFailure(appId: String, txnVer: Long, latestTxnVer: Long)(fn: => Any): Unit = { - val ex = intercept[ConcurrentTransactionException] { - fn - } - assert( - ex.getMessage.contains( - s"This error occurs when multiple updates are using the " + - s"same transaction identifier to write into " + - s"this table.\nApplication ID: $appId, " + - s"Attempted version: $txnVer, Latest version in table: $latestTxnVer" - ) - ) - } + expData = expData ++ data.flatMap(_._2).flatMap(_.toTestRows) - // Create a transaction with id (txnAppId1, 0) and commit it - addDataWithTxnId(newTbl = true, appId = "txnAppId1", txnVer = 0, expTblVer = 0) + verifyCommitResult(commitResult, expVersion = expTblVer, expIsReadyForCheckpoint = false) + val expPartCols = if (newTbl) testPartitionColumns else null + verifyCommitInfo(tblPath, version = expTblVer, expPartCols, operation = WRITE) + verifyWrittenContent(tblPath, testPartitionSchema, expData, testPartitionColumns) + } - // Try to create a transaction with id (txnAppId1, 0) and commit it - should be valid - addDataWithTxnId(newTbl = false, appId = "txnAppId1", txnVer = 1, expTblVer = 1) + def addDataWithTxnId( + newTbl: Boolean, + appId: String, + txnVer: Long, + expTblVer: Long): Unit = { + val (txn, combinedActions) = prepTxnAndActions(newTbl, appId, txnVer) + commitAndVerify(newTbl, txn, combinedActions, expTblVer) + } - // Try to create a transaction with id (txnAppId1, 1) and try to commit it - // Should fail the it is already committed above. - expFailure("txnAppId1", txnVer = 1, latestTxnVer = 1) { - addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 1, expTblVer = 2) - } + def expFailure(appId: String, txnVer: Long, latestTxnVer: Long)(fn: => Any): Unit = { + val ex = intercept[ConcurrentTransactionException] { + fn + } + assert( + ex.getMessage.contains( + s"This error occurs when multiple updates are using the " + + s"same transaction identifier to write into this table.\nApplication ID: $appId, " + + s"Attempted version: $txnVer, Latest version in table: $latestTxnVer" + ) + ) + } - // append with no txn id - addDataWithTxnId(newTbl = false, appId = null, txnVer = 0, expTblVer = 2) + // Create a transaction with id (txnAppId1, 0) and commit it + addDataWithTxnId(newTbl = true, appId = "txnAppId1", txnVer = 0, expTblVer = 0) - // Try to create a transaction with id (txnAppId2, 1) and commit it - // Should be successful as the transaction app id is different - addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 1, expTblVer = 3) + // Try to create a transaction with id (txnAppId1, 0) and commit it - should be valid + addDataWithTxnId(newTbl = false, appId = "txnAppId1", txnVer = 1, expTblVer = 1) - // Try to create a transaction with id (txnAppId2, 0) and commit it - // Should fail as the transaction app id is same - // but the version is less than the committed - expFailure("txnAppId2", txnVer = 0, latestTxnVer = 1) { - addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 0, expTblVer = 4) - } + // Try to create a transaction with id (txnAppId1, 1) and try to commit it + // Should fail the it is already committed above. + expFailure("txnAppId1", txnVer = 1, latestTxnVer = 1) { + addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 1, expTblVer = 2) + } - // Start a transaction (txnAppId2, 2), but don't commit it yet - val (txn, combinedActions) = prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 2) - // Now start a new transaction with the same id (txnAppId2, 2) and commit it - addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 2, expTblVer = 4) - // Now try to commit the previous transaction (txnAppId2, 2) - should fail - expFailure("txnAppId2", txnVer = 2, latestTxnVer = 2) { - commitAndVerify( - newTbl = false, - txn, - combinedActions, - expTblVer = 5, - checksumWritten = false - ) - } + // append with no txn id + addDataWithTxnId(newTbl = false, appId = null, txnVer = 0, expTblVer = 2) - // Start a transaction (txnAppId2, 3), but don't commit it yet - val (txn2, combinedActions2) = - prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 3) - // Now start a new transaction with the different id (txnAppId1, 10) and commit it - addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 10, expTblVer = 5) - // Now try to commit the previous transaction (txnAppId2, 3) - should pass - // checksum file is not written - commitAndVerify( - newTbl = false, - txn2, - combinedActions2, - expTblVer = 6, - checksumWritten = false - ) + // Try to create a transaction with id (txnAppId2, 1) and commit it + // Should be successful as the transaction app id is different + addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 1, expTblVer = 3) + + // Try to create a transaction with id (txnAppId2, 0) and commit it + // Should fail as the transaction app id is same but the version is less than the committed + expFailure("txnAppId2", txnVer = 0, latestTxnVer = 1) { + addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 0, expTblVer = 4) } - } - ) + + // Start a transaction (txnAppId2, 2), but don't commit it yet + val (txn, combinedActions) = prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 2) + // Now start a new transaction with the same id (txnAppId2, 2) and commit it + addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 2, expTblVer = 4) + // Now try to commit the previous transaction (txnAppId2, 2) - should fail + expFailure("txnAppId2", txnVer = 2, latestTxnVer = 2) { + commitAndVerify(newTbl = false, txn, combinedActions, expTblVer = 5) + } + + // Start a transaction (txnAppId2, 3), but don't commit it yet + val (txn2, combinedActions2) = prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 3) + // Now start a new transaction with the different id (txnAppId1, 10) and commit it + addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 10, expTblVer = 5) + // Now try to commit the previous transaction (txnAppId2, 3) - should pass + commitAndVerify(newTbl = false, txn2, combinedActions2, expTblVer = 6) + } + } 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 @@ -1184,6 +965,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] @@ -1231,11 +1014,11 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } case _: TimestampNTZType => None // ignore case s: StructType => - val newType = removeUnsupportedTypes(s); - if (newType.length() > 0) { - Some(newType) - } else { - None + val newType = removeUnsupportedTypes(s); + if (newType.length() > 0) { + Some(newType) + } else { + None } case _ => Some(dataType) } @@ -1251,7 +1034,9 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } def createTestTxn( - engine: Engine, tablePath: String, schema: Option[StructType] = None): Transaction = { + engine: Engine, + tablePath: String, + schema: Option[StructType] = None): Transaction = { val table = Table.forPath(engine, tablePath) var txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) schema.foreach(s => txnBuilder = txnBuilder.withSchema(engine, s)) @@ -1482,12 +1267,13 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa createTxn(engine, tablePath, isNewTable = true, schema, partCols = Seq.empty, tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "id", - TableConfig.ICEBERG_COMPAT_V2_ENABLED.getKey -> "true")) - .commit(engine, emptyIterable()) + TableConfig.ICEBERG_COMPAT_V2_ENABLED.getKey -> "true" + ) + ).commit(engine, emptyIterable()) - val structType = table.getLatestSnapshot(engine).getSchema() - assertColumnMapping(structType.get("a"), 1) - assertColumnMapping(structType.get("b"), 2) + val structType = table.getLatestSnapshot(engine).getSchema() + assertColumnMapping(structType.get("a"), 1) + assertColumnMapping(structType.get("b"), 2) val innerStruct = structType.get("b").getDataType.asInstanceOf[StructType] assertColumnMapping(innerStruct.get("d"), 3) assertColumnMapping(innerStruct.get("e"), 4) @@ -1496,9 +1282,9 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } private def assertColumnMapping( - field: StructField, - expId: Long, - expPhyName: String = "UUID"): Unit = { + field: StructField, + expId: Long, + expPhyName: String = "UUID"): Unit = { val meta = field.getMetadata assert(meta.get(ColumnMapping.COLUMN_MAPPING_ID_KEY) == expId) // For new tables the physical column name is a UUID. For existing tables, we @@ -1509,4 +1295,4 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa assert(meta.get(ColumnMapping.COLUMN_MAPPING_PHYSICAL_NAME_KEY) == expPhyName) } } -} +} \ No newline at end of file diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala index 065f59e60aa..a5b04951d82 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala @@ -22,7 +22,7 @@ import java.util.{Optional, TimeZone, UUID} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import io.delta.golden.GoldenTableUtils -import io.delta.kernel.{Scan, Snapshot, Table} +import io.delta.kernel.{Scan, Snapshot, Table, Transaction, TransactionCommitResult} import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, MapValue, Row} import io.delta.kernel.defaults.engine.DefaultEngine import io.delta.kernel.defaults.internal.data.vector.DefaultGenericVector @@ -33,7 +33,7 @@ import io.delta.kernel.internal.data.ScanStateRow import io.delta.kernel.internal.util.{ColumnMapping, Utils} import io.delta.kernel.internal.util.Utils.singletonCloseableIterator import io.delta.kernel.types._ -import io.delta.kernel.utils.CloseableIterator +import io.delta.kernel.utils.{CloseableIterable, CloseableIterator} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.shaded.org.apache.commons.io.FileUtils @@ -43,6 +43,8 @@ import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.delta.util.FileNames import org.scalatest.Assertions +import scala.language.implicitConversions + trait TestUtils extends Assertions with SQLHelper { lazy val configuration = new Configuration() From bf6613bc9473a8a518fcfb3a5505fc086229e5d1 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 15:39:57 -0800 Subject: [PATCH 40/73] refactor test --- .../defaults/DeltaTableWritesSuite.scala | 514 +++++++++--------- 1 file changed, 249 insertions(+), 265 deletions(-) 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 a07b5ea5951..abf72c1fff7 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 @@ -25,6 +25,7 @@ import io.delta.kernel.engine.Engine import io.delta.kernel.exceptions._ import io.delta.kernel.expressions.Literal import io.delta.kernel.expressions.Literal._ +import io.delta.kernel.hook.PostCommitHook.PostCommitHookType import io.delta.kernel.internal.checkpoints.CheckpointerSuite.selectSingleElement import io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames import io.delta.kernel.internal.{SnapshotImpl, TableConfig} @@ -128,7 +129,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa assert(txn.getSchema(engine) === testSchema) assert(txn.getPartitionColumns(engine) === Seq.empty.asJava) - val txnResult = commitTransaction(txn, engine, emptyIterable()) + val txnResult = commitTransaction(txn, engine, emptyIterable()) assert(txnResult.getVersion === 0) assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) @@ -333,27 +334,27 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val table = Table.forPath(engine, tablePath) val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) - val schema = new StructType() - .add("id", INTEGER) - .add("Part1", INTEGER) // partition column - .add("part2", INTEGER) // partition column + val schema = new StructType() + .add("id", INTEGER) + .add("Part1", INTEGER) // partition column + .add("part2", INTEGER) // partition column - val txn = txnBuilder - .withSchema(engine, schema) - // partition columns should preserve the same case the one in the schema - .withPartitionColumns(engine, Seq("part1", "PART2").asJava) - .build(engine) + val txn = txnBuilder + .withSchema(engine, schema) + // partition columns should preserve the same case the one in the schema + .withPartitionColumns(engine, Seq("part1", "PART2").asJava) + .build(engine) - 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 = commitTransaction(txn, engine, emptyIterable()) + 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 = commitTransaction(txn, engine, emptyIterable()) - assert(txnResult.getVersion === 0) - assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) + assert(txnResult.getVersion === 0) + assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) - verifyCommitInfo(tablePath, version = 0, Seq("Part1", "part2")) - verifyWrittenContent(tablePath, schema, Seq.empty, Seq("Part1", "part2")) + verifyCommitInfo(tablePath, version = 0, Seq("Part1", "part2")) + verifyWrittenContent(tablePath, schema, Seq.empty, Seq("Part1", "part2")) } } @@ -362,16 +363,16 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val parquetAllTypes = goldenTablePath("parquet-all-types") val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) - val table = Table.forPath(engine, tablePath) - val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) - val txn = txnBuilder.withSchema(engine, schema).build(engine) - val txnResult = commitTransaction(txn, engine, emptyIterable()) + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + val txn = txnBuilder.withSchema(engine, schema).build(engine) + val txnResult = commitTransaction(txn, engine, emptyIterable()) - assert(txnResult.getVersion === 0) - assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) + assert(txnResult.getVersion === 0) + assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) - verifyCommitInfo(tablePath, version = 0) - verifyWrittenContent(tablePath, schema, Seq.empty) + verifyCommitInfo(tablePath, version = 0) + verifyWrittenContent(tablePath, schema, Seq.empty) } } @@ -389,13 +390,11 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa data = Seq(Map.empty[String, Literal] -> (dataBatches1 ++ dataBatches2)) ) - val expectedAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap( - _.toTestRows - ) + val expectedAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap(_.toTestRows) - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) - verifyWrittenContent(tblPath, testSchema, expectedAnswer) + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) + verifyWrittenContent(tblPath, testSchema, expectedAnswer) } } @@ -410,21 +409,21 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa data = Seq(Map.empty[String, Literal] -> dataBatches1) ) - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) - verifyWrittenContent(tblPath, testSchema, dataBatches1.flatMap(_.toTestRows)) + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, partitionCols = Seq.empty, operation = WRITE) + verifyWrittenContent(tblPath, testSchema, dataBatches1.flatMap(_.toTestRows)) - val commitResult1 = appendData( - engine, - tblPath, - data = Seq(Map.empty[String, Literal] -> dataBatches2) - ) + val commitResult1 = appendData( + engine, + tblPath, + data = Seq(Map.empty[String, Literal] -> dataBatches2) + ) - val expAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap(_.toTestRows) + val expAnswer = dataBatches1.flatMap(_.toTestRows) ++ dataBatches2.flatMap(_.toTestRows) - verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) - verifyWrittenContent(tblPath, testSchema, expAnswer) + verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) + verifyWrittenContent(tblPath, testSchema, expAnswer) } } @@ -440,17 +439,15 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val stagedFiles = stageData(txnState, Map.empty, dataBatches1) val stagedActionsIterable = inMemoryIterable(stagedFiles) - val commitResult = commitTransaction(txn, 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] { - commitTransaction(txn, engine, stagedActionsIterable) - } - assert( - ex.getMessage - .contains("Transaction is already attempted to commit. Create a new transaction.") - ) + commitTransaction(txn, engine, stagedActionsIterable) + } + assert(ex.getMessage.contains( + "Transaction is already attempted to commit. Create a new transaction.")) } } @@ -468,12 +465,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa ) ) - val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ - dataPartitionBatches2.flatMap(_.toTestRows) + val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ + dataPartitionBatches2.flatMap(_.toTestRows) - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, testPartitionColumns, operation = WRITE) - verifyWrittenContent(tblPath, testPartitionSchema, expData, testPartitionColumns) + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, testPartitionColumns, operation = WRITE) + verifyWrittenContent(tblPath, testPartitionSchema, expData, testPartitionColumns) } } @@ -482,28 +479,28 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val tblPath = tempTblPath + "/table+ with special chars" val partitionCols = Seq("part1", "part2") - { - val commitResult0 = appendData( - engine, - tblPath, - isNewTable = true, - testPartitionSchema, - testPartitionColumns, - data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) - ) - - val expData = dataPartitionBatches1.flatMap(_.toTestRows) - - verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) - verifyCommitInfo(tblPath, version = 0, partitionCols, operation = WRITE) - verifyWrittenContent(tblPath, testPartitionSchema, expData, partitionCols) - } - { - val commitResult1 = appendData( - engine, - tblPath, - data = Seq(Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2) - ) + { + val commitResult0 = appendData( + engine, + tblPath, + isNewTable = true, + testPartitionSchema, + testPartitionColumns, + data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) + ) + + val expData = dataPartitionBatches1.flatMap(_.toTestRows) + + verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) + verifyCommitInfo(tblPath, version = 0, partitionCols, operation = WRITE) + verifyWrittenContent(tblPath, testPartitionSchema, expData, partitionCols) + } + { + val commitResult1 = appendData( + engine, + tblPath, + data = Seq(Map("part1" -> ofInt(4), "part2" -> ofInt(5)) -> dataPartitionBatches2) + ) val expData = dataPartitionBatches1.flatMap(_.toTestRows) ++ dataPartitionBatches2.flatMap(_.toTestRows) @@ -523,39 +520,52 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa .add("Part1", DOUBLE) // partition column .add("parT2", TIMESTAMP) // partition column - val partCols = Seq("part1", "Part2") // given as input to the txn builder - - // expected partition cols in the commit info or elsewhere in the Delta log. - // it is expected to contain the partition columns in the same case as the schema - val expPartCols = Seq("Part1", "parT2") - - val v0Part0Values = Map("PART1" -> ofDouble(1.0), "pART2" -> ofTimestamp(1231212L)) - val v0Part0Data = - generateData(schema, expPartCols, v0Part0Values, batchSize = 200, numBatches = 3) - - val v0Part1Values = Map("Part1" -> ofDouble(7), "PARt2" -> ofTimestamp(123112L)) - val v0Part1Data = - generateData(schema, expPartCols, v0Part1Values, batchSize = 100, numBatches = 7) - - val v1Part0Values = Map("PART1" -> ofNull(DOUBLE), "pART2" -> ofTimestamp(1231212L)) - val v1Part0Data = - generateData(schema, expPartCols, v1Part0Values, batchSize = 200, numBatches = 3) - - val v1Part1Values = Map("Part1" -> ofDouble(7), "PARt2" -> ofNull(TIMESTAMP)) - val v1Part1Data = - generateData(schema, expPartCols, v1Part1Values, batchSize = 100, numBatches = 7) - - val dataPerVersion = Map( - 0 -> Seq(v0Part0Values -> v0Part0Data, v0Part1Values -> v0Part1Data), - 1 -> Seq(v1Part0Values -> v1Part0Data, v1Part1Values -> v1Part1Data) - ) + val partCols = Seq("part1", "Part2") // given as input to the txn builder + + // expected partition cols in the commit info or elsewhere in the Delta log. + // it is expected to contain the partition columns in the same case as the schema + val expPartCols = Seq("Part1", "parT2") + + val v0Part0Values = Map( + "PART1" -> ofDouble(1.0), + "pART2" -> ofTimestamp(1231212L)) + val v0Part0Data = + generateData(schema, expPartCols, v0Part0Values, batchSize = 200, numBatches = 3) + + val v0Part1Values = Map( + "Part1" -> ofDouble(7), + "PARt2" -> ofTimestamp(123112L)) + val v0Part1Data = + generateData(schema, expPartCols, v0Part1Values, batchSize = 100, numBatches = 7) + + val v1Part0Values = Map( + "PART1" -> ofNull(DOUBLE), + "pART2" -> ofTimestamp(1231212L)) + val v1Part0Data = + generateData(schema, expPartCols, v1Part0Values, batchSize = 200, numBatches = 3) + + val v1Part1Values = Map( + "Part1" -> ofDouble(7), + "PARt2" -> ofNull(TIMESTAMP)) + val v1Part1Data = + generateData(schema, expPartCols, v1Part1Values, batchSize = 100, numBatches = 7) + + val dataPerVersion = Map( + 0 -> Seq(v0Part0Values -> v0Part0Data, v0Part1Values -> v0Part1Data), + 1 -> Seq(v1Part0Values -> v1Part0Data, v1Part1Values -> v1Part1Data) + ) - val expV0Data = v0Part0Data.flatMap(_.toTestRows) ++ v0Part1Data.flatMap(_.toTestRows) - val expV1Data = v1Part0Data.flatMap(_.toTestRows) ++ v1Part1Data.flatMap(_.toTestRows) + val expV0Data = v0Part0Data.flatMap(_.toTestRows) ++ v0Part1Data.flatMap(_.toTestRows) + val expV1Data = v1Part0Data.flatMap(_.toTestRows) ++ v1Part1Data.flatMap(_.toTestRows) - for (i <- 0 until 2) { - val commitResult = - appendData(engine, tblPath, isNewTable = i == 0, schema, partCols, dataPerVersion(i)) + for (i <- 0 until 2) { + val commitResult = appendData( + engine, + tblPath, + isNewTable = i == 0, + schema, + partCols, + dataPerVersion(i)) verifyCommitResult(commitResult, expVersion = i, expIsReadyForCheckpoint = false) // partition cols are not written in the commit info for inserts @@ -617,7 +627,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa // delete all commit files before version 30 in both cases and expect the read to pass as // there is a checkpoint at version 30 and should be used for state reconstruction. deleteDeltaFilesBefore(tblPath, beforeVersion = 30) - verifyWrittenContent(tblPath, schema, expData, partCols) + verifyWrittenContent(tblPath, schema, expData, partCols) } } } @@ -627,27 +637,27 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val parquetAllTypes = goldenTablePath("parquet-all-types") val schema = removeUnsupportedTypes(tableSchema(parquetAllTypes)) - val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] - val dataWithPartInfo = Seq(Map.empty[String, Literal] -> data) + val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] + val dataWithPartInfo = Seq(Map.empty[String, Literal] -> data) - appendData(engine, tblPath, isNewTable = true, schema, Seq.empty, dataWithPartInfo) - var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + appendData(engine, tblPath, isNewTable = true, schema, Seq.empty, dataWithPartInfo) + var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - val checkpointInterval = 4 - setCheckpointInterval(tblPath, checkpointInterval) + val checkpointInterval = 4 + setCheckpointInterval(tblPath, checkpointInterval) - for (i <- 2 until 5) { - // insert until a checkpoint is required - val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) + for (i <- 2 until 5) { + // insert until a checkpoint is required + val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - checkpointIfReady(engine, tblPath, commitResult, expSize = i /* one file per version */ ) + checkpointIfReady(engine, tblPath, commitResult, expSize = i /* one file per version */) - verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) - verifyCommitInfo(tblPath, version = i, null, operation = WRITE) - verifyWrittenContent(tblPath, schema, expData) - } - assertCheckpointExists(tblPath, atVersion = checkpointInterval) + verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) + verifyCommitInfo(tblPath, version = i, null, operation = WRITE) + verifyWrittenContent(tblPath, schema, expData) + } + assertCheckpointExists(tblPath, atVersion = checkpointInterval) } } @@ -672,11 +682,11 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val casePreservingPartCols = casePreservingPartitionColNames(schema, partCols.asJava).asScala.to[Seq] - // get the partition values from the data batch at the given rowId - def getPartitionValues(batch: ColumnarBatch, rowId: Int): Map[String, Literal] = { - casePreservingPartCols.map { partCol => - val colIndex = schema.indexOf(partCol) - val vector = batch.getColumnVector(colIndex) + // get the partition values from the data batch at the given rowId + def getPartitionValues(batch: ColumnarBatch, rowId: Int): Map[String, Literal] = { + casePreservingPartCols.map { partCol => + val colIndex = schema.indexOf(partCol) + val vector = batch.getColumnVector(colIndex) val literal = if (vector.isNullAt(rowId)) { Literal.ofNull(vector.getDataType) @@ -701,38 +711,37 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } (partCol, literal) }.toMap - } + } - val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] - - // From the above table read data, convert each row as a new batch with partition info - // Take the values of the partitionCols from the data and create a new batch with the - // selection vector to just select a single row. - var dataWithPartInfo = Seq.empty[(Map[String, Literal], Seq[FilteredColumnarBatch])] - - data.foreach { filteredBatch => - val batch = filteredBatch.getData - Seq.range(0, batch.getSize).foreach { rowId => - val partValues = getPartitionValues(batch, rowId) - val filteredBatch = new FilteredColumnarBatch( - batch, - Optional.of(selectSingleElement(batch.getSize, rowId)) - ) - dataWithPartInfo = dataWithPartInfo :+ (partValues, Seq(filteredBatch)) - } + val data = readTableUsingKernel(engine, parquetAllTypes, schema).to[Seq] + + // From the above table read data, convert each row as a new batch with partition info + // Take the values of the partitionCols from the data and create a new batch with the + // selection vector to just select a single row. + var dataWithPartInfo = Seq.empty[(Map[String, Literal], Seq[FilteredColumnarBatch])] + + data.foreach { filteredBatch => + val batch = filteredBatch.getData + Seq.range(0, batch.getSize).foreach { rowId => + val partValues = getPartitionValues(batch, rowId) + val filteredBatch = new FilteredColumnarBatch( + batch, + Optional.of(selectSingleElement(batch.getSize, rowId))) + dataWithPartInfo = dataWithPartInfo :+ (partValues, Seq(filteredBatch)) } + } - appendData(engine, tblPath, isNewTable = true, schema, partCols, dataWithPartInfo) - verifyCommitInfo(tblPath, version = 0, casePreservingPartCols, operation = WRITE) + appendData(engine, tblPath, isNewTable = true, schema, partCols, dataWithPartInfo) + verifyCommitInfo(tblPath, version = 0, casePreservingPartCols, operation = WRITE) - var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) + var expData = dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) - val checkpointInterval = 2 - setCheckpointInterval(tblPath, checkpointInterval) // version 1 + val checkpointInterval = 2 + setCheckpointInterval(tblPath, checkpointInterval) // version 1 - for (i <- 2 until 4) { - // insert until a checkpoint is required - val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) + for (i <- 2 until 4) { + // insert until a checkpoint is required + val commitResult = appendData(engine, tblPath, data = dataWithPartInfo) expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows) @@ -744,7 +753,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa verifyWrittenContent(tblPath, schema, expData, partCols) } - assertCheckpointExists(tblPath, atVersion = checkpointInterval) + assertCheckpointExists(tblPath, atVersion = checkpointInterval) } } @@ -786,14 +795,10 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa isNewTable = true, testPartitionSchema, testPartitionColumns, - data - ) - } - assert( - ex.getMessage.contains( - "Partition column part2 is of type integer but the value provided is of type string" - ) - ) + data) + } + assert(ex.getMessage.contains( + "Partition column part2 is of type integer but the value provided is of type string")) } } @@ -805,110 +810,92 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa val data = Seq(Map("part1" -> ofInt(1), "part2" -> ofInt(2)) -> dataPartitionBatches1) var expData = Seq.empty[TestRow] // as the data in inserted, update this. - def prepTxnAndActions( - newTbl: Boolean, - appId: String, - txnVer: Long): (Transaction, CloseableIterable[Row]) = { - var txnBuilder = createWriteTxnBuilder(Table.forPath(engine, tblPath)) + def prepTxnAndActions(newTbl: Boolean, appId: String, txnVer: Long) + : (Transaction, CloseableIterable[Row]) = { + var txnBuilder = createWriteTxnBuilder(Table.forPath(engine, tblPath)) if (appId != null) txnBuilder = txnBuilder.withTransactionId(engine, appId, txnVer) if (newTbl) { - txnBuilder = txnBuilder - .withSchema(engine, testPartitionSchema) - .withPartitionColumns(engine, testPartitionColumns.asJava) - } - val txn = txnBuilder.build(engine) - - val combinedActions = inMemoryIterable( - data - .map { - case (partValues, partData) => - stageData(txn.getTransactionState(engine), partValues, partData) - } - .reduceLeft(_ combine _) - ) - - (txn, combinedActions) + txnBuilder = txnBuilder.withSchema(engine, testPartitionSchema) + .withPartitionColumns(engine, testPartitionColumns.asJava) } + val txn = txnBuilder.build(engine) + + val combinedActions = inMemoryIterable( + data.map { case (partValues, partData) => + stageData(txn.getTransactionState(engine), partValues, partData) + }.reduceLeft(_ combine _)) + + (txn, combinedActions) + } - def commitAndVerify( - newTbl: Boolean, - txn: Transaction, - actions: CloseableIterable[Row], - expTblVer: Long): Unit = { - val commitResult = commitTransaction(txn, engine, actions) + def commitAndVerify(newTbl: Boolean, txn: Transaction, + actions: CloseableIterable[Row], expTblVer: Long): Unit = { + val commitResult = commitTransaction(txn, engine, actions) - expData = expData ++ data.flatMap(_._2).flatMap(_.toTestRows) + expData = expData ++ data.flatMap(_._2).flatMap(_.toTestRows) - verifyCommitResult(commitResult, expVersion = expTblVer, expIsReadyForCheckpoint = false) + verifyCommitResult(commitResult, expVersion = expTblVer, expIsReadyForCheckpoint = false) val expPartCols = if (newTbl) testPartitionColumns else null verifyCommitInfo(tblPath, version = expTblVer, expPartCols, operation = WRITE) - verifyWrittenContent(tblPath, testPartitionSchema, expData, testPartitionColumns) - } + verifyWrittenContent(tblPath, testPartitionSchema, expData) + } - def addDataWithTxnId( - newTbl: Boolean, - appId: String, - txnVer: Long, - expTblVer: Long): Unit = { - val (txn, combinedActions) = prepTxnAndActions(newTbl, appId, txnVer) - commitAndVerify(newTbl, txn, combinedActions, expTblVer) - } + def addDataWithTxnId(newTbl: Boolean, appId: String, txnVer: Long, expTblVer: Long): Unit = { + val (txn, combinedActions) = prepTxnAndActions(newTbl, appId, txnVer) + commitAndVerify(newTbl, txn, combinedActions, expTblVer) + } - def expFailure(appId: String, txnVer: Long, latestTxnVer: Long)(fn: => Any): Unit = { - val ex = intercept[ConcurrentTransactionException] { - fn - } - assert( - ex.getMessage.contains( - s"This error occurs when multiple updates are using the " + - s"same transaction identifier to write into this table.\nApplication ID: $appId, " + - s"Attempted version: $txnVer, Latest version in table: $latestTxnVer" - ) - ) + def expFailure(appId: String, txnVer: Long, latestTxnVer: Long)(fn: => Any): Unit = { + val ex = intercept[ConcurrentTransactionException] { + fn } + assert(ex.getMessage.contains(s"This error occurs when multiple updates are using the " + + s"same transaction identifier to write into this table.\nApplication ID: $appId, " + + s"Attempted version: $txnVer, Latest version in table: $latestTxnVer")) + } - // Create a transaction with id (txnAppId1, 0) and commit it - addDataWithTxnId(newTbl = true, appId = "txnAppId1", txnVer = 0, expTblVer = 0) + // Create a transaction with id (txnAppId1, 0) and commit it + addDataWithTxnId(newTbl = true, appId = "txnAppId1", txnVer = 0, expTblVer = 0) - // Try to create a transaction with id (txnAppId1, 0) and commit it - should be valid - addDataWithTxnId(newTbl = false, appId = "txnAppId1", txnVer = 1, expTblVer = 1) + // Try to create a transaction with id (txnAppId1, 0) and commit it - should be valid + addDataWithTxnId(newTbl = false, appId = "txnAppId1", txnVer = 1, expTblVer = 1) - // Try to create a transaction with id (txnAppId1, 1) and try to commit it - // Should fail the it is already committed above. - expFailure("txnAppId1", txnVer = 1, latestTxnVer = 1) { - addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 1, expTblVer = 2) - } + // Try to create a transaction with id (txnAppId1, 1) and try to commit it + // Should fail the it is already committed above. + expFailure("txnAppId1", txnVer = 1, latestTxnVer = 1) { + addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 1, expTblVer = 2) + } - // append with no txn id - addDataWithTxnId(newTbl = false, appId = null, txnVer = 0, expTblVer = 2) + // append with no txn id + addDataWithTxnId(newTbl = false, appId = null, txnVer = 0, expTblVer = 2) - // Try to create a transaction with id (txnAppId2, 1) and commit it - // Should be successful as the transaction app id is different - addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 1, expTblVer = 3) + // Try to create a transaction with id (txnAppId2, 1) and commit it + // Should be successful as the transaction app id is different + addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 1, expTblVer = 3) - // Try to create a transaction with id (txnAppId2, 0) and commit it - // Should fail as the transaction app id is same but the version is less than the committed - expFailure("txnAppId2", txnVer = 0, latestTxnVer = 1) { - addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 0, expTblVer = 4) - } + // Try to create a transaction with id (txnAppId2, 0) and commit it + // Should fail as the transaction app id is same but the version is less than the committed + expFailure("txnAppId2", txnVer = 0, latestTxnVer = 1) { + addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 0, expTblVer = 4) + } - // Start a transaction (txnAppId2, 2), but don't commit it yet - val (txn, combinedActions) = prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 2) - // Now start a new transaction with the same id (txnAppId2, 2) and commit it - addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 2, expTblVer = 4) - // Now try to commit the previous transaction (txnAppId2, 2) - should fail - expFailure("txnAppId2", txnVer = 2, latestTxnVer = 2) { - commitAndVerify(newTbl = false, txn, combinedActions, expTblVer = 5) - } + // Start a transaction (txnAppId2, 2), but don't commit it yet + val (txn, combinedActions) = prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 2) + // Now start a new transaction with the same id (txnAppId2, 2) and commit it + addDataWithTxnId(newTbl = false, "txnAppId2", txnVer = 2, expTblVer = 4) + // Now try to commit the previous transaction (txnAppId2, 2) - should fail + expFailure("txnAppId2", txnVer = 2, latestTxnVer = 2) { + commitAndVerify(newTbl = false, txn, combinedActions, expTblVer = 5) + } - // Start a transaction (txnAppId2, 3), but don't commit it yet - val (txn2, combinedActions2) = prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 3) - // Now start a new transaction with the different id (txnAppId1, 10) and commit it - addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 10, expTblVer = 5) - // Now try to commit the previous transaction (txnAppId2, 3) - should pass - commitAndVerify(newTbl = false, txn2, combinedActions2, expTblVer = 6) + // Start a transaction (txnAppId2, 3), but don't commit it yet + val (txn2, combinedActions2) = prepTxnAndActions(newTbl = false, "txnAppId2", txnVer = 3) + // Now start a new transaction with the different id (txnAppId1, 10) and commit it + addDataWithTxnId(newTbl = false, "txnAppId1", txnVer = 10, expTblVer = 5) + // Now try to commit the previous transaction (txnAppId2, 3) - should pass + commitAndVerify(newTbl = false, txn2, combinedActions2, expTblVer = 6) } } @@ -1014,11 +1001,11 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } case _: TimestampNTZType => None // ignore case s: StructType => - val newType = removeUnsupportedTypes(s); - if (newType.length() > 0) { - Some(newType) - } else { - None + val newType = removeUnsupportedTypes(s); + if (newType.length() > 0) { + Some(newType) + } else { + None } case _ => Some(dataType) } @@ -1034,9 +1021,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } def createTestTxn( - engine: Engine, - tablePath: String, - schema: Option[StructType] = None): Transaction = { + engine: Engine, tablePath: String, schema: Option[StructType] = None): Transaction = { val table = Table.forPath(engine, tablePath) var txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) schema.foreach(s => txnBuilder = txnBuilder.withSchema(engine, s)) @@ -1267,13 +1252,12 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa createTxn(engine, tablePath, isNewTable = true, schema, partCols = Seq.empty, tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "id", - TableConfig.ICEBERG_COMPAT_V2_ENABLED.getKey -> "true" - ) - ).commit(engine, emptyIterable()) + TableConfig.ICEBERG_COMPAT_V2_ENABLED.getKey -> "true")) + .commit(engine, emptyIterable()) - val structType = table.getLatestSnapshot(engine).getSchema() - assertColumnMapping(structType.get("a"), 1) - assertColumnMapping(structType.get("b"), 2) + val structType = table.getLatestSnapshot(engine).getSchema() + assertColumnMapping(structType.get("a"), 1) + assertColumnMapping(structType.get("b"), 2) val innerStruct = structType.get("b").getDataType.asInstanceOf[StructType] assertColumnMapping(innerStruct.get("d"), 3) assertColumnMapping(innerStruct.get("e"), 4) @@ -1282,9 +1266,9 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } private def assertColumnMapping( - field: StructField, - expId: Long, - expPhyName: String = "UUID"): Unit = { + field: StructField, + expId: Long, + expPhyName: String = "UUID"): Unit = { val meta = field.getMetadata assert(meta.get(ColumnMapping.COLUMN_MAPPING_ID_KEY) == expId) // For new tables the physical column name is a UUID. For existing tables, we From daa03fd37789b27ce891d23b3f307884ca579201 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 16:54:12 -0800 Subject: [PATCH 41/73] add checks --- .../kernel/internal/checksum/CRCInfo.java | 5 +- .../internal/checksum/ChecksumWriter.java | 1 - .../ChecksumSimpleComparisonSuite.scala | 97 ++++++++----------- 3 files changed, 47 insertions(+), 56 deletions(-) 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 index 1b971506a5f..22365c33c00 100644 --- 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 @@ -15,6 +15,7 @@ */ 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.ColumnarBatch; @@ -87,12 +88,14 @@ public CRCInfo( 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 = txnId; + this.txnId = requireNonNull(txnId); } /** The version of the Delta table that this CRCInfo represents. */ 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 index 1c84a0edb28..11dcca8d176 100644 --- 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 @@ -40,7 +40,6 @@ public ChecksumWriter(Path logPath) { /** Writes a checksum file */ public void writeCheckSum(Engine engine, CRCInfo crcInfo) throws IOException { - checkArgument(crcInfo.getNumFiles() >= 0 && crcInfo.getTableSizeBytes() >= 0); Path newChecksumPath = FileNames.checksumFile(logPath, crcInfo.getVersion()); logger.info("Writing checksum file to path: {}", newChecksumPath); wrapEngineExceptionThrowsIO( 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 index 7c0f77fb659..a90105c5100 100644 --- 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 @@ -100,21 +100,17 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt } /** - * Insert into partition spark table, copy the commit log to kernel table - * and verify the checksum files for are consistent between spark and kernel + * Insert into unpartitioned spark table, read the added file from the commit log, + * commit them to kernel table and verify the checksum files are consistent + * between spark and kernel * */ - private def insertIntoPartitionedTableAndCheckCrc( + private def insertIntoUnpartitionedTableAndCheckCrc( engine: Engine, sparkTablePath: String, kernelTablePath: String, versionAtCommit: Long): Unit = { - var valueToAppend = "(0, 0)" - var addedPartition = Set(0) - (0L to versionAtCommit).foreach(i => { - val partitionValue = 2 * i - addedPartition = addedPartition + partitionValue.toInt - valueToAppend = valueToAppend + s",($i, $partitionValue)" - }) + var valueToAppend = "(0)" + (0L to versionAtCommit).foreach(i => valueToAppend = valueToAppend + s",($i)") spark.sql( s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend" ) @@ -124,27 +120,27 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt .createTransactionBuilder(engine, "test-engine", Operation.WRITE) .build(engine) - convertSparkDeltaLogToKernelCommit( - txn, - engine, - sparkTablePath, - versionAtCommit, - Some(addedPartition) - ) + convertSparkDeltaLogToKernelCommit(txn, engine, sparkTablePath, versionAtCommit) assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } /** - * Insert into unpartitioned spark table, copy the commit log to kernel table - * and verify the checksum files for are consistent between spark and kernel + * Insert into partitioned spark table, read the added file from the commit log, + * commit them to kernel table and verify the checksum files are consistent + * between spark and kernel * */ - private def insertIntoUnpartitionedTableAndCheckCrc( + private def insertIntoPartitionedTableAndCheckCrc( engine: Engine, sparkTablePath: String, kernelTablePath: String, versionAtCommit: Long): Unit = { - var valueToAppend = "(0)" - (0L to versionAtCommit).foreach(i => valueToAppend = valueToAppend + s",($i)") + var valueToAppend = "(0, 0)" + var addedPartition = Set(0) + (0L to versionAtCommit).foreach(i => { + val partitionValue = 2 * i + addedPartition = addedPartition + partitionValue.toInt + valueToAppend = valueToAppend + s",($i, $partitionValue)" + }) spark.sql( s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend" ) @@ -154,7 +150,13 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt .createTransactionBuilder(engine, "test-engine", Operation.WRITE) .build(engine) - convertSparkDeltaLogToKernelCommit(txn, engine, sparkTablePath, versionAtCommit) + convertSparkDeltaLogToKernelCommit( + txn, + engine, + sparkTablePath, + versionAtCommit, + Some(addedPartition) + ) assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } @@ -272,49 +274,36 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt ) while (columnarBatches.hasNext) { - processColumnarBatch(columnarBatches.next(), partition, addFiles) + collectAddFilesFromLogRows(columnarBatches.next(), partition, addFiles) } - toCloseableIterator(addFiles.iterator()) } - private def processColumnarBatch( - batch: ColumnarBatch, + private def collectAddFilesFromLogRows( + logFileRows: ColumnarBatch, partition: Option[String], addFiles: util.ArrayList[DataFileStatus]): Unit = { - val rows = batch.getRows + val rows = logFileRows.getRows while (rows.hasNext) { val row = rows.next() val addIndex = row.getSchema.indexOf("add") if (!row.isNullAt(addIndex)) { - processAddFile(row.getStruct(addIndex), partition, addFiles) + val addFile = new AddFile(row.getStruct(addIndex)) + if (partition.isEmpty || + partition.get == VectorUtils + .toJavaMap(addFile.getPartitionValues) + .get(PARTITION_COLUMN)) { + addFiles.add( + new DataFileStatus( + addFile.getPath, + addFile.getSize, + addFile.getModificationTime, + Optional.empty() // TODO: populate stats once #4139 is fixed + ) + ) + } } } } - - private def processAddFile( - addFileRow: Row, - partition: Option[String], - addFiles: util.ArrayList[DataFileStatus]): Unit = { - - val addFile = new AddFile(addFileRow) - - if (shouldIncludeFile(addFile, partition)) { - addFiles.add( - new DataFileStatus( - addFile.getPath, - addFile.getSize, - addFile.getModificationTime, - Optional.empty() // TODO: populate stats once #4139 is fixed - ) - ) - } - } - - private def shouldIncludeFile(addFile: AddFile, partition: Option[String]): Boolean = { - partition.isEmpty || - partition.get == VectorUtils.toJavaMap(addFile.getPartitionValues).get(PARTITION_COLUMN) - } - } From 0ac1005f4d4ef61aff4c6f196181fed35dc14f27 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 17:26:24 -0800 Subject: [PATCH 42/73] resolve review --- .../kernel/internal/TransactionImpl.java | 5 ++- .../kernel/internal/checksum/CRCInfo.java | 5 ++- .../checksum/ChecksumWriterSuite.scala | 44 ++++++++++--------- .../kernel/defaults/utils/TestUtils.scala | 6 +-- 4 files changed, 32 insertions(+), 28 deletions(-) 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 68f6978219b..00cbaf19e50 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 @@ -365,7 +365,8 @@ private TransactionCommitResult doCommit( postCommitHooks.add(new CheckpointHook(dataPath, commitAsVersion)); } - buildPostCommitCrcInfo(commitAsVersion, transactionMetrics.captureTransactionMetricsResult()) + buildPostCommitCrcInfoIfCurrentCrcAvailable( + commitAsVersion, transactionMetrics.captureTransactionMetricsResult()) .ifPresent(crcInfo -> postCommitHooks.add(new ChecksumSimpleHook(crcInfo, logPath))); return new TransactionCommitResult(commitAsVersion, postCommitHooks); @@ -443,7 +444,7 @@ private void recordTransactionReport( engine.getMetricsReporters().forEach(reporter -> reporter.report(transactionReport)); } - private Optional buildPostCommitCrcInfo( + private Optional buildPostCommitCrcInfoIfCurrentCrcAvailable( long commitAtVersion, TransactionMetricsResult metricsResult) { // Create table if (isNewTable) { 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 index 1b971506a5f..2523da797ca 100644 --- 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 @@ -63,8 +63,9 @@ public static Optional fromColumnarBatch( batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TABLE_SIZE_BYTES)).getLong(rowId); long numFiles = batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(NUM_FILES)).getLong(rowId); Optional txnId = - Optional.ofNullable( - batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TXN_ID)).getString(rowId)); + batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TXN_ID)).isNullAt(rowId) + ? Optional.empty() + : Optional.of(batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TXN_ID)).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); 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 index d571e1d1256..5f84067c94a 100644 --- 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 @@ -38,13 +38,13 @@ 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 = CRC_FILE_SCHEMA.indexOf("tableSizeBytes") - private val NUM_FILES = CRC_FILE_SCHEMA.indexOf("numFiles") - private val NUM_METADATA = CRC_FILE_SCHEMA.indexOf("numMetadata") - private val NUM_PROTOCOL = CRC_FILE_SCHEMA.indexOf("numProtocol") - private val TXN_ID = CRC_FILE_SCHEMA.indexOf("txnId") - private val METADATA = CRC_FILE_SCHEMA.indexOf("metadata") - private val PROTOCOL = CRC_FILE_SCHEMA.indexOf("protocol") + 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() @@ -63,8 +63,9 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { ) verifyChecksumFile(jsonHandler, version) - verifyChecksumContent(jsonHandler.capturedCrcRow, tableSizeBytes, numFiles, txn) - verifyMetadataAndProtocol(jsonHandler.capturedCrcRow, metadata, protocol) + assert(jsonHandler.capturedCrcRow.isDefined) + verifyChecksumContent(jsonHandler.capturedCrcRow.get, tableSizeBytes, numFiles, txn) + verifyMetadataAndProtocol(jsonHandler.capturedCrcRow.get, metadata, protocol) } // Test with and without transaction ID @@ -74,7 +75,8 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { private def verifyChecksumFile(jsonHandler: MockCheckSumFileJsonWriter, version: Long): Unit = { assert(jsonHandler.checksumFilePath == s"$FAKE_DELTA_LOG_PATH/${"%020d".format(version)}.crc") - assert(jsonHandler.capturedCrcRow.getSchema == CRC_FILE_SCHEMA) + assert(jsonHandler.capturedCrcRow.isDefined) + assert(jsonHandler.capturedCrcRow.get.getSchema == CRC_FILE_SCHEMA) } private def verifyChecksumContent( @@ -82,15 +84,15 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { expectedTableSizeBytes: Long, expectedNumFiles: Long, expectedTxnId: Optional[String]): Unit = { - assert(actualCheckSumRow.getLong(TABLE_SIZE_BYTES) == expectedTableSizeBytes) - assert(actualCheckSumRow.getLong(NUM_FILES) == expectedNumFiles) - assert(actualCheckSumRow.getLong(NUM_METADATA) == 1L) - assert(actualCheckSumRow.getLong(NUM_PROTOCOL) == 1L) + assert(actualCheckSumRow.getLong(TABLE_SIZE_BYTES_IDX) == expectedTableSizeBytes) + assert(actualCheckSumRow.getLong(NUM_FILES_IDX) == expectedNumFiles) + assert(actualCheckSumRow.getLong(NUM_METADATA_IDX) == 1L) + assert(actualCheckSumRow.getLong(NUM_PROTOCOL_IDX) == 1L) if (expectedTxnId.isPresent) { - assert(actualCheckSumRow.getString(TXN_ID) == expectedTxnId.get()) + assert(actualCheckSumRow.getString(TXN_ID_IDX) == expectedTxnId.get()) } else { - assert(actualCheckSumRow.isNullAt(TXN_ID)) + assert(actualCheckSumRow.isNullAt(TXN_ID_IDX)) } } @@ -98,10 +100,11 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { actualRow: Row, expectedMetadata: Metadata, expectedProtocol: Protocol): Unit = { - checkMetadata(expectedMetadata, actualRow.getStruct(METADATA)) - checkProtocol(expectedProtocol, actualRow.getStruct(PROTOCOL)) + 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) @@ -141,6 +144,7 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { ) } + // TODO: implement compare in Protocol and remove this method private def checkProtocol(expectedProtocol: Protocol, actualProtocolRow: Row): Unit = { assert(actualProtocolRow.getSchema == Protocol.FULL_SCHEMA) assert( @@ -183,7 +187,7 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { * Mock implementation of JsonHandler for testing checksum file writing. */ class MockCheckSumFileJsonWriter extends BaseMockJsonHandler { - var capturedCrcRow: Row = new GenericRow(new StructType(), new util.HashMap[Integer, AnyRef]) + var capturedCrcRow: Option[Row] = None var checksumFilePath: String = "" override def writeJsonFileAtomically( @@ -192,7 +196,7 @@ class MockCheckSumFileJsonWriter extends BaseMockJsonHandler { overwrite: Boolean): Unit = { checksumFilePath = filePath assert(data.hasNext, "Expected data iterator to contain exactly one row") - capturedCrcRow = data.next() + 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/utils/TestUtils.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala index a5b04951d82..065f59e60aa 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala @@ -22,7 +22,7 @@ import java.util.{Optional, TimeZone, UUID} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import io.delta.golden.GoldenTableUtils -import io.delta.kernel.{Scan, Snapshot, Table, Transaction, TransactionCommitResult} +import io.delta.kernel.{Scan, Snapshot, Table} import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, MapValue, Row} import io.delta.kernel.defaults.engine.DefaultEngine import io.delta.kernel.defaults.internal.data.vector.DefaultGenericVector @@ -33,7 +33,7 @@ import io.delta.kernel.internal.data.ScanStateRow import io.delta.kernel.internal.util.{ColumnMapping, Utils} import io.delta.kernel.internal.util.Utils.singletonCloseableIterator import io.delta.kernel.types._ -import io.delta.kernel.utils.{CloseableIterable, CloseableIterator} +import io.delta.kernel.utils.CloseableIterator import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.shaded.org.apache.commons.io.FileUtils @@ -43,8 +43,6 @@ import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.delta.util.FileNames import org.scalatest.Assertions -import scala.language.implicitConversions - trait TestUtils extends Assertions with SQLHelper { lazy val configuration = new Configuration() From 622d28d10d2cb649d664b9e802e22feec7599b06 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 17:35:58 -0800 Subject: [PATCH 43/73] revert unnecessary code --- .../defaults/DeltaTableWriteSuiteBase.scala | 28 ++++--------------- 1 file changed, 5 insertions(+), 23 deletions(-) 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 e25b8f3fb4a..6f78d94f144 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 @@ -22,22 +22,16 @@ import io.delta.kernel.defaults.utils.{TestRow, TestUtils} import io.delta.kernel.engine.Engine import io.delta.kernel.internal.actions.{Metadata, Protocol, SingleAction} import io.delta.kernel.internal.fs.{Path => DeltaPath} -import io.delta.kernel.internal.util.{Clock, FileNames, VectorUtils} +import io.delta.kernel.internal.util.FileNames import io.delta.kernel.internal.util.Utils.singletonCloseableIterator -import io.delta.kernel.internal.{SnapshotImpl, TableConfig, TableImpl, TransactionImpl} +import io.delta.kernel.internal.{SnapshotImpl, TableConfig, TableImpl} import io.delta.kernel.utils.{CloseableIterable, CloseableIterator, FileStatus} -import io.delta.kernel.{ - Meta, - Operation, - Table, - Transaction, - TransactionBuilder, - TransactionCommitResult -} +import io.delta.kernel.{Meta, Operation, Table, Transaction, TransactionBuilder, TransactionCommitResult} import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, Row} import io.delta.kernel.defaults.internal.data.DefaultColumnarBatch import io.delta.kernel.expressions.Literal import io.delta.kernel.expressions.Literal.ofInt +import io.delta.kernel.internal.util.Clock import io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.types.IntegerType.INTEGER @@ -45,17 +39,15 @@ import io.delta.kernel.types.StructType import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} import io.delta.kernel.Operation.CREATE_TABLE import io.delta.kernel.hook.PostCommitHook.PostCommitHookType -import io.delta.kernel.internal.checksum.CRCInfo import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.delta.VersionNotFoundException import org.scalatest.funsuite.AnyFunSuite -import org.scalatest.matchers.should.Matchers.convertToAnyShouldWrapper import java.io.File import java.nio.file.{Files, Paths} -import java.util.{Locale, Optional} +import java.util.Optional import scala.collection.JavaConverters._ import scala.collection.immutable.{ListMap, Seq} @@ -461,16 +453,6 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { ) } - def assertChecksumSimpleReadiness(txnResult: TransactionCommitResult): Unit = { - assert( - txnResult.getPostCommitHooks - .stream() - .anyMatch( - hook => hook.getType == PostCommitHookType.CHECKSUM_SIMPLE - ) - ) - } - protected def commitTransaction( txn: Transaction, engine: Engine, From 13bb0a79d60615e311593329c160dc7ec42e82ff Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 17:40:15 -0800 Subject: [PATCH 44/73] revert unnecessary code 2/n --- .../kernel/defaults/DeltaTableWritesSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 abf72c1fff7..1958939bf3e 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 @@ -831,7 +831,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } def commitAndVerify(newTbl: Boolean, txn: Transaction, - actions: CloseableIterable[Row], expTblVer: Long): Unit = { + actions: CloseableIterable[Row], expTblVer: Long): Unit = { val commitResult = commitTransaction(txn, engine, actions) expData = expData ++ data.flatMap(_._2).flatMap(_.toTestRows) @@ -1021,7 +1021,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } def createTestTxn( - engine: Engine, tablePath: String, schema: Option[StructType] = None): Transaction = { + engine: Engine, tablePath: String, schema: Option[StructType] = None): Transaction = { val table = Table.forPath(engine, tablePath) var txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) schema.foreach(s => txnBuilder = txnBuilder.withSchema(engine, s)) @@ -1266,9 +1266,9 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } private def assertColumnMapping( - field: StructField, - expId: Long, - expPhyName: String = "UUID"): Unit = { + field: StructField, + expId: Long, + expPhyName: String = "UUID"): Unit = { val meta = field.getMetadata assert(meta.get(ColumnMapping.COLUMN_MAPPING_ID_KEY) == expId) // For new tables the physical column name is a UUID. For existing tables, we @@ -1279,4 +1279,4 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa assert(meta.get(ColumnMapping.COLUMN_MAPPING_PHYSICAL_NAME_KEY) == expPhyName) } } -} \ No newline at end of file +} From 972109afe62ae273c90e8298a82c4bc63cf209bb Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 17:41:04 -0800 Subject: [PATCH 45/73] revert unnecessary code 3/n --- .../scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 1958939bf3e..1dc7a86b0de 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 @@ -751,7 +751,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) verifyCommitInfo(tblPath, version = i, partitionCols = null, operation = WRITE) verifyWrittenContent(tblPath, schema, expData, partCols) - } + } assertCheckpointExists(tblPath, atVersion = checkpointInterval) } From c52c58edbe6abc60ded77f66acee24e70eae7bef Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 17:41:52 -0800 Subject: [PATCH 46/73] revert unnecessary code 4/n --- .../io/delta/kernel/defaults/DeltaTableWritesSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 1dc7a86b0de..23cf3ee7de1 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 @@ -1266,9 +1266,9 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa } private def assertColumnMapping( - field: StructField, - expId: Long, - expPhyName: String = "UUID"): Unit = { + field: StructField, + expId: Long, + expPhyName: String = "UUID"): Unit = { val meta = field.getMetadata assert(meta.get(ColumnMapping.COLUMN_MAPPING_ID_KEY) == expId) // For new tables the physical column name is a UUID. For existing tables, we From bba00bd022b1338a4c2b142e37627f60e3eb1e7c Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 17:45:37 -0800 Subject: [PATCH 47/73] add null check --- .../main/java/io/delta/kernel/internal/checksum/CRCInfo.java | 2 ++ 1 file changed, 2 insertions(+) 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 index d7364a24edb..c4e390484b9 100644 --- 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 @@ -60,8 +60,10 @@ public static Optional fromColumnarBatch( Protocol.fromColumnVector(batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(PROTOCOL)), rowId); Metadata metadata = Metadata.fromColumnVector(batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(METADATA)), rowId); + checkArgument(!batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TABLE_SIZE_BYTES)).isNullAt(rowId)); long tableSizeBytes = batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TABLE_SIZE_BYTES)).getLong(rowId); + checkArgument(!batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(NUM_FILES)).isNullAt(rowId)); long numFiles = batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(NUM_FILES)).getLong(rowId); Optional txnId = batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TXN_ID)).isNullAt(rowId) From 93b9aeb040d848ba0f7eb66492b4e0587a1e26ce Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 17:52:01 -0800 Subject: [PATCH 48/73] fixing doc --- .../delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index a90105c5100..b0ce3067346 100644 --- 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 @@ -125,7 +125,7 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt } /** - * Insert into partitioned spark table, read the added file from the commit log, + * Insert into partitioned spark table, read the added files from the commit log, * commit them to kernel table and verify the checksum files are consistent * between spark and kernel * */ From a11badd50752576dfa5c847189e30ff5c6159b7f Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 17:53:00 -0800 Subject: [PATCH 49/73] fixing doc --- .../kernel/defaults/ChecksumSimpleComparisonSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 index b0ce3067346..ad3a8c2d182 100644 --- 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 @@ -125,9 +125,9 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt } /** - * Insert into partitioned spark table, read the added files from the commit log, - * commit them to kernel table and verify the checksum files are consistent - * between spark and kernel + * Insert into partitioned spark table, load the added files from the spark table's commit log, + * commit them to kernel table and verify the checksum files are consistent between spark + * and kernel * */ private def insertIntoPartitionedTableAndCheckCrc( engine: Engine, From cf336248a97dda83948d3e0bc5d860ced6d4f11d Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 17:54:07 -0800 Subject: [PATCH 50/73] fixing doc --- .../kernel/defaults/ChecksumSimpleComparisonSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 index ad3a8c2d182..ff344dfc83f 100644 --- 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 @@ -100,9 +100,9 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt } /** - * Insert into unpartitioned spark table, read the added file from the commit log, - * commit them to kernel table and verify the checksum files are consistent - * between spark and kernel + * Insert into unpartitioned spark table, load the added files from the spark table's commit log, + * commit them to kernel table and verify the checksum files are consistent between spark + * and kernel. * */ private def insertIntoUnpartitionedTableAndCheckCrc( engine: Engine, @@ -127,7 +127,7 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt /** * Insert into partitioned spark table, load the added files from the spark table's commit log, * commit them to kernel table and verify the checksum files are consistent between spark - * and kernel + * and kernel. * */ private def insertIntoPartitionedTableAndCheckCrc( engine: Engine, From a6189d7e4e5b6d2f40fa4b19e4873cfb76577b5d Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 17:58:45 -0800 Subject: [PATCH 51/73] fixing header --- .../defaults/DeltaTableWriteWithCrcSuite.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) 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 index cfe13e80214..d423b970238 100644 --- 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 @@ -1,3 +1,18 @@ +/* + * 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 io.delta.kernel.data.Row import io.delta.kernel.{Transaction, TransactionCommitResult} From 26cd8b8ab49ed5e41fdbf921c9d3b7621e136f31 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Tue, 18 Feb 2025 18:06:22 -0800 Subject: [PATCH 52/73] fixing java --- .../main/java/io/delta/kernel/internal/checksum/CRCInfo.java | 5 +++-- .../io/delta/kernel/internal/checksum/ChecksumWriter.java | 1 - 2 files changed, 3 insertions(+), 3 deletions(-) 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 index c4e390484b9..b6258bad4a4 100644 --- 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 @@ -60,7 +60,8 @@ public static Optional fromColumnarBatch( Protocol.fromColumnVector(batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(PROTOCOL)), rowId); Metadata metadata = Metadata.fromColumnVector(batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(METADATA)), rowId); - checkArgument(!batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TABLE_SIZE_BYTES)).isNullAt(rowId)); + checkArgument( + !batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TABLE_SIZE_BYTES)).isNullAt(rowId)); long tableSizeBytes = batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TABLE_SIZE_BYTES)).getLong(rowId); checkArgument(!batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(NUM_FILES)).isNullAt(rowId)); @@ -92,7 +93,7 @@ public CRCInfo( long numFiles, Optional txnId) { checkArgument(tableSizeBytes >= 0); - checkArgument(numFiles >=0 ); + checkArgument(numFiles >= 0); this.version = version; this.metadata = requireNonNull(metadata); this.protocol = requireNonNull(protocol); 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 index 11dcca8d176..caec865694a 100644 --- 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 @@ -16,7 +16,6 @@ package io.delta.kernel.internal.checksum; import static io.delta.kernel.internal.DeltaErrors.wrapEngineExceptionThrowsIO; -import static io.delta.kernel.internal.util.Preconditions.checkArgument; import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator; import static java.util.Objects.requireNonNull; From c0178ec1e56628a27ee678ec1fbcc2f3f5a275a1 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 12:22:20 -0800 Subject: [PATCH 53/73] group method --- .../kernel/internal/TransactionImpl.java | 2 +- .../kernel/internal/checksum/CRCInfo.java | 24 ++++++++++--------- 2 files changed, 14 insertions(+), 12 deletions(-) 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 00cbaf19e50..470e9d2e656 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 @@ -345,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( @@ -446,7 +447,6 @@ private void recordTransactionReport( private Optional buildPostCommitCrcInfoIfCurrentCrcAvailable( long commitAtVersion, TransactionMetricsResult metricsResult) { - // Create table if (isNewTable) { return Optional.of( new CRCInfo( 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 index b6258bad4a4..d96cb3cf860 100644 --- 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 @@ -18,6 +18,7 @@ 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; @@ -56,20 +57,21 @@ public class CRCInfo { public static Optional fromColumnarBatch( long version, ColumnarBatch batch, int rowId, String crcFilePath) { + checkArgument(!batch.getColumnVector(getSchemaIndex(TABLE_SIZE_BYTES)).isNullAt(rowId)); + checkArgument(!batch.getColumnVector(getSchemaIndex(NUM_FILES)).isNullAt(rowId)); + // Read required fields. Protocol protocol = - Protocol.fromColumnVector(batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(PROTOCOL)), rowId); + Protocol.fromColumnVector(batch.getColumnVector(getSchemaIndex(PROTOCOL)), rowId); Metadata metadata = - Metadata.fromColumnVector(batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(METADATA)), rowId); - checkArgument( - !batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TABLE_SIZE_BYTES)).isNullAt(rowId)); - long tableSizeBytes = - batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TABLE_SIZE_BYTES)).getLong(rowId); - checkArgument(!batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(NUM_FILES)).isNullAt(rowId)); - long numFiles = batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(NUM_FILES)).getLong(rowId); + Metadata.fromColumnVector(batch.getColumnVector(getSchemaIndex(METADATA)), rowId); + long tableSizeBytes = batch.getColumnVector(getSchemaIndex(TABLE_SIZE_BYTES)).getLong(rowId); + long numFiles = batch.getColumnVector(getSchemaIndex(NUM_FILES)).getLong(rowId); + // Read optional field + ColumnVector txnIdColumnVector = batch.getColumnVector(getSchemaIndex(TXN_ID)); Optional txnId = - batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TXN_ID)).isNullAt(rowId) + txnIdColumnVector.isNullAt(rowId) ? Optional.empty() - : Optional.of(batch.getColumnVector(CRC_FILE_SCHEMA.indexOf(TXN_ID)).getString(rowId)); + : 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); @@ -149,7 +151,7 @@ public Row toRow() { return new GenericRow(CRC_FILE_SCHEMA, values); } - private int getSchemaIndex(String fieldName) { + private static int getSchemaIndex(String fieldName) { return CRC_FILE_SCHEMA.indexOf(fieldName); } } From 658f7703e8940abf2c814045ce91ecef16f12c4b Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 12:27:53 -0800 Subject: [PATCH 54/73] revert --- .../io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) 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 24b6a6e94eb..b7d479adeda 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 @@ -330,13 +330,9 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { partCols: Seq[String] = null, data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])], clock: Clock = () => System.currentTimeMillis, - tableProperties: Map[String, String] = null, - executePostCommitHook: Boolean = false): TransactionCommitResult = { + tableProperties: Map[String, String] = null): TransactionCommitResult = { val txn = createTxn(engine, tablePath, isNewTable, schema, partCols, tableProperties, clock) val commitResult = commitAppendData(engine, txn, data) - if (executePostCommitHook) { - commitResult.getPostCommitHooks.forEach(hook => hook.threadSafeInvoke(engine)) - } commitResult } From 6c70ba5e5ee9e423edc0cf4a709de5679715678e Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 12:31:24 -0800 Subject: [PATCH 55/73] revert --- .../io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 b7d479adeda..12b4de3c82d 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 @@ -332,8 +332,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { clock: Clock = () => System.currentTimeMillis, tableProperties: Map[String, String] = null): TransactionCommitResult = { val txn = createTxn(engine, tablePath, isNewTable, schema, partCols, tableProperties, clock) - val commitResult = commitAppendData(engine, txn, data) - commitResult + commitAppendData(engine, txn, data) } def assertMetadataProp( From abc2bd933f4843a9d216cdbf0dc4ce6f07859f5d Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 12:31:51 -0800 Subject: [PATCH 56/73] revert --- .../io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 12b4de3c82d..4864547021a 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 @@ -411,8 +411,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { partitionCols: Seq[String] = Seq.empty, isBlindAppend: Boolean = true, operation: Operation = CREATE_TABLE): Unit = { - val row = spark - .sql(s"DESCRIBE HISTORY delta.`$tablePath`") + val row = spark.sql(s"DESCRIBE HISTORY delta.`$tablePath`") .filter(s"version = $version") .select( "version", From 305997ea8641547a3fc22fcaafd0136ce4a10f98 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 12:33:03 -0800 Subject: [PATCH 57/73] revert --- .../io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala | 1 + 1 file changed, 1 insertion(+) 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 4864547021a..1a5a4d9e848 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 @@ -331,6 +331,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])], clock: Clock = () => System.currentTimeMillis, tableProperties: Map[String, String] = null): TransactionCommitResult = { + val txn = createTxn(engine, tablePath, isNewTable, schema, partCols, tableProperties, clock) commitAppendData(engine, txn, data) } From e7710cbd1849ac01543be797c5c1f926f0e23cf8 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 12:34:25 -0800 Subject: [PATCH 58/73] typo --- .../main/java/io/delta/kernel/internal/checksum/CRCInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index d96cb3cf860..bf4a546e0db 100644 --- 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 @@ -66,7 +66,7 @@ public static Optional fromColumnarBatch( Metadata.fromColumnVector(batch.getColumnVector(getSchemaIndex(METADATA)), rowId); long tableSizeBytes = batch.getColumnVector(getSchemaIndex(TABLE_SIZE_BYTES)).getLong(rowId); long numFiles = batch.getColumnVector(getSchemaIndex(NUM_FILES)).getLong(rowId); - // Read optional field + // Read optional fields ColumnVector txnIdColumnVector = batch.getColumnVector(getSchemaIndex(TXN_ID)); Optional txnId = txnIdColumnVector.isNullAt(rowId) From 0bec8989199606f2959d612434776c6325b0c52e Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 12:35:28 -0800 Subject: [PATCH 59/73] grouping --- .../java/io/delta/kernel/internal/checksum/CRCInfo.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 index bf4a546e0db..83805795211 100644 --- 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 @@ -57,8 +57,6 @@ public class CRCInfo { public static Optional fromColumnarBatch( long version, ColumnarBatch batch, int rowId, String crcFilePath) { - checkArgument(!batch.getColumnVector(getSchemaIndex(TABLE_SIZE_BYTES)).isNullAt(rowId)); - checkArgument(!batch.getColumnVector(getSchemaIndex(NUM_FILES)).isNullAt(rowId)); // Read required fields. Protocol protocol = Protocol.fromColumnVector(batch.getColumnVector(getSchemaIndex(PROTOCOL)), rowId); @@ -66,12 +64,16 @@ public static Optional fromColumnarBatch( Metadata.fromColumnVector(batch.getColumnVector(getSchemaIndex(METADATA)), rowId); long tableSizeBytes = batch.getColumnVector(getSchemaIndex(TABLE_SIZE_BYTES)).getLong(rowId); long numFiles = batch.getColumnVector(getSchemaIndex(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)); + + checkArgument(!batch.getColumnVector(getSchemaIndex(TABLE_SIZE_BYTES)).isNullAt(rowId)); + checkArgument(!batch.getColumnVector(getSchemaIndex(NUM_FILES)).isNullAt(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); From d392927656c3a8feb43459f26a0612cba565c0ec Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 12:38:26 -0800 Subject: [PATCH 60/73] todo --- .../src/main/java/io/delta/kernel/internal/TransactionImpl.java | 1 + 1 file changed, 1 insertion(+) 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 470e9d2e656..7e088e6d246 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 @@ -467,6 +467,7 @@ private Optional buildPostCommitCrcInfoIfCurrentCrcAvailable( 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()))); From f60b9c61bf3591ff8b7ed923be9b07eb1430c409 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 14:11:11 -0800 Subject: [PATCH 61/73] fix doc --- .../src/main/java/io/delta/kernel/internal/TransactionImpl.java | 2 ++ 1 file changed, 2 insertions(+) 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 7e088e6d246..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 @@ -460,6 +460,8 @@ private Optional buildPostCommitCrcInfoIfCurrentCrcAvailable( 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 -> From c4ae5e5d5ea4111914603e7424e4908db5391974 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 14:23:25 -0800 Subject: [PATCH 62/73] fix check --- .../io/delta/kernel/internal/checksum/CRCInfo.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) 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 index 83805795211..b4d72dc89ad 100644 --- 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 @@ -24,6 +24,7 @@ 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; @@ -62,8 +63,14 @@ public static Optional fromColumnarBatch( Protocol.fromColumnVector(batch.getColumnVector(getSchemaIndex(PROTOCOL)), rowId); Metadata metadata = Metadata.fromColumnVector(batch.getColumnVector(getSchemaIndex(METADATA)), rowId); - long tableSizeBytes = batch.getColumnVector(getSchemaIndex(TABLE_SIZE_BYTES)).getLong(rowId); - long numFiles = batch.getColumnVector(getSchemaIndex(NUM_FILES)).getLong(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)); @@ -72,8 +79,6 @@ public static Optional fromColumnarBatch( ? Optional.empty() : Optional.of(txnIdColumnVector.getString(rowId)); - checkArgument(!batch.getColumnVector(getSchemaIndex(TABLE_SIZE_BYTES)).isNullAt(rowId)); - checkArgument(!batch.getColumnVector(getSchemaIndex(NUM_FILES)).isNullAt(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); From a6be52bb4f7f4c5bfa8464c6c8ebb8bfa889fb09 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 16:34:40 -0800 Subject: [PATCH 63/73] fix test --- .../checksum/ChecksumWriterSuite.scala | 13 ++++++---- .../DeltaTableWriteWithCrcSuite.scala | 24 ++++++------------- 2 files changed, 15 insertions(+), 22 deletions(-) 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 index 5862b993b3d..22fa0da61f1 100644 --- 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 @@ -63,7 +63,6 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { new CRCInfo(version, metadata, protocol, tableSizeBytes, numFiles, txn)) verifyChecksumFile(jsonHandler, version) - assert(jsonHandler.capturedCrcRow.isDefined) verifyChecksumContent(jsonHandler.capturedCrcRow.get, tableSizeBytes, numFiles, txn) verifyMetadataAndProtocol(jsonHandler.capturedCrcRow.get, metadata, protocol) } @@ -84,10 +83,14 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { expectedTableSizeBytes: Long, expectedNumFiles: Long, expectedTxnId: Optional[String]): Unit = { - assert(actualCheckSumRow.getLong(TABLE_SIZE_BYTES_IDX) == expectedTableSizeBytes) - assert(actualCheckSumRow.getLong(NUM_FILES_IDX) == expectedNumFiles) - assert(actualCheckSumRow.getLong(NUM_METADATA_IDX) == 1L) - assert(actualCheckSumRow.getLong(NUM_PROTOCOL_IDX) == 1L) + 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()) 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 index 605fc386a03..9e909d87904 100644 --- 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 @@ -17,17 +17,15 @@ package io.delta.kernel.defaults import java.io.File import java.nio.file.Files import java.util.{Locale, Optional} - import scala.collection.immutable.Seq import scala.jdk.CollectionConverters.{asScalaBufferConverter, asScalaSetConverter} 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.TransactionImpl +import io.delta.kernel.internal.{SnapshotImpl, TransactionImpl} import io.delta.kernel.internal.actions.Metadata import io.delta.kernel.internal.checksum.CRCInfo import io.delta.kernel.internal.util.Utils.singletonCloseableIterator @@ -36,24 +34,16 @@ import io.delta.kernel.utils.{CloseableIterable, FileStatus} class DeltaTableWriteWithCrcSuite extends DeltaTableWritesSuite { - implicit class TransactionOps(txn: Transaction) { - def commitAndGenerateCrc( - 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 commitTransaction( txn: Transaction, engine: Engine, dataActions: CloseableIterable[Row]): TransactionCommitResult = { - txn.commitAndGenerateCrc(engine, dataActions) + 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( From eb1bca11d035fd144369ce73dc6f75a80ed44a6c Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 16:49:47 -0800 Subject: [PATCH 64/73] fix test --- .../defaults/DeltaTableWriteSuiteBase.scala | 11 +-- .../DeltaTableWriteWithCrcSuite.scala | 76 ++++--------------- .../defaults/DeltaTableWritesSuite.scala | 15 ++-- 3 files changed, 25 insertions(+), 77 deletions(-) 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 1a5a4d9e848..82a5d503d22 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 @@ -383,9 +383,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { protected def verifyWrittenContent( path: String, expSchema: StructType, - expData: Seq[TestRow], - expPartitionColumns: Seq[String] = Seq(), - version: Option[Long] = Option.empty): Unit = { + expData: Seq[TestRow]): Unit = { val actSchema = tableSchema(path) assert(actSchema === expSchema) @@ -396,12 +394,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { // Spark reads the timestamp partition columns in local timezone vs. Kernel reads in UTC. We // need to set the timezone to UTC before reading the data using Spark to make the tests pass withSparkTimeZone("UTC") { - val resultSpark = spark - .sql(s"SELECT * FROM delta.`$path`" + { - if (version.isDefined) s" VERSION AS OF ${version.get}" else "" - }) - .collect() - .map(TestRow(_)) + val resultSpark = spark.sql(s"SELECT * FROM delta.`$path`").collect().map(TestRow(_)) checkAnswer(resultSpark, expData) } } 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 index 9e909d87904..a4061359ddf 100644 --- 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 @@ -17,9 +17,11 @@ package io.delta.kernel.defaults import java.io.File import java.nio.file.Files import java.util.{Locale, Optional} + import scala.collection.immutable.Seq import scala.jdk.CollectionConverters.{asScalaBufferConverter, asScalaSetConverter} import scala.language.implicitConversions + import io.delta.kernel.{Transaction, TransactionCommitResult} import io.delta.kernel.data.Row import io.delta.kernel.defaults.utils.TestRow @@ -27,7 +29,8 @@ import io.delta.kernel.engine.Engine import io.delta.kernel.hook.PostCommitHook.PostCommitHookType import io.delta.kernel.internal.{SnapshotImpl, TransactionImpl} import io.delta.kernel.internal.actions.Metadata -import io.delta.kernel.internal.checksum.CRCInfo +import io.delta.kernel.internal.checksum.{ChecksumReader, CRCInfo} +import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.util.Utils.singletonCloseableIterator import io.delta.kernel.types.StructType import io.delta.kernel.utils.{CloseableIterable, FileStatus} @@ -49,67 +52,20 @@ class DeltaTableWriteWithCrcSuite extends DeltaTableWritesSuite { override def verifyWrittenContent( path: String, expSchema: StructType, - expData: Seq[TestRow], - expPartitionColumns: Seq[String] = Seq(), - version: Option[Long] = Option.empty): Unit = { - val actSchema = tableSchema(path) - assert(actSchema === expSchema) - - // verify data using Kernel reader - checkTable(path, expData) - - // verify data using Spark reader. - // Spark reads the timestamp partition columns in local timezone vs. Kernel reads in UTC. We - // need to set the timezone to UTC before reading the data using Spark to make the tests pass - withSparkTimeZone("UTC") { - val resultSpark = spark - .sql(s"SELECT * FROM delta.`$path`" + { - if (version.isDefined) s" VERSION AS OF ${version.get}" else "" - }) - .collect() - .map(TestRow(_)) - checkAnswer(resultSpark, expData) - } - - checkChecksumContent(path, version, expSchema, expPartitionColumns) + expData: Seq[TestRow]): Unit = { + super.verifyWrittenContent(path, expSchema, expData) + checkChecksum(path, expSchema) } - def checkChecksumContent( + def checkChecksum( tablePath: String, - version: Option[Long], - expSchema: StructType, - expPartitionColumns: Seq[String]): Unit = { - val checksumVersion = version.getOrElse(latestSnapshot(tablePath, defaultEngine).getVersion) - val checksumFile = new File(f"$tablePath/_delta_log/$checksumVersion%020d.crc") - - assert(Files.exists(checksumFile.toPath), s"Checksum file not found: ${checksumFile.getPath}") - - val columnarBatches = defaultEngine - .getJsonHandler() - .readJsonFiles( - singletonCloseableIterator(FileStatus.of(checksumFile.getPath)), - CRCInfo.CRC_FILE_SCHEMA, - Optional.empty()) - - assert(columnarBatches.hasNext, "Empty checksum file") - val crcRow = columnarBatches.next() - assert(crcRow.getSize === 1, s"Expected single row, found ${crcRow.getSize}") - - val metadata = Metadata.fromColumnVector( - crcRow.getColumnVector(CRCInfo.CRC_FILE_SCHEMA.indexOf("metadata")), - /* rowId= */ 0) - - assert( - metadata.getSchema === expSchema, - s"Schema mismatch.\nExpected: $expSchema\nActual: ${metadata.getSchema}") - - val normalizedPartitions = expPartitionColumns.map(_.toLowerCase(Locale.ROOT)).toSet - assert( - metadata.getPartitionColNames.asScala === normalizedPartitions, - s"Partition columns mismatch.\n" + - s"Expected: $normalizedPartitions\n" + - s"Actual: ${metadata.getPartitionColNames.asScala}") - - assert(!columnarBatches.hasNext, "Unexpected additional data in checksum file") + expSchema: StructType): 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 8009786c692..1552f5b0e1e 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 @@ -352,7 +352,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false) verifyCommitInfo(tablePath, version = 0, Seq("Part1", "part2")) - verifyWrittenContent(tablePath, schema, Seq.empty, Seq("Part1", "part2")) + verifyWrittenContent(tablePath, schema, Seq.empty) } } @@ -463,7 +463,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) verifyCommitInfo(tblPath, version = 0, testPartitionColumns, operation = WRITE) - verifyWrittenContent(tblPath, testPartitionSchema, expData, testPartitionColumns) + verifyWrittenContent(tblPath, testPartitionSchema, expData) } } @@ -485,7 +485,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa verifyCommitResult(commitResult0, expVersion = 0, expIsReadyForCheckpoint = false) verifyCommitInfo(tblPath, version = 0, partitionCols, operation = WRITE) - verifyWrittenContent(tblPath, testPartitionSchema, expData, partitionCols) + verifyWrittenContent(tblPath, testPartitionSchema, expData) } { val commitResult1 = appendData( @@ -498,7 +498,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa verifyCommitResult(commitResult1, expVersion = 1, expIsReadyForCheckpoint = false) verifyCommitInfo(tblPath, version = 1, partitionCols = null, operation = WRITE) - verifyWrittenContent(tblPath, testPartitionSchema, expData, partitionCols) + verifyWrittenContent(tblPath, testPartitionSchema, expData) } } } @@ -564,8 +564,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa verifyWrittenContent( tblPath, schema, - if (i == 0) expV0Data else expV0Data ++ expV1Data, - partCols) + if (i == 0) expV0Data else expV0Data ++ expV1Data) } } } @@ -620,7 +619,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa // delete all commit files before version 30 in both cases and expect the read to pass as // there is a checkpoint at version 30 and should be used for state reconstruction. deleteDeltaFilesBefore(tblPath, beforeVersion = 30) - verifyWrittenContent(tblPath, schema, expData, partCols) + verifyWrittenContent(tblPath, schema, expData) } } } @@ -742,7 +741,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0) verifyCommitInfo(tblPath, version = i, partitionCols = null, operation = WRITE) - verifyWrittenContent(tblPath, schema, expData, partCols) + verifyWrittenContent(tblPath, schema, expData) } assertCheckpointExists(tblPath, atVersion = checkpointInterval) From 941d9c844d7a4ff709e7cab9168681efb78a006c Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 17:41:38 -0800 Subject: [PATCH 65/73] fix test --- .../ChecksumSimpleComparisonSuite.scala | 129 ++++-------------- 1 file changed, 26 insertions(+), 103 deletions(-) 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 index dd5cc07e6b2..29f0110d3aa 100644 --- 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 @@ -18,24 +18,24 @@ package io.delta.kernel.defaults import java.io.File import java.nio.file.Files import java.util -import java.util.Collections.{emptyMap, singletonMap} +import java.util.Collections.emptyMap import java.util.Optional import scala.collection.immutable.Seq +import scala.jdk.CollectionConverters.setAsJavaSetConverter import io.delta.kernel.{Operation, Table, Transaction} -import io.delta.kernel.data.{ColumnarBatch, Row} +import io.delta.kernel.data.Row import io.delta.kernel.defaults.utils.TestUtils import io.delta.kernel.engine.Engine -import io.delta.kernel.expressions.Literal +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, VectorUtils} -import io.delta.kernel.internal.util.Utils.{singletonCloseableIterator, toCloseableIterator} +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.{CloseableIterator, DataFileStatus, FileStatus} import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} /** @@ -113,7 +113,7 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt .createTransactionBuilder(engine, "test-engine", Operation.WRITE) .build(engine) - convertSparkDeltaLogToKernelCommit(txn, engine, sparkTablePath, versionAtCommit) + commitSparkChangeToKernel(txn, engine, sparkTablePath, versionAtCommit) assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } @@ -142,12 +142,11 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt .createTransactionBuilder(engine, "test-engine", Operation.WRITE) .build(engine) - convertSparkDeltaLogToKernelCommit( + commitSparkChangeToKernel( txn, engine, sparkTablePath, - versionAtCommit, - Some(addedPartition)) + versionAtCommit) assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) } @@ -188,107 +187,31 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt assert(crc1.getMetadata.getPartitionColNames === crc2.getMetadata.getPartitionColNames) } - private def convertSparkDeltaLogToKernelCommit( + private def commitSparkChangeToKernel( txn: Transaction, engine: Engine, sparkTablePath: String, - versionToConvert: Long, - addedPartition: Option[(Set[Int])] = None): Unit = { + versionToConvert: Long): Unit = { - val txnState = txn.getTransactionState(engine) - - val dataActionsIterator = addedPartition match { - case None => - // Unpartitioned table case - val writeContext = Transaction.getWriteContext(engine, txnState, emptyMap()) - Transaction.generateAppendActions( - engine, - txnState, - convertSparkTableDeltaLogToKernelAppendActions( - engine, - sparkTablePath, - versionToConvert, - None), - writeContext) - - case Some(partitions) => - // Partitioned table case - val actions = new util.ArrayList[Row]() - partitions.foreach { partition => - val writeContext = Transaction.getWriteContext( - engine, - txnState, - singletonMap(PARTITION_COLUMN, Literal.ofInt(partition))) - - Transaction - .generateAppendActions( - engine, - txnState, - convertSparkTableDeltaLogToKernelAppendActions( - engine, - sparkTablePath, - versionToConvert, - Some(partition.toString)), - writeContext) - .forEach(action => actions.add(action)) + val tableChange = Table.forPath(engine, sparkTablePath).asInstanceOf[TableImpl].getChanges( + engine, + versionToConvert, + versionToConvert, + 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)) } - actions.iterator() - } + })) txn - .commit(engine, inMemoryIterable(toCloseableIterator(dataActionsIterator))) + .commit(engine, inMemoryIterable(toCloseableIterator(addFilesRows.iterator()))) .getPostCommitHooks .forEach(_.threadSafeInvoke(engine)) } - - private def convertSparkTableDeltaLogToKernelAppendActions( - engine: Engine, - sparkTablePath: String, - version: Long, - partition: Option[String]): CloseableIterator[DataFileStatus] = { - - val logPath = new Path(sparkTablePath, "_delta_log") - val deltaFile = FileStatus.of(FileNames.deltaFile(logPath, version), 0, 0) - - val addFiles = new util.ArrayList[DataFileStatus]() - - val columnarBatches = engine.getJsonHandler.readJsonFiles( - singletonCloseableIterator(deltaFile), - SingleAction.FULL_SCHEMA, - Optional.empty()) - - while (columnarBatches.hasNext) { - collectAddFilesFromLogRows(columnarBatches.next(), partition, addFiles) - } - toCloseableIterator(addFiles.iterator()) - } - - private def collectAddFilesFromLogRows( - logFileRows: ColumnarBatch, - partition: Option[String], - addFiles: util.ArrayList[DataFileStatus]): Unit = { - val rows = logFileRows.getRows - while (rows.hasNext) { - val row = rows.next() - val addIndex = row.getSchema.indexOf("add") - - if (!row.isNullAt(addIndex)) { - val addFile = new AddFile(row.getStruct(addIndex)) - if ( - partition.isEmpty || - partition.get == VectorUtils - .toJavaMap(addFile.getPartitionValues) - .get(PARTITION_COLUMN) - ) { - addFiles.add( - new DataFileStatus( - addFile.getPath, - addFile.getSize, - addFile.getModificationTime, - Optional.empty() // TODO: populate stats once #4139 is fixed - )) - } - } - } - } } From a14d42ecba8b7eb2b5a115e09850044a6a8cd9ce Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 17:56:07 -0800 Subject: [PATCH 66/73] utils method --- .../delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 index 29f0110d3aa..13883066483 100644 --- 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 @@ -37,6 +37,7 @@ 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.{emptyIterable, inMemoryIterable} +import io.delta.kernel.internal.util.FileNames; /** * Test suite to verify checksum file correctness by comparing @@ -175,7 +176,7 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt } private def buildCrcPath(basePath: String, version: Long): java.nio.file.Path = { - new File(f"$basePath/_delta_log/$version%020d.crc").toPath + new File(FileNames.checksumFile(new Path(basePath), version).toString).toPath } // TODO: Add equals/hashCode to metadata, protocol then CRCInfo. From c4462a77f48a06c5b87a3518d1469d569884d120 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 19:01:41 -0800 Subject: [PATCH 67/73] fix scala --- .../delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 index 13883066483..2c1df6f3f36 100644 --- 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 @@ -33,11 +33,11 @@ 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.IntegerType.INTEGER import io.delta.kernel.types.StructType -import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} -import io.delta.kernel.internal.util.FileNames; +import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable}; /** * Test suite to verify checksum file correctness by comparing From 744beaf670061d44ca2e91387d8186bb455183f2 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 19:40:24 -0800 Subject: [PATCH 68/73] fix test --- .../delta/kernel/defaults/ChecksumSimpleComparisonSuite.scala | 2 +- spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) 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 index 2c1df6f3f36..2102f94cf33 100644 --- 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 @@ -176,7 +176,7 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt } private def buildCrcPath(basePath: String, version: Long): java.nio.file.Path = { - new File(FileNames.checksumFile(new Path(basePath), version).toString).toPath + new File(FileNames.checksumFile(new Path(f"$basePath/_delta_log"), version).toString).toPath } // TODO: Add equals/hashCode to metadata, protocol then CRCInfo. diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala b/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala index 8c46b4403b8..38c83cde43e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala @@ -94,6 +94,7 @@ trait RecordChecksum extends DeltaLogging { private def getChecksum(snapshot: Snapshot): VersionChecksum = snapshot.computeChecksum protected def writeChecksumFile(txnId: String, snapshot: Snapshot): Unit = { + // scalastyle:off println if (!spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_WRITE_CHECKSUM_ENABLED)) { return } From 562cfe4c745faba6ed0d42bcf1a67cae18b2d8e9 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Wed, 19 Feb 2025 19:40:58 -0800 Subject: [PATCH 69/73] revert --- spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala b/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala index 38c83cde43e..8c46b4403b8 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala @@ -94,7 +94,6 @@ trait RecordChecksum extends DeltaLogging { private def getChecksum(snapshot: Snapshot): VersionChecksum = snapshot.computeChecksum protected def writeChecksumFile(txnId: String, snapshot: Snapshot): Unit = { - // scalastyle:off println if (!spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_WRITE_CHECKSUM_ENABLED)) { return } From b63911039407012b066868110f4c9a3675c8cbd3 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Thu, 20 Feb 2025 11:53:46 -0800 Subject: [PATCH 70/73] add docs --- .../defaults/DeltaTableWriteSuiteBase.scala | 4 ++++ .../DeltaTableWriteWithCrcSuite.scala | 24 ++++++++----------- .../defaults/DeltaTableWritesSuite.scala | 1 + 3 files changed, 15 insertions(+), 14 deletions(-) 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 82a5d503d22..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 @@ -463,6 +463,10 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils { .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, 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 index a4061359ddf..53750cbdbbb 100644 --- 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 @@ -14,12 +14,7 @@ * limitations under the License. */ package io.delta.kernel.defaults -import java.io.File -import java.nio.file.Files -import java.util.{Locale, Optional} - import scala.collection.immutable.Seq -import scala.jdk.CollectionConverters.{asScalaBufferConverter, asScalaSetConverter} import scala.language.implicitConversions import io.delta.kernel.{Transaction, TransactionCommitResult} @@ -27,14 +22,15 @@ 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.{SnapshotImpl, TransactionImpl} -import io.delta.kernel.internal.actions.Metadata -import io.delta.kernel.internal.checksum.{ChecksumReader, CRCInfo} +import io.delta.kernel.internal.checksum.ChecksumReader import io.delta.kernel.internal.fs.Path -import io.delta.kernel.internal.util.Utils.singletonCloseableIterator import io.delta.kernel.types.StructType -import io.delta.kernel.utils.{CloseableIterable, FileStatus} +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( @@ -54,12 +50,12 @@ class DeltaTableWriteWithCrcSuite extends DeltaTableWritesSuite { expSchema: StructType, expData: Seq[TestRow]): Unit = { super.verifyWrittenContent(path, expSchema, expData) - checkChecksum(path, expSchema) + verifyChecksumValid(path) } - def checkChecksum( - tablePath: String, - expSchema: StructType): Unit = { + /** Ensure checksum is readable by CRC reader. */ + def verifyChecksumValid( + tablePath: String): Unit = { val checksumVersion = latestSnapshot(tablePath, defaultEngine).getVersion val crcInfo = ChecksumReader.getCRCInfo( defaultEngine, 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 1552f5b0e1e..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 { /////////////////////////////////////////////////////////////////////////// From 5a34d7ee136d53d3cd12a940ce3bffb7502edc17 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Thu, 20 Feb 2025 13:01:52 -0800 Subject: [PATCH 71/73] patch https://github.com/delta-io/delta/commit/aae9d649fd6a9ca4b87c742beb048afffcfd571b --- ...d649fd6a9ca4b87c742beb048afffcfd571b.patch | 176 ++++++++++++++++++ .../ChecksumSimpleComparisonSuite.scala | 99 +++------- 2 files changed, 203 insertions(+), 72 deletions(-) create mode 100644 aae9d649fd6a9ca4b87c742beb048afffcfd571b.patch diff --git a/aae9d649fd6a9ca4b87c742beb048afffcfd571b.patch b/aae9d649fd6a9ca4b87c742beb048afffcfd571b.patch new file mode 100644 index 00000000000..64cde006a9a --- /dev/null +++ b/aae9d649fd6a9ca4b87c742beb048afffcfd571b.patch @@ -0,0 +1,176 @@ +From aae9d649fd6a9ca4b87c742beb048afffcfd571b Mon Sep 17 00:00:00 2001 +From: Allison Portis +Date: Thu, 20 Feb 2025 11:32:18 -0800 +Subject: [PATCH] Simplify + +--- + .../ChecksumSimpleComparisonSuite.scala | 99 +++++-------------- + 1 file changed, 27 insertions(+), 72 deletions(-) + +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 +index 2102f94cf3..754a1572b0 100644 +--- 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 +@@ -18,13 +18,11 @@ package io.delta.kernel.defaults + import java.io.File + import java.nio.file.Files + import java.util +-import java.util.Collections.emptyMap +-import java.util.Optional + + import scala.collection.immutable.Seq + import scala.jdk.CollectionConverters.setAsJavaSetConverter + +-import io.delta.kernel.{Operation, Table, Transaction} ++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 +@@ -35,9 +33,11 @@ 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.IntegerType.INTEGER ++import io.delta.kernel.types.LongType.LONG + import io.delta.kernel.types.StructType +-import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable}; ++import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} ++ ++import org.apache.spark.sql.functions.col + + /** + * Test suite to verify checksum file correctness by comparing +@@ -58,16 +58,18 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt + engine, + kernelTablePath, + isNewTable = true, +- schema = new StructType().add("id", INTEGER), ++ 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 Integer) USING DELTA") ++ spark.sql(s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` (id LONG) USING DELTA") + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + +- (1 to 10).foreach(version => +- insertIntoUnpartitionedTableAndCheckCrc(engine, sparkTablePath, kernelTablePath, version)) +- ++ (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) ++ } + } + } + +@@ -80,77 +82,24 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt + engine, + kernelTablePath, + isNewTable = true, +- schema = new StructType().add("id", INTEGER).add(PARTITION_COLUMN, INTEGER), ++ 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 Integer, part Integer) USING DELTA PARTITIONED BY (part)") ++ s"(id LONG, part LONG) USING DELTA PARTITIONED BY (part)") + assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) + +- (1 to 10).foreach(version => +- insertIntoPartitionedTableAndCheckCrc(engine, sparkTablePath, kernelTablePath, version)) ++ (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) ++ } + } + } + +- /** +- * Insert into unpartitioned spark table, load the added files from the spark table's commit log, +- * commit them to kernel table and verify the checksum files are consistent between spark +- * and kernel. +- */ +- private def insertIntoUnpartitionedTableAndCheckCrc( +- engine: Engine, +- sparkTablePath: String, +- kernelTablePath: String, +- versionAtCommit: Long): Unit = { +- var valueToAppend = "(0)" +- (0L to versionAtCommit).foreach(i => valueToAppend = valueToAppend + s",($i)") +- spark.sql( +- s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend") +- +- val txn = Table +- .forPath(engine, kernelTablePath) +- .createTransactionBuilder(engine, "test-engine", Operation.WRITE) +- .build(engine) +- +- commitSparkChangeToKernel(txn, engine, sparkTablePath, versionAtCommit) +- assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) +- } +- +- /** +- * Insert into partitioned spark table, load the added files from the spark table's commit log, +- * commit them to kernel table and verify the checksum files are consistent between spark +- * and kernel. +- */ +- private def insertIntoPartitionedTableAndCheckCrc( +- engine: Engine, +- sparkTablePath: String, +- kernelTablePath: String, +- versionAtCommit: Long): Unit = { +- var valueToAppend = "(0, 0)" +- var addedPartition = Set(0) +- (0L to versionAtCommit).foreach(i => { +- val partitionValue = 2 * i +- addedPartition = addedPartition + partitionValue.toInt +- valueToAppend = valueToAppend + s",($i, $partitionValue)" +- }) +- spark.sql( +- s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend") +- +- val txn = Table +- .forPath(engine, kernelTablePath) +- .createTransactionBuilder(engine, "test-engine", Operation.WRITE) +- .build(engine) +- +- commitSparkChangeToKernel( +- txn, +- engine, +- sparkTablePath, +- versionAtCommit) +- assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) +- } +- + private def assertChecksumEquals( + engine: Engine, + sparkTablePath: String, +@@ -188,16 +137,22 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt + assert(crc1.getMetadata.getPartitionColNames === crc2.getMetadata.getPartitionColNames) + } + ++ // TODO docs + private def commitSparkChangeToKernel( +- txn: Transaction, ++ 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]() 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 index 2102f94cf33..754a1572b00 100644 --- 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 @@ -18,13 +18,11 @@ package io.delta.kernel.defaults import java.io.File import java.nio.file.Files import java.util -import java.util.Collections.emptyMap -import java.util.Optional import scala.collection.immutable.Seq import scala.jdk.CollectionConverters.setAsJavaSetConverter -import io.delta.kernel.{Operation, Table, Transaction} +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 @@ -35,9 +33,11 @@ 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.IntegerType.INTEGER +import io.delta.kernel.types.LongType.LONG import io.delta.kernel.types.StructType -import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable}; +import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} + +import org.apache.spark.sql.functions.col /** * Test suite to verify checksum file correctness by comparing @@ -58,16 +58,18 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt engine, kernelTablePath, isNewTable = true, - schema = new StructType().add("id", INTEGER), + 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 Integer) USING DELTA") + spark.sql(s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` (id LONG) USING DELTA") assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) - (1 to 10).foreach(version => - insertIntoUnpartitionedTableAndCheckCrc(engine, sparkTablePath, kernelTablePath, version)) - + (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) + } } } @@ -80,77 +82,24 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt engine, kernelTablePath, isNewTable = true, - schema = new StructType().add("id", INTEGER).add(PARTITION_COLUMN, INTEGER), + 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 Integer, part Integer) USING DELTA PARTITIONED BY (part)") + s"(id LONG, part LONG) USING DELTA PARTITIONED BY (part)") assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) - (1 to 10).foreach(version => - insertIntoPartitionedTableAndCheckCrc(engine, sparkTablePath, kernelTablePath, version)) + (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) + } } } - /** - * Insert into unpartitioned spark table, load the added files from the spark table's commit log, - * commit them to kernel table and verify the checksum files are consistent between spark - * and kernel. - */ - private def insertIntoUnpartitionedTableAndCheckCrc( - engine: Engine, - sparkTablePath: String, - kernelTablePath: String, - versionAtCommit: Long): Unit = { - var valueToAppend = "(0)" - (0L to versionAtCommit).foreach(i => valueToAppend = valueToAppend + s",($i)") - spark.sql( - s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend") - - val txn = Table - .forPath(engine, kernelTablePath) - .createTransactionBuilder(engine, "test-engine", Operation.WRITE) - .build(engine) - - commitSparkChangeToKernel(txn, engine, sparkTablePath, versionAtCommit) - assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) - } - - /** - * Insert into partitioned spark table, load the added files from the spark table's commit log, - * commit them to kernel table and verify the checksum files are consistent between spark - * and kernel. - */ - private def insertIntoPartitionedTableAndCheckCrc( - engine: Engine, - sparkTablePath: String, - kernelTablePath: String, - versionAtCommit: Long): Unit = { - var valueToAppend = "(0, 0)" - var addedPartition = Set(0) - (0L to versionAtCommit).foreach(i => { - val partitionValue = 2 * i - addedPartition = addedPartition + partitionValue.toInt - valueToAppend = valueToAppend + s",($i, $partitionValue)" - }) - spark.sql( - s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend") - - val txn = Table - .forPath(engine, kernelTablePath) - .createTransactionBuilder(engine, "test-engine", Operation.WRITE) - .build(engine) - - commitSparkChangeToKernel( - txn, - engine, - sparkTablePath, - versionAtCommit) - assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) - } - private def assertChecksumEquals( engine: Engine, sparkTablePath: String, @@ -188,16 +137,22 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt assert(crc1.getMetadata.getPartitionColNames === crc2.getMetadata.getPartitionColNames) } + // TODO docs private def commitSparkChangeToKernel( - txn: Transaction, + 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]() From b9b4aac6fe0e37bb5dda505f93072ccaf91e17e5 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Thu, 20 Feb 2025 13:08:14 -0800 Subject: [PATCH 72/73] remove patch file --- ...d649fd6a9ca4b87c742beb048afffcfd571b.patch | 176 ------------------ 1 file changed, 176 deletions(-) delete mode 100644 aae9d649fd6a9ca4b87c742beb048afffcfd571b.patch diff --git a/aae9d649fd6a9ca4b87c742beb048afffcfd571b.patch b/aae9d649fd6a9ca4b87c742beb048afffcfd571b.patch deleted file mode 100644 index 64cde006a9a..00000000000 --- a/aae9d649fd6a9ca4b87c742beb048afffcfd571b.patch +++ /dev/null @@ -1,176 +0,0 @@ -From aae9d649fd6a9ca4b87c742beb048afffcfd571b Mon Sep 17 00:00:00 2001 -From: Allison Portis -Date: Thu, 20 Feb 2025 11:32:18 -0800 -Subject: [PATCH] Simplify - ---- - .../ChecksumSimpleComparisonSuite.scala | 99 +++++-------------- - 1 file changed, 27 insertions(+), 72 deletions(-) - -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 -index 2102f94cf3..754a1572b0 100644 ---- 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 -@@ -18,13 +18,11 @@ package io.delta.kernel.defaults - import java.io.File - import java.nio.file.Files - import java.util --import java.util.Collections.emptyMap --import java.util.Optional - - import scala.collection.immutable.Seq - import scala.jdk.CollectionConverters.setAsJavaSetConverter - --import io.delta.kernel.{Operation, Table, Transaction} -+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 -@@ -35,9 +33,11 @@ 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.IntegerType.INTEGER -+import io.delta.kernel.types.LongType.LONG - import io.delta.kernel.types.StructType --import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable}; -+import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} -+ -+import org.apache.spark.sql.functions.col - - /** - * Test suite to verify checksum file correctness by comparing -@@ -58,16 +58,18 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt - engine, - kernelTablePath, - isNewTable = true, -- schema = new StructType().add("id", INTEGER), -+ 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 Integer) USING DELTA") -+ spark.sql(s"CREATE OR REPLACE TABLE delta.`${sparkTablePath}` (id LONG) USING DELTA") - assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) - -- (1 to 10).foreach(version => -- insertIntoUnpartitionedTableAndCheckCrc(engine, sparkTablePath, kernelTablePath, version)) -- -+ (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) -+ } - } - } - -@@ -80,77 +82,24 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt - engine, - kernelTablePath, - isNewTable = true, -- schema = new StructType().add("id", INTEGER).add(PARTITION_COLUMN, INTEGER), -+ 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 Integer, part Integer) USING DELTA PARTITIONED BY (part)") -+ s"(id LONG, part LONG) USING DELTA PARTITIONED BY (part)") - assertChecksumEquals(engine, sparkTablePath, kernelTablePath, 0) - -- (1 to 10).foreach(version => -- insertIntoPartitionedTableAndCheckCrc(engine, sparkTablePath, kernelTablePath, version)) -+ (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) -+ } - } - } - -- /** -- * Insert into unpartitioned spark table, load the added files from the spark table's commit log, -- * commit them to kernel table and verify the checksum files are consistent between spark -- * and kernel. -- */ -- private def insertIntoUnpartitionedTableAndCheckCrc( -- engine: Engine, -- sparkTablePath: String, -- kernelTablePath: String, -- versionAtCommit: Long): Unit = { -- var valueToAppend = "(0)" -- (0L to versionAtCommit).foreach(i => valueToAppend = valueToAppend + s",($i)") -- spark.sql( -- s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend") -- -- val txn = Table -- .forPath(engine, kernelTablePath) -- .createTransactionBuilder(engine, "test-engine", Operation.WRITE) -- .build(engine) -- -- commitSparkChangeToKernel(txn, engine, sparkTablePath, versionAtCommit) -- assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) -- } -- -- /** -- * Insert into partitioned spark table, load the added files from the spark table's commit log, -- * commit them to kernel table and verify the checksum files are consistent between spark -- * and kernel. -- */ -- private def insertIntoPartitionedTableAndCheckCrc( -- engine: Engine, -- sparkTablePath: String, -- kernelTablePath: String, -- versionAtCommit: Long): Unit = { -- var valueToAppend = "(0, 0)" -- var addedPartition = Set(0) -- (0L to versionAtCommit).foreach(i => { -- val partitionValue = 2 * i -- addedPartition = addedPartition + partitionValue.toInt -- valueToAppend = valueToAppend + s",($i, $partitionValue)" -- }) -- spark.sql( -- s"INSERT INTO delta.`$sparkTablePath` values $valueToAppend") -- -- val txn = Table -- .forPath(engine, kernelTablePath) -- .createTransactionBuilder(engine, "test-engine", Operation.WRITE) -- .build(engine) -- -- commitSparkChangeToKernel( -- txn, -- engine, -- sparkTablePath, -- versionAtCommit) -- assertChecksumEquals(engine, sparkTablePath, kernelTablePath, versionAtCommit) -- } -- - private def assertChecksumEquals( - engine: Engine, - sparkTablePath: String, -@@ -188,16 +137,22 @@ class ChecksumSimpleComparisonSuite extends DeltaTableWriteSuiteBase with TestUt - assert(crc1.getMetadata.getPartitionColNames === crc2.getMetadata.getPartitionColNames) - } - -+ // TODO docs - private def commitSparkChangeToKernel( -- txn: Transaction, -+ 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]() From a2e28834d023ecf912d479dcd9ba23c709f4eb98 Mon Sep 17 00:00:00 2001 From: Xin Huang Date: Thu, 20 Feb 2025 14:08:23 -0800 Subject: [PATCH 73/73] resolve commit --- .../delta/kernel/internal/checksum/ChecksumWriterSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 index 22fa0da61f1..1e73f99de19 100644 --- 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 @@ -173,8 +173,8 @@ class ChecksumWriterSuite extends AnyFunSuite with MockEngineUtils { new Protocol( /* minReaderVersion= */ 1, /* minWriterVersion= */ 2, - Collections.emptyList(), - Collections.emptyList()) + Collections.emptySet(), + Collections.emptySet()) } }