From 596374ebc5c2e0880646bf209e93b4aab857fee4 Mon Sep 17 00:00:00 2001 From: Jiaheng Tang Date: Fri, 3 May 2024 14:19:25 -0700 Subject: [PATCH 01/11] [Example][branch-3.2] Update clustering example for 3.2 (#2993) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (Example) ## Description Update clustering example for delta 3.2 ## How was this patch tested? Manually ran the example ``` python3 run-integration-tests.py --use-local ``` ## Does this PR introduce _any_ user-facing changes? No --- .../src/main/scala/example/Clustering.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/examples/scala/src/main/scala/example/Clustering.scala b/examples/scala/src/main/scala/example/Clustering.scala index 12ffafee19f..5c59e16e172 100644 --- a/examples/scala/src/main/scala/example/Clustering.scala +++ b/examples/scala/src/main/scala/example/Clustering.scala @@ -15,6 +15,8 @@ */ package example +import io.delta.tables.DeltaTable + import org.apache.spark.sql.SparkSession object Clustering { @@ -51,6 +53,29 @@ object Clustering { // Optimize the table println("Optimize the table") deltaSpark.sql(s"OPTIMIZE $tableName") + + // Change the clustering columns + println("Change the clustering columns") + deltaSpark.sql( + s"""ALTER TABLE $tableName CLUSTER BY (col2, col1)""".stripMargin) + + + // Check the clustering columns + println("Check the clustering columns") + deltaSpark.sql(s"DESCRIBE DETAIL $tableName").show(false) + } finally { + // Cleanup + deltaSpark.sql(s"DROP TABLE IF EXISTS $tableName") + } + + // DeltaTable clusterBy Scala API + try { + val table = io.delta.tables.DeltaTable.create() + .tableName(tableName) + .addColumn("col1", "INT") + .addColumn("col2", "STRING") + .clusterBy("col1", "col2") + .execute() } finally { // Cleanup deltaSpark.sql(s"DROP TABLE IF EXISTS $tableName") From 3a668e9917a98121c3db36883213b9680ff6812c Mon Sep 17 00:00:00 2001 From: Venki Korukanti Date: Thu, 2 May 2024 14:52:12 -0700 Subject: [PATCH 02/11] [Kernel] Add a meta file containing the Kernel version. (#3032) This is useful when we need to get the current version of Kernel. Approach is similar to how we do it for other modules in the delta. Manually verified. (cherry picked from commit 8cdf411d2c3e483a483595d117619c4aa6e15faa) --- build.sbt | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/build.sbt b/build.sbt index c63954d9040..7f9c660adc4 100644 --- a/build.sbt +++ b/build.sbt @@ -341,6 +341,18 @@ lazy val kernelApi = (project in file("kernel/kernel-api")) "com.novocode" % "junit-interface" % "0.11" % "test", "org.slf4j" % "slf4j-log4j12" % "1.7.36" % "test" ), + // Generate the package object to provide the version information in runtime. + Compile / sourceGenerators += Def.task { + val file = (Compile / sourceManaged).value / "io" / "delta" / "kernel" / "Meta.java" + IO.write(file, + s"""package io.delta.kernel; + | + |final public class Meta { + | public static final String KERNEL_VERSION = "${version.value}"; + |} + |""".stripMargin) + Seq(file) + }, javaCheckstyleSettings("kernel/dev/checkstyle.xml"), // Unidoc settings unidocSourceFilePatterns := Seq(SourceFilePattern("io/delta/kernel/")), From 1523e071dba1b001767d7042ca6fef9fc1139acf Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Thu, 2 May 2024 17:08:08 -0700 Subject: [PATCH 03/11] [Kernel] Rename `TableClient` to `Engine` (#3015) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description Renames `TableClient` to `Engine` in package names and code. A follow-up PR will be needed to update the `USER_GUIDE` and `README` ## How was this patch tested? Existing tests suffice. ## Does this PR introduce _any_ user-facing changes? Yes, renames public interface. (cherry picked from commit f727b84ba248c118043ef27c16b7df83bc378589) --- .../internal/KernelDeltaLogDelegator.scala | 18 ++-- .../flink/internal/KernelSnapshotWrapper.java | 2 +- .../kernel/examples/BaseTableReader.java | 8 +- .../examples/MultiThreadedTableReader.java | 36 ++++---- .../examples/SingleThreadedTableReader.java | 22 ++--- .../delta/kernel/examples/utils/RowSerDe.java | 6 +- .../src/main/java/io/delta/kernel/Scan.java | 24 +++--- .../java/io/delta/kernel/ScanBuilder.java | 10 +-- .../main/java/io/delta/kernel/Snapshot.java | 14 +-- .../src/main/java/io/delta/kernel/Table.java | 30 +++---- .../TableClient.java => engine/Engine.java} | 4 +- .../{client => engine}/ExpressionHandler.java | 2 +- .../{client => engine}/FileReadRequest.java | 2 +- .../{client => engine}/FileSystemClient.java | 2 +- .../{client => engine}/JsonHandler.java | 2 +- .../{client => engine}/ParquetHandler.java | 2 +- .../{client => engine}/package-info.java | 2 +- .../delta/kernel/expressions/Predicate.java | 2 +- .../kernel/internal/DeltaHistoryManager.java | 22 ++--- .../internal/InternalScanFileUtils.java | 6 +- .../kernel/internal/ScanBuilderImpl.java | 12 +-- .../io/delta/kernel/internal/ScanImpl.java | 27 +++--- .../delta/kernel/internal/SnapshotImpl.java | 23 +++-- .../io/delta/kernel/internal/TableImpl.java | 24 +++--- .../kernel/internal/actions/Metadata.java | 6 +- .../internal/checkpoints/Checkpointer.java | 30 +++---- .../kernel/internal/data/ScanStateRow.java | 32 +++---- .../DeletionVectorStoredBitmap.java | 4 +- .../deletionvectors/DeletionVectorUtils.java | 6 +- .../internal/replay/ActionsIterator.java | 16 ++-- .../replay/ActiveAddFilesIterator.java | 12 +-- .../replay/CreateCheckpointIterator.java | 13 ++- .../kernel/internal/replay/LogReplay.java | 21 ++--- .../internal/skipping/DataSkippingUtils.java | 6 +- .../internal/snapshot/SnapshotManager.java | 86 +++++++++---------- .../kernel/internal/util/PartitionUtils.java | 6 +- .../internal/DeltaHistoryManagerSuite.scala | 8 +- .../internal/SnapshotManagerSuite.scala | 14 +-- .../checkpoints/CheckpointerSuite.scala | 22 ++--- ...lientUtils.scala => MockEngineUtils.scala} | 18 ++-- .../test/MockFileSystemClientUtils.scala | 26 +++--- .../DefaultEngine.java} | 20 ++--- .../DefaultExpressionHandler.java | 4 +- .../DefaultFileSystemClient.java | 6 +- .../DefaultJsonHandler.java | 4 +- .../DefaultParquetHandler.java | 4 +- .../{client => engine}/package-info.java | 6 +- ...ntErrors.java => DefaultEngineErrors.java} | 4 +- .../data/DefaultRowBasedColumnarBatch.java | 2 +- .../DefaultExpressionEvaluator.java | 2 +- .../expressions/ImplicitCastExpression.java | 2 +- .../internal/logstore/LogStoreProvider.java | 6 +- .../BenchmarkParallelCheckpointReading.java | 26 +++--- .../defaults/CheckpointV2ReadSuite.scala | 8 +- .../defaults/CreateCheckpointSuite.scala | 37 ++++---- .../kernel/defaults/DeletionVectorSuite.scala | 2 +- .../defaults/DeltaTableReadsSuite.scala | 38 ++++---- .../defaults/LogReplayMetricsSuite.scala | 56 ++++++------ .../kernel/defaults/LogReplaySuite.scala | 52 +++++------ .../io/delta/kernel/defaults/ScanSuite.scala | 78 ++++++++--------- .../DefaultExpressionHandlerSuite.scala | 2 +- .../DefaultFileSystemClientSuite.scala | 4 +- .../DefaultJsonHandlerSuite.scala | 4 +- .../DefaultParquetHandlerSuite.scala | 2 +- .../internal/parquet/ParquetSuiteBase.scala | 2 +- .../kernel/defaults/utils/TestUtils.scala | 75 ++++++++-------- project/FlinkMimaExcludes.scala | 7 +- 67 files changed, 536 insertions(+), 545 deletions(-) rename kernel/kernel-api/src/main/java/io/delta/kernel/{client/TableClient.java => engine/Engine.java} (96%) rename kernel/kernel-api/src/main/java/io/delta/kernel/{client => engine}/ExpressionHandler.java (99%) rename kernel/kernel-api/src/main/java/io/delta/kernel/{client => engine}/FileReadRequest.java (97%) rename kernel/kernel-api/src/main/java/io/delta/kernel/{client => engine}/FileSystemClient.java (98%) rename kernel/kernel-api/src/main/java/io/delta/kernel/{client => engine}/JsonHandler.java (99%) rename kernel/kernel-api/src/main/java/io/delta/kernel/{client => engine}/ParquetHandler.java (99%) rename kernel/kernel-api/src/main/java/io/delta/kernel/{client => engine}/package-info.java (96%) rename kernel/kernel-api/src/test/scala/io/delta/kernel/test/{MockTableClientUtils.scala => MockEngineUtils.scala} (91%) rename kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/{client/DefaultTableClient.java => engine/DefaultEngine.java} (72%) rename kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/{client => engine}/DefaultExpressionHandler.java (97%) rename kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/{client => engine}/DefaultFileSystemClient.java (97%) rename kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/{client => engine}/DefaultJsonHandler.java (99%) rename kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/{client => engine}/DefaultParquetHandler.java (98%) rename kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/{client => engine}/package-info.java (74%) rename kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/{DefaultTableClientErrors.java => DefaultEngineErrors.java} (88%) rename kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/{client => engine}/DefaultExpressionHandlerSuite.scala (99%) rename kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/{client => engine}/DefaultFileSystemClientSuite.scala (96%) rename kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/{client => engine}/DefaultJsonHandlerSuite.scala (99%) rename kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/{client => engine}/DefaultParquetHandlerSuite.scala (98%) diff --git a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala index 5fceee6a88d..21a0c164972 100644 --- a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala +++ b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala @@ -22,11 +22,11 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import io.delta.kernel.{Table, TableNotFoundException} -import io.delta.kernel.defaults.client.DefaultTableClient -import io.delta.kernel.internal.{SnapshotImpl => SnapshotImplKernel, TableImpl} +import io.delta.kernel.defaults.engine.DefaultEngine +import io.delta.kernel.internal.{TableImpl, SnapshotImpl => SnapshotImplKernel} import io.delta.standalone.VersionLog import io.delta.standalone.actions.{CommitInfo => CommitInfoJ} -import io.delta.standalone.internal.{SnapshotImpl => StandaloneSnapshotImpl, InitialSnapshotImpl => StandaloneInitialSnapshotImpl} +import io.delta.standalone.internal.{InitialSnapshotImpl => StandaloneInitialSnapshotImpl, SnapshotImpl => StandaloneSnapshotImpl} import io.delta.standalone.internal.util.{Clock, SystemClock} class KernelOptTxn(kernelDeltaLog: KernelDeltaLogDelegator, kernelSnapshot: KernelSnapshotDelegator) @@ -44,7 +44,7 @@ class KernelOptTxn(kernelDeltaLog: KernelDeltaLogDelegator, kernelSnapshot: Kern * provides features used by flink+startTransaction. */ class KernelDeltaLogDelegator( - tableClient: DefaultTableClient, + engine: DefaultEngine, table: TableImpl, standaloneDeltaLog: DeltaLogImpl, hadoopConf: Configuration, @@ -68,7 +68,7 @@ class KernelDeltaLogDelegator( override def update(): StandaloneSnapshotImpl = { // get latest snapshot via kernel val kernelSnapshot = try { - table.getLatestSnapshot(tableClient).asInstanceOf[SnapshotImplKernel] + table.getLatestSnapshot(engine).asInstanceOf[SnapshotImplKernel] } catch { case e: TableNotFoundException => return new StandaloneInitialSnapshotImpl(hadoopConf, logPath, this) @@ -82,7 +82,7 @@ class KernelDeltaLogDelegator( kernelSnapshotWrapper, hadoopConf, logPath, - kernelSnapshot.getVersion(tableClient), // note: tableClient isn't used + kernelSnapshot.getVersion(engine), // note: engine isn't used this, standaloneDeltaLog )) @@ -128,9 +128,9 @@ object KernelDeltaLogDelegator { // the kernel does not val standaloneDeltaLog = new DeltaLogImpl(hadoopConf, logPath, dataPathFromLog, clock) standaloneDeltaLog.ensureLogDirectoryExist() - val tableClient = DefaultTableClient.create(hadoopConf) - val table = Table.forPath(tableClient, dataPath).asInstanceOf[TableImpl] + val engine = DefaultEngine.create(hadoopConf) + val table = Table.forPath(engine, dataPath).asInstanceOf[TableImpl] // Todo: Potentially we could get the resolved paths out of the table above - new KernelDeltaLogDelegator(tableClient, table, standaloneDeltaLog, hadoopConf, logPath, dataPathFromLog, clock) + new KernelDeltaLogDelegator(engine, table, standaloneDeltaLog, hadoopConf, logPath, dataPathFromLog, clock) } } diff --git a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java index 8d8bf84c449..43ff2096670 100644 --- a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java +++ b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java @@ -73,7 +73,7 @@ public Metadata getMetadata() { */ @Override public long getVersion() { - // WARNING: getVersion in SnapshotImpl currently doesn't use the table client, so we can + // WARNING: getVersion in SnapshotImpl currently doesn't use the engine so we can // pass null, but if this changes this code could break return kernelSnapshot.getVersion(null); } diff --git a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/BaseTableReader.java b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/BaseTableReader.java index 833d174c233..ebd2a805b34 100644 --- a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/BaseTableReader.java +++ b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/BaseTableReader.java @@ -37,14 +37,14 @@ import org.apache.hadoop.conf.Configuration; import io.delta.kernel.TableNotFoundException; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.data.FilteredColumnarBatch; import io.delta.kernel.data.Row; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.types.*; import io.delta.kernel.utils.CloseableIterator; -import io.delta.kernel.defaults.client.DefaultTableClient; +import io.delta.kernel.defaults.engine.DefaultEngine; /** * Base class for reading Delta Lake tables using the Delta Kernel APIs. @@ -53,11 +53,11 @@ public abstract class BaseTableReader { public static final int DEFAULT_LIMIT = 20; protected final String tablePath; - protected final TableClient tableClient; + protected final Engine engine; public BaseTableReader(String tablePath) { this.tablePath = requireNonNull(tablePath); - this.tableClient = DefaultTableClient.create(new Configuration()); + this.engine = DefaultEngine.create(new Configuration()); } /** diff --git a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java index b1bf732ccdc..687b8a87d76 100644 --- a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java +++ b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java @@ -26,7 +26,7 @@ import org.apache.commons.cli.Options; import io.delta.kernel.*; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.data.ColumnarBatch; import io.delta.kernel.data.FilteredColumnarBatch; import io.delta.kernel.data.Row; @@ -78,15 +78,15 @@ public MultiThreadedTableReader(int numThreads, String tablePath) { public int show(int limit, Optional> columnsOpt, Optional predicate) throws TableNotFoundException { - Table table = Table.forPath(tableClient, tablePath); - Snapshot snapshot = table.getLatestSnapshot(tableClient); - StructType readSchema = pruneSchema(snapshot.getSchema(tableClient), columnsOpt); + Table table = Table.forPath(engine, tablePath); + Snapshot snapshot = table.getLatestSnapshot(engine); + StructType readSchema = pruneSchema(snapshot.getSchema(engine), columnsOpt); - ScanBuilder scanBuilder = snapshot.getScanBuilder(tableClient) - .withReadSchema(tableClient, readSchema); + ScanBuilder scanBuilder = snapshot.getScanBuilder(engine) + .withReadSchema(engine, readSchema); if (predicate.isPresent()) { - scanBuilder = scanBuilder.withFilter(tableClient, predicate.get()); + scanBuilder = scanBuilder.withFilter(engine, predicate.get()); } return new Reader(limit) @@ -140,15 +140,15 @@ private static class ScanFile { /** * Get the deserialized scan state as {@link Row} object */ - Row getScanRow(TableClient tableClient) { - return RowSerDe.deserializeRowFromJson(tableClient, stateJson); + Row getScanRow(Engine engine) { + return RowSerDe.deserializeRowFromJson(engine, stateJson); } /** * Get the deserialized scan file as {@link Row} object */ - Row getScanFileRow(TableClient tableClient) { - return RowSerDe.deserializeRowFromJson(tableClient, fileJson); + Row getScanFileRow(Engine engine) { + return RowSerDe.deserializeRowFromJson(engine, fileJson); } } @@ -199,9 +199,9 @@ int readData(StructType readSchema, Scan scan) { private Runnable workGenerator(Scan scan) { return (() -> { - Row scanStateRow = scan.getScanState(tableClient); + Row scanStateRow = scan.getScanState(engine); try(CloseableIterator scanFileIter = - scan.getScanFiles(tableClient)) { + scan.getScanFiles(engine)) { while (scanFileIter.hasNext() && !stopSignal.get()) { try (CloseableIterator scanFileRows = scanFileIter.next().getRows()) { @@ -231,15 +231,15 @@ private Runnable workConsumer(int workerId) { if (work == ScanFile.POISON_PILL) { return; // exit as there are no more work units } - Row scanState = work.getScanRow(tableClient); - Row scanFile = work.getScanFileRow(tableClient); + Row scanState = work.getScanRow(engine); + Row scanFile = work.getScanFileRow(engine); FileStatus fileStatus = InternalScanFileUtils.getAddFileStatus(scanFile); StructType physicalReadSchema = - ScanStateRow.getPhysicalDataReadSchema(tableClient, scanState); + ScanStateRow.getPhysicalDataReadSchema(engine, scanState); CloseableIterator physicalDataIter = - tableClient.getParquetHandler().readParquetFiles( + engine.getParquetHandler().readParquetFiles( singletonCloseableIterator(fileStatus), physicalReadSchema, Optional.empty()); @@ -247,7 +247,7 @@ private Runnable workConsumer(int workerId) { try ( CloseableIterator dataIter = Scan.transformPhysicalData( - tableClient, + engine, scanState, scanFile, physicalDataIter)) { diff --git a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/SingleThreadedTableReader.java b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/SingleThreadedTableReader.java index 37b75e01765..13cd2e80644 100644 --- a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/SingleThreadedTableReader.java +++ b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/SingleThreadedTableReader.java @@ -56,15 +56,15 @@ public SingleThreadedTableReader(String tablePath) { @Override public int show(int limit, Optional> columnsOpt, Optional predicate) throws TableNotFoundException, IOException { - Table table = Table.forPath(tableClient, tablePath); - Snapshot snapshot = table.getLatestSnapshot(tableClient); - StructType readSchema = pruneSchema(snapshot.getSchema(tableClient), columnsOpt); + Table table = Table.forPath(engine, tablePath); + Snapshot snapshot = table.getLatestSnapshot(engine); + StructType readSchema = pruneSchema(snapshot.getSchema(engine), columnsOpt); - ScanBuilder scanBuilder = snapshot.getScanBuilder(tableClient) - .withReadSchema(tableClient, readSchema); + ScanBuilder scanBuilder = snapshot.getScanBuilder(engine) + .withReadSchema(engine, readSchema); if (predicate.isPresent()) { - scanBuilder = scanBuilder.withFilter(tableClient, predicate.get()); + scanBuilder = scanBuilder.withFilter(engine, predicate.get()); } return readData(readSchema, scanBuilder.build(), limit); @@ -95,13 +95,13 @@ public static void main(String[] args) private int readData(StructType readSchema, Scan scan, int maxRowCount) throws IOException { printSchema(readSchema); - Row scanState = scan.getScanState(tableClient); - CloseableIterator scanFileIter = scan.getScanFiles(tableClient); + Row scanState = scan.getScanState(engine); + CloseableIterator scanFileIter = scan.getScanFiles(engine); int readRecordCount = 0; try { StructType physicalReadSchema = - ScanStateRow.getPhysicalDataReadSchema(tableClient, scanState); + ScanStateRow.getPhysicalDataReadSchema(engine, scanState); while (scanFileIter.hasNext()) { FilteredColumnarBatch scanFilesBatch = scanFileIter.next(); try (CloseableIterator scanFileRows = scanFilesBatch.getRows()) { @@ -110,14 +110,14 @@ private int readData(StructType readSchema, Scan scan, int maxRowCount) throws I FileStatus fileStatus = InternalScanFileUtils.getAddFileStatus(scanFileRow); CloseableIterator physicalDataIter = - tableClient.getParquetHandler().readParquetFiles( + engine.getParquetHandler().readParquetFiles( singletonCloseableIterator(fileStatus), physicalReadSchema, Optional.empty()); try ( CloseableIterator transformedData = Scan.transformPhysicalData( - tableClient, + engine, scanState, scanFileRow, physicalDataIter)) { diff --git a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/utils/RowSerDe.java b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/utils/RowSerDe.java index 144f8313097..28e1ee08a61 100644 --- a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/utils/RowSerDe.java +++ b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/utils/RowSerDe.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.data.Row; import io.delta.kernel.types.*; @@ -59,12 +59,12 @@ public static String serializeRowToJson(Row row) { /** * Utility method to deserialize a {@link Row} object from the JSON form. */ - public static Row deserializeRowFromJson(TableClient tableClient, String jsonRowWithSchema) { + public static Row deserializeRowFromJson(Engine engine, String jsonRowWithSchema) { try { JsonNode jsonNode = OBJECT_MAPPER.readTree(jsonRowWithSchema); JsonNode schemaNode = jsonNode.get("schema"); StructType schema = - tableClient.getJsonHandler().deserializeStructType(schemaNode.asText()); + engine.getJsonHandler().deserializeStructType(schemaNode.asText()); return parseRowFromJsonWithSchema((ObjectNode) jsonNode.get("row"), schema); } catch (JsonProcessingException ex) { throw new UncheckedIOException(ex); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Scan.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Scan.java index 4b776254c81..d53ee4da7d7 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/Scan.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Scan.java @@ -20,8 +20,8 @@ import java.util.Optional; import io.delta.kernel.annotation.Evolving; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.*; +import io.delta.kernel.engine.Engine; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.types.StructField; import io.delta.kernel.types.StructType; @@ -47,7 +47,7 @@ public interface Scan { /** * Get an iterator of data files to scan. * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @return iterator of {@link FilteredColumnarBatch}s where each selected row in * the batch corresponds to one scan file. Schema of each row is defined as follows: *

@@ -87,7 +87,7 @@ public interface Scan { * * */ - CloseableIterator getScanFiles(TableClient tableClient); + CloseableIterator getScanFiles(Engine engine); /** * Get the remaining filter that is not guaranteed to be satisfied for the data Delta Kernel @@ -101,17 +101,17 @@ public interface Scan { * Get the scan state associated with the current scan. This state is common across all * files in the scan to be read. * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @return Scan state in {@link Row} format. */ - Row getScanState(TableClient tableClient); + Row getScanState(Engine engine); /** * Transform the physical data read from the table data file into the logical data that expected * out of the Delta table. * - * @param tableClient Connector provided {@link TableClient} implementation. - * @param scanState Scan state returned by {@link Scan#getScanState(TableClient)} + * @param engine Connector provided {@link Engine} implementation. + * @param scanState Scan state returned by {@link Scan#getScanState(Engine)} * @param scanFile Scan file from where the physical data {@code physicalDataIter} is * read from. * @param physicalDataIter Iterator of {@link ColumnarBatch}s containing the physical data read @@ -123,7 +123,7 @@ public interface Scan { * @throws IOException when error occurs while reading the data. */ static CloseableIterator transformPhysicalData( - TableClient tableClient, + Engine engine, Row scanState, Row scanFile, CloseableIterator physicalDataIter) throws IOException { @@ -142,8 +142,8 @@ private void initIfRequired() { if (inited) { return; } - physicalReadSchema = ScanStateRow.getPhysicalSchema(tableClient, scanState); - logicalReadSchema = ScanStateRow.getLogicalSchema(tableClient, scanState); + physicalReadSchema = ScanStateRow.getPhysicalSchema(engine, scanState); + logicalReadSchema = ScanStateRow.getLogicalSchema(engine, scanState); tablePath = ScanStateRow.getTableRoot(scanState); inited = true; @@ -182,7 +182,7 @@ public FilteredColumnarBatch next() { } if (!dv.equals(currDV)) { Tuple2 dvInfo = - DeletionVectorUtils.loadNewDvAndBitmap(tableClient, tablePath, dv); + DeletionVectorUtils.loadNewDvAndBitmap(engine, tablePath, dv); this.currDV = dvInfo._1; this.currBitmap = dvInfo._2; } @@ -197,7 +197,7 @@ public FilteredColumnarBatch next() { // Add partition columns nextDataBatch = PartitionUtils.withPartitionColumns( - tableClient.getExpressionHandler(), + engine.getExpressionHandler(), nextDataBatch, InternalScanFileUtils.getPartitionValues(scanFile), physicalReadSchema diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/ScanBuilder.java b/kernel/kernel-api/src/main/java/io/delta/kernel/ScanBuilder.java index deff60fdd16..7c0eb5c6db0 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/ScanBuilder.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/ScanBuilder.java @@ -17,7 +17,7 @@ package io.delta.kernel; import io.delta.kernel.annotation.Evolving; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.types.StructType; @@ -33,21 +33,21 @@ public interface ScanBuilder { * Apply the given filter expression to prune any files that do not contain data satisfying * the given filter. * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @param predicate a {@link Predicate} to prune the metadata or data. * @return A {@link ScanBuilder} with filter applied. */ - ScanBuilder withFilter(TableClient tableClient, Predicate predicate); + ScanBuilder withFilter(Engine engine, Predicate predicate); /** * Apply the given readSchema. If the builder already has a projection applied, calling * this again replaces the existing projection. * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @param readSchema Subset of columns to read from the Delta table. * @return A {@link ScanBuilder} with projection pruning. */ - ScanBuilder withReadSchema(TableClient tableClient, StructType readSchema); + ScanBuilder withReadSchema(Engine engine, StructType readSchema); /** * @return Build the {@link Scan instance} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java index a4263f12df7..f03b0299eb6 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Snapshot.java @@ -17,7 +17,7 @@ package io.delta.kernel; import io.delta.kernel.annotation.Evolving; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.types.StructType; /** @@ -31,24 +31,24 @@ public interface Snapshot { /** * Get the version of this snapshot in the table. * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @return version of this snapshot in the Delta table */ - long getVersion(TableClient tableClient); + long getVersion(Engine engine); /** * Get the schema of the table at this snapshot. * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @return Schema of the Delta table at this snapshot. */ - StructType getSchema(TableClient tableClient); + StructType getSchema(Engine engine); /** * Create a scan builder to construct a {@link Scan} to read data from this snapshot. * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @return an instance of {@link ScanBuilder} */ - ScanBuilder getScanBuilder(TableClient tableClient); + ScanBuilder getScanBuilder(Engine engine); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java index c039a17dade..d390aa163da 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java @@ -18,7 +18,7 @@ import java.io.IOException; import io.delta.kernel.annotation.Evolving; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.internal.TableImpl; @@ -51,42 +51,42 @@ public interface Table { * * * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @param path location of the table. Path is resolved to fully - * qualified path using the given {@code tableClient}. + * qualified path using the given {@code engine}. * @return an instance of {@link Table} representing the Delta table at given path */ - static Table forPath(TableClient tableClient, String path) { - return TableImpl.forPath(tableClient, path); + static Table forPath(Engine engine, String path) { + return TableImpl.forPath(engine, path); } /** * The fully qualified path of this {@link Table} instance. * - * @param tableClient {@link TableClient} instance. + * @param engine {@link Engine} instance. * @return the table path. * @since 3.2.0 */ - String getPath(TableClient tableClient); + String getPath(Engine engine); /** * Get the latest snapshot of the table. * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @return an instance of {@link Snapshot} */ - Snapshot getLatestSnapshot(TableClient tableClient) + Snapshot getLatestSnapshot(Engine engine) throws TableNotFoundException; /** * Get the snapshot at the given {@code versionId}. * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @param versionId snapshot version to retrieve * @return an instance of {@link Snapshot} * @since 3.2.0 */ - Snapshot getSnapshotAsOfVersion(TableClient tableClient, long versionId) + Snapshot getSnapshotAsOfVersion(Engine engine, long versionId) throws TableNotFoundException; /** @@ -105,25 +105,25 @@ Snapshot getSnapshotAsOfVersion(TableClient tableClient, long versionId) * latest version of the table, we throw an error * . * - * @param tableClient {@link TableClient} instance to use in Delta Kernel. + * @param engine {@link Engine} instance to use in Delta Kernel. * @param millisSinceEpochUTC timestamp to fetch the snapshot for in milliseconds since the * unix epoch * @return an instance of {@link Snapshot} * @since 3.2.0 */ - Snapshot getSnapshotAsOfTimestamp(TableClient tableClient, long millisSinceEpochUTC) + Snapshot getSnapshotAsOfTimestamp(Engine engine, long millisSinceEpochUTC) throws TableNotFoundException; /** * Checkpoint the table at given version. It writes a single checkpoint file. * - * @param tableClient {@link TableClient} instance to use. + * @param engine {@link Engine} instance to use. * @param version Version to checkpoint. * @throws TableNotFoundException if the table is not found * @throws CheckpointAlreadyExistsException if a checkpoint already exists at the given version * @throws IOException for any I/O error. * @since 3.2.0 */ - void checkpoint(TableClient tableClient, long version) + void checkpoint(Engine engine, long version) throws TableNotFoundException, CheckpointAlreadyExistsException, IOException; } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/client/TableClient.java b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/Engine.java similarity index 96% rename from kernel/kernel-api/src/main/java/io/delta/kernel/client/TableClient.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/engine/Engine.java index 83cc73d8ff1..3a7801dd797 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/client/TableClient.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/Engine.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.delta.kernel.client; +package io.delta.kernel.engine; import io.delta.kernel.annotation.Evolving; @@ -26,7 +26,7 @@ * @since 3.0.0 */ @Evolving -public interface TableClient { +public interface Engine { /** * Get the connector provided {@link ExpressionHandler}. diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/client/ExpressionHandler.java b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/ExpressionHandler.java similarity index 99% rename from kernel/kernel-api/src/main/java/io/delta/kernel/client/ExpressionHandler.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/engine/ExpressionHandler.java index 07164e19e4f..d5f921de53a 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/client/ExpressionHandler.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/ExpressionHandler.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.client; +package io.delta.kernel.engine; import io.delta.kernel.annotation.Evolving; import io.delta.kernel.data.ColumnVector; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/client/FileReadRequest.java b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/FileReadRequest.java similarity index 97% rename from kernel/kernel-api/src/main/java/io/delta/kernel/client/FileReadRequest.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/engine/FileReadRequest.java index 3429f09bbfd..e61520f164b 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/client/FileReadRequest.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/FileReadRequest.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.client; +package io.delta.kernel.engine; import io.delta.kernel.annotation.Evolving; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/client/FileSystemClient.java b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/FileSystemClient.java similarity index 98% rename from kernel/kernel-api/src/main/java/io/delta/kernel/client/FileSystemClient.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/engine/FileSystemClient.java index 9153cebc014..63774ef4fa0 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/client/FileSystemClient.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/FileSystemClient.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.delta.kernel.client; +package io.delta.kernel.engine; import java.io.ByteArrayInputStream; import java.io.FileNotFoundException; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/client/JsonHandler.java b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/JsonHandler.java similarity index 99% rename from kernel/kernel-api/src/main/java/io/delta/kernel/client/JsonHandler.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/engine/JsonHandler.java index 2f2a51c4200..9bbbb467663 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/client/JsonHandler.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/JsonHandler.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.delta.kernel.client; +package io.delta.kernel.engine; import java.io.IOException; import java.nio.file.FileAlreadyExistsException; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/client/ParquetHandler.java b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/ParquetHandler.java similarity index 99% rename from kernel/kernel-api/src/main/java/io/delta/kernel/client/ParquetHandler.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/engine/ParquetHandler.java index 4faa605a726..88a53724516 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/client/ParquetHandler.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/ParquetHandler.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.delta.kernel.client; +package io.delta.kernel.engine; import java.io.IOException; import java.nio.file.FileAlreadyExistsException; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/client/package-info.java b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/package-info.java similarity index 96% rename from kernel/kernel-api/src/main/java/io/delta/kernel/client/package-info.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/engine/package-info.java index 98226959d87..3e75f5ea522 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/client/package-info.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/package-info.java @@ -19,4 +19,4 @@ * as reading parquet files, listing files in a file system, parsing a JSON string etc. to Delta * Kernel. */ -package io.delta.kernel.client; +package io.delta.kernel.engine; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/expressions/Predicate.java b/kernel/kernel-api/src/main/java/io/delta/kernel/expressions/Predicate.java index 969eeaf7bc9..b36cee0b3e3 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/expressions/Predicate.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/expressions/Predicate.java @@ -22,7 +22,7 @@ import java.util.stream.Stream; import io.delta.kernel.annotation.Evolving; -import io.delta.kernel.client.ExpressionHandler; +import io.delta.kernel.engine.ExpressionHandler; /** * Defines predicate scalar expression which is an extension of {@link ScalarExpression} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaHistoryManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaHistoryManager.java index 9d6bbc05b63..6bbc1ceac0e 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaHistoryManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaHistoryManager.java @@ -23,7 +23,7 @@ import org.slf4j.LoggerFactory; import io.delta.kernel.TableNotFoundException; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.utils.FileStatus; import io.delta.kernel.internal.checkpoints.CheckpointInstance; @@ -44,19 +44,19 @@ private DeltaHistoryManager() {} * exception. If the provided timestamp is before the timestamp of the earliest version of the * table throws an exception. * - * @param tableClient instance of {@link TableClient} to use + * @param engine instance of {@link Engine} to use * @param logPath the _delta_log path of the table * @param timestamp the timestamp find the version for in milliseconds since the unix epoch * @return the active recreatable commit version at the provided timestamp * @throws TableNotFoundException when there is no Delta table at the given path */ public static long getActiveCommitAtTimestamp( - TableClient tableClient, Path logPath, long timestamp) throws TableNotFoundException { + Engine engine, Path logPath, long timestamp) throws TableNotFoundException { - long earliestRecreatableCommit = getEarliestRecreatableCommit(tableClient, logPath); + long earliestRecreatableCommit = getEarliestRecreatableCommit(engine, logPath); // Search for the commit - List commits = getCommits(tableClient, logPath, earliestRecreatableCommit); + List commits = getCommits(engine, logPath, earliestRecreatableCommit); Commit commit = lastCommitBeforeOrAtTimestamp(commits, timestamp) .orElseThrow(() -> DeltaErrors.timestampEarlierThanTableFirstCommitException( @@ -85,9 +85,9 @@ public static long getActiveCommitAtTimestamp( * We search for the earliest checkpoint we have, or whether we have the 0th delta file. This * method assumes that the commits are contiguous. */ - private static long getEarliestRecreatableCommit(TableClient tableClient, Path logPath) + private static long getEarliestRecreatableCommit(Engine engine, Path logPath) throws TableNotFoundException { - try (CloseableIterator files = listFrom(tableClient, logPath, 0) + try (CloseableIterator files = listFrom(engine, logPath, 0) .filter(fs -> FileNames.isCommitFile(getName(fs.getPath())) || FileNames.isCheckpointFile(getName(fs.getPath())) @@ -167,12 +167,12 @@ private static long getEarliestRecreatableCommit(TableClient tableClient, Path l * exist or is empty. */ private static CloseableIterator listFrom( - TableClient tableClient, + Engine engine, Path logPath, long startVersion) throws TableNotFoundException { Path tablePath = logPath.getParent(); try { - CloseableIterator files = tableClient + CloseableIterator files = engine .getFileSystemClient() .listFrom(FileNames.listingPrefix(logPath, startVersion)); if (!files.hasNext()) { @@ -192,9 +192,9 @@ private static CloseableIterator listFrom( * Guarantees that the commits returned have both monotonically increasing versions and * timestamps. */ - private static List getCommits(TableClient tableClient, Path logPath, long start) + private static List getCommits(Engine engine, Path logPath, long start) throws TableNotFoundException{ - CloseableIterator commits = listFrom(tableClient, logPath, start) + CloseableIterator commits = listFrom(engine, logPath, start) .filter(fs -> FileNames.isCommitFile(getName(fs.getPath()))) .map(fs -> new Commit(FileNames.deltaVersion(fs.getPath()), fs.getModificationTime())); return monotonizeCommitTimestamps(commits); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/InternalScanFileUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/InternalScanFileUtils.java index 3932ee9cb2f..5cff673fe44 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/InternalScanFileUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/InternalScanFileUtils.java @@ -19,8 +19,8 @@ import java.util.Map; import io.delta.kernel.Scan; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.Row; +import io.delta.kernel.engine.Engine; import io.delta.kernel.expressions.Column; import io.delta.kernel.types.DataType; import io.delta.kernel.types.StringType; @@ -36,7 +36,7 @@ /** * Utilities to extract information out of the scan file rows returned by - * {@link Scan#getScanFiles(TableClient)}. + * {@link Scan#getScanFiles(Engine)}. */ public class InternalScanFileUtils { private InternalScanFileUtils() {} @@ -67,7 +67,7 @@ private InternalScanFileUtils() {} .add(TABLE_ROOT_STRUCT_FIELD); /** - * Schema of the returned scan files when {@link ScanImpl#getScanFiles(TableClient, boolean)} + * Schema of the returned scan files when {@link ScanImpl#getScanFiles(Engine, boolean)} * is called with {@code includeStats=true}. */ public static final StructType SCAN_FILE_SCHEMA_WITH_STATS = new StructType() diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java index ffb88b629ea..4d0b282a47c 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java @@ -20,7 +20,7 @@ import io.delta.kernel.Scan; import io.delta.kernel.ScanBuilder; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.types.StructType; @@ -40,7 +40,7 @@ public class ScanBuilderImpl private final Metadata metadata; private final StructType snapshotSchema; private final LogReplay logReplay; - private final TableClient tableClient; + private final Engine engine; private StructType readSchema; private Optional predicate; @@ -51,19 +51,19 @@ public ScanBuilderImpl( Metadata metadata, StructType snapshotSchema, LogReplay logReplay, - TableClient tableClient) { + Engine engine) { this.dataPath = dataPath; this.protocol = protocol; this.metadata = metadata; this.snapshotSchema = snapshotSchema; this.logReplay = logReplay; - this.tableClient = tableClient; + this.engine = engine; this.readSchema = snapshotSchema; this.predicate = Optional.empty(); } @Override - public ScanBuilder withFilter(TableClient tableClient, Predicate predicate) { + public ScanBuilder withFilter(Engine engine, Predicate predicate) { if (this.predicate.isPresent()) { throw new IllegalArgumentException("There already exists a filter in current builder"); } @@ -72,7 +72,7 @@ public ScanBuilder withFilter(TableClient tableClient, Predicate predicate) { } @Override - public ScanBuilder withReadSchema(TableClient tableClient, StructType readSchema) { + public ScanBuilder withReadSchema(Engine engine, StructType readSchema) { // TODO: validate the readSchema is a subset of the table schema this.readSchema = readSchema; return this; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanImpl.java index 394514661b1..1c98f650283 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanImpl.java @@ -22,8 +22,8 @@ import static java.util.stream.Collectors.toMap; import io.delta.kernel.Scan; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.*; +import io.delta.kernel.engine.Engine; import io.delta.kernel.expressions.*; import io.delta.kernel.types.StructField; import io.delta.kernel.types.StructType; @@ -93,8 +93,8 @@ public ScanImpl( * @return data in {@link ColumnarBatch} batch format. Each row correspond to one survived file. */ @Override - public CloseableIterator getScanFiles(TableClient tableClient) { - return getScanFiles(tableClient, false); + public CloseableIterator getScanFiles(Engine engine) { + return getScanFiles(engine, false); } /** @@ -106,12 +106,12 @@ public CloseableIterator getScanFiles(TableClient tableCl * When {@code includeStats=false} the JSON file statistics may or may not be present in the * returned columnar batches. * - * @param tableClient the {@link TableClient} instance to use + * @param engine the {@link Engine} instance to use * @param includeStats whether to read and include the JSON statistics * @return the surviving scan files as {@link FilteredColumnarBatch}s */ public CloseableIterator getScanFiles( - TableClient tableClient, boolean includeStats) { + Engine engine, boolean includeStats) { if (accessedScanFiles) { throw new IllegalStateException("Scan files are already fetched from this instance"); } @@ -134,12 +134,12 @@ public CloseableIterator getScanFiles( partitionColToStructFieldMap.get()))); // Apply partition pruning - scanFileIter = applyPartitionPruning(tableClient, scanFileIter); + scanFileIter = applyPartitionPruning(engine, scanFileIter); // Apply data skipping if (hasDataSkippingFilter) { // there was a usable data skipping filter --> apply data skipping - scanFileIter = applyDataSkipping(tableClient, scanFileIter, dataSkippingFilter.get()); + scanFileIter = applyDataSkipping(engine, scanFileIter, dataSkippingFilter.get()); } // TODO when !includeStats drop the stats column if present before returning @@ -147,7 +147,7 @@ public CloseableIterator getScanFiles( } @Override - public Row getScanState(TableClient tableClient) { + public Row getScanState(Engine engine) { // Physical equivalent of the logical read schema. StructType physicalReadSchema = ColumnMapping.convertToPhysicalSchema( readSchema, @@ -207,7 +207,7 @@ private Optional removeAlwaysTrue(Optional predicate) { } private CloseableIterator applyPartitionPruning( - TableClient tableClient, + Engine engine, CloseableIterator scanFileIter) { Optional partitionPredicate = getPartitionsFilters(); if (!partitionPredicate.isPresent()) { @@ -232,7 +232,7 @@ public FilteredColumnarBatch next() { FilteredColumnarBatch next = scanFileIter.next(); if (predicateEvaluator == null) { predicateEvaluator = - tableClient.getExpressionHandler().getPredicateEvaluator( + engine.getExpressionHandler().getPredicateEvaluator( next.getData().getSchema(), predicateOnScanFileBatch); } @@ -258,7 +258,7 @@ private Optional getDataSkippingFilter() { } private CloseableIterator applyDataSkipping( - TableClient tableClient, + Engine engine, CloseableIterator scanFileIter, DataSkippingPredicate dataSkippingFilter) { // Get the stats schema @@ -280,15 +280,14 @@ private CloseableIterator applyDataSkipping( Arrays.asList(dataSkippingFilter, Literal.ofBoolean(true))), AlwaysTrue.ALWAYS_TRUE); - PredicateEvaluator predicateEvaluator = tableClient + PredicateEvaluator predicateEvaluator = engine .getExpressionHandler() .getPredicateEvaluator(prunedStatsSchema, filterToEval); return scanFileIter.map(filteredScanFileBatch -> { ColumnVector newSelectionVector = predicateEvaluator.eval( - DataSkippingUtils.parseJsonStats( - tableClient, + DataSkippingUtils.parseJsonStats(engine, filteredScanFileBatch, prunedStatsSchema ), 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 0363d1cfb09..431e13f4d27 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 @@ -19,7 +19,7 @@ import io.delta.kernel.ScanBuilder; import io.delta.kernel.Snapshot; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.types.StructType; import io.delta.kernel.internal.actions.Metadata; @@ -47,7 +47,7 @@ public SnapshotImpl( Path dataPath, long version, LogSegment logSegment, - TableClient tableClient, + Engine engine, long timestamp, Optional snapshotHint) { this.dataPath = dataPath; @@ -56,8 +56,7 @@ public SnapshotImpl( this.logReplay = new LogReplay( logPath, dataPath, - version, - tableClient, + version, engine, logSegment, snapshotHint); this.protocol = logReplay.getProtocol(); @@ -65,24 +64,23 @@ public SnapshotImpl( } @Override - public long getVersion(TableClient tableClient) { + public long getVersion(Engine engine) { return version; } @Override - public StructType getSchema(TableClient tableClient) { + public StructType getSchema(Engine engine) { return getMetadata().getSchema(); } @Override - public ScanBuilder getScanBuilder(TableClient tableClient) { + public ScanBuilder getScanBuilder(Engine engine) { return new ScanBuilderImpl( dataPath, protocol, metadata, - getSchema(tableClient), - logReplay, - tableClient + getSchema(engine), + logReplay, engine ); } @@ -95,11 +93,10 @@ public Protocol getProtocol() { } public CreateCheckpointIterator getCreateCheckpointIterator( - TableClient tableClient) { + Engine engine) { long minFileRetentionTimestampMillis = System.currentTimeMillis() - TOMBSTONE_RETENTION.fromMetadata(metadata); - return new CreateCheckpointIterator( - tableClient, + return new CreateCheckpointIterator(engine, logSegment, minFileRetentionTimestampMillis ); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java index 03124b02479..89f556d3ca2 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java @@ -18,16 +18,16 @@ import java.io.IOException; import io.delta.kernel.*; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.snapshot.SnapshotManager; public class TableImpl implements Table { - public static Table forPath(TableClient tableClient, String path) { + public static Table forPath(Engine engine, String path) { String resolvedPath; try { - resolvedPath = tableClient.getFileSystemClient().resolvePath(path); + resolvedPath = engine.getFileSystemClient().resolvePath(path); } catch (IOException io) { throw new RuntimeException(io); } @@ -45,30 +45,30 @@ public TableImpl(String tablePath) { } @Override - public String getPath(TableClient tableClient) { + public String getPath(Engine engine) { return tablePath; } @Override - public Snapshot getLatestSnapshot(TableClient tableClient) throws TableNotFoundException { - return snapshotManager.buildLatestSnapshot(tableClient); + public Snapshot getLatestSnapshot(Engine engine) throws TableNotFoundException { + return snapshotManager.buildLatestSnapshot(engine); } @Override - public Snapshot getSnapshotAsOfVersion(TableClient tableClient, long versionId) + public Snapshot getSnapshotAsOfVersion(Engine engine, long versionId) throws TableNotFoundException { - return snapshotManager.getSnapshotAt(tableClient, versionId); + return snapshotManager.getSnapshotAt(engine, versionId); } @Override - public Snapshot getSnapshotAsOfTimestamp(TableClient tableClient, long millisSinceEpochUTC) + public Snapshot getSnapshotAsOfTimestamp(Engine engine, long millisSinceEpochUTC) throws TableNotFoundException { - return snapshotManager.getSnapshotForTimestamp(tableClient, millisSinceEpochUTC); + return snapshotManager.getSnapshotForTimestamp(engine, millisSinceEpochUTC); } @Override - public void checkpoint(TableClient tableClient, long version) + public void checkpoint(Engine engine, long version) throws TableNotFoundException, CheckpointAlreadyExistsException, IOException { - snapshotManager.checkpoint(tableClient, version); + snapshotManager.checkpoint(engine, version); } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Metadata.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Metadata.java index 3464b682dfc..39fb5fd2506 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Metadata.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Metadata.java @@ -19,8 +19,8 @@ import java.util.stream.Collectors; import static java.util.Objects.requireNonNull; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.*; +import io.delta.kernel.engine.Engine; import io.delta.kernel.types.*; import io.delta.kernel.internal.data.GenericRow; @@ -32,14 +32,14 @@ public class Metadata { public static Metadata fromColumnVector( - ColumnVector vector, int rowId, TableClient tableClient) { + ColumnVector vector, int rowId, Engine engine) { if (vector.isNullAt(rowId)) { return null; } final String schemaJson = requireNonNull(vector.getChild(4), rowId, "schemaString") .getString(rowId); - StructType schema = tableClient.getJsonHandler().deserializeStructType(schemaJson); + StructType schema = engine.getJsonHandler().deserializeStructType(schemaJson); return new Metadata( requireNonNull(vector.getChild(0), rowId, "id").getString(rowId), diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checkpoints/Checkpointer.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checkpoints/Checkpointer.java index c9a4e13cd47..7fcc648076c 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checkpoints/Checkpointer.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/checkpoints/Checkpointer.java @@ -23,9 +23,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.ColumnarBatch; import io.delta.kernel.data.Row; +import io.delta.kernel.engine.Engine; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.utils.FileStatus; @@ -81,10 +81,10 @@ public static Optional getLatestCompleteCheckpointFromList( * Find the last complete checkpoint before (strictly less than) a given version. */ public static Optional findLastCompleteCheckpointBefore( - TableClient tableClient, + Engine engine, Path tableLogPath, long version) { - return findLastCompleteCheckpointBeforeHelper(tableClient, tableLogPath, version)._1; + return findLastCompleteCheckpointBeforeHelper(engine, tableLogPath, version)._1; } /** @@ -93,7 +93,7 @@ public static Optional findLastCompleteCheckpointBefore( */ protected static Tuple2, Long> findLastCompleteCheckpointBeforeHelper( - TableClient tableClient, + Engine engine, Path tableLogPath, long version) { CheckpointInstance upperBoundCheckpoint = new CheckpointInstance(version); @@ -111,7 +111,7 @@ public static Optional findLastCompleteCheckpointBefore( while (currentVersion >= 0) { try { long searchLowerBound = Math.max(0, currentVersion - 1000); - CloseableIterator deltaLogFileIter = tableClient.getFileSystemClient() + CloseableIterator deltaLogFileIter = engine.getFileSystemClient() .listFrom(FileNames.listingPrefix(tableLogPath, searchLowerBound)); List checkpoints = new ArrayList<>(); @@ -181,20 +181,20 @@ public Checkpointer(Path tableLogPath) { /** * Returns information about the most recent checkpoint. */ - public Optional readLastCheckpointFile(TableClient tableClient) { - return loadMetadataFromFile(tableClient, 0 /* tries */); + public Optional readLastCheckpointFile(Engine engine) { + return loadMetadataFromFile(engine, 0 /* tries */); } /** * Write the given data to last checkpoint metadata file. - * @param tableClient {@link TableClient} instance to use for writing + * @param engine {@link Engine} instance to use for writing * @param checkpointMetaData Checkpoint metadata to write * @throws IOException For any I/O issues. */ public void writeLastCheckpointFile( - TableClient tableClient, + Engine engine, CheckpointMetaData checkpointMetaData) throws IOException { - tableClient.getJsonHandler() + engine.getJsonHandler() .writeJsonFileAtomically( lastCheckpointFilePath.toString(), singletonCloseableIterator(checkpointMetaData.toRow()), @@ -204,10 +204,10 @@ public void writeLastCheckpointFile( /** * Loads the checkpoint metadata from the _last_checkpoint file. *

- * @param tableClient {@link TableClient instance to use} + * @param engine {@link Engine instance to use} * @param tries Number of times already tried to load the metadata before this call. */ - private Optional loadMetadataFromFile(TableClient tableClient, int tries) { + private Optional loadMetadataFromFile(Engine engine, int tries) { if (tries >= 3) { // We have tried 3 times and failed. Assume the checkpoint metadata file is corrupt. logger.warn( @@ -222,7 +222,7 @@ private Optional loadMetadataFromFile(TableClient tableClien lastCheckpointFilePath.toString(), 0 /* size */, 0 /* modTime */); try (CloseableIterator jsonIter = - tableClient.getJsonHandler().readJsonFiles( + engine.getJsonHandler().readJsonFiles( singletonCloseableIterator(lastCheckpointFile), CheckpointMetaData.READ_SCHEMA, Optional.empty())) { @@ -240,7 +240,7 @@ private Optional loadMetadataFromFile(TableClient tableClien lastCheckpointFilePath, tries); Thread.sleep(1000); - return loadMetadataFromFile(tableClient, tries + 1); + return loadMetadataFromFile(engine, tries + 1); } } catch (FileNotFoundException ex) { return Optional.empty(); // there is no point retrying @@ -254,7 +254,7 @@ private Optional loadMetadataFromFile(TableClient tableClien lastCheckpointFilePath, tries); logger.warn(msg, ex); // we can retry until max tries are exhausted - return loadMetadataFromFile(tableClient, tries + 1); + return loadMetadataFromFile(engine, tries + 1); } } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/ScanStateRow.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/ScanStateRow.java index 3bff36b264f..0d3ab71ee66 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/ScanStateRow.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/ScanStateRow.java @@ -20,8 +20,8 @@ import static java.util.stream.Collectors.toMap; import io.delta.kernel.Scan; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.Row; +import io.delta.kernel.engine.Engine; import io.delta.kernel.types.*; import io.delta.kernel.internal.actions.Metadata; @@ -74,50 +74,50 @@ public ScanStateRow(HashMap valueMap) { /** * Utility method to get the logical schema from the scan state {@link Row} returned by - * {@link Scan#getScanState(TableClient)}. + * {@link Scan#getScanState(Engine)}. * - * @param tableClient instance of {@link TableClient} to use. + * @param engine instance of {@link Engine} to use. * @param scanState Scan state {@link Row} * @return Logical schema to read from the data files. */ - public static StructType getLogicalSchema(TableClient tableClient, Row scanState) { + public static StructType getLogicalSchema(Engine engine, Row scanState) { String serializedSchema = scanState.getString(COL_NAME_TO_ORDINAL.get("logicalSchemaString")); - return tableClient.getJsonHandler().deserializeStructType(serializedSchema); + return engine.getJsonHandler().deserializeStructType(serializedSchema); } /** * Utility method to get the physical schema from the scan state {@link Row} returned by - * {@link Scan#getScanState(TableClient)}. + * {@link Scan#getScanState(Engine)}. * - * @param tableClient instance of {@link TableClient} to use. + * @param engine instance of {@link Engine} to use. * @param scanState Scan state {@link Row} * @return Physical schema to read from the data files. */ - public static StructType getPhysicalSchema(TableClient tableClient, Row scanState) { + public static StructType getPhysicalSchema(Engine engine, Row scanState) { String serializedSchema = scanState.getString(COL_NAME_TO_ORDINAL.get("physicalSchemaString")); - return tableClient.getJsonHandler().deserializeStructType(serializedSchema); + return engine.getJsonHandler().deserializeStructType(serializedSchema); } /** * Utility method to get the physical data read schema from the scan state {@link Row} - * returned by {@link Scan#getScanState(TableClient)}. This schema is used to request data + * returned by {@link Scan#getScanState(Engine)}. This schema is used to request data * from the scan files for the query. * - * @param tableClient instance of {@link TableClient} to use. + * @param engine instance of {@link Engine} to use. * @param scanState Scan state {@link Row} * @return Physical schema to read from the data files. */ - public static StructType getPhysicalDataReadSchema(TableClient tableClient, Row scanState) { + public static StructType getPhysicalDataReadSchema(Engine engine, Row scanState) { String serializedSchema = scanState.getString(COL_NAME_TO_ORDINAL.get("physicalDataReadSchemaString")); - return tableClient.getJsonHandler().deserializeStructType(serializedSchema); + return engine.getJsonHandler().deserializeStructType(serializedSchema); } /** * Get the list of partition column names from the scan state {@link Row} returned by - * {@link Scan#getScanState(TableClient)}. + * {@link Scan#getScanState(Engine)}. * * @param scanState Scan state {@link Row} * @return List of partition column names according to the scan state. @@ -129,7 +129,7 @@ public static List getPartitionColumns(Row scanState) { /** * Get the column mapping mode from the scan state {@link Row} returned by - * {@link Scan#getScanState(TableClient)}. + * {@link Scan#getScanState(Engine)}. */ public static String getColumnMappingMode(Row scanState) { Map configuration = VectorUtils.toJavaMap( @@ -139,7 +139,7 @@ public static String getColumnMappingMode(Row scanState) { /** * Get the table root from scan state {@link Row} returned by - * {@link Scan#getScanState(TableClient)} + * {@link Scan#getScanState(Engine)} * * @param scanState Scan state {@link Row} * @return Fully qualified path to the location of the table. diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/deletionvectors/DeletionVectorStoredBitmap.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/deletionvectors/DeletionVectorStoredBitmap.java index 53e107b8994..fb994d028c0 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/deletionvectors/DeletionVectorStoredBitmap.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/deletionvectors/DeletionVectorStoredBitmap.java @@ -22,8 +22,8 @@ import java.util.Optional; import java.util.zip.CRC32; -import io.delta.kernel.client.FileReadRequest; -import io.delta.kernel.client.FileSystemClient; +import io.delta.kernel.engine.FileReadRequest; +import io.delta.kernel.engine.FileSystemClient; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.internal.actions.DeletionVectorDescriptor; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/deletionvectors/DeletionVectorUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/deletionvectors/DeletionVectorUtils.java index 86b9bae08a3..b80b5c436df 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/deletionvectors/DeletionVectorUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/deletionvectors/DeletionVectorUtils.java @@ -19,7 +19,7 @@ import java.io.IOException; import java.util.Optional; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.internal.actions.DeletionVectorDescriptor; import io.delta.kernel.internal.util.Tuple2; @@ -29,14 +29,14 @@ */ public class DeletionVectorUtils { public static Tuple2 loadNewDvAndBitmap( - TableClient tableClient, + Engine engine, String tablePath, DeletionVectorDescriptor dv) { DeletionVectorStoredBitmap storedBitmap = new DeletionVectorStoredBitmap(dv, Optional.of(tablePath)); try { RoaringBitmapArray bitmap = storedBitmap - .load(tableClient.getFileSystemClient()); + .load(engine.getFileSystemClient()); return new Tuple2<>(dv, bitmap); } catch (IOException e) { throw new RuntimeException("Couldn't load dv", e); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ActionsIterator.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ActionsIterator.java index 9ce9cc22c54..f95a894d196 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ActionsIterator.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ActionsIterator.java @@ -21,9 +21,9 @@ import java.util.*; import java.util.stream.Collectors; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.ColumnVector; import io.delta.kernel.data.ColumnarBatch; +import io.delta.kernel.engine.Engine; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterator; @@ -48,7 +48,7 @@ * Users must pass in a `readSchema` to select which actions and sub-fields they want to consume. */ class ActionsIterator implements CloseableIterator { - private final TableClient tableClient; + private final Engine engine; private final Optional checkpointPredicate; @@ -76,11 +76,11 @@ class ActionsIterator implements CloseableIterator { private boolean closed; ActionsIterator( - TableClient tableClient, + Engine engine, List files, StructType readSchema, Optional checkpointPredicate) { - this.tableClient = tableClient; + this.engine = engine; this.checkpointPredicate = checkpointPredicate; this.filesList = new LinkedList<>(); this.filesList.addAll( @@ -188,12 +188,12 @@ private CloseableIterator getActionsIterFromSinglePartOrV2Checkpo final CloseableIterator topLevelIter; if (fileName.endsWith(".parquet")) { - topLevelIter = tableClient.getParquetHandler().readParquetFiles( + topLevelIter = engine.getParquetHandler().readParquetFiles( singletonCloseableIterator(file), modifiedReadSchema, checkpointPredicate); } else if (fileName.endsWith(".json")) { - topLevelIter = tableClient.getJsonHandler().readJsonFiles( + topLevelIter = engine.getJsonHandler().readJsonFiles( singletonCloseableIterator(file), modifiedReadSchema, checkpointPredicate); @@ -279,7 +279,7 @@ private CloseableIterator getNextActionsIter() { // on (faster metadata & protocol loading in subsequent runs by remembering // the version of the last version where the metadata and protocol are found). final CloseableIterator dataIter = - tableClient.getJsonHandler().readJsonFiles( + engine.getJsonHandler().readJsonFiles( singletonCloseableIterator(nextFile), readSchema, Optional.empty()); @@ -303,7 +303,7 @@ private CloseableIterator getNextActionsIter() { // optimizations like reading multiple files in parallel. CloseableIterator checkpointFiles = retrieveRemainingCheckpointFiles(nextLogFile); - CloseableIterator dataIter = tableClient.getParquetHandler() + CloseableIterator dataIter = engine.getParquetHandler() .readParquetFiles(checkpointFiles, readSchema, checkpointPredicate); long version = checkpointVersion(nextFilePath); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ActiveAddFilesIterator.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ActiveAddFilesIterator.java index d19f52fc6c6..23847402adb 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ActiveAddFilesIterator.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ActiveAddFilesIterator.java @@ -19,10 +19,10 @@ import java.net.URI; import java.util.*; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.ColumnVector; import io.delta.kernel.data.ColumnarBatch; import io.delta.kernel.data.FilteredColumnarBatch; +import io.delta.kernel.engine.Engine; import io.delta.kernel.expressions.ExpressionEvaluator; import io.delta.kernel.expressions.Literal; import io.delta.kernel.types.StringType; @@ -50,7 +50,7 @@ * (have not been tombstoned). */ class ActiveAddFilesIterator implements CloseableIterator { - private final TableClient tableClient; + private final Engine engine; private final Path tableRoot; private final CloseableIterator iter; @@ -68,10 +68,10 @@ class ActiveAddFilesIterator implements CloseableIterator private boolean closed; ActiveAddFilesIterator( - TableClient tableClient, + Engine engine, CloseableIterator iter, Path tableRoot) { - this.tableClient = tableClient; + this.engine = engine; this.tableRoot = tableRoot; this.iter = iter; this.tombstonesFromJson = new HashSet<>(); @@ -217,7 +217,7 @@ private void prepareNext() { // Step 4: TODO: remove this step. This is a temporary requirement until the path // in `add` is converted to absolute path. if (tableRootVectorGenerator == null) { - tableRootVectorGenerator = tableClient.getExpressionHandler() + tableRootVectorGenerator = engine.getExpressionHandler() .getEvaluator( scanAddFiles.getSchema(), Literal.ofString(tableRoot.toUri().toString()), @@ -230,7 +230,7 @@ private void prepareNext() { tableRootVector); Optional selectionColumnVector = atLeastOneUnselected ? - Optional.of(tableClient.getExpressionHandler() + Optional.of(engine.getExpressionHandler() .createSelectionVector(selectionVectorBuffer, 0, addsVector.getSize())) : Optional.empty(); next = Optional.of(new FilteredColumnarBatch(scanAddFiles, selectionColumnVector)); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CreateCheckpointIterator.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CreateCheckpointIterator.java index 8bc6229f142..9eef17f378b 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CreateCheckpointIterator.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/CreateCheckpointIterator.java @@ -19,8 +19,8 @@ import java.io.IOException; import java.util.*; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.*; +import io.delta.kernel.engine.Engine; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.internal.actions.SetTransaction; @@ -78,7 +78,7 @@ public class CreateCheckpointIterator implements CloseableIterator protocolAndMetadata; public LogReplay( Path logPath, Path dataPath, long snapshotVersion, - TableClient tableClient, + Engine engine, LogSegment logSegment, Optional snapshotHint) { assertLogFilesBelongToTable(logPath, logSegment.allLogFilesUnsorted()); this.dataPath = dataPath; this.logSegment = logSegment; - this.tableClient = tableClient; + this.engine = engine; this.protocolAndMetadata = loadTableProtocolAndMetadata(snapshotHint, snapshotVersion); } @@ -168,12 +168,11 @@ public CloseableIterator getAddFilesAsColumnarBatches( boolean shouldReadStats, Optional checkpointPredicate) { final CloseableIterator addRemoveIter = - new ActionsIterator( - tableClient, + new ActionsIterator(engine, logSegment.allLogFilesReversed(), getAddRemoveReadSchema(shouldReadStats), checkpointPredicate); - return new ActiveAddFilesIterator(tableClient, addRemoveIter, dataPath); + return new ActiveAddFilesIterator(engine, addRemoveIter, dataPath); } //////////////////// @@ -204,8 +203,7 @@ private Tuple2 loadTableProtocolAndMetadata( Metadata metadata = null; try (CloseableIterator reverseIter = - new ActionsIterator( - tableClient, + new ActionsIterator(engine, logSegment.allLogFilesReversed(), PROTOCOL_METADATA_READ_SCHEMA, Optional.empty())) { @@ -245,7 +243,7 @@ private Tuple2 loadTableProtocolAndMetadata( for (int i = 0; i < metadataVector.getSize(); i++) { if (!metadataVector.isNullAt(i)) { - metadata = Metadata.fromColumnVector(metadataVector, i, tableClient); + metadata = Metadata.fromColumnVector(metadataVector, i, engine); if (protocol != null) { // Stop since we have found the latest Protocol and Metadata. @@ -288,8 +286,7 @@ private Tuple2 loadTableProtocolAndMetadata( private Optional loadLatestTransactionVersion(String applicationId) { try (CloseableIterator reverseIter = - new ActionsIterator( - tableClient, + new ActionsIterator(engine, logSegment.allLogFilesReversed(), SET_TRANSACTION_READ_SCHEMA, Optional.empty())) { diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/skipping/DataSkippingUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/skipping/DataSkippingUtils.java index 6289151a86b..a9e7dffbfac 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/skipping/DataSkippingUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/skipping/DataSkippingUtils.java @@ -17,10 +17,10 @@ import java.util.*; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.ColumnVector; import io.delta.kernel.data.ColumnarBatch; import io.delta.kernel.data.FilteredColumnarBatch; +import io.delta.kernel.engine.Engine; import io.delta.kernel.expressions.*; import io.delta.kernel.types.*; import static io.delta.kernel.internal.InternalScanFileUtils.ADD_FILE_ORDINAL; @@ -34,11 +34,11 @@ public class DataSkippingUtils { * return the parsed JSON stats from the scan files. */ public static ColumnarBatch parseJsonStats( - TableClient tableClient, FilteredColumnarBatch scanFileBatch, StructType statsSchema) { + Engine engine, FilteredColumnarBatch scanFileBatch, StructType statsSchema) { ColumnVector statsVector = scanFileBatch.getData() .getColumnVector(ADD_FILE_ORDINAL) .getChild(ADD_FILE_STATS_ORDINAL); - return tableClient.getJsonHandler() + return engine.getJsonHandler() .parseJson(statsVector, statsSchema, scanFileBatch.getSelectionVector()); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java index 97bdb7e1b11..4953dc43b74 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java @@ -29,7 +29,7 @@ import org.slf4j.LoggerFactory; import io.delta.kernel.*; -import io.delta.kernel.client.TableClient; +import io.delta.kernel.engine.Engine; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.utils.FileStatus; @@ -101,76 +101,75 @@ public static void verifyDeltaVersions( /** * Construct the latest snapshot for given table. * - * @param tableClient Instance of {@link TableClient} to use. + * @param engine Instance of {@link Engine} to use. * @return * @throws TableNotFoundException */ - public Snapshot buildLatestSnapshot(TableClient tableClient) + public Snapshot buildLatestSnapshot(Engine engine) throws TableNotFoundException { - return getSnapshotAtInit(tableClient); + return getSnapshotAtInit(engine); } /** * Construct the snapshot for the given table at the version provided. * - * @param tableClient Instance of {@link TableClient} to use. + * @param engine Instance of {@link Engine} to use. * @param version The snapshot version to construct * @return a {@link Snapshot} of the table at version {@code version} * @throws TableNotFoundException */ public Snapshot getSnapshotAt( - TableClient tableClient, + Engine engine, long version) throws TableNotFoundException { - Optional logSegmentOpt = getLogSegmentForVersion( - tableClient, + Optional logSegmentOpt = getLogSegmentForVersion(engine, Optional.empty(), /* startCheckpointOpt */ Optional.of(version) /* versionToLoadOpt */); return logSegmentOpt - .map(logSegment -> createSnapshot(logSegment, tableClient)) + .map(logSegment -> createSnapshot(logSegment, engine)) .orElseThrow(() -> new TableNotFoundException(tablePath.toString())); } /** * Construct the snapshot for the given table at the provided timestamp. * - * @param tableClient Instance of {@link TableClient} to use. + * @param engine Instance of {@link Engine} to use. * @param millisSinceEpochUTC timestamp to fetch the snapshot for in milliseconds since the * unix epoch * @return a {@link Snapshot} of the table at the provided timestamp * @throws TableNotFoundException */ public Snapshot getSnapshotForTimestamp( - TableClient tableClient, long millisSinceEpochUTC) throws TableNotFoundException { + Engine engine, long millisSinceEpochUTC) throws TableNotFoundException { long startTimeMillis = System.currentTimeMillis(); long versionToRead = DeltaHistoryManager.getActiveCommitAtTimestamp( - tableClient, logPath, millisSinceEpochUTC); + engine, logPath, millisSinceEpochUTC); logger.info("{}: Took {}ms to fetch version at timestamp {}", tablePath, System.currentTimeMillis() - startTimeMillis, millisSinceEpochUTC); - return getSnapshotAt(tableClient, versionToRead); + return getSnapshotAt(engine, versionToRead); } - public void checkpoint(TableClient tableClient, long version) + public void checkpoint(Engine engine, long version) throws TableNotFoundException, IOException { logger.info("{}: Starting checkpoint for version: {}", tablePath, version); // Get the snapshot corresponding the version - SnapshotImpl snapshot = (SnapshotImpl) getSnapshotAt(tableClient, version); + SnapshotImpl snapshot = (SnapshotImpl) getSnapshotAt(engine, version); // Check if writing to the given table protocol version/features is supported in Kernel validateWriteSupportedTable( - snapshot.getProtocol(), snapshot.getMetadata(), snapshot.getSchema(tableClient)); + snapshot.getProtocol(), snapshot.getMetadata(), snapshot.getSchema(engine)); Path checkpointPath = FileNames.checkpointFileSingular(logPath, version); long numberOfAddFiles = 0; try (CreateCheckpointIterator checkpointDataIter = - snapshot.getCreateCheckpointIterator(tableClient)) { + snapshot.getCreateCheckpointIterator(engine)) { // Write the iterator actions to the checkpoint using the Parquet handler - tableClient.getParquetHandler() + engine.getParquetHandler() .writeParquetFileAtomically( checkpointPath.toString(), checkpointDataIter); @@ -187,7 +186,7 @@ public void checkpoint(TableClient tableClient, long version) new CheckpointMetaData(version, numberOfAddFiles, Optional.empty()); Checkpointer checkpointer = new Checkpointer(logPath); - checkpointer.writeLastCheckpointFile(tableClient, checkpointMetaData); + checkpointer.writeLastCheckpointFile(engine, checkpointMetaData); logger.info("{}: Last checkpoint metadata file is written for version: {}", tablePath, version); @@ -217,11 +216,11 @@ private void registerHint(SnapshotHint newHint) { * Get an iterator of files in the _delta_log directory starting with the startVersion. */ private CloseableIterator listFrom( - TableClient tableClient, + Engine engine, long startVersion) throws IOException { logger.debug("{}: startVersion: {}", tablePath, startVersion); - return tableClient + return engine .getFileSystemClient() .listFrom(FileNames.listingPrefix(logPath, startVersion)); } @@ -243,13 +242,12 @@ private boolean isDeltaCommitOrCheckpointFile(String fileName) { * with the startVersion. Returns None if no files are found or the directory is missing. */ private Optional> listFromOrNone( - TableClient tableClient, + Engine engine, long startVersion) { // LIST the directory, starting from the provided lower bound (treat missing dir as empty). // NOTE: "empty/missing" is _NOT_ equivalent to "contains no useful commit files." try { - CloseableIterator results = listFrom( - tableClient, + CloseableIterator results = listFrom(engine, startVersion); if (results.hasNext()) { return Optional.of(results); @@ -277,7 +275,7 @@ private Optional> listFromOrNone( * None if the listing returned no files at all. */ protected final Optional> listDeltaAndCheckpointFiles( - TableClient tableClient, + Engine engine, long startVersion, Optional versionToLoad) { versionToLoad.ifPresent(v -> @@ -290,8 +288,7 @@ protected final Optional> listDeltaAndCheckpointFiles( )); logger.debug("startVersion: {}, versionToLoad: {}", startVersion, versionToLoad); - return listFromOrNone( - tableClient, + return listFromOrNone(engine, startVersion).map(fileStatusesIter -> { final List output = new ArrayList<>(); @@ -337,28 +334,27 @@ protected final Optional> listDeltaAndCheckpointFiles( * Load the Snapshot for this Delta table at initialization. This method uses the * `lastCheckpoint` file as a hint on where to start listing the transaction log directory. */ - private SnapshotImpl getSnapshotAtInit(TableClient tableClient) + private SnapshotImpl getSnapshotAtInit(Engine engine) throws TableNotFoundException { Checkpointer checkpointer = new Checkpointer(logPath); Optional lastCheckpointOpt = - checkpointer.readLastCheckpointFile(tableClient); + checkpointer.readLastCheckpointFile(engine); if (!lastCheckpointOpt.isPresent()) { logger.warn("{}: Last checkpoint file is missing or corrupted. " + "Will search for the checkpoint files directly.", tablePath); } Optional logSegmentOpt = - getLogSegmentFrom(tableClient, lastCheckpointOpt); + getLogSegmentFrom(engine, lastCheckpointOpt); return logSegmentOpt .map(logSegment -> createSnapshot( - logSegment, - tableClient)) + logSegment, engine)) .orElseThrow(() -> new TableNotFoundException(tablePath.toString())); } private SnapshotImpl createSnapshot( LogSegment initSegment, - TableClient tableClient) { + Engine engine) { final String startingFromStr = initSegment .checkpointVersionOpt .map(v -> format("starting from checkpoint version %s.", v)) @@ -373,8 +369,7 @@ private SnapshotImpl createSnapshot( logPath, tablePath, initSegment.version, - initSegment, - tableClient, + initSegment, engine, initSegment.lastCommitTimestamp, Optional.ofNullable(latestSnapshotHint.get()) ); @@ -386,7 +381,7 @@ private SnapshotImpl createSnapshot( startingFromStr); final SnapshotHint hint = new SnapshotHint( - snapshot.getVersion(tableClient), + snapshot.getVersion(engine), snapshot.getProtocol(), snapshot.getMetadata()); @@ -402,10 +397,9 @@ private SnapshotImpl createSnapshot( * @param startingCheckpoint A checkpoint that we can start our listing from */ private Optional getLogSegmentFrom( - TableClient tableClient, + Engine engine, Optional startingCheckpoint) { - return getLogSegmentForVersion( - tableClient, + return getLogSegmentForVersion(engine, startingCheckpoint.map(x -> x.version), Optional.empty()); } @@ -427,7 +421,7 @@ private Optional getLogSegmentFrom( * startCheckpoint. None, if the delta log directory was missing or empty. */ public Optional getLogSegmentForVersion( - TableClient tableClient, + Engine engine, Optional startCheckpoint, Optional versionToLoad) { // Only use startCheckpoint if it is <= versionToLoad @@ -440,7 +434,7 @@ public Optional getLogSegmentForVersion( long beforeVersion = versionToLoad.get() + 1; long startTimeMillis = System.currentTimeMillis(); startCheckpointToUse = - findLastCompleteCheckpointBefore(tableClient, logPath, beforeVersion) + findLastCompleteCheckpointBefore(engine, logPath, beforeVersion) .map(x -> x.version); logger.info("{}: Took {}ms to load last checkpoint before version {}", @@ -456,15 +450,14 @@ public Optional getLogSegmentForVersion( long startTimeMillis = System.currentTimeMillis(); final Optional> newFiles = - listDeltaAndCheckpointFiles(tableClient, startVersion, versionToLoad); + listDeltaAndCheckpointFiles(engine, startVersion, versionToLoad); logger.info("{}: Took {}ms to list the files after starting checkpoint", tablePath, System.currentTimeMillis() - startTimeMillis); startTimeMillis = System.currentTimeMillis(); try { - return getLogSegmentForVersion( - tableClient, + return getLogSegmentForVersion(engine, startCheckpointToUse, versionToLoad, newFiles); @@ -480,7 +473,7 @@ public Optional getLogSegmentForVersion( * and will then try to construct a new LogSegment using that. */ protected Optional getLogSegmentForVersion( - TableClient tableClient, + Engine engine, Optional startCheckpointOpt, Optional versionToLoadOpt, Optional> filesOpt) { @@ -519,8 +512,7 @@ protected Optional getLogSegmentForVersion( } else if (newFiles.isEmpty()) { // The directory may be deleted and recreated and we may have stale state in our // DeltaLog singleton, so try listing from the first version - return getLogSegmentForVersion( - tableClient, + return getLogSegmentForVersion(engine, Optional.empty(), versionToLoadOpt); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/PartitionUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/PartitionUtils.java index e02b5c3b4c1..2409570767f 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/PartitionUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/PartitionUtils.java @@ -28,9 +28,9 @@ import java.util.stream.IntStream; import static java.util.Arrays.asList; -import io.delta.kernel.client.ExpressionHandler; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.*; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.engine.ExpressionHandler; import io.delta.kernel.expressions.*; import io.delta.kernel.types.*; import static io.delta.kernel.expressions.AlwaysFalse.ALWAYS_FALSE; @@ -191,7 +191,7 @@ private static Expression rewriteColRefOnPartitionValuesParsed( /** * Utility method to rewrite the partition predicate referring to the table schema as predicate * referring to the {@code partitionValues} in scan files read from Delta log. The scan file - * batch is returned by the {@link io.delta.kernel.Scan#getScanFiles(TableClient)}. + * batch is returned by the {@link io.delta.kernel.Scan#getScanFiles(Engine)}. *

* E.g. given predicate on partition columns: * {@code p1 = 'new york' && p2 >= 26} where p1 is of type string and p2 is of int diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala index 0fde3af3826..f9d2580a60e 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala @@ -32,7 +32,7 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil timestamp: Long, expectedVersion: Long): Unit = { val activeCommit = DeltaHistoryManager.getActiveCommitAtTimestamp( - createMockFSListFromTableClient(fileList), + createMockFSListFromEngine(fileList), logPath, timestamp ) @@ -46,7 +46,7 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil expectedErrorMessageContains: String)(implicit classTag: ClassTag[T]): Unit = { val e = intercept[T] { DeltaHistoryManager.getActiveCommitAtTimestamp( - createMockFSListFromTableClient(fileList), + createMockFSListFromEngine(fileList), logPath, timestamp ) @@ -155,7 +155,7 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil // Non-existent path intercept[TableNotFoundException]( DeltaHistoryManager.getActiveCommitAtTimestamp( - createMockFSListFromTableClient(p => throw new FileNotFoundException(p)), + createMockFSListFromEngine(p => throw new FileNotFoundException(p)), logPath, 0 ) @@ -163,7 +163,7 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil // Empty _delta_log directory intercept[TableNotFoundException]( DeltaHistoryManager.getActiveCommitAtTimestamp( - createMockFSListFromTableClient(p => Seq()), + createMockFSListFromEngine(p => Seq()), logPath, 0 ) diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala index e4a3136bebf..7fb1cfe7b16 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala @@ -202,7 +202,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { }.getOrElse((Seq.empty, Seq.empty)) val logSegmentOpt = snapshotManager.getLogSegmentForVersion( - createMockFSListFromTableClient(listFromProvider(deltas ++ checkpointFiles)("/"), + createMockFSListFromEngine(listFromProvider(deltas ++ checkpointFiles)("/"), new MockSidecarParquetHandler(expectedSidecars), new MockSidecarJsonHandler(expectedSidecars)), Optional.empty(), @@ -300,7 +300,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { expectedErrorMessageContains: String = "")(implicit classTag: ClassTag[T]): Unit = { val e = intercept[T] { snapshotManager.getLogSegmentForVersion( - createMockFSListFromTableClient(files), + createMockFSListFromEngine(files), startCheckpoint, versionToLoad ) @@ -424,7 +424,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("getLogSegmentForVersion: empty delta log") { // listDeltaAndCheckpointFiles = Optional.empty() val logSegmentOpt = snapshotManager.getLogSegmentForVersion( - createMockFSListFromTableClient(Seq.empty), + createMockFSListFromEngine(Seq.empty), Optional.empty(), Optional.empty() ) @@ -475,7 +475,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { } for (checkpointV <- Seq(10, 20)) { val logSegmentOpt = snapshotManager.getLogSegmentForVersion( - createMockFSListFromTableClient(listFrom(checkpointV)(_)), + createMockFSListFromEngine(listFrom(checkpointV)(_)), Optional.of(checkpointV), Optional.empty() ) @@ -687,7 +687,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { expectedErrorMessageContains = "Could not find any delta files for version 10" ) val logSegment = snapshotManager.getLogSegmentForVersion( - createMockFSListFromTableClient(fileList), + createMockFSListFromEngine(fileList), Optional.empty(), Optional.empty() ) @@ -833,7 +833,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { val checkpoints = singularCheckpointFileStatuses(validVersions) val deltas = deltaFileStatuses(deltaVersions) val logSegmentOpt = snapshotManager.getLogSegmentForVersion( - createMockFSListFromTableClient(deltas ++ corruptedCheckpoint ++ checkpoints), + createMockFSListFromEngine(deltas ++ corruptedCheckpoint ++ checkpoints), Optional.empty(), Optional.empty() ) @@ -854,7 +854,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("getLogSegmentForVersion: corrupt _last_checkpoint with empty delta log") { // listDeltaAndCheckpointFiles = Optional.empty() val logSegmentOpt = snapshotManager.getLogSegmentForVersion( - createMockFSListFromTableClient(Seq.empty), + createMockFSListFromEngine(Seq.empty), Optional.of(1), Optional.empty() ) diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointerSuite.scala index 3f19e50d827..d4f9bd51c5c 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/checkpoints/CheckpointerSuite.scala @@ -21,7 +21,7 @@ import io.delta.kernel.internal.checkpoints.Checkpointer.findLastCompleteCheckpo import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.util.FileNames.checkpointFileSingular import io.delta.kernel.internal.util.Utils -import io.delta.kernel.test.{BaseMockJsonHandler, MockFileSystemClientUtils, MockTableClientUtils, VectorTestUtils} +import io.delta.kernel.test.{BaseMockJsonHandler, MockFileSystemClientUtils, MockEngineUtils, VectorTestUtils} import io.delta.kernel.types.StructType import io.delta.kernel.utils.{CloseableIterator, FileStatus} import org.scalatest.funsuite.AnyFunSuite @@ -38,7 +38,7 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("load a valid last checkpoint metadata file") { val jsonHandler = new MockLastCheckpointMetadataFileReader(maxFailures = 0) val lastCheckpoint = new Checkpointer(VALID_LAST_CHECKPOINT_FILE_TABLE) - .readLastCheckpointFile(mockTableClient(jsonHandler = jsonHandler)) + .readLastCheckpointFile(mockEngine(jsonHandler = jsonHandler)) assertValidCheckpointMetadata(lastCheckpoint) assert(jsonHandler.currentFailCount == 0) } @@ -46,7 +46,7 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("load a zero-sized last checkpoint metadata file") { val jsonHandler = new MockLastCheckpointMetadataFileReader(maxFailures = 0) val lastCheckpoint = new Checkpointer(ZERO_SIZED_LAST_CHECKPOINT_FILE_TABLE) - .readLastCheckpointFile(mockTableClient(jsonHandler = jsonHandler)) + .readLastCheckpointFile(mockEngine(jsonHandler = jsonHandler)) assert(!lastCheckpoint.isPresent) assert(jsonHandler.currentFailCount == 0) } @@ -54,7 +54,7 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("load an invalid last checkpoint metadata file") { val jsonHandler = new MockLastCheckpointMetadataFileReader(maxFailures = 0) val lastCheckpoint = new Checkpointer(INVALID_LAST_CHECKPOINT_FILE_TABLE) - .readLastCheckpointFile(mockTableClient(jsonHandler = jsonHandler)) + .readLastCheckpointFile(mockEngine(jsonHandler = jsonHandler)) assert(!lastCheckpoint.isPresent) assert(jsonHandler.currentFailCount == 0) } @@ -62,7 +62,7 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("retry last checkpoint metadata loading - succeeds at third attempt") { val jsonHandler = new MockLastCheckpointMetadataFileReader(maxFailures = 2) val lastCheckpoint = new Checkpointer(VALID_LAST_CHECKPOINT_FILE_TABLE) - .readLastCheckpointFile(mockTableClient(jsonHandler = jsonHandler)) + .readLastCheckpointFile(mockEngine(jsonHandler = jsonHandler)) assertValidCheckpointMetadata(lastCheckpoint) assert(jsonHandler.currentFailCount == 2) } @@ -70,7 +70,7 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("retry last checkpoint metadata loading - exceeds max failures") { val jsonHandler = new MockLastCheckpointMetadataFileReader(maxFailures = 4) val lastCheckpoint = new Checkpointer(VALID_LAST_CHECKPOINT_FILE_TABLE) - .readLastCheckpointFile(mockTableClient(jsonHandler = jsonHandler)) + .readLastCheckpointFile(mockEngine(jsonHandler = jsonHandler)) assert(!lastCheckpoint.isPresent) assert(jsonHandler.currentFailCount == 3) // 3 is the max retries } @@ -78,7 +78,7 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("try to load last checkpoint metadata when the file is missing") { val jsonHandler = new MockLastCheckpointMetadataFileReader(maxFailures = 0) val lastCheckpoint = new Checkpointer(LAST_CHECKPOINT_FILE_NOT_FOUND_TABLE) - .readLastCheckpointFile(mockTableClient(jsonHandler = jsonHandler)) + .readLastCheckpointFile(mockEngine(jsonHandler = jsonHandler)) assert(!lastCheckpoint.isPresent) assert(jsonHandler.currentFailCount == 0) } @@ -202,8 +202,8 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { beforeVersion: Long, expCheckpointVersion: Long, expNumFilesListed: Long): Unit = { - val tableClient = createMockFSListFromTableClient(deltaLogFiles) - val result = findLastCompleteCheckpointBeforeHelper(tableClient, logPath, beforeVersion) + val engine = createMockFSListFromEngine(deltaLogFiles) + val result = findLastCompleteCheckpointBeforeHelper(engine, logPath, beforeVersion) assert(result._1.isPresent, s"Checkpoint should be found for version=$beforeVersion") assert( result._1.get().version === expCheckpointVersion, @@ -215,8 +215,8 @@ class CheckpointerSuite extends AnyFunSuite with MockFileSystemClientUtils { deltaLogFiles: Seq[FileStatus], beforeVersion: Long, expNumFilesListed: Long): Unit = { - val tableClient = createMockFSListFromTableClient(deltaLogFiles) - val result = findLastCompleteCheckpointBeforeHelper(tableClient, logPath, beforeVersion) + val engine = createMockFSListFromEngine(deltaLogFiles) + val result = findLastCompleteCheckpointBeforeHelper(engine, logPath, beforeVersion) assert(!result._1.isPresent, s"No checkpoint should be found for version=$beforeVersion") assert(result._2 == expNumFilesListed, s"Invalid number of files listed: $beforeVersion") } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockTableClientUtils.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala similarity index 91% rename from kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockTableClientUtils.scala rename to kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala index 5d8ed140ea2..ed80a5de092 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockTableClientUtils.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala @@ -15,7 +15,7 @@ */ package io.delta.kernel.test -import io.delta.kernel.client._ +import io.delta.kernel.engine._ import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, Row} import io.delta.kernel.expressions.{Column, Expression, ExpressionEvaluator, Predicate, PredicateEvaluator} import io.delta.kernel.types.{DataType, StructType} @@ -26,7 +26,7 @@ import java.util import java.util.Optional /** - * Contains broiler plate code for mocking [[TableClient]] and its sub-interfaces. + * Contains broiler plate code for mocking [[Engine]] and its sub-interfaces. * * A concrete class is created for each sub-interface (e.g. [[FileSystemClient]]) with * default implementation (unsupported). Test suites can override a specific API(s) @@ -40,20 +40,20 @@ import java.util.Optional * } * } * - * val myMockTableClient = mockTableClient(fileSystemClient = myMockFileSystemClient) + * val myMockEngine = mockEngine(fileSystemClient = myMockFileSystemClient) * }}} */ -trait MockTableClientUtils { +trait MockEngineUtils { /** - * Create a mock TableClient with the given components. If a component is not provided, it will + * Create a mock Engine with the given components. If a component is not provided, it will * throw an exception when accessed. */ - def mockTableClient( + def mockEngine( fileSystemClient: FileSystemClient = null, jsonHandler: JsonHandler = null, parquetHandler: ParquetHandler = null, - expressionHandler: ExpressionHandler = null): TableClient = { - new TableClient() { + expressionHandler: ExpressionHandler = null): Engine = { + new Engine() { override def getExpressionHandler: ExpressionHandler = Option(expressionHandler).getOrElse( throw new UnsupportedOperationException("not supported in this test suite")) @@ -102,7 +102,7 @@ trait BaseMockJsonHandler extends JsonHandler { /** * Base class for mocking [[ParquetHandler]] */ -trait BaseMockParquetHandler extends ParquetHandler with MockTableClientUtils { +trait BaseMockParquetHandler extends ParquetHandler with MockEngineUtils { override def readParquetFiles( fileIter: CloseableIterator[FileStatus], physicalSchema: StructType, diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala index ec7fcc18860..00f42d0638a 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala @@ -17,7 +17,7 @@ package io.delta.kernel.test import java.util.UUID -import io.delta.kernel.client._ +import io.delta.kernel.engine._ import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.util.FileNames import io.delta.kernel.internal.util.Utils.toCloseableIterator @@ -31,7 +31,7 @@ import scala.collection.JavaConverters._ * * [[MockListFromFileSystemClient]] - mocks the `listFrom` API within [[FileSystemClient]]. */ -trait MockFileSystemClientUtils extends MockTableClientUtils { +trait MockFileSystemClientUtils extends MockEngineUtils { val dataPath = new Path("/fake/path/to/table/") val logPath = new Path(dataPath, "_delta_log") @@ -86,7 +86,7 @@ trait MockFileSystemClientUtils extends MockTableClientUtils { } } - /* Create input function for createMockTableClient to implement listFrom from a list of + /* Create input function for createMockEngine to implement listFrom from a list of * file statuses. */ def listFromProvider(files: Seq[FileStatus])(filePath: String): Seq[FileStatus] = { @@ -94,34 +94,34 @@ trait MockFileSystemClientUtils extends MockTableClientUtils { } /** - * Create a mock [[TableClient]] to mock the [[FileSystemClient.listFrom]] calls using + * Create a mock [[Engine]] to mock the [[FileSystemClient.listFrom]] calls using * the given contents. The contents are filtered depending upon the list from path prefix. */ - def createMockFSListFromTableClient( + def createMockFSListFromEngine( contents: Seq[FileStatus], parquetHandler: ParquetHandler, - jsonHandler: JsonHandler): TableClient = { - mockTableClient(fileSystemClient = + jsonHandler: JsonHandler): Engine = { + mockEngine(fileSystemClient = new MockListFromFileSystemClient(listFromProvider(contents)), parquetHandler = parquetHandler, jsonHandler = jsonHandler) } /** - * Create a mock [[TableClient]] to mock the [[FileSystemClient.listFrom]] calls using + * Create a mock [[Engine]] to mock the [[FileSystemClient.listFrom]] calls using * the given contents. The contents are filtered depending upon the list from path prefix. */ - def createMockFSListFromTableClient(contents: Seq[FileStatus]): TableClient = { - mockTableClient(fileSystemClient = + def createMockFSListFromEngine(contents: Seq[FileStatus]): Engine = { + mockEngine(fileSystemClient = new MockListFromFileSystemClient(listFromProvider(contents))) } /** - * Create a mock [[TableClient]] to mock the [[FileSystemClient.listFrom]] calls using + * Create a mock [[Engine]] to mock the [[FileSystemClient.listFrom]] calls using * [[MockListFromFileSystemClient]]. */ - def createMockFSListFromTableClient(listFromProvider: String => Seq[FileStatus]): TableClient = { - mockTableClient(fileSystemClient = new MockListFromFileSystemClient(listFromProvider)) + def createMockFSListFromEngine(listFromProvider: String => Seq[FileStatus]): Engine = { + mockEngine(fileSystemClient = new MockListFromFileSystemClient(listFromProvider)) } } diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultTableClient.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultEngine.java similarity index 72% rename from kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultTableClient.java rename to kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultEngine.java index 972fd06ef99..13a920dc38d 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultTableClient.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultEngine.java @@ -13,20 +13,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.defaults.client; +package io.delta.kernel.defaults.engine; import org.apache.hadoop.conf.Configuration; -import io.delta.kernel.client.*; +import io.delta.kernel.engine.*; /** - * Default implementation of {@link TableClient} based on Hadoop APIs. + * Default implementation of {@link Engine} based on Hadoop APIs. */ -public class DefaultTableClient - implements TableClient { +public class DefaultEngine + implements Engine { private final Configuration hadoopConf; - protected DefaultTableClient(Configuration hadoopConf) { + protected DefaultEngine(Configuration hadoopConf) { this.hadoopConf = hadoopConf; } @@ -51,12 +51,12 @@ public ParquetHandler getParquetHandler() { } /** - * Create an instance of {@link DefaultTableClient}. + * Create an instance of {@link DefaultEngine}. * * @param hadoopConf Hadoop configuration to use. - * @return an instance of {@link DefaultTableClient}. + * @return an instance of {@link DefaultEngine}. */ - public static DefaultTableClient create(Configuration hadoopConf) { - return new DefaultTableClient(hadoopConf); + public static DefaultEngine create(Configuration hadoopConf) { + return new DefaultEngine(hadoopConf); } } diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultExpressionHandler.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultExpressionHandler.java similarity index 97% rename from kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultExpressionHandler.java rename to kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultExpressionHandler.java index 4eeedebdd06..02afa50c3f3 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultExpressionHandler.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultExpressionHandler.java @@ -13,15 +13,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.defaults.client; +package io.delta.kernel.defaults.engine; import java.util.Arrays; import java.util.Optional; import static java.lang.String.format; import static java.util.Objects.requireNonNull; -import io.delta.kernel.client.ExpressionHandler; import io.delta.kernel.data.ColumnVector; +import io.delta.kernel.engine.ExpressionHandler; import io.delta.kernel.expressions.Expression; import io.delta.kernel.expressions.ExpressionEvaluator; import io.delta.kernel.expressions.Predicate; diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultFileSystemClient.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultFileSystemClient.java similarity index 97% rename from kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultFileSystemClient.java rename to kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultFileSystemClient.java index 9924b4b227d..c2bfa7a80bc 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultFileSystemClient.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultFileSystemClient.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.defaults.client; +package io.delta.kernel.defaults.engine; import java.io.*; @@ -22,8 +22,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import io.delta.kernel.client.FileReadRequest; -import io.delta.kernel.client.FileSystemClient; +import io.delta.kernel.engine.FileReadRequest; +import io.delta.kernel.engine.FileSystemClient; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.utils.FileStatus; diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultJsonHandler.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultJsonHandler.java similarity index 99% rename from kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultJsonHandler.java rename to kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultJsonHandler.java index 69d572661dc..ea793bcaa69 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultJsonHandler.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultJsonHandler.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.defaults.client; +package io.delta.kernel.defaults.engine; import java.io.*; import java.nio.charset.StandardCharsets; @@ -26,8 +26,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; -import io.delta.kernel.client.JsonHandler; import io.delta.kernel.data.*; +import io.delta.kernel.engine.JsonHandler; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.types.*; import io.delta.kernel.utils.CloseableIterator; diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultParquetHandler.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultParquetHandler.java similarity index 98% rename from kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultParquetHandler.java rename to kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultParquetHandler.java index dadb013e310..b39e3b4a4a3 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/DefaultParquetHandler.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultParquetHandler.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.defaults.client; +package io.delta.kernel.defaults.engine; import java.io.IOException; import java.io.UncheckedIOException; @@ -24,9 +24,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; -import io.delta.kernel.client.ParquetHandler; import io.delta.kernel.data.ColumnarBatch; import io.delta.kernel.data.FilteredColumnarBatch; +import io.delta.kernel.engine.ParquetHandler; import io.delta.kernel.expressions.Column; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.types.StructType; diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/package-info.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/package-info.java similarity index 74% rename from kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/package-info.java rename to kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/package-info.java index 550e47b1b5a..abf172f48f0 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/client/package-info.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/package-info.java @@ -15,7 +15,7 @@ */ /** - * Default implementation of {@link io.delta.kernel.client.TableClient} interface and - * the sub-interfaces exposed by the {@link io.delta.kernel.client.TableClient}. + * Default implementation of {@link io.delta.kernel.engine.Engine} interface and + * the sub-interfaces exposed by the {@link io.delta.kernel.engine.Engine}. */ -package io.delta.kernel.defaults.client; +package io.delta.kernel.defaults.engine; diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/DefaultTableClientErrors.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/DefaultEngineErrors.java similarity index 88% rename from kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/DefaultTableClientErrors.java rename to kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/DefaultEngineErrors.java index 0f8bc1e419e..2b08ef04dc1 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/DefaultTableClientErrors.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/DefaultEngineErrors.java @@ -17,9 +17,9 @@ import static java.lang.String.format; -public class DefaultTableClientErrors { +public class DefaultEngineErrors { - // TODO update to be table client exception with future exception framework + // TODO update to be engine exception with future exception framework // (see delta-io/delta#2231) public static IllegalArgumentException canNotInstantiateLogStore(String logStoreClassName) { return new IllegalArgumentException( diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/data/DefaultRowBasedColumnarBatch.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/data/DefaultRowBasedColumnarBatch.java index 948697b446b..a79f7ac7941 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/data/DefaultRowBasedColumnarBatch.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/data/DefaultRowBasedColumnarBatch.java @@ -29,7 +29,7 @@ /** * {@link ColumnarBatch} wrapper around list of {@link Row} objects. - * TODO: We should change the {@link io.delta.kernel.defaults.client.DefaultJsonHandler} to + * TODO: We should change the {@link io.delta.kernel.defaults.engine.DefaultJsonHandler} to * generate data in true columnar format than wrapping a set of rows with a columnar batch * interface. */ diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluator.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluator.java index f801b67b34d..47d56a532ac 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluator.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluator.java @@ -21,9 +21,9 @@ import static java.lang.String.format; import static java.util.Objects.requireNonNull; -import io.delta.kernel.client.ExpressionHandler; import io.delta.kernel.data.ColumnVector; import io.delta.kernel.data.ColumnarBatch; +import io.delta.kernel.engine.ExpressionHandler; import io.delta.kernel.expressions.*; import io.delta.kernel.types.*; diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/ImplicitCastExpression.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/ImplicitCastExpression.java index 63dca50a748..6358ae1d956 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/ImplicitCastExpression.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/ImplicitCastExpression.java @@ -24,7 +24,7 @@ import io.delta.kernel.expressions.Expression; import io.delta.kernel.types.DataType; -import io.delta.kernel.defaults.client.DefaultExpressionHandler; +import io.delta.kernel.defaults.engine.DefaultExpressionHandler; /** * An implicit cast expression to convert the input type to another given type. Here is the valid diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/logstore/LogStoreProvider.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/logstore/LogStoreProvider.java index 812b5750d80..9f4d9738230 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/logstore/LogStoreProvider.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/logstore/LogStoreProvider.java @@ -20,7 +20,7 @@ import io.delta.storage.*; import org.apache.hadoop.conf.Configuration; -import io.delta.kernel.defaults.internal.DefaultTableClientErrors; +import io.delta.kernel.defaults.internal.DefaultEngineErrors; /** * Utility class to provide the correct {@link LogStore} based on the scheme of the path. @@ -66,7 +66,7 @@ public static LogStore getLogStore(Configuration hadoopConf, String scheme) { .getConstructor(Configuration.class) .newInstance(hadoopConf); } catch (Exception e) { - throw DefaultTableClientErrors.canNotInstantiateLogStore(classNameFromConfig); + throw DefaultEngineErrors.canNotInstantiateLogStore(classNameFromConfig); } } @@ -85,7 +85,7 @@ public static LogStore getLogStore(Configuration hadoopConf, String scheme) { .getConstructor(Configuration.class) .newInstance(hadoopConf); } catch (Exception e) { - throw DefaultTableClientErrors.canNotInstantiateLogStore(defaultClassName); + throw DefaultEngineErrors.canNotInstantiateLogStore(defaultClassName); } } diff --git a/kernel/kernel-defaults/src/test/java/io/delta/kernel/defaults/benchmarks/BenchmarkParallelCheckpointReading.java b/kernel/kernel-defaults/src/test/java/io/delta/kernel/defaults/benchmarks/BenchmarkParallelCheckpointReading.java index 989b0ee2b8a..cdfb58d94e0 100644 --- a/kernel/kernel-defaults/src/test/java/io/delta/kernel/defaults/benchmarks/BenchmarkParallelCheckpointReading.java +++ b/kernel/kernel-defaults/src/test/java/io/delta/kernel/defaults/benchmarks/BenchmarkParallelCheckpointReading.java @@ -26,9 +26,9 @@ import org.openjdk.jmh.infra.Blackhole; import io.delta.kernel.*; -import io.delta.kernel.client.ParquetHandler; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.*; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.engine.ParquetHandler; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterator; @@ -36,8 +36,8 @@ import io.delta.kernel.internal.util.Utils; -import io.delta.kernel.defaults.client.DefaultParquetHandler; -import io.delta.kernel.defaults.client.DefaultTableClient; +import io.delta.kernel.defaults.engine.DefaultEngine; +import io.delta.kernel.defaults.engine.DefaultParquetHandler; import io.delta.kernel.defaults.internal.parquet.ParquetFileReader; @@ -112,18 +112,18 @@ public static class BenchmarkData { @Benchmark @BenchmarkMode(Mode.AverageTime) public void benchmark(BenchmarkData benchmarkData, Blackhole blackhole) throws Exception { - TableClient tableClient = createTableClient(benchmarkData.parallelReaderCount); - Table table = Table.forPath(tableClient, testTablePath); + Engine engine = createEngine(benchmarkData.parallelReaderCount); + Table table = Table.forPath(engine, testTablePath); - Snapshot snapshot = table.getLatestSnapshot(tableClient); - ScanBuilder scanBuilder = snapshot.getScanBuilder(tableClient); + Snapshot snapshot = table.getLatestSnapshot(engine); + ScanBuilder scanBuilder = snapshot.getScanBuilder(engine); Scan scan = scanBuilder.build(); // Scan state is not used, but get it so that we simulate the real use case. - Row row = scan.getScanState(tableClient); + Row row = scan.getScanState(engine); blackhole.consume(row); // To avoid dead code elimination by the JIT compiler long fileSize = 0; - try (CloseableIterator batchIter = scan.getScanFiles(tableClient)) { + try (CloseableIterator batchIter = scan.getScanFiles(engine)) { while (batchIter.hasNext()) { FilteredColumnarBatch batch = batchIter.next(); try (CloseableIterator rowIter = batch.getRows()) { @@ -144,13 +144,13 @@ public static void main(String[] args) throws Exception { org.openjdk.jmh.Main.main(args); } - private static TableClient createTableClient(int numberOfParallelThreads) { + private static Engine createEngine(int numberOfParallelThreads) { Configuration hadoopConf = new Configuration(); if (numberOfParallelThreads <= 0) { - return DefaultTableClient.create(hadoopConf); + return DefaultEngine.create(hadoopConf); } - return new DefaultTableClient(hadoopConf) { + return new DefaultEngine(hadoopConf) { @Override public ParquetHandler getParquetHandler() { return new ParallelParquetHandler(hadoopConf, numberOfParallelThreads); diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala index b1f6e3b3f08..0391c171d83 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala @@ -19,7 +19,7 @@ import java.io.File import scala.collection.JavaConverters._ -import io.delta.kernel.defaults.client.DefaultTableClient +import io.delta.kernel.defaults.engine.DefaultEngine import io.delta.kernel.defaults.utils.{TestRow, TestUtils} import io.delta.kernel.internal.checkpoints.CheckpointInstance import io.delta.kernel.internal.{InternalScanFileUtils, SnapshotImpl} @@ -39,7 +39,7 @@ import org.apache.spark.sql.types.{BooleanType, IntegerType, LongType, MapType, class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils { private final val supportedFileFormats = Seq("json", "parquet") - override lazy val defaultTableClient = DefaultTableClient.create(new Configuration() { + override lazy val defaultEngine = DefaultEngine.create(new Configuration() { { // Set the batch sizes to small so that we get to test the multiple batch scenarios. set("delta.kernel.default.parquet.reader.batch-size", "2"); @@ -78,7 +78,7 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils { snapshotFromSpark.protocol.readerFeatureNames) assert(snapshotImpl.getProtocol.getWriterFeatures.asScala.toSet == snapshotFromSpark.protocol.writerFeatureNames) - assert(snapshot.getVersion(defaultTableClient) == snapshotFromSpark.version) + assert(snapshot.getVersion(defaultEngine) == snapshotFromSpark.version) // Validate that snapshot read from most recent checkpoint. For most cases, given a checkpoint // interval of 2, this will be the most recent even version. @@ -93,7 +93,7 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils { // Validate AddFiles from sidecars found against Spark connector. - val scan = snapshot.getScanBuilder(defaultTableClient).build() + val scan = snapshot.getScanBuilder(defaultEngine).build() val foundFiles = collectScanFileRows(scan).map(InternalScanFileUtils.getAddFileStatus).map( _.getPath.split('/').last).toSet diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala index 704bfc4c9fe..b413afddb99 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala @@ -16,24 +16,25 @@ package io.delta.kernel.defaults import io.delta.golden.GoldenTableUtils.goldenTablePath -import io.delta.kernel.client.TableClient -import io.delta.kernel.defaults.client.DefaultTableClient import io.delta.kernel.defaults.utils.{TestRow, TestUtils} import io.delta.kernel.{CheckpointAlreadyExistsException, Table, TableNotFoundException} import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path + import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate import org.apache.spark.sql.delta.actions.{AddFile, Metadata, RemoveFile} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.{DeltaLog, VersionNotFoundException} import org.apache.spark.sql.types.{IntegerType, StructType} import org.scalatest.funsuite.AnyFunSuite - import java.io.File +import io.delta.kernel.defaults.engine.DefaultEngine +import io.delta.kernel.engine.Engine + /** - * Test suite for `io.delta.kernel.Table.checkpoint(TableClient, version)` + * Test suite for `io.delta.kernel.Table.checkpoint(engine, version)` */ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { @@ -44,7 +45,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { Seq(true, false).foreach { includeRemoves => val testMsgUpdate = if (includeRemoves) " and removes" else "" test(s"commits containing adds$testMsgUpdate, and no previous checkpoint") { - withTempDirAndTableClient { (tablePath, tc) => + withTempDirAndEngine { (tablePath, tc) => addData(tablePath, alternateBetweenAddsAndRemoves = includeRemoves, numberIter = 10) // before creating checkpoint, read and save the expected results using Spark @@ -79,7 +80,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { test(s"commits containing adds$testMsgUpdate, and a previous checkpoint " + s"created using Spark (actions/perfile): $sparkCheckpointActionPerFile") { - withTempDirAndTableClient { (tablePath, tc) => + withTempDirAndEngine { (tablePath, tc) => addData(tablePath, includeRemoves, numberIter = 6) // checkpoint using Spark @@ -110,7 +111,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { } test("commits with metadata updates") { - withTempDirAndTableClient { (tablePath, tc) => + withTempDirAndEngine { (tablePath, tc) => addData(path = tablePath, alternateBetweenAddsAndRemoves = true, numberIter = 16) // makes the latest table version 16 @@ -141,7 +142,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { } test("commits with protocol updates") { - withTempDirAndTableClient { (tablePath, tc) => + withTempDirAndEngine { (tablePath, tc) => addData(path = tablePath, alternateBetweenAddsAndRemoves = true, numberIter = 16) spark.sql( @@ -175,7 +176,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { } test("commits with set transactions") { - withTempDirAndTableClient { (tablePath, tc) => + withTempDirAndEngine { (tablePath, tc) => def idempotentAppend(appId: String, version: Int): Unit = { spark.range(end = 10).repartition(2).write.format("delta") .option("txnAppId", appId) @@ -220,7 +221,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { Seq(None, Some("2 days"), Some("0 days")).foreach { retentionInterval => test(s"checkpoint contains all not expired tombstones: $retentionInterval") { - withTempDirAndTableClient { (tablePath, tc) => + withTempDirAndEngine { (tablePath, tc) => def addFile(path: String): AddFile = AddFile( path = path, partitionValues = Map.empty, @@ -296,7 +297,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { } test("try creating checkpoint on a non-existent table") { - withTempDirAndTableClient { (path, tc) => + withTempDirAndEngine { (path, tc) => Seq(0, 1, 2).foreach { checkpointVersion => val ex = intercept[TableNotFoundException] { kernelCheckpoint(tc, path, checkpointVersion) @@ -308,7 +309,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { test("try creating checkpoint at version that already has a " + "checkpoint or a version that doesn't exist") { - withTempDirAndTableClient { (path, tc) => + withTempDirAndEngine { (path, tc) => for (_ <- 0 to 3) { appendCommit(path) } @@ -328,7 +329,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { } test("create a checkpoint on a existing table") { - withTempDirAndTableClient { (tablePath, tc) => + withTempDirAndEngine { (tablePath, tc) => copyTable("time-travel-start-start20-start40", tablePath) // before creating checkpoint, read and save the expected results using Spark @@ -342,7 +343,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { } test("try create a checkpoint on a unsupported table feature table") { - withTempDirAndTableClient { (tablePath, tc) => + withTempDirAndEngine { (tablePath, tc) => copyTable("dv-with-columnmapping", tablePath) val ex2 = intercept[Exception] { @@ -379,7 +380,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { } } - def kernelCheckpoint(tc: TableClient, tablePath: String, checkpointVersion: Long): Unit = { + def kernelCheckpoint(tc: Engine, tablePath: String, checkpointVersion: Long): Unit = { Table.forPath(tc, tablePath).checkpoint(tc, checkpointVersion) } @@ -428,15 +429,15 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { s"Cannot time travel Delta table to version ${beforeVersion - 1}")) } - def withTempDirAndTableClient(f: (String, TableClient) => Unit): Unit = { - val tableClient = DefaultTableClient.create(new Configuration() { + def withTempDirAndEngine(f: (String, Engine) => Unit): Unit = { + val engine = DefaultEngine.create(new Configuration() { { // Set the batch sizes to small so that we get to test the multiple batch scenarios. set("delta.kernel.default.parquet.reader.batch-size", "200"); set("delta.kernel.default.json.reader.batch-size", "200"); } }) - withTempDir { dir => f(dir.getAbsolutePath, tableClient) } + withTempDir { dir => f(dir.getAbsolutePath, engine) } } def checkpointFilePath(tablePath: String, checkpointVersion: Long): String = { diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeletionVectorSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeletionVectorSuite.scala index ffb36c56cfe..fb99caf38d8 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeletionVectorSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeletionVectorSuite.scala @@ -50,7 +50,7 @@ class DeletionVectorSuite extends AnyFunSuite with TestUtils { checkTable( path = goldenTablePath("dv-partitioned-with-checkpoint"), expectedAnswer = expectedResult.map(TestRow.fromTuple(_)), - tableClient = defaultTableClient + engine = defaultEngine ) } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala index a071338226d..ab278d7204f 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala @@ -189,7 +189,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { test("invalid path") { val invalidPath = "/path/to/non-existent-directory" - val table = Table.forPath(defaultTableClient, invalidPath) + val table = Table.forPath(defaultEngine, invalidPath) def expectTableNotFoundException(fn: () => Unit): Unit = { val ex = intercept[TableNotFoundException] { @@ -198,11 +198,11 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { assert(ex.getMessage().contains(s"Delta table at path `file:$invalidPath` is not found")) } - expectTableNotFoundException(() => table.getLatestSnapshot(defaultTableClient)) + expectTableNotFoundException(() => table.getLatestSnapshot(defaultEngine)) expectTableNotFoundException(() => - table.getSnapshotAsOfTimestamp(defaultTableClient, 1)) + table.getSnapshotAsOfTimestamp(defaultEngine, 1)) expectTableNotFoundException(() => - table.getSnapshotAsOfVersion(defaultTableClient, 1)) + table.getSnapshotAsOfVersion(defaultEngine, 1)) } test("table deleted after the `Table` creation") { @@ -211,11 +211,11 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { val target = new File(temp.getCanonicalPath) FileUtils.copyDirectory(source, target) - val table = Table.forPath(defaultTableClient, target.getCanonicalPath) + val table = Table.forPath(defaultEngine, target.getCanonicalPath) // delete the table and try to get the snapshot. Expect a failure. FileUtils.deleteDirectory(target) val ex = intercept[TableNotFoundException] { - table.getLatestSnapshot(defaultTableClient) + table.getLatestSnapshot(defaultEngine) } assert(ex.getMessage.contains( s"Delta table at path `file:${target.getCanonicalPath}` is not found")) @@ -264,8 +264,8 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { val path = "file:" + goldenTablePath("data-reader-partition-values") // for now we don't support timestamp type partition columns so remove from read columns - val readCols = Table.forPath(defaultTableClient, path).getLatestSnapshot(defaultTableClient) - .getSchema(defaultTableClient) + val readCols = Table.forPath(defaultEngine, path).getLatestSnapshot(defaultEngine) + .getSchema(defaultEngine) .withoutField("as_timestamp") .fields() .asScala @@ -582,7 +582,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { test("table protocol version greater than reader protocol version") { val e = intercept[Exception] { latestSnapshot(goldenTablePath("deltalog-invalid-protocol-version")) - .getScanBuilder(defaultTableClient) + .getScanBuilder(defaultEngine) .build() } assert(e.getMessage.contains("Unsupported reader protocol version")) @@ -624,8 +624,8 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { ) // Cannot read a version that does not exist val e = intercept[RuntimeException] { - Table.forPath(defaultTableClient, path) - .getSnapshotAsOfVersion(defaultTableClient, 11) + Table.forPath(defaultEngine, path) + .getSnapshotAsOfVersion(defaultEngine, 11) } assert(e.getMessage.contains( "Trying to load a non-existent version 11. The latest version available is 10")) @@ -657,8 +657,8 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { // Cannot read a version that has been truncated val e = intercept[RuntimeException] { - Table.forPath(defaultTableClient, tablePath) - .getSnapshotAsOfVersion(defaultTableClient, 9) + Table.forPath(defaultEngine, tablePath) + .getSnapshotAsOfVersion(defaultEngine, 9) } assert(e.getMessage.contains("Unable to reconstruct state at version 9")) // Can read version 10 @@ -793,8 +793,8 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { withTempDir { dir => new File(dir, "_delta_log").mkdirs() intercept[TableNotFoundException] { - Table.forPath(defaultTableClient, dir.getCanonicalPath) - .getSnapshotAsOfTimestamp(defaultTableClient, 0L) + Table.forPath(defaultEngine, dir.getCanonicalPath) + .getSnapshotAsOfTimestamp(defaultEngine, 0L) } } } @@ -802,8 +802,8 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { test("getSnapshotAtTimestamp: empty folder no _delta_log dir") { withTempDir { dir => intercept[TableNotFoundException] { - Table.forPath(defaultTableClient, dir.getCanonicalPath) - .getSnapshotAsOfTimestamp(defaultTableClient, 0L) + Table.forPath(defaultEngine, dir.getCanonicalPath) + .getSnapshotAsOfTimestamp(defaultEngine, 0L) } } } @@ -812,8 +812,8 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { withTempDir { dir => spark.range(20).write.format("parquet").mode("overwrite").save(dir.getCanonicalPath) intercept[TableNotFoundException] { - Table.forPath(defaultTableClient, dir.getCanonicalPath) - .getSnapshotAsOfTimestamp(defaultTableClient, 0L) + Table.forPath(defaultEngine, dir.getCanonicalPath) + .getSnapshotAsOfTimestamp(defaultEngine, 0L) } } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayMetricsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayMetricsSuite.scala index 51193d3ac90..39ca5de76c7 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayMetricsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayMetricsSuite.scala @@ -18,9 +18,9 @@ package io.delta.kernel.defaults import java.io.File import io.delta.kernel.Table -import io.delta.kernel.client.{ExpressionHandler, FileSystemClient, TableClient} +import io.delta.kernel.engine.{ExpressionHandler, FileSystemClient, Engine} import io.delta.kernel.data.ColumnarBatch -import io.delta.kernel.defaults.client.{DefaultJsonHandler, DefaultParquetHandler, DefaultTableClient} +import io.delta.kernel.defaults.engine.{DefaultJsonHandler, DefaultParquetHandler, DefaultEngine} import io.delta.kernel.expressions.Predicate import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.util.FileNames @@ -47,59 +47,59 @@ class LogReplayMetricsSuite extends QueryTest // Test Helper Methods // ///////////////////////// - private def withTempDirAndTableClient(f: (File, MetricsTableClient) => Unit): Unit = { - val tableClient = new MetricsTableClient(new Configuration() { + private def withTempDirAndEngine(f: (File, MetricsEngine) => Unit): Unit = { + val engine = new MetricsEngine(new Configuration() { { // Set the batch sizes to small so that we get to test the multiple batch scenarios. set("delta.kernel.default.parquet.reader.batch-size", "2"); set("delta.kernel.default.json.reader.batch-size", "2"); } }) - withTempDir { dir => f(dir, tableClient) } + withTempDir { dir => f(dir, engine) } } private def loadPandMCheckMetrics( - tableClient: MetricsTableClient, + engine: MetricsEngine, table: Table, expJsonVersionsRead: Seq[Long], expParquetVersionsRead: Seq[Long], expParquetReadSetSizes: Seq[Long] = Nil): Unit = { - tableClient.resetMetrics() - table.getLatestSnapshot(tableClient).getSchema(tableClient) + engine.resetMetrics() + table.getLatestSnapshot(engine).getSchema(engine) assertMetrics( - tableClient, + engine, expJsonVersionsRead, expParquetVersionsRead, expParquetReadSetSizes) } private def loadScanFilesCheckMetrics( - tableClient: MetricsTableClient, + engine: MetricsEngine, table: Table, expJsonVersionsRead: Seq[Long], expParquetVersionsRead: Seq[Long], expParquetReadSetSizes: Seq[Long]): Unit = { - tableClient.resetMetrics() - val scan = table.getLatestSnapshot(tableClient).getScanBuilder(tableClient).build() + engine.resetMetrics() + val scan = table.getLatestSnapshot(engine).getScanBuilder(engine).build() // get all scan files and iterate through them to trigger the metrics collection - val scanFiles = scan.getScanFiles(tableClient) + val scanFiles = scan.getScanFiles(engine) while (scanFiles.hasNext) scanFiles.next() assertMetrics( - tableClient, + engine, expJsonVersionsRead, expParquetVersionsRead, expParquetReadSetSizes) } def assertMetrics( - tableClient: MetricsTableClient, + engine: MetricsEngine, expJsonVersionsRead: Seq[Long], expParquetVersionsRead: Seq[Long], expParquetReadSetSizes: Seq[Long]): Unit = { - val actualJsonVersionsRead = tableClient.getJsonHandler.getVersionsRead - val actualParquetVersionsRead = tableClient.getParquetHandler.getVersionsRead + val actualJsonVersionsRead = engine.getJsonHandler.getVersionsRead + val actualParquetVersionsRead = engine.getParquetHandler.getVersionsRead assert( actualJsonVersionsRead === expJsonVersionsRead, s"Expected to read json versions " + @@ -111,7 +111,7 @@ class LogReplayMetricsSuite extends QueryTest ) if (expParquetReadSetSizes.nonEmpty) { - val actualParquetReadSetSizes = tableClient.getParquetHandler.checkpointReadRequestSizes + val actualParquetReadSetSizes = engine.getParquetHandler.checkpointReadRequestSizes assert( actualParquetReadSetSizes === expParquetReadSetSizes, s"Expected parquet read set sizes " + s"$expParquetReadSetSizes but read $actualParquetReadSetSizes" @@ -133,7 +133,7 @@ class LogReplayMetricsSuite extends QueryTest /////////// test("no hint, no checkpoint, reads all files") { - withTempDirAndTableClient { (dir, tc) => + withTempDirAndEngine { (dir, tc) => val path = dir.getAbsolutePath for (_ <- 0 to 9) { appendCommit(path) } @@ -144,7 +144,7 @@ class LogReplayMetricsSuite extends QueryTest } test("no hint, existing checkpoint, reads all files up to that checkpoint") { - withTempDirAndTableClient { (dir, tc) => + withTempDirAndEngine { (dir, tc) => val path = dir.getAbsolutePath for (_ <- 0 to 14) { appendCommit(path) } @@ -155,7 +155,7 @@ class LogReplayMetricsSuite extends QueryTest } test("no hint, existing checkpoint, newer P & M update, reads up to P & M commit") { - withTempDirAndTableClient { (dir, tc) => + withTempDirAndEngine { (dir, tc) => val path = dir.getAbsolutePath for (_ <- 0 to 12) { appendCommit(path) } @@ -177,7 +177,7 @@ class LogReplayMetricsSuite extends QueryTest } test("hint with no new commits, should read no files") { - withTempDirAndTableClient { (dir, tc) => + withTempDirAndEngine { (dir, tc) => val path = dir.getAbsolutePath for (_ <- 0 to 14) { @@ -195,7 +195,7 @@ class LogReplayMetricsSuite extends QueryTest } test("hint with no P or M updates") { - withTempDirAndTableClient { (dir, tc) => + withTempDirAndEngine { (dir, tc) => val path = dir.getAbsolutePath for (_ <- 0 to 14) { appendCommit(path) } @@ -226,7 +226,7 @@ class LogReplayMetricsSuite extends QueryTest } test("hint with a P or M update") { - withTempDirAndTableClient { (dir, tc) => + withTempDirAndEngine { (dir, tc) => val path = dir.getAbsolutePath for (_ <- 0 to 3) { appendCommit(path) } @@ -264,7 +264,7 @@ class LogReplayMetricsSuite extends QueryTest } test("read a table with multi-part checkpoint") { - withTempDirAndTableClient { (dir, tc) => + withTempDirAndEngine { (dir, tc) => val path = dir.getAbsolutePath for (_ <- 0 to 14) { appendCommit(path) } @@ -300,9 +300,9 @@ class LogReplayMetricsSuite extends QueryTest // Helper Classes // //////////////////// -/** A table client that records the Delta commit (.json) and checkpoint (.parquet) files read */ -class MetricsTableClient(config: Configuration) extends TableClient { - private val impl = DefaultTableClient.create(config) +/** An engine that records the Delta commit (.json) and checkpoint (.parquet) files read */ +class MetricsEngine(config: Configuration) extends Engine { + private val impl = DefaultEngine.create(config) private val jsonHandler = new MetricsJsonHandler(config) private val parquetHandler = new MetricsParquetHandler(config) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala index d44a9567323..b8d09fcebf2 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala @@ -16,23 +16,23 @@ package io.delta.kernel.defaults import java.io.File +import java.util.Optional + import scala.collection.JavaConverters._ import io.delta.golden.GoldenTableUtils.goldenTablePath import org.scalatest.funsuite.AnyFunSuite import org.apache.hadoop.conf.Configuration + import io.delta.kernel.types.{LongType, StructType} import io.delta.kernel.internal.{InternalScanFileUtils, SnapshotImpl} import io.delta.kernel.internal.data.ScanStateRow -import io.delta.kernel.defaults.client.DefaultTableClient +import io.delta.kernel.defaults.engine.DefaultEngine import io.delta.kernel.defaults.utils.{TestRow, TestUtils} - import io.delta.kernel.Table -import java.util.Optional - class LogReplaySuite extends AnyFunSuite with TestUtils { - override lazy val defaultTableClient = DefaultTableClient.create(new Configuration() {{ + override lazy val defaultEngine = DefaultEngine.create(new Configuration() {{ // Set the batch sizes to small so that we get to test the multiple batch scenarios. set("delta.kernel.default.parquet.reader.batch-size", "2"); set("delta.kernel.default.json.reader.batch-size", "2"); @@ -87,7 +87,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test(s"missing $action should fail") { val path = goldenTablePath(s"deltalog-state-reconstruction-without-$action") val e = intercept[IllegalStateException] { - latestSnapshot(path).getSchema(defaultTableClient) + latestSnapshot(path).getSchema(defaultEngine) } assert(e.getMessage.contains(s"No $action found")) } @@ -100,7 +100,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test(s"missing $action should fail missing from checkpoint") { val path = goldenTablePath(s"deltalog-state-reconstruction-from-checkpoint-missing-$action") val e = intercept[IllegalStateException] { - latestSnapshot(path).getSchema(defaultTableClient) + latestSnapshot(path).getSchema(defaultEngine) } assert(e.getMessage.contains(s"No $action found")) } @@ -109,11 +109,11 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test("fetches the latest protocol and metadata") { val path = goldenTablePath("log-replay-latest-metadata-protocol") val snapshot = latestSnapshot(path) - val scanStateRow = snapshot.getScanBuilder(defaultTableClient).build() - .getScanState(defaultTableClient) + val scanStateRow = snapshot.getScanBuilder(defaultEngine).build() + .getScanState(defaultEngine) // schema is updated - assert(ScanStateRow.getLogicalSchema(defaultTableClient, scanStateRow) + assert(ScanStateRow.getLogicalSchema(defaultEngine, scanStateRow) .fieldNames().asScala.toSet == Set("col1", "col2") ) @@ -126,8 +126,8 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test("standalone DeltaLogSuite: 'checkpoint'") { val path = goldenTablePath("checkpoint") val snapshot = latestSnapshot(path) - assert(snapshot.getVersion(defaultTableClient) == 14) - val scan = snapshot.getScanBuilder(defaultTableClient).build() + assert(snapshot.getVersion(defaultEngine) == 14) + val scan = snapshot.getScanBuilder(defaultEngine).build() assert(collectScanFileRows(scan).length == 1) } @@ -144,9 +144,9 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { expectedFiles: Array[File], expectedVersion: Int): Unit = { val snapshot = latestSnapshot(tablePath) - assert(snapshot.getVersion(defaultTableClient) == expectedVersion) + assert(snapshot.getVersion(defaultEngine) == expectedVersion) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(defaultTableClient).build()) + snapshot.getScanBuilder(defaultEngine).build()) assert(scanFileRows.length == expectedFiles.length) val scanFilePaths = scanFileRows .map(InternalScanFileUtils.getAddFileStatus) @@ -199,9 +199,9 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { // Repartition into 2 files withGoldenTable("snapshot-repartitioned") { tablePath => val snapshot = latestSnapshot(tablePath) - assert(snapshot.getVersion(defaultTableClient) == 5) + assert(snapshot.getVersion(defaultEngine) == 5) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(defaultTableClient).build()) + snapshot.getScanBuilder(defaultEngine).build()) assert(scanFileRows.length == 2) } @@ -216,7 +216,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test("DV cases with same path different DV keys") { val snapshot = latestSnapshot(goldenTablePath("log-replay-dv-key-cases")) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(defaultTableClient).build() + snapshot.getScanBuilder(defaultEngine).build() ) assert(scanFileRows.length == 1) // there should only be 1 add file val dv = InternalScanFileUtils.getDeletionVectorDescriptorFromRow(scanFileRows.head) @@ -227,14 +227,14 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { withGoldenTable("log-replay-special-characters-a") { path => val snapshot = latestSnapshot(path) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(defaultTableClient).build() + snapshot.getScanBuilder(defaultEngine).build() ) assert(scanFileRows.isEmpty) } withGoldenTable("log-replay-special-characters-b") { path => val snapshot = latestSnapshot(path) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(defaultTableClient).build() + snapshot.getScanBuilder(defaultEngine).build() ) assert(scanFileRows.length == 1) val addFileStatus = InternalScanFileUtils.getAddFileStatus(scanFileRows.head) @@ -247,8 +247,8 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { ignore("path should be canonicalized - normal characters") { Seq("canonicalized-paths-normal-a", "canonicalized-paths-normal-b").foreach { path => val snapshot = latestSnapshot(goldenTablePath(path)) - assert(snapshot.getVersion(defaultTableClient) == 1) - val scanFileRows = collectScanFileRows(snapshot.getScanBuilder(defaultTableClient).build()) + assert(snapshot.getVersion(defaultEngine) == 1) + val scanFileRows = collectScanFileRows(snapshot.getScanBuilder(defaultEngine).build()) assert(scanFileRows.isEmpty) } } @@ -256,8 +256,8 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { ignore("path should be canonicalized - special characters") { Seq("canonicalized-paths-special-a", "canonicalized-paths-special-b").foreach { path => val snapshot = latestSnapshot(goldenTablePath(path)) - assert(snapshot.getVersion(defaultTableClient) == 1) - val scanFileRows = collectScanFileRows(snapshot.getScanBuilder(defaultTableClient).build()) + assert(snapshot.getVersion(defaultEngine) == 1) + val scanFileRows = collectScanFileRows(snapshot.getScanBuilder(defaultEngine).build()) assert(scanFileRows.isEmpty) } } @@ -274,7 +274,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test("delete and re-add same file in different transactions") { val path = goldenTablePath("delete-re-add-same-file-different-transactions") val snapshot = latestSnapshot(path) - val scan = snapshot.getScanBuilder(defaultTableClient).build() + val scan = snapshot.getScanBuilder(defaultEngine).build() val foundFiles = collectScanFileRows(scan).map(InternalScanFileUtils.getAddFileStatus) @@ -289,9 +289,9 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test("get the last transaction version for appID") { val unresolvedPath = goldenTablePath("deltalog-getChanges") - val table = Table.forPath(defaultTableClient, unresolvedPath) + val table = Table.forPath(defaultEngine, unresolvedPath) - val snapshot = table.getLatestSnapshot(defaultTableClient) + val snapshot = table.getLatestSnapshot(defaultEngine) assert(snapshot.isInstanceOf[SnapshotImpl]) val snapshotImpl = snapshot.asInstanceOf[SnapshotImpl] diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala index 4a88d8d1400..79119d05ea5 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog} import org.apache.spark.sql.types.{IntegerType => SparkIntegerType, StructField => SparkStructField, StructType => SparkStructType} import org.scalatest.funsuite.AnyFunSuite -import io.delta.kernel.client.{JsonHandler, ParquetHandler, TableClient} +import io.delta.kernel.engine.{JsonHandler, ParquetHandler, Engine} import io.delta.kernel.data.{ColumnarBatch, ColumnVector, FilteredColumnarBatch, Row} import io.delta.kernel.expressions.{AlwaysFalse, AlwaysTrue, And, Column, Or, Predicate, ScalarExpression} import io.delta.kernel.expressions.Literal._ @@ -42,7 +42,7 @@ import io.delta.kernel.utils.{CloseableIterator, FileStatus} import io.delta.kernel.{Scan, Snapshot, Table} import io.delta.kernel.internal.util.InternalUtils import io.delta.kernel.internal.{InternalScanFileUtils, ScanImpl} -import io.delta.kernel.defaults.client.{DefaultJsonHandler, DefaultParquetHandler, DefaultTableClient} +import io.delta.kernel.defaults.engine.{DefaultJsonHandler, DefaultParquetHandler, DefaultEngine} import io.delta.kernel.defaults.utils.{ExpressionTestUtils, TestUtils} class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with SQLHelper { @@ -99,15 +99,15 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with val snapshot = latestSnapshot(tablePath) hits.foreach { predicate => val scanFiles = collectScanFileRows( - snapshot.getScanBuilder(defaultTableClient) - .withFilter(defaultTableClient, predicate) + snapshot.getScanBuilder(defaultEngine) + .withFilter(defaultEngine, predicate) .build()) assert(scanFiles.nonEmpty, s"Expected hit but got miss for $predicate") } misses.foreach { predicate => val scanFiles = collectScanFileRows( - snapshot.getScanBuilder(defaultTableClient) - .withFilter(defaultTableClient, predicate) + snapshot.getScanBuilder(defaultEngine) + .withFilter(defaultEngine, predicate) .build()) assert(scanFiles.isEmpty, s"Expected miss but got hit for $predicate\n" + s"Returned scan files have stats: ${getScanFileStats(scanFiles)}" @@ -123,8 +123,8 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with val snapshot = latestSnapshot(tablePath) filterToNumExpFiles.foreach { case (filter, numExpFiles) => val scanFiles = collectScanFileRows( - snapshot.getScanBuilder(defaultTableClient) - .withFilter(defaultTableClient, filter) + snapshot.getScanBuilder(defaultEngine) + .withFilter(defaultEngine, filter) .build()) assert(scanFiles.length == numExpFiles, s"Expected $numExpFiles but found ${scanFiles.length} for $filter") @@ -1006,8 +1006,8 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with predicate: Predicate, expNumPartitions: Int, expNumFiles: Long): Unit = { val snapshot = latestSnapshot(tableDir.getCanonicalPath) val scanFiles = collectScanFileRows( - snapshot.getScanBuilder(defaultTableClient) - .withFilter(defaultTableClient, predicate) + snapshot.getScanBuilder(defaultEngine) + .withFilter(defaultEngine, predicate) .build()) assert(scanFiles.length == expNumFiles, s"Expected $expNumFiles but found ${scanFiles.length} for $predicate") @@ -1465,10 +1465,10 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with test("don't read stats column when there is no usable data skipping filter") { val path = goldenTablePath("data-skipping-basic-stats-all-types") - val tableClient = tableClientDisallowedStatsReads + val engine = engineDisallowedStatsReads - def snapshot(tableClient: TableClient): Snapshot = { - Table.forPath(tableClient, path).getLatestSnapshot(tableClient) + def snapshot(engine: Engine): Snapshot = { + Table.forPath(engine, path).getLatestSnapshot(engine) } def verifyNoStatsColumn(scanFiles: CloseableIterator[FilteredColumnarBatch]): Unit = { @@ -1480,25 +1480,25 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with // no filter --> don't read stats verifyNoStatsColumn( - snapshot(tableClientDisallowedStatsReads) - .getScanBuilder(tableClient).build() - .getScanFiles(tableClient)) + snapshot(engineDisallowedStatsReads) + .getScanBuilder(engine).build() + .getScanFiles(engine)) // partition filter only --> don't read stats val partFilter = equals(new Column("part"), ofInt(1)) verifyNoStatsColumn( - snapshot(tableClientDisallowedStatsReads) - .getScanBuilder(tableClient).withFilter(tableClient, partFilter).build() - .getScanFiles(tableClient)) + snapshot(engineDisallowedStatsReads) + .getScanBuilder(engine).withFilter(engine, partFilter).build() + .getScanFiles(engine)) // no eligible data skipping filter --> don't read stats val nonEligibleFilter = lessThan( new ScalarExpression("%", Seq(col("as_int"), ofInt(10)).asJava), ofInt(1)) verifyNoStatsColumn( - snapshot(tableClientDisallowedStatsReads) - .getScanBuilder(tableClient).withFilter(tableClient, nonEligibleFilter).build() - .getScanFiles(tableClient)) + snapshot(engineDisallowedStatsReads) + .getScanBuilder(engine).withFilter(engine, nonEligibleFilter).build() + .getScanFiles(engine)) } test("data skipping - prune schema correctly for various predicates") { @@ -1532,18 +1532,18 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with greaterThan(col("as_long"), ofInt(0)) ) -> Set(nestedCol("minValues.as_int"), nestedCol("maxValues.as_long")) ).foreach { case (predicate, expectedCols) => - val tableClient = tableClientVerifyJsonParseSchema(verifySchema(expectedCols)) + val engine = engineVerifyJsonParseSchema(verifySchema(expectedCols)) collectScanFileRows( - Table.forPath(tableClient, path).getLatestSnapshot(tableClient) - .getScanBuilder(tableClient) - .withFilter(tableClient, predicate) + Table.forPath(engine, path).getLatestSnapshot(engine) + .getScanBuilder(engine) + .withFilter(engine, predicate) .build(), - tableClient = tableClient) + engine = engine) } } ////////////////////////////////////////////////////////////////////////////////////////// - // Check the includeStats parameter on ScanImpl.getScanFiles(tableClient, includeStats) + // Check the includeStats parameter on ScanImpl.getScanFiles(engine, includeStats) ////////////////////////////////////////////////////////////////////////////////////////// test("check ScanImpl.getScanFiles for includeStats=true") { @@ -1551,7 +1551,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with withTempDir { tempDir => spark.range(10).write.format("delta").save(tempDir.getCanonicalPath) def checkStatsPresent(scan: Scan): Unit = { - val scanFileBatches = scan.asInstanceOf[ScanImpl].getScanFiles(defaultTableClient, true) + val scanFileBatches = scan.asInstanceOf[ScanImpl].getScanFiles(defaultEngine, true) scanFileBatches.forEach { batch => assert(batch.getData().getSchema() == InternalScanFileUtils.SCAN_FILE_SCHEMA_WITH_STATS) } @@ -1559,15 +1559,15 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with // No query filter checkStatsPresent( latestSnapshot(tempDir.getCanonicalPath) - .getScanBuilder(defaultTableClient) + .getScanBuilder(defaultEngine) .build() ) // Query filter but no valid data skipping filter checkStatsPresent( latestSnapshot(tempDir.getCanonicalPath) - .getScanBuilder(defaultTableClient) + .getScanBuilder(defaultEngine) .withFilter( - defaultTableClient, + defaultEngine, greaterThan( new ScalarExpression("+", Seq(col("id"), ofInt(10)).asJava), ofInt(100) @@ -1577,9 +1577,9 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with // With valid data skipping filter present checkStatsPresent( latestSnapshot(tempDir.getCanonicalPath) - .getScanBuilder(defaultTableClient) + .getScanBuilder(defaultEngine) .withFilter( - defaultTableClient, + defaultEngine, greaterThan( col("id"), ofInt(0) @@ -1600,12 +1600,12 @@ object ScanSuite { } /** - * Returns a custom table client implementation that doesn't allow "add.stats" in the read schema + * Returns a custom engine implementation that doesn't allow "add.stats" in the read schema * for parquet or json handlers. */ - def tableClientDisallowedStatsReads: TableClient = { + def engineDisallowedStatsReads: Engine = { val hadoopConf = new Configuration() - new DefaultTableClient(hadoopConf) { + new DefaultEngine(hadoopConf) { override def getParquetHandler: ParquetHandler = { new DefaultParquetHandler(hadoopConf) { @@ -1633,9 +1633,9 @@ object ScanSuite { } } - def tableClientVerifyJsonParseSchema(verifyFx: StructType => Unit): TableClient = { + def engineVerifyJsonParseSchema(verifyFx: StructType => Unit): Engine = { val hadoopConf = new Configuration() - new DefaultTableClient(hadoopConf) { + new DefaultEngine(hadoopConf) { override def getJsonHandler: JsonHandler = { new DefaultJsonHandler(hadoopConf) { override def parseJson(stringVector: ColumnVector, schema: StructType, diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultExpressionHandlerSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala similarity index 99% rename from kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultExpressionHandlerSuite.scala rename to kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala index 4010594fb43..67807e2b0cd 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultExpressionHandlerSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.defaults.client +package io.delta.kernel.defaults.engine import io.delta.kernel.defaults.utils.ExpressionTestUtils import io.delta.kernel.types.BooleanType.BOOLEAN diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultFileSystemClientSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultFileSystemClientSuite.scala similarity index 96% rename from kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultFileSystemClientSuite.scala rename to kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultFileSystemClientSuite.scala index 97ee2a959db..53a64003605 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultFileSystemClientSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultFileSystemClientSuite.scala @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.defaults.client +package io.delta.kernel.defaults.engine import java.io.FileNotFoundException @@ -25,7 +25,7 @@ import org.scalatest.funsuite.AnyFunSuite class DefaultFileSystemClientSuite extends AnyFunSuite with TestUtils { - val fsClient = defaultTableClient.getFileSystemClient + val fsClient = defaultEngine.getFileSystemClient val fs = FileSystem.get(configuration) test("list from file") { diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultJsonHandlerSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultJsonHandlerSuite.scala similarity index 99% rename from kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultJsonHandlerSuite.scala rename to kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultJsonHandlerSuite.scala index 86a79308b5f..1902cfe5c21 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultJsonHandlerSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultJsonHandlerSuite.scala @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.defaults.client +package io.delta.kernel.defaults.engine import java.math.{BigDecimal => JBigDecimal} import java.util.Optional @@ -34,7 +34,7 @@ class DefaultJsonHandlerSuite extends AnyFunSuite with TestUtils with DefaultVec val jsonHandler = new DefaultJsonHandler(new Configuration { set("delta.kernel.default.json.reader.batch-size", "1") }) - val fsClient = defaultTableClient.getFileSystemClient + val fsClient = defaultEngine.getFileSystemClient ///////////////////////////////////////////////////////////////////////////////////////////////// // Tests for parseJson for statistics eligible types (additional in TestDefaultJsonHandler.java) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultParquetHandlerSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultParquetHandlerSuite.scala similarity index 98% rename from kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultParquetHandlerSuite.scala rename to kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultParquetHandlerSuite.scala index 39be552053d..fef6bf07dc1 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/client/DefaultParquetHandlerSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultParquetHandlerSuite.scala @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel.defaults.client +package io.delta.kernel.defaults.engine import io.delta.golden.GoldenTableUtils.goldenTableFile import io.delta.kernel.defaults.internal.parquet.ParquetSuiteBase diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSuiteBase.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSuiteBase.scala index 094c1f113e0..f3afde8094f 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSuiteBase.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/parquet/ParquetSuiteBase.scala @@ -216,7 +216,7 @@ trait ParquetSuiteBase extends TestUtils { .filter(path => path.endsWith(".parquet")) .map(path => FileStatus.of(path, 0L, 0L)) - val data = defaultTableClient.getParquetHandler.readParquetFiles( + val data = defaultEngine.getParquetHandler.readParquetFiles( toCloseableIterator(parquetFiles.asJava), readSchema, predicate) 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 7a11471fff6..ac231447a02 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 @@ -25,10 +25,10 @@ import scala.collection.mutable.ArrayBuffer import io.delta.golden.GoldenTableUtils import io.delta.kernel.{Scan, Snapshot, Table} -import io.delta.kernel.client.TableClient -import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, MapValue, Row} -import io.delta.kernel.defaults.client.DefaultTableClient +import io.delta.kernel.data.{ColumnarBatch, ColumnVector, FilteredColumnarBatch, MapValue, Row} +import io.delta.kernel.defaults.engine.DefaultEngine import io.delta.kernel.defaults.internal.data.vector.DefaultGenericVector +import io.delta.kernel.engine.Engine import io.delta.kernel.expressions.{Column, Predicate} import io.delta.kernel.internal.InternalScanFileUtils import io.delta.kernel.internal.data.ScanStateRow @@ -38,6 +38,7 @@ import io.delta.kernel.types._ import io.delta.kernel.utils.CloseableIterator import org.apache.hadoop.conf.Configuration import org.apache.hadoop.shaded.org.apache.commons.io.FileUtils + import org.apache.spark.sql.SparkSession import org.apache.spark.sql.{types => sparktypes} import org.apache.spark.sql.catalyst.plans.SQLHelper @@ -46,7 +47,7 @@ import org.scalatest.Assertions trait TestUtils extends Assertions with SQLHelper { lazy val configuration = new Configuration() - lazy val defaultTableClient = DefaultTableClient.create(configuration) + lazy val defaultEngine = DefaultEngine.create(configuration) lazy val spark = SparkSession .builder() @@ -101,7 +102,7 @@ trait TestUtils extends Assertions with SQLHelper { if (predicate.isEmpty) { new FilteredColumnarBatch(batch, Optional.empty()) } else { - val predicateEvaluator = defaultTableClient.getExpressionHandler + val predicateEvaluator = defaultEngine.getExpressionHandler .getPredicateEvaluator(batch.getSchema, predicate.get) val selVector = predicateEvaluator.eval(batch, Optional.empty()) new FilteredColumnarBatch(batch, Optional.of(selVector)) @@ -122,20 +123,20 @@ trait TestUtils extends Assertions with SQLHelper { testFunc(tablePath) } - def latestSnapshot(path: String, tableClient: TableClient = defaultTableClient): Snapshot = { - Table.forPath(tableClient, path) - .getLatestSnapshot(tableClient) + def latestSnapshot(path: String, engine: Engine = defaultEngine): Snapshot = { + Table.forPath(engine, path) + .getLatestSnapshot(engine) } def tableSchema(path: String): StructType = { - Table.forPath(defaultTableClient, path) - .getLatestSnapshot(defaultTableClient) - .getSchema(defaultTableClient) + Table.forPath(defaultEngine, path) + .getLatestSnapshot(defaultEngine) + .getSchema(defaultEngine) } def hasColumnMappingId(str: String): Boolean = { - val table = Table.forPath(defaultTableClient, str) - val schema = table.getLatestSnapshot(defaultTableClient).getSchema(defaultTableClient) + val table = Table.forPath(defaultEngine, str) + val schema = table.getLatestSnapshot(defaultEngine).getSchema(defaultEngine) schema.fields().asScala.exists { field => field.getMetadata.contains(ColumnMapping.COLUMN_MAPPING_ID_KEY) } @@ -156,8 +157,8 @@ trait TestUtils extends Assertions with SQLHelper { } } - def collectScanFileRows(scan: Scan, tableClient: TableClient = defaultTableClient): Seq[Row] = { - scan.getScanFiles(tableClient).toSeq + def collectScanFileRows(scan: Scan, engine: Engine = defaultEngine): Seq[Row] = { + scan.getScanFiles(engine).toSeq .flatMap(_.getRows.toSeq) } @@ -166,18 +167,18 @@ trait TestUtils extends Assertions with SQLHelper { readSchema: StructType = null, filter: Predicate = null, expectedRemainingFilter: Predicate = null, - tableClient: TableClient = defaultTableClient): Seq[Row] = { + engine: Engine = defaultEngine): Seq[Row] = { val result = ArrayBuffer[Row]() - var scanBuilder = snapshot.getScanBuilder(tableClient) + var scanBuilder = snapshot.getScanBuilder(engine) if (readSchema != null) { - scanBuilder = scanBuilder.withReadSchema(tableClient, readSchema) + scanBuilder = scanBuilder.withReadSchema(engine, readSchema) } if (filter != null) { - scanBuilder = scanBuilder.withFilter(tableClient, filter) + scanBuilder = scanBuilder.withFilter(engine, filter) } val scan = scanBuilder.build() @@ -188,21 +189,21 @@ trait TestUtils extends Assertions with SQLHelper { actRemainingPredicate.toString === Optional.ofNullable(expectedRemainingFilter).toString) } - val scanState = scan.getScanState(tableClient); - val fileIter = scan.getScanFiles(tableClient) + val scanState = scan.getScanState(engine); + val fileIter = scan.getScanFiles(engine) - val physicalDataReadSchema = ScanStateRow.getPhysicalDataReadSchema(tableClient, scanState) + val physicalDataReadSchema = ScanStateRow.getPhysicalDataReadSchema(engine, scanState) fileIter.forEach { fileColumnarBatch => fileColumnarBatch.getRows().forEach { scanFileRow => val fileStatus = InternalScanFileUtils.getAddFileStatus(scanFileRow) - val physicalDataIter = tableClient.getParquetHandler().readParquetFiles( + val physicalDataIter = engine.getParquetHandler().readParquetFiles( singletonCloseableIterator(fileStatus), physicalDataReadSchema, Optional.empty()) var dataBatches: CloseableIterator[FilteredColumnarBatch] = null try { dataBatches = Scan.transformPhysicalData( - tableClient, + engine, scanState, scanFileRow, physicalDataIter) @@ -282,7 +283,7 @@ trait TestUtils extends Assertions with SQLHelper { * @param path fully qualified path of the table to check * @param expectedAnswer expected rows * @param readCols subset of columns to read; if null then all columns will be read - * @param tableClient table client to use to read the table + * @param engine engine to use to read the table * @param expectedSchema expected schema to check for; if null then no check is performed * @param filter Filter to select a subset of rows form the table * @param expectedRemainingFilter Remaining predicate out of the `filter` that is not enforced @@ -293,7 +294,7 @@ trait TestUtils extends Assertions with SQLHelper { path: String, expectedAnswer: Seq[TestRow], readCols: Seq[String] = null, - tableClient: TableClient = defaultTableClient, + engine: Engine = defaultEngine, expectedSchema: StructType = null, filter: Predicate = null, version: Option[Long] = None, @@ -304,40 +305,40 @@ trait TestUtils extends Assertions with SQLHelper { assert(version.isEmpty || timestamp.isEmpty, "Cannot provide both a version and timestamp") val snapshot = if (version.isDefined) { - Table.forPath(tableClient, path) - .getSnapshotAsOfVersion(tableClient, version.get) + Table.forPath(engine, path) + .getSnapshotAsOfVersion(engine, version.get) } else if (timestamp.isDefined) { - Table.forPath(tableClient, path) - .getSnapshotAsOfTimestamp(tableClient, timestamp.get) + Table.forPath(engine, path) + .getSnapshotAsOfTimestamp(engine, timestamp.get) } else { - latestSnapshot(path, tableClient) + latestSnapshot(path, engine) } val readSchema = if (readCols == null) { null } else { - val schema = snapshot.getSchema(tableClient) + val schema = snapshot.getSchema(engine) new StructType(readCols.map(schema.get(_)).asJava) } if (expectedSchema != null) { assert( - expectedSchema == snapshot.getSchema(tableClient), + expectedSchema == snapshot.getSchema(engine), s""" |Expected schema does not match actual schema: |Expected schema: $expectedSchema - |Actual schema: ${snapshot.getSchema(tableClient)} + |Actual schema: ${snapshot.getSchema(engine)} |""".stripMargin ) } expectedVersion.foreach { version => - assert(version == snapshot.getVersion(defaultTableClient), + assert(version == snapshot.getVersion(defaultEngine), s"Expected version $version does not match actual version" + - s" ${snapshot.getVersion(defaultTableClient)}") + s" ${snapshot.getVersion(defaultEngine)}") } - val result = readSnapshot(snapshot, readSchema, filter, expectedRemainingFilter, tableClient) + val result = readSnapshot(snapshot, readSchema, filter, expectedRemainingFilter, engine) checkAnswer(result, expectedAnswer) } diff --git a/project/FlinkMimaExcludes.scala b/project/FlinkMimaExcludes.scala index f9a6296e4d7..dd2443fda16 100644 --- a/project/FlinkMimaExcludes.scala +++ b/project/FlinkMimaExcludes.scala @@ -20,5 +20,10 @@ import com.typesafe.tools.mima.core._ * The list of Mima errors to exclude in the Flink project. */ object FlinkMimaExcludes { - val ignoredABIProblems = Seq() + // scalastyle:off line.size.limit + + val ignoredABIProblems = Seq( + // We can ignore internal changes + ProblemFilters.exclude[IncompatibleMethTypeProblem]("io.delta.standalone.internal.KernelDeltaLogDelegator.this") + ) } From db10a33afd166b18e42ffbdb86f8d8cf022524e1 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Thu, 2 May 2024 17:09:05 -0700 Subject: [PATCH 04/11] [Build] Fix Java checkstyle plugin to work with SBT upgrade (#3019) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [X] Other (fill in here) ## Description https://github.com/delta-io/delta/pull/2828 upgrades the SBT version from 1.5.5 to 1.9.9 which causes `projectName/checkstyle` to fail with ``` sbt:delta> kernelApi/checkstyle [error] stack trace is suppressed; run last kernelApi / checkstyle for the full output [error] (kernelApi / checkstyle) org.xml.sax.SAXParseException; lineNumber: 18; columnNumber: 10; DOCTYPE is disallowed when the feature "http://apache.org/xml/features/disallow-doctype-decl" set to true. [error] Total time: 0 s, completed May 1, 2024 2:59:48 PM ``` This failure was silent in our CI runs for some reason, if you search the logs before that commit you can see "checkstyle" in them but no instances after. This is a little concerning but don't really have time to figure out why this was silent. For now, upgrades versions to match Spark's current plugins which fixes the issue. See the matching Spark PR here https://github.com/apache/spark/pull/38481. ## How was this patch tested? Ran `kernelApi/checkstyle` locally. TODO: verify it's present in the CI runs after as well ## Does this PR introduce _any_ user-facing changes? No. (cherry picked from commit 12cabb79f955105cd640396f599eff88c6c220c7) --- build.sbt | 12 ++++++------ .../delta/flink/internal/KernelSnapshotWrapper.java | 3 +-- project/plugins.sbt | 4 ++-- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/build.sbt b/build.sbt index 7f9c660adc4..cc0a7c968eb 100644 --- a/build.sbt +++ b/build.sbt @@ -1224,9 +1224,9 @@ lazy val flink = (project in file("connectors/flink")) IO.write(file, s"""package io.delta.flink.internal; | - |final public class Meta { - | public static final String FLINK_VERSION = "${flinkVersion}"; - | public static final String CONNECTOR_VERSION = "${version.value}"; + |public final class Meta { + | public static final String FLINK_VERSION = "${flinkVersion}"; + | public static final String CONNECTOR_VERSION = "${version.value}"; |} |""".stripMargin) Seq(file) @@ -1320,9 +1320,9 @@ def javaCheckstyleSettings(checkstyleFile: String): Def.SettingsDefinition = { // and during tests (e.g. build/sbt test) Seq( checkstyleConfigLocation := CheckstyleConfigLocation.File(checkstyleFile), - checkstyleSeverityLevel := Some(CheckstyleSeverityLevel.Error), - (Compile / checkstyle) := (Compile / checkstyle).triggeredBy(Compile / compile).value, - (Test / checkstyle) := (Test / checkstyle).triggeredBy(Test / compile).value + checkstyleSeverityLevel := CheckstyleSeverityLevel.Error, + (Compile / compile) := ((Compile / compile) dependsOn (Compile / checkstyle)).value, + (Test / test) := ((Test / test) dependsOn (Test / checkstyle)).value ) } diff --git a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java index 43ff2096670..1b85100e9cd 100644 --- a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java +++ b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java @@ -23,11 +23,10 @@ import java.util.List; import java.util.Optional; +import io.delta.kernel.data.ColumnVector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import io.delta.kernel.data.ColumnVector; - import io.delta.standalone.DeltaScan; import io.delta.standalone.actions.AddFile; import io.delta.standalone.actions.Metadata; diff --git a/project/plugins.sbt b/project/plugins.sbt index 674539b6a59..79e0da3f54d 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -45,7 +45,7 @@ libraryDependencySchemes += "org.scala-lang.modules" %% "scala-xml" % VersionSch addSbtPlugin("net.aichler" % "sbt-jupiter-interface" % "0.9.1") -addSbtPlugin("com.etsy" % "sbt-checkstyle-plugin" % "3.1.1") +addSbtPlugin("software.purpledragon" % "sbt-checkstyle-plugin" % "4.0.1") // By default, sbt-checkstyle-plugin uses checkstyle version 6.15, but we should set it to use the // same version as Spark -dependencyOverrides += "com.puppycrawl.tools" % "checkstyle" % "8.43" +dependencyOverrides += "com.puppycrawl.tools" % "checkstyle" % "9.3" From c1018a15667b6bc07d5b42e7a329a765a7b11435 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Thu, 2 May 2024 18:25:35 -0700 Subject: [PATCH 05/11] [Kernel][Infra] Fix the java checkstyle for Kernel's Meta file (#3034) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description Merging https://github.com/delta-io/delta/pull/3019 broke the `kernelApi/checkstyle` which is failing CI jobs since it's original tests ran before https://github.com/delta-io/delta/commit/8cdf411d2c3e483a483595d117619c4aa6e15faa was committed. ## How was this patch tested? Checked locally that `kernelApi/checkstyle` passes. (cherry picked from commit fe88cc3496940a082b0db414f603c0c4712a95cf) --- build.sbt | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/build.sbt b/build.sbt index cc0a7c968eb..146355c2404 100644 --- a/build.sbt +++ b/build.sbt @@ -345,10 +345,25 @@ lazy val kernelApi = (project in file("kernel/kernel-api")) Compile / sourceGenerators += Def.task { val file = (Compile / sourceManaged).value / "io" / "delta" / "kernel" / "Meta.java" IO.write(file, - s"""package io.delta.kernel; + s"""/* + | * Copyright (2024) The Delta Lake Project Authors. + | * + | * Licensed under the Apache License, Version 2.0 (the "License"); + | * you may not use this file except in compliance with the License. + | * You may obtain a copy of the License at + | * + | * http://www.apache.org/licenses/LICENSE-2.0 + | * + | * Unless required by applicable law or agreed to in writing, software + | * distributed under the License is distributed on an "AS IS" BASIS, + | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + | * See the License for the specific language governing permissions and + | * limitations under the License. + | */ + |package io.delta.kernel; | - |final public class Meta { - | public static final String KERNEL_VERSION = "${version.value}"; + |public final class Meta { + | public static final String KERNEL_VERSION = "${version.value}"; |} |""".stripMargin) Seq(file) From f4555f545895ac474bce01fa4f84e7795a4fb0ea Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Fri, 3 May 2024 11:27:25 -0700 Subject: [PATCH 06/11] [Kernel] Remove unused `ExpressionHandler.isSupported(...)` for now (#3018) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description Removes a currently unused API in `ExpressionHandler`. See https://github.com/delta-io/delta/issues/3017 for details. **TLDR** we added this API before achieving consensus on what we want to implement and it is currently unused. Remove it for now and we can add it back later if needed. ## How was this patch tested? N/A ## Does this PR introduce _any_ user-facing changes? No (cherry picked from commit e7cafecdb10a7698eef7aadcf1a2876c9e067fbb) --- .../kernel/engine/ExpressionHandler.java | 9 ------ .../delta/kernel/test/MockEngineUtils.scala | 6 ---- .../engine/DefaultExpressionHandler.java | 11 ------- .../DefaultExpressionHandlerSuite.scala | 30 ------------------- 4 files changed, 56 deletions(-) diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/engine/ExpressionHandler.java b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/ExpressionHandler.java index d5f921de53a..d39bf1ed26a 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/engine/ExpressionHandler.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/ExpressionHandler.java @@ -34,15 +34,6 @@ */ @Evolving public interface ExpressionHandler { - /** - * Is the given expression evaluation supported on the data with given schema? - * - * @param inputSchema Schema of input data on which the expression is evaluated. - * @param expression Expression to check whether it is supported for evaluation. - * @param outputType Expected result data type. - * @return true if supported and false otherwise. - */ - boolean isSupported(StructType inputSchema, Expression expression, DataType outputType); /** * Create an {@link ExpressionEvaluator} that can evaluate the given expression on diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala index ed80a5de092..dd49f2df15e 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockEngineUtils.scala @@ -136,12 +136,6 @@ trait BaseMockExpressionHandler extends ExpressionHandler { outputType: DataType): ExpressionEvaluator = throw new UnsupportedOperationException("not supported in this test suite") - override def isSupported( - inputSchema: StructType, - expression: Expression, - outputType: DataType): Boolean = - throw new UnsupportedOperationException("not supported in this test suite") - override def createSelectionVector(values: Array[Boolean], from: Int, to: Int): ColumnVector = throw new UnsupportedOperationException("not supported in this test suite") } diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultExpressionHandler.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultExpressionHandler.java index 02afa50c3f3..e7386662a52 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultExpressionHandler.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultExpressionHandler.java @@ -38,17 +38,6 @@ * Default implementation of {@link ExpressionHandler} */ public class DefaultExpressionHandler implements ExpressionHandler { - @Override - public boolean isSupported(StructType inputSchema, Expression expression, DataType outputType) { - // There is no extra cost to create an expression handler in default implementation in - // addition to checking the expression support. So we can just use `getEvaluator`. - try { - getEvaluator(inputSchema, expression, outputType); - return true; - } catch (UnsupportedOperationException e) { - return false; - } - } @Override public ExpressionEvaluator getEvaluator( diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala index 67807e2b0cd..49ea61a5aa8 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/engine/DefaultExpressionHandlerSuite.scala @@ -76,36 +76,6 @@ class DefaultExpressionHandlerSuite extends AnyFunSuite with ExpressionTestUtils assert(ex.getMessage.contains("values is null")) } - val tableSchema = new StructType() - .add("d1", INTEGER) - .add("d2", STRING) - .add("d3", new StructType() - .add("d31", BOOLEAN) - .add("d32", LONG)) - .add("p1", INTEGER) - .add("p2", STRING) - val unsupportedExpr = Map( - (unsupported("d1"), BOOLEAN) -> false, // unsupported function - (lt(col("d1"), int(12)), BOOLEAN) -> true, - (lt(col("d1"), int(12)), INTEGER) -> false, // output type is not supported - (lt(nestedCol("d3.d32"), int(12)), BOOLEAN) -> true, // implicit conversion from int to long - (gt(col("d1"), str("sss")), STRING) -> false, // unexpected input type to > operator - // unsupported expression in one of the AND inputs - (and(gt(col("d2"), str("sss")), unsupported("d2")), BOOLEAN) -> false, - // both unsupported expressions in AND inputs - (and(gt(nestedCol("d3.d31"), str("sss")), unsupported("d2")), BOOLEAN) -> false, - // unsupported expression in one of the OR inputs - (or(gt(col("p2"), str("sss")), unsupported("d2")), BOOLEAN) -> false, - // both unsupported expressions in OR inputs - (or(gt(nestedCol("d3.d31"), str("sss")), unsupported("d2")), BOOLEAN) -> false - ).foreach { - case ((expr, outputType), expected) => - test(s"is expression supported: $expr -> $outputType") { - assert( - new DefaultExpressionHandler().isSupported(tableSchema, expr, outputType) == expected) - } - } - private def selectionVector(values: Array[Boolean], from: Int, to: Int) = { new DefaultExpressionHandler().createSelectionVector(values, from, to) } From c8bbd5b6db29ab4607a34ffb5b5023f78fe61869 Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Fri, 3 May 2024 15:57:30 -0700 Subject: [PATCH 07/11] [Kernel] Refactor all user-facing exceptions to be "KernelExceptions" (#3014) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description In order to provide a unified and user-friendly experience we are making all user-facing exceptions instances of one parent class `KernelException`. This PR also attempts to unify/improve some of these error messsages. ## How was this patch tested? Existing tests should suffice (updated to match the changes here). ## Does this PR introduce _any_ user-facing changes? Yes, changes exception types + messages. (cherry picked from commit c426f030cd48ac6fc09bd6e7bf9fcff9252ca211) --- .../internal/KernelDeltaLogDelegator.scala | 4 +- .../kernel/examples/BaseTableReader.java | 2 +- .../examples/MultiThreadedTableReader.java | 1 + .../examples/SingleThreadedTableReader.java | 1 + .../src/main/java/io/delta/kernel/Table.java | 8 +- .../CheckpointAlreadyExistsException.java | 4 +- .../kernel/exceptions/KernelException.java | 39 +++++ .../TableNotFoundException.java | 17 +- .../io/delta/kernel/internal/DeltaErrors.java | 152 +++++++++--------- .../kernel/internal/DeltaHistoryManager.java | 11 +- .../delta/kernel/internal/TableFeatures.java | 29 ++-- .../io/delta/kernel/internal/TableImpl.java | 3 +- .../kernel/internal/replay/LogReplay.java | 3 +- .../internal/snapshot/SnapshotManager.java | 14 +- .../internal/DeltaHistoryManagerSuite.scala | 23 ++- .../internal/SnapshotManagerSuite.scala | 8 +- .../kernel/internal/TableFeaturesSuite.scala | 7 +- .../internal/DefaultEngineErrors.java | 18 ++- .../DefaultExpressionEvaluator.java | 38 ++--- .../expressions/ElementAtEvaluator.java | 5 +- .../defaults/CreateCheckpointSuite.scala | 9 +- .../defaults/DeltaTableReadsSuite.scala | 13 +- 22 files changed, 243 insertions(+), 166 deletions(-) rename kernel/kernel-api/src/main/java/io/delta/kernel/{ => exceptions}/CheckpointAlreadyExistsException.java (90%) create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/KernelException.java rename kernel/kernel-api/src/main/java/io/delta/kernel/{ => exceptions}/TableNotFoundException.java (73%) diff --git a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala index 21a0c164972..9c51405259d 100644 --- a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala +++ b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala @@ -20,9 +20,9 @@ package io.delta.standalone.internal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path - -import io.delta.kernel.{Table, TableNotFoundException} +import io.delta.kernel.Table import io.delta.kernel.defaults.engine.DefaultEngine +import io.delta.kernel.exceptions.TableNotFoundException import io.delta.kernel.internal.{TableImpl, SnapshotImpl => SnapshotImplKernel} import io.delta.standalone.VersionLog import io.delta.standalone.actions.{CommitInfo => CommitInfoJ} diff --git a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/BaseTableReader.java b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/BaseTableReader.java index ebd2a805b34..97e55ba2ed4 100644 --- a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/BaseTableReader.java +++ b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/BaseTableReader.java @@ -36,8 +36,8 @@ import org.apache.commons.cli.ParseException; import org.apache.hadoop.conf.Configuration; -import io.delta.kernel.TableNotFoundException; import io.delta.kernel.engine.Engine; +import io.delta.kernel.exceptions.TableNotFoundException; import io.delta.kernel.data.FilteredColumnarBatch; import io.delta.kernel.data.Row; import io.delta.kernel.expressions.Predicate; diff --git a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java index 687b8a87d76..c8b0cceceef 100644 --- a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java +++ b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java @@ -31,6 +31,7 @@ import io.delta.kernel.data.FilteredColumnarBatch; import io.delta.kernel.data.Row; import io.delta.kernel.examples.utils.RowSerDe; +import io.delta.kernel.exceptions.TableNotFoundException; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterator; diff --git a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/SingleThreadedTableReader.java b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/SingleThreadedTableReader.java index 13cd2e80644..d2976d9cf68 100644 --- a/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/SingleThreadedTableReader.java +++ b/kernel/examples/table-reader/src/main/java/io/delta/kernel/examples/SingleThreadedTableReader.java @@ -25,6 +25,7 @@ import io.delta.kernel.data.ColumnarBatch; import io.delta.kernel.data.FilteredColumnarBatch; import io.delta.kernel.data.Row; +import io.delta.kernel.exceptions.TableNotFoundException; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterator; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java index d390aa163da..00309e5aa70 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java @@ -19,7 +19,9 @@ import io.delta.kernel.annotation.Evolving; import io.delta.kernel.engine.Engine; - +import io.delta.kernel.exceptions.CheckpointAlreadyExistsException; +import io.delta.kernel.exceptions.KernelException; +import io.delta.kernel.exceptions.TableNotFoundException; import io.delta.kernel.internal.TableImpl; /** @@ -84,6 +86,8 @@ Snapshot getLatestSnapshot(Engine engine) * @param engine {@link Engine} instance to use in Delta Kernel. * @param versionId snapshot version to retrieve * @return an instance of {@link Snapshot} + * @throws KernelException if the provided version is less than the first available version + * or greater than the last available version * @since 3.2.0 */ Snapshot getSnapshotAsOfVersion(Engine engine, long versionId) @@ -109,6 +113,8 @@ Snapshot getSnapshotAsOfVersion(Engine engine, long versionId) * @param millisSinceEpochUTC timestamp to fetch the snapshot for in milliseconds since the * unix epoch * @return an instance of {@link Snapshot} + * @throws KernelException if the provided timestamp is before the earliest available version or + * after the latest available version * @since 3.2.0 */ Snapshot getSnapshotAsOfTimestamp(Engine engine, long millisSinceEpochUTC) diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/CheckpointAlreadyExistsException.java b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/CheckpointAlreadyExistsException.java similarity index 90% rename from kernel/kernel-api/src/main/java/io/delta/kernel/CheckpointAlreadyExistsException.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/CheckpointAlreadyExistsException.java index f871b341b21..47db394672a 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/CheckpointAlreadyExistsException.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/CheckpointAlreadyExistsException.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.kernel; +package io.delta.kernel.exceptions; import static java.lang.String.format; @@ -26,7 +26,7 @@ * @since 3.2.0 */ @Evolving -public class CheckpointAlreadyExistsException extends IllegalArgumentException { +public class CheckpointAlreadyExistsException extends KernelException { public CheckpointAlreadyExistsException(long version) { super(format("Checkpoint for given version %d already exists in the table", version)); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/KernelException.java b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/KernelException.java new file mode 100644 index 00000000000..3d894eb87b3 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/KernelException.java @@ -0,0 +1,39 @@ +/* + * Copyright (2024) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.exceptions; + +/** + * Thrown when Kernel cannot execute the requested operation due to the operation being invalid + * or unsupported. + */ +public class KernelException extends RuntimeException { + + public KernelException() { + super(); + } + + public KernelException(String message) { + super(message); + } + + public KernelException(Throwable cause) { + super(cause); + } + + public KernelException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/TableNotFoundException.java b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableNotFoundException.java similarity index 73% rename from kernel/kernel-api/src/main/java/io/delta/kernel/TableNotFoundException.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableNotFoundException.java index f47da001fb5..feaeb78f728 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/TableNotFoundException.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableNotFoundException.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.delta.kernel; +package io.delta.kernel.exceptions; import io.delta.kernel.annotation.Evolving; @@ -24,22 +24,23 @@ * @since 3.0.0 */ @Evolving -public class TableNotFoundException - extends Exception { +public class TableNotFoundException extends KernelException { private final String tablePath; public TableNotFoundException(String tablePath) { - this.tablePath = tablePath; + this(tablePath, null); } public TableNotFoundException(String tablePath, Throwable cause) { - super(cause); + super(String.format("Delta table at path `%s` is not found", tablePath), cause); this.tablePath = tablePath; } - @Override - public String getMessage() { - return String.format("Delta table at path `%s` is not found", tablePath); + /** + * @return the provided path where no Delta table was found + */ + public String getTablePath() { + return tablePath; } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java index 8253cb783ab..5c4eb9374f5 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java @@ -16,120 +16,122 @@ package io.delta.kernel.internal; import java.sql.Timestamp; -import java.util.Optional; -import io.delta.kernel.expressions.Expression; +import io.delta.kernel.exceptions.KernelException; +/** + * Contains methods to create user-facing Delta exceptions. + */ public final class DeltaErrors { private DeltaErrors() {} - // TODO update to be user-facing exception with future exception framework - // (see delta-io/delta#2231) & document in method docs as needed (Table::getSnapshotAtVersion) - public static RuntimeException nonReconstructableStateException( - String tablePath, long version) { + public static KernelException versionBeforeFirstAvailableCommit( + String tablePath, long versionToLoad, long earliestVersion) { String message = String.format( - "%s: Unable to reconstruct state at version %s as the transaction log has been " + - "truncated due to manual deletion or the log retention policy and checkpoint " + - "retention policy.", + "%s: Cannot load table version %s as the transaction log has been truncated due to " + + "manual deletion or the log/checkpoint retention policy. The earliest available " + + "version is %s.", tablePath, - version); - return new RuntimeException(message); + versionToLoad, + earliestVersion); + return new KernelException(message); } - // TODO update to be user-facing exception with future exception framework - // (see delta-io/delta#2231) & document in method docs as needed (Table::getSnapshotAtVersion) - public static RuntimeException nonExistentVersionException( + public static KernelException versionAfterLatestCommit( String tablePath, long versionToLoad, long latestVersion) { String message = String.format( - "%s: Trying to load a non-existent version %s. The latest version available is %s", + "%s: Cannot load table version %s as it does not exist. " + + "The latest available version is %s.", tablePath, versionToLoad, latestVersion); - return new RuntimeException(message); + return new KernelException(message); } - // TODO update to be user-facing exception with future exception framework - // (see delta-io/delta#2231) & document in method docs as needed - // (Table::getSnapshotAtTimestamp) - public static RuntimeException timestampEarlierThanTableFirstCommitException( - String tablePath, long providedTimestamp, long commitTimestamp) { + public static KernelException timestampBeforeFirstAvailableCommit( + String tablePath, + long providedTimestamp, + long earliestCommitTimestamp, + long earliestCommitVersion) { String message = String.format( - "%s: The provided timestamp %s ms (%s) is before the earliest version available. " + - "Please use a timestamp greater than or equal to %s ms (%s)", + "%s: The provided timestamp %s ms (%s) is before the earliest available version %s. " + + "Please use a timestamp greater than or equal to %s ms (%s).", tablePath, providedTimestamp, formatTimestamp(providedTimestamp), - commitTimestamp, - formatTimestamp(commitTimestamp)); - return new RuntimeException(message); + earliestCommitVersion, + earliestCommitTimestamp, + formatTimestamp(earliestCommitTimestamp)); + return new KernelException(message); } - // TODO update to be user-facing exception with future exception framework - // (see delta-io/delta#2231) & document in method docs as needed - // (Table::getSnapshotAtTimestamp) - public static RuntimeException timestampLaterThanTableLastCommit( - String tablePath, long providedTimestamp, long commitTimestamp, long commitVersion) { - String commitTimestampStr = formatTimestamp(commitTimestamp); + public static KernelException timestampAfterLatestCommit( + String tablePath, + long providedTimestamp, + long latestCommitTimestamp, + long latestCommitVersion) { String message = String.format( - "%s: The provided timestamp %s ms (%s) is after the latest commit with " + - "timestamp %s ms (%s). If you wish to query this version of the table please " + - "either provide the version %s or use the exact timestamp of the last " + - "commit %s ms (%s)", + "%s: The provided timestamp %s ms (%s) is after the latest available version %s. " + + "Please use a timestamp less than or equal to %s ms (%s).", tablePath, providedTimestamp, formatTimestamp(providedTimestamp), - commitTimestamp, - commitTimestampStr, - commitVersion, - commitTimestamp, - commitTimestampStr); - return new RuntimeException(message); + latestCommitVersion, + latestCommitTimestamp, + formatTimestamp(latestCommitTimestamp)); + return new KernelException(message); } - // TODO: Change the exception to proper type as part of the exception framework - // (see delta-io/delta#2231) - /** - * Exception thrown when the expression evaluator doesn't support the given expression. - * @param expression - * @param reason Optional additional reason for why the expression is not supported. - * @return - */ - public static UnsupportedOperationException unsupportedExpression( - Expression expression, - Optional reason) { + /* ------------------------ PROTOCOL EXCEPTIONS ----------------------------- */ + + public static KernelException unsupportedReaderProtocol( + String tablePath, int tableReaderVersion) { String message = String.format( - "Expression evaluator doesn't support the expression: %s.%s", - expression, - reason.map(r -> " Reason: " + r).orElse("")); - return new UnsupportedOperationException(message); + "Unsupported Delta protocol reader version: table `%s` requires reader version %s " + + "which is unsupported by this version of Delta Kernel.", + tablePath, + tableReaderVersion); + return new KernelException(message); } - public static UnsupportedOperationException unsupportedReaderProtocol(int readVersion) { - throw new UnsupportedOperationException( - "Unsupported reader protocol version: " + readVersion); + public static KernelException unsupportedReaderFeature( + String tablePath, String readerFeature) { + String message = String.format( + "Unsupported Delta reader feature: table `%s` requires reader table feature \"%s\" " + + "which is unsupported by this version of Delta Kernel.", + tablePath, + readerFeature); + return new KernelException(message); } - public static UnsupportedOperationException unsupportedReadFeature( - int readProtocolVersion, - String readFeature) { - throw new UnsupportedOperationException(String.format( - "Unsupported reader protocol version: %s with feature: %s", - readProtocolVersion, readFeature)); + public static KernelException unsupportedWriterProtocol( + String tablePath, int tableWriterVersion) { + String message = String.format( + "Unsupported Delta protocol writer version: table `%s` requires writer version %s " + + "which is unsupported by this version of Delta Kernel.", + tablePath, + tableWriterVersion); + return new KernelException(message); } - public static UnsupportedOperationException unsupportedWriterProtocol(int writeVersion) { - throw new UnsupportedOperationException( - "Unsupported writer protocol version: " + writeVersion); + public static KernelException unsupportedWriterFeature( + String tablePath, String writerFeature) { + String message = String.format( + "Unsupported Delta writer feature: table `%s` requires writer table feature \"%s\" " + + "which is unsupported by this version of Delta Kernel.", + tablePath, + writerFeature); + return new KernelException(message); } - public static UnsupportedOperationException unsupportedWriteFeature( - int writeProtocolVersion, - String writeFeature) { - throw new UnsupportedOperationException(String.format( - "Unsupported writer protocol version: %s with feature: %s", - writeProtocolVersion, writeFeature)); + public static KernelException columnInvariantsNotSupported() { + String message = "This version of Delta Kernel does not support writing to tables with " + + "column invariants present."; + return new KernelException(message); } + /* ------------------------ HELPER METHODS ----------------------------- */ + private static String formatTimestamp(long millisSinceEpochUTC) { return new Timestamp(millisSinceEpochUTC).toInstant().toString(); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaHistoryManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaHistoryManager.java index 6bbc1ceac0e..00386df19dc 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaHistoryManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaHistoryManager.java @@ -22,8 +22,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import io.delta.kernel.TableNotFoundException; import io.delta.kernel.engine.Engine; +import io.delta.kernel.exceptions.TableNotFoundException; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.utils.FileStatus; import io.delta.kernel.internal.checkpoints.CheckpointInstance; @@ -59,15 +59,16 @@ public static long getActiveCommitAtTimestamp( List commits = getCommits(engine, logPath, earliestRecreatableCommit); Commit commit = lastCommitBeforeOrAtTimestamp(commits, timestamp) .orElseThrow(() -> - DeltaErrors.timestampEarlierThanTableFirstCommitException( + DeltaErrors.timestampBeforeFirstAvailableCommit( logPath.getParent().toString(), /* use dataPath */ timestamp, - commits.get(0).timestamp) + commits.get(0).timestamp, + commits.get(0).version) ); // If timestamp is after the last commit of the table if (commit == commits.get(commits.size() - 1) && commit.timestamp < timestamp) { - throw DeltaErrors.timestampLaterThanTableLastCommit( + throw DeltaErrors.timestampAfterLatestCommit( logPath.getParent().toString(), /* use dataPath */ timestamp, commit.timestamp, @@ -85,7 +86,7 @@ public static long getActiveCommitAtTimestamp( * We search for the earliest checkpoint we have, or whether we have the 0th delta file. This * method assumes that the commits are contiguous. */ - private static long getEarliestRecreatableCommit(Engine engine, Path logPath) + public static long getEarliestRecreatableCommit(Engine engine, Path logPath) throws TableNotFoundException { try (CloseableIterator files = listFrom(engine, logPath, 0) .filter(fs -> diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableFeatures.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableFeatures.java index 13339dfcb44..07ba06b449b 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableFeatures.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableFeatures.java @@ -34,7 +34,8 @@ public class TableFeatures { // Helper Methods // //////////////////// - public static void validateReadSupportedTable(Protocol protocol, Metadata metadata) { + public static void validateReadSupportedTable( + Protocol protocol, Metadata metadata, String tablePath) { switch (protocol.getMinReaderVersion()) { case 1: break; @@ -54,12 +55,13 @@ public static void validateReadSupportedTable(Protocol protocol, Metadata metada case "v2Checkpoint": break; default: - throw DeltaErrors.unsupportedReadFeature(3, readerFeature); + throw DeltaErrors.unsupportedReaderFeature(tablePath, readerFeature); } } break; default: - throw unsupportedReaderProtocol(protocol.getMinReaderVersion()); + throw DeltaErrors.unsupportedReaderProtocol( + tablePath, protocol.getMinReaderVersion()); } } @@ -79,7 +81,8 @@ public static void validateReadSupportedTable(Protocol protocol, Metadata metada public static void validateWriteSupportedTable( Protocol protocol, Metadata metadata, - StructType tableSchema) { + StructType tableSchema, + String tablePath) { int minWriterVersion = protocol.getMinWriterVersion(); switch (minWriterVersion) { case 1: @@ -87,20 +90,20 @@ public static void validateWriteSupportedTable( case 2: // Append-only and column invariants are the writer features added in version 2 // Append-only is supported, but not the invariants - validateNoInvariants(minWriterVersion, tableSchema); + validateNoInvariants(tableSchema); break; case 3: // Check constraints are added in version 3 - throw unsupportedWriterProtocol(minWriterVersion); + throw unsupportedWriterProtocol(tablePath, minWriterVersion); case 4: // CDF and generated columns are writer features added in version 4 - throw unsupportedWriterProtocol(minWriterVersion); + throw unsupportedWriterProtocol(tablePath, minWriterVersion); case 5: // Column mapping is the only one writer feature added in version 5 - throw unsupportedWriterProtocol(minWriterVersion); + throw unsupportedWriterProtocol(tablePath, minWriterVersion); case 6: // Identity is the only one writer feature added in version 6 - throw unsupportedWriterProtocol(minWriterVersion); + throw unsupportedWriterProtocol(tablePath, minWriterVersion); case 7: for (String writerFeature : protocol.getWriterFeatures()) { switch (writerFeature) { @@ -108,20 +111,20 @@ public static void validateWriteSupportedTable( case "appendOnly": break; default: - throw unsupportedWriteFeature(7, writerFeature); + throw unsupportedWriterFeature(tablePath, writerFeature); } } break; default: - throw unsupportedWriterProtocol(minWriterVersion); + throw unsupportedWriterProtocol(tablePath, minWriterVersion); } } - private static void validateNoInvariants(int minWriterVersion, StructType tableSchema) { + private static void validateNoInvariants(StructType tableSchema) { boolean hasInvariants = tableSchema.fields().stream().anyMatch( field -> field.getMetadata().contains("delta.invariants")); if (hasInvariants) { - throw unsupportedWriteFeature(minWriterVersion, "invariants"); + throw columnInvariantsNotSupported(); } } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java index 89f556d3ca2..f022a3b99e9 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java @@ -19,7 +19,8 @@ import io.delta.kernel.*; import io.delta.kernel.engine.Engine; - +import io.delta.kernel.exceptions.CheckpointAlreadyExistsException; +import io.delta.kernel.exceptions.TableNotFoundException; import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.snapshot.SnapshotManager; 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 ce4cca9c3f0..a3952bd5f76 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 @@ -247,7 +247,8 @@ private Tuple2 loadTableProtocolAndMetadata( if (protocol != null) { // Stop since we have found the latest Protocol and Metadata. - TableFeatures.validateReadSupportedTable(protocol, metadata); + TableFeatures.validateReadSupportedTable( + protocol, metadata, dataPath.toString()); return new Tuple2<>(protocol, metadata); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java index 4953dc43b74..49695ae7bce 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java @@ -30,6 +30,8 @@ import io.delta.kernel.*; import io.delta.kernel.engine.Engine; +import io.delta.kernel.exceptions.CheckpointAlreadyExistsException; +import io.delta.kernel.exceptions.TableNotFoundException; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.utils.FileStatus; @@ -160,8 +162,8 @@ public void checkpoint(Engine engine, long version) SnapshotImpl snapshot = (SnapshotImpl) getSnapshotAt(engine, version); // Check if writing to the given table protocol version/features is supported in Kernel - validateWriteSupportedTable( - snapshot.getProtocol(), snapshot.getMetadata(), snapshot.getSchema(engine)); + validateWriteSupportedTable(snapshot.getProtocol(), snapshot.getMetadata(), + snapshot.getSchema(engine), tablePath.toString()); Path checkpointPath = FileNames.checkpointFileSingular(logPath, version); @@ -317,8 +319,10 @@ protected final Optional> listDeltaAndCheckpointFiles( // than the versionToLoad then the versionToLoad is not reconstructable // from the existing logs if (output.isEmpty()) { - throw DeltaErrors.nonReconstructableStateException( - tablePath.toString(), versionToLoad.get()); + long earliestVersion = DeltaHistoryManager.getEarliestRecreatableCommit( + engine, logPath); + throw DeltaErrors.versionBeforeFirstAvailableCommit( + tablePath.toString(), versionToLoad.get(), earliestVersion); } break; } @@ -624,7 +628,7 @@ protected Optional getLogSegmentForVersion( } versionToLoadOpt.filter(v -> v != newVersion).ifPresent(v -> { - throw DeltaErrors.nonExistentVersionException(tablePath.toString(), v, newVersion); + throw DeltaErrors.versionAfterLatestCommit(tablePath.toString(), v, newVersion); }); // We may just be getting a checkpoint file after the filtering diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala index f9d2580a60e..b66f758b94b 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaHistoryManagerSuite.scala @@ -19,10 +19,9 @@ import java.io.FileNotFoundException import scala.reflect.ClassTag +import io.delta.kernel.exceptions.TableNotFoundException import org.scalatest.funsuite.AnyFunSuite - import io.delta.kernel.utils.FileStatus -import io.delta.kernel.TableNotFoundException import io.delta.kernel.test.MockFileSystemClientUtils class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { @@ -66,12 +65,12 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, -1, - DeltaErrors.timestampEarlierThanTableFirstCommitException(dataPath.toString, -1, 0).getMessage + DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, -1, 0, 0).getMessage ) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 21, - DeltaErrors.timestampLaterThanTableLastCommit(dataPath.toString, 21, 20, 2).getMessage + DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 21, 20, 2).getMessage ) } @@ -87,12 +86,12 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, -1, - DeltaErrors.timestampEarlierThanTableFirstCommitException(dataPath.toString, -1, 0).getMessage + DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, -1, 0, 0).getMessage ) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 21, - DeltaErrors.timestampLaterThanTableLastCommit(dataPath.toString, 21, 20, 2).getMessage + DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 21, 20, 2).getMessage ) } @@ -106,12 +105,12 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 8, - DeltaErrors.timestampEarlierThanTableFirstCommitException(dataPath.toString, 8, 20).getMessage + DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, 8, 20, 2).getMessage ) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 31, - DeltaErrors.timestampLaterThanTableLastCommit(dataPath.toString, 31, 30, 3).getMessage + DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 31, 30, 3).getMessage ) } @@ -123,12 +122,12 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 8, - DeltaErrors.timestampEarlierThanTableFirstCommitException(dataPath.toString, 8, 20).getMessage + DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, 8, 20, 2).getMessage ) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 21, - DeltaErrors.timestampLaterThanTableLastCommit(dataPath.toString, 21, 20, 2).getMessage + DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 21, 20, 2).getMessage ) } @@ -142,12 +141,12 @@ class DeltaHistoryManagerSuite extends AnyFunSuite with MockFileSystemClientUtil checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 8, - DeltaErrors.timestampEarlierThanTableFirstCommitException(dataPath.toString, 8, 20).getMessage + DeltaErrors.timestampBeforeFirstAvailableCommit(dataPath.toString, 8, 20, 2).getMessage ) checkGetActiveCommitAtTimestampError[RuntimeException]( deltaFiles, 31, - DeltaErrors.timestampLaterThanTableLastCommit(dataPath.toString, 31, 30, 3).getMessage + DeltaErrors.timestampAfterLatestCommit(dataPath.toString, 31, 30, 3).getMessage ) } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala index 7fb1cfe7b16..db2814ab775 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala @@ -453,13 +453,13 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { files = deltaFileStatuses(Seq(0L)), versionToLoad = Optional.of(15), expectedErrorMessageContains = - "Trying to load a non-existent version 15. The latest version available is 0" + "Cannot load table version 15 as it does not exist. The latest available version is 0" ) testExpectedError[RuntimeException]( files = deltaFileStatuses((10L until 13L)) ++ singularCheckpointFileStatuses(Seq(10L)), versionToLoad = Optional.of(15), expectedErrorMessageContains = - "Trying to load a non-existent version 15. The latest version available is 12" + "Cannot load table version 15 as it does not exist. The latest available version is 12" ) } @@ -512,13 +512,13 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { testExpectedError[RuntimeException]( files, versionToLoad = Optional.of(15), - expectedErrorMessageContains = "Unable to reconstruct state at version 15" + expectedErrorMessageContains = "Cannot load table version 15" ) testExpectedError[RuntimeException]( files, startCheckpoint = Optional.of(20), versionToLoad = Optional.of(15), - expectedErrorMessageContains = "Unable to reconstruct state at version 15" + expectedErrorMessageContains = "Cannot load table version 15" ) } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableFeaturesSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableFeaturesSuite.scala index 2035c73eb8e..e54bca72753 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableFeaturesSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/TableFeaturesSuite.scala @@ -16,6 +16,7 @@ package io.delta.kernel.internal import io.delta.kernel.data.{ArrayValue, ColumnVector, MapValue} +import io.delta.kernel.exceptions.KernelException import io.delta.kernel.internal.TableFeatures.validateWriteSupportedTable import io.delta.kernel.internal.actions.{Format, Metadata, Protocol} import io.delta.kernel.internal.util.InternalUtils.singletonStringColumnVector @@ -84,15 +85,15 @@ class TableFeaturesSuite extends AnyFunSuite { protocol: Protocol, metadata: Metadata = null, schema: StructType = createTestSchema()): Unit = { - validateWriteSupportedTable(protocol, metadata, schema) + validateWriteSupportedTable(protocol, metadata, schema, "/test/table") } def checkUnsupported( protocol: Protocol, metadata: Metadata = null, schema: StructType = createTestSchema()): Unit = { - intercept[UnsupportedOperationException] { - validateWriteSupportedTable(protocol, metadata, schema) + intercept[KernelException] { + validateWriteSupportedTable(protocol, metadata, schema, "/test/table") } } diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/DefaultEngineErrors.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/DefaultEngineErrors.java index 2b08ef04dc1..74eedba0f8e 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/DefaultEngineErrors.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/DefaultEngineErrors.java @@ -17,12 +17,26 @@ import static java.lang.String.format; +import io.delta.kernel.expressions.Expression; + public class DefaultEngineErrors { - // TODO update to be engine exception with future exception framework - // (see delta-io/delta#2231) public static IllegalArgumentException canNotInstantiateLogStore(String logStoreClassName) { return new IllegalArgumentException( format("Can not instantiate `LogStore` class: %s", logStoreClassName)); } + + /** + * Exception for when the default expression evaluator cannot evaluate an expression. + * @param expression the unsupported expression + * @param reason reason for why the expression is not supported/cannot be evaluated + */ + public static UnsupportedOperationException unsupportedExpressionException( + Expression expression, String reason) { + String message = format( + "Default expression evaluator cannot evaluate the expression: %s. Reason: %s", + expression, + reason); + return new UnsupportedOperationException(message); + } } diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluator.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluator.java index 47d56a532ac..b873109d355 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluator.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/DefaultExpressionEvaluator.java @@ -27,14 +27,15 @@ import io.delta.kernel.expressions.*; import io.delta.kernel.types.*; -import io.delta.kernel.internal.DeltaErrors; import static io.delta.kernel.internal.util.ExpressionUtils.getLeft; import static io.delta.kernel.internal.util.ExpressionUtils.getRight; import static io.delta.kernel.internal.util.ExpressionUtils.getUnaryChild; import static io.delta.kernel.internal.util.Preconditions.checkArgument; + import io.delta.kernel.defaults.internal.data.vector.DefaultBooleanVector; import io.delta.kernel.defaults.internal.data.vector.DefaultConstantVector; +import static io.delta.kernel.defaults.internal.DefaultEngineErrors.unsupportedExpressionException; import static io.delta.kernel.defaults.internal.expressions.DefaultExpressionUtils.booleanWrapperVector; import static io.delta.kernel.defaults.internal.expressions.DefaultExpressionUtils.childAt; import static io.delta.kernel.defaults.internal.expressions.DefaultExpressionUtils.compare; @@ -64,9 +65,9 @@ public DefaultExpressionEvaluator( ExpressionTransformResult transformResult = new ExpressionTransformer(inputSchema).visit(expression); if (!transformResult.outputType.equivalent(outputType)) { - String reason = format( - "Can not create an expression handler returns result of type %s", outputType); - throw DeltaErrors.unsupportedExpression(expression, Optional.of(reason)); + String reason = String.format( + "Expression %s does not match expected output type %s", expression, outputType); + throw unsupportedExpressionException(expression, reason); } this.expression = transformResult.expression; } @@ -151,8 +152,9 @@ ExpressionTransformResult visitComparator(Predicate predicate) { transformBinaryComparator(predicate), BooleanType.BOOLEAN); default: - throw DeltaErrors.unsupportedExpression( - predicate, Optional.of("unsupported expression encountered")); + // We should never reach this based on the ExpressionVisitor + throw new IllegalStateException( + String.format("%s is not a recognized comparator", predicate.getName())); } } @@ -193,9 +195,9 @@ ExpressionTransformResult visitPartitionValue(PartitionValueExpression partition if (partitionColType instanceof StructType || partitionColType instanceof ArrayType || partitionColType instanceof MapType) { - throw DeltaErrors.unsupportedExpression( + throw unsupportedExpressionException( partitionValue, - Optional.of("unsupported partition data type: " + partitionColType)); + "unsupported partition data type: " + partitionColType); } return new ExpressionTransformResult( new PartitionValueExpression(serializedPartValueInput.expression, partitionColType), @@ -251,22 +253,22 @@ ExpressionTransformResult visitCoalesce(ScalarExpression coalesce) { .map(this::visit) .collect(Collectors.toList()); if (children.size() == 0) { - throw DeltaErrors.unsupportedExpression( - coalesce, Optional.of("Coalesce requires at least one expression")); + throw unsupportedExpressionException( + coalesce, "Coalesce requires at least one expression"); } // TODO support least-common-type resolution long numDistinctTypes = children.stream().map(e -> e.outputType) .distinct() .count(); if (numDistinctTypes > 1) { - throw DeltaErrors.unsupportedExpression( + throw unsupportedExpressionException( coalesce, - Optional.of("Coalesce is only supported for arguments of the same type")); + "Coalesce is only supported for arguments of the same type"); } // TODO support other data types besides boolean (just needs tests) if (!(children.get(0).outputType instanceof BooleanType)) { - throw new UnsupportedOperationException( - "Coalesce is only supported for boolean type expressions"); + throw unsupportedExpressionException( + coalesce, "Coalesce is only supported for boolean type expressions"); } return new ExpressionTransformResult( new ScalarExpression( @@ -305,7 +307,7 @@ private Expression transformBinaryComparator(Predicate predicate) { String msg = format("operands are of different types which are not " + "comparable: left type=%s, right type=%s", leftResult.outputType, rightResult.outputType); - throw DeltaErrors.unsupportedExpression(predicate, Optional.of(msg)); + throw unsupportedExpressionException(predicate, msg); } } return new Predicate(predicate.getName(), left, right); @@ -437,9 +439,9 @@ ColumnVector visitComparator(Predicate predicate) { } break; default: - throw DeltaErrors.unsupportedExpression( - predicate, - Optional.of("unsupported expression encountered")); + // We should never reach this based on the ExpressionVisitor + throw new IllegalStateException( + String.format("%s is not a recognized comparator", predicate.getName())); } return new DefaultBooleanVector(numRows, Optional.of(nullability), result); diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/ElementAtEvaluator.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/ElementAtEvaluator.java index 199c726c881..afdb34ed213 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/ElementAtEvaluator.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/expressions/ElementAtEvaluator.java @@ -16,7 +16,6 @@ package io.delta.kernel.defaults.internal.expressions; import java.util.Arrays; -import java.util.Optional; import static java.lang.String.format; import io.delta.kernel.data.ColumnVector; @@ -27,10 +26,10 @@ import io.delta.kernel.types.MapType; import io.delta.kernel.types.StringType; -import io.delta.kernel.internal.DeltaErrors; import io.delta.kernel.internal.util.Utils; import static io.delta.kernel.internal.util.Preconditions.checkArgument; +import static io.delta.kernel.defaults.internal.DefaultEngineErrors.unsupportedExpressionException; import static io.delta.kernel.defaults.internal.expressions.ImplicitCastExpression.canCastTo; /** @@ -60,7 +59,7 @@ static ScalarExpression validateAndTransform( String reason = format( "lookup key type (%s) is different from the map key type (%s)", lookupKeyType, asMapType.getKeyType()); - throw DeltaErrors.unsupportedExpression(elementAt, Optional.of(reason)); + throw unsupportedExpressionException(elementAt, reason); } } return new ScalarExpression(elementAt.getName(), Arrays.asList(mapInput, lookupKey)); diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala index b413afddb99..eb4b8dd60a1 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala @@ -17,7 +17,7 @@ package io.delta.kernel.defaults import io.delta.golden.GoldenTableUtils.goldenTablePath import io.delta.kernel.defaults.utils.{TestRow, TestUtils} -import io.delta.kernel.{CheckpointAlreadyExistsException, Table, TableNotFoundException} +import io.delta.kernel.Table import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -32,6 +32,7 @@ import java.io.File import io.delta.kernel.defaults.engine.DefaultEngine import io.delta.kernel.engine.Engine +import io.delta.kernel.exceptions.{CheckpointAlreadyExistsException, TableNotFoundException} /** * Test suite for `io.delta.kernel.Table.checkpoint(engine, version)` @@ -324,7 +325,7 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { val ex2 = intercept[Exception] { kernelCheckpoint(tc, path, checkpointVersion = 5) } - assert(ex2.getMessage.contains("Trying to load a non-existent version 5")) + assert(ex2.getMessage.contains("Cannot load table version 5 as it does not exist")) } } @@ -349,8 +350,8 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils { val ex2 = intercept[Exception] { kernelCheckpoint(tc, tablePath, checkpointVersion = 5) } - assert(ex2.getMessage.contains( - "Unsupported writer protocol version: 7 with feature: deletionVectors")) + assert(ex2.getMessage.contains("Unsupported Delta writer feature") && + ex2.getMessage.contains("writer table feature \"deletionVectors\"")) } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala index ab278d7204f..89794e578d8 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala @@ -16,12 +16,13 @@ package io.delta.kernel.defaults import io.delta.golden.GoldenTableUtils.goldenTablePath +import io.delta.kernel.exceptions.TableNotFoundException import io.delta.kernel.defaults.utils.{TestRow, TestUtils} import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.util.InternalUtils.daysSinceEpoch import io.delta.kernel.internal.util.{DateTimeConstants, FileNames} import io.delta.kernel.types.{LongType, StructType} -import io.delta.kernel.{Table, TableNotFoundException} +import io.delta.kernel.Table import org.apache.hadoop.shaded.org.apache.commons.io.FileUtils import org.apache.spark.sql.functions.col import org.scalatest.funsuite.AnyFunSuite @@ -585,7 +586,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { .getScanBuilder(defaultEngine) .build() } - assert(e.getMessage.contains("Unsupported reader protocol version")) + assert(e.getMessage.contains("Unsupported Delta protocol reader version")) } ////////////////////////////////////////////////////////////////////////////////// @@ -628,7 +629,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { .getSnapshotAsOfVersion(defaultEngine, 11) } assert(e.getMessage.contains( - "Trying to load a non-existent version 11. The latest version available is 10")) + "Cannot load table version 11 as it does not exist. The latest available version is 10")) } } @@ -660,7 +661,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { Table.forPath(defaultEngine, tablePath) .getSnapshotAsOfVersion(defaultEngine, 9) } - assert(e.getMessage.contains("Unable to reconstruct state at version 9")) + assert(e.getMessage.contains("Cannot load table version 9")) // Can read version 10 checkTable( path = tablePath, @@ -774,7 +775,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { } assert(e1.getMessage.contains( s"The provided timestamp ${start + 50 * minuteInMilliseconds} ms " + - s"(2018-10-24T22:04:18Z) is after the latest commit")) + s"(2018-10-24T22:04:18Z) is after the latest available version")) // Timestamp before the first commit fails val e2 = intercept[RuntimeException] { checkTable( @@ -785,7 +786,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { } assert(e2.getMessage.contains( s"The provided timestamp ${start - 1L} ms (2018-10-24T21:14:17.999Z) is before " + - s"the earliest version available.")) + s"the earliest available version")) } } From fe5d931603c8ccc1c6af04f9cb3586b1d78210b3 Mon Sep 17 00:00:00 2001 From: Venki Korukanti Date: Fri, 3 May 2024 19:03:06 -0700 Subject: [PATCH 08/11] [3.2][Kernel][Writes] APIs and impl. for creating new tables (#3016) (Split from #2944) APIs and implementation for creating partitioned or unpartitioned tables. No data insertion yet. Will come in the next PR. Test suite --- .../io/delta/kernel/DataWriteContext.java | 53 ++++ .../main/java/io/delta/kernel/Operation.java | 61 ++++ .../src/main/java/io/delta/kernel/Table.java | 15 + .../java/io/delta/kernel/Transaction.java | 151 +++++++++ .../io/delta/kernel/TransactionBuilder.java | 57 ++++ .../delta/kernel/TransactionCommitResult.java | 58 ++++ .../ConcurrentWriteException.java} | 18 +- .../TableAlreadyExistsException.java | 48 +++ .../exceptions/TableNotFoundException.java | 8 +- .../kernel/internal/DataWriteContextImpl.java | 88 ++++++ .../io/delta/kernel/internal/DeltaErrors.java | 45 ++- .../delta/kernel/internal/SnapshotImpl.java | 32 +- .../io/delta/kernel/internal/TableConfig.java | 12 + .../io/delta/kernel/internal/TableImpl.java | 16 + .../internal/TransactionBuilderImpl.java | 189 +++++++++++ .../kernel/internal/TransactionImpl.java | 174 ++++++++++ .../kernel/internal/actions/CommitInfo.java | 50 ++- .../kernel/internal/actions/Protocol.java | 7 +- .../internal/data/TransactionStateRow.java | 73 +++++ .../kernel/internal/replay/LogReplay.java | 2 +- .../internal/snapshot/SnapshotManager.java | 21 +- .../kernel/internal/util/SchemaUtils.java | 130 +++++++- .../kernel/internal/util/VectorUtils.java | 3 + .../delta/kernel/utils/CloseableIterable.java | 108 +++++++ .../delta/kernel/utils/CloseableIterator.java | 34 ++ .../internal/util/SchemaUtilsSuite.scala | 9 +- .../internal/parquet/ParquetFileWriter.java | 5 +- .../defaults/DeltaTableWritesSuite.scala | 296 ++++++++++++++++++ .../kernel/defaults/utils/TestUtils.scala | 6 + 29 files changed, 1702 insertions(+), 67 deletions(-) create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/DataWriteContext.java create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/Operation.java create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/TransactionCommitResult.java rename kernel/kernel-api/src/main/java/io/delta/kernel/{internal/lang/CloseableIterable.java => exceptions/ConcurrentWriteException.java} (55%) create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableAlreadyExistsException.java create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/internal/DataWriteContextImpl.java create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java create mode 100644 kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterable.java create mode 100644 kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/DataWriteContext.java b/kernel/kernel-api/src/main/java/io/delta/kernel/DataWriteContext.java new file mode 100644 index 00000000000..22003d6c038 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/DataWriteContext.java @@ -0,0 +1,53 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel; + +import java.util.List; +import java.util.Map; + +import io.delta.kernel.annotation.Evolving; +import io.delta.kernel.data.Row; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.expressions.Column; + +/** + * Contains the context for writing data to Delta table. The context is created for each partition + * for partitioned table or once per table for un-partitioned table. It is created using + * {@link Transaction#getWriteContext(Engine, Row, Map)} (String, Map, List)}. + * + * @since 3.2.0 + */ +@Evolving +public interface DataWriteContext { + /** + * Returns the target directory where the data should be written. + * + * @return fully qualified path of the target directory + */ + String getTargetDirectory(); + + /** + * Returns the list of {@link Column} that the connector can optionally collect statistics. Each + * {@link Column} is a reference to a top-level or nested column in the table. + *

+ * Statistics collections can be skipped or collected for a partial list of the returned + * {@link Column}s. When stats are present in the written Delta log, they can be used to + * optimize query performance. + * + * @return schema of the statistics + */ + List getStatisticsColumns(); +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Operation.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Operation.java new file mode 100644 index 00000000000..e231f2921d3 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Operation.java @@ -0,0 +1,61 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel; + +/** + * An operation that can be performed on a Delta table. + *

+ * An operation is tracked as the first line in commit info action inside the Delta Log + * It also shows up when {@code DESCRIBE HISTORY} on the table is executed. + */ +public enum Operation { + + /** + * Recorded when the table is created. + */ + CREATE_TABLE("CREATE TABLE"), + + /** + * Recorded during batch inserts. + */ + WRITE("WRITE"), + + /** + * Recorded during streaming inserts. + */ + STREAMING_UPDATE("STREAMING UPDATE"), + + /** + * For any operation that doesn't fit the above categories. + */ + MANUAL_UPDATE("Manual Update"); + + /** + * Actual value that will be recorded in the transaction log + */ + private final String description; + + Operation(String description) { + this.description = description; + } + + /** + * Returns the string that will be recorded in the transaction log. + */ + public String getDescription() { + return description; + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java index 00309e5aa70..acf2a15dd3a 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Table.java @@ -120,6 +120,21 @@ Snapshot getSnapshotAsOfVersion(Engine engine, long versionId) Snapshot getSnapshotAsOfTimestamp(Engine engine, long millisSinceEpochUTC) throws TableNotFoundException; + /** + * Create a {@link TransactionBuilder} which can create a {@link Transaction} object to mutate + * the table. + * + * @param engine {@link Engine} instance to use. + * @param engineInfo information about the engine that is making the updates. + * @param operation metadata of operation that is being performed. E.g. "insert", "delete". + * @return {@link TransactionBuilder} instance to build the transaction. + * @since 3.2.0 + */ + TransactionBuilder createTransactionBuilder( + Engine engine, + String engineInfo, + Operation operation); + /** * Checkpoint the table at given version. It writes a single checkpoint file. * diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java new file mode 100644 index 00000000000..efea669dd7d --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java @@ -0,0 +1,151 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel; + +import java.util.List; +import java.util.Map; + +import io.delta.kernel.annotation.Evolving; +import io.delta.kernel.data.FilteredColumnarBatch; +import io.delta.kernel.data.Row; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.exceptions.ConcurrentWriteException; +import io.delta.kernel.expressions.Literal; +import io.delta.kernel.types.StructType; +import io.delta.kernel.utils.*; + +/** + * Represents a transaction to mutate a Delta table. + * + * @since 3.2.0 + */ +@Evolving +public interface Transaction { + /** + * Get the schema of the table. If the connector is adding any data to the table through this + * transaction, it should have the same schema as the table schema. + */ + StructType getSchema(Engine engine); + + /** + * Get the list of logical names of the partition columns. This helps the connector to do + * physical partitioning of the data before asking the Kernel to stage the data per partition. + */ + List getPartitionColumns(Engine engine); + + /** + * Get the state of the transaction. The state helps Kernel do the transformations to logical + * data according to the Delta protocol and table features enabled on the table. The engine + * should use this at the data writer task to transform the logical data that the engine wants + * to write to the table in to physical data that goes in data files using + * {@link Transaction#transformLogicalData(Engine, Row, CloseableIterator, Map)} + */ + Row getTransactionState(Engine engine); + + /** + * Commit the transaction including the data action rows generated by + * {@link Transaction#generateAppendActions}. + * + * @param engine {@link Engine} instance. + * @param dataActions Iterable of data actions to commit. These data actions are generated by + * the + * {@link Transaction#generateAppendActions(Engine, Row, CloseableIterator, + * DataWriteContext)}. The {@link CloseableIterable} allows the Kernel to + * access the list of actions multiple times (in case of retries to resolve + * the conflicts due to other writers to the table). Kernel provides a + * in-memory based implementation of {@link CloseableIterable} with utility + * API {@link CloseableIterable#inMemoryIterable(CloseableIterator)} + * @return {@link TransactionCommitResult} status of the successful transaction. + * @throws ConcurrentWriteException when the transaction has encountered a non-retryable + * conflicts or exceeded the maximum number of retries reached. + * The connector needs to rerun the query on top of the latest + * table state and retry the transaction. + */ + TransactionCommitResult commit(Engine engine, CloseableIterable dataActions) + throws ConcurrentWriteException; + + /** + * Given the logical data that needs to be written to the table, convert it into the required + * physical data depending upon the table Delta protocol and features enabled on the table. + * Kernel takes care of adding any additional column or removing existing columns that doesn't + * need to be in physical data files. All these transformations are driven by the Delta protocol + * and table features enabled on the table. + *

+ * The given data should belong to exactly one partition. It is the job of the connector to do + * partitioning of the data before calling the API. Partition values are provided as map of + * column name to partition value (as {@link Literal}). If the table is an un-partitioned table, + * then map should be empty. + * + * @param engine {@link Engine} instance to use. + * @param transactionState The transaction state + * @param dataIter Iterator of logical data (with schema same as the table schema) + * to transform to physical data. All the data n this iterator should + * belong to one physical partition and it should also include the + * partition data. + * @param partitionValues The partition values for the data. If the table is un-partitioned, + * the map should be empty + * @return Iterator of physical data to write to the data files. + */ + static CloseableIterator transformLogicalData( + Engine engine, + Row transactionState, + CloseableIterator dataIter, + Map partitionValues) { + throw new UnsupportedOperationException("Not implemented yet"); + } + + /** + * Get the context for writing data into a table. The context tells the connector where the data + * should be written. For partitioned table context is generated per partition. So, the + * connector should call this API for each partition. For un-partitioned table, the context is + * same for all the data. + * + * @param engine {@link Engine} instance to use. + * @param transactionState The transaction state + * @param partitionValues The partition values for the data. If the table is un-partitioned, + * the map should be empty + * @return {@link DataWriteContext} containing metadata about where and how the data for + * partition should be written. + */ + static DataWriteContext getWriteContext( + Engine engine, + Row transactionState, + Map partitionValues) { + throw new UnsupportedOperationException("Not implemented yet"); + } + + /** + * For given data files, generate Delta actions that can be committed in a transaction. + * These data files are the result of writing the data returned by + * {@link Transaction#transformLogicalData} with the context returned by + * {@link Transaction#getWriteContext}. + * + * @param engine {@link Engine} instance. + * @param transactionState State of the transaction. + * @param fileStatusIter Iterator of row objects representing each data file written. + * @param dataWriteContext The context used when writing the data files given in + * {@code fileStatusIter} + * @return {@link CloseableIterator} of {@link Row} representing the actions to commit using + * {@link Transaction#commit}. + */ + static CloseableIterator generateAppendActions( + Engine engine, + Row transactionState, + CloseableIterator fileStatusIter, + DataWriteContext dataWriteContext) { + throw new UnsupportedOperationException("Not implemented yet"); + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java b/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java new file mode 100644 index 00000000000..9c4ed484cc7 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionBuilder.java @@ -0,0 +1,57 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel; + +import java.util.List; + +import io.delta.kernel.annotation.Evolving; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.types.StructType; + +/** + * Builder for creating a {@link Transaction} to mutate a Delta table. + * + * @since 3.2.0 + */ +@Evolving +public interface TransactionBuilder { + /** + * Set the schema of the table when creating a new table. + * + * @param engine {@link Engine} instance to use. + * @param schema The new schema of the table. + * @return updated {@link TransactionBuilder} instance. + */ + TransactionBuilder withSchema(Engine engine, StructType schema); + + /** + * Set the list of partitions columns when create a new partitioned table. + * + * @param engine {@link Engine} instance to use. + * @param partitionColumns The partition columns of the table. These should be a subset of the + * columns in the schema. + * @return updated {@link TransactionBuilder} instance. + */ + TransactionBuilder withPartitionColumns(Engine engine, List partitionColumns); + + /** + * Build the transaction. Also validates the given info to ensure that a valida transaction + * can be created. + * + * @param engine {@link Engine} instance to use. + */ + Transaction build(Engine engine); +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionCommitResult.java b/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionCommitResult.java new file mode 100644 index 00000000000..961491982b2 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/TransactionCommitResult.java @@ -0,0 +1,58 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel; + +import io.delta.kernel.annotation.Evolving; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.utils.CloseableIterable; + +/** + * Contains the result of a successful transaction commit. Returned by + * {@link Transaction#commit(Engine, CloseableIterable)}. + * + * @since 3.2.0 + */ +@Evolving +public class TransactionCommitResult { + private final long version; + private final boolean isReadyForCheckpoint; + + public TransactionCommitResult(long version, boolean isReadyForCheckpoint) { + this.version = version; + this.isReadyForCheckpoint = isReadyForCheckpoint; + } + + /** + * Contains the version of the transaction committed as. + * + * @return version the transaction is committed as. + */ + public long getVersion() { + return version; + } + + /** + * Is the table ready for checkpoint (i.e. there are enough commits since the last checkpoint)? + * If yes the connector can choose to checkpoint as the version the transaction is committed as + * using {@link Table#checkpoint(Engine, long)} + * + * @return Is the table ready for checkpointing? + */ + public boolean isReadyForCheckpoint() { + return isReadyForCheckpoint; + } +} + diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/lang/CloseableIterable.java b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/ConcurrentWriteException.java similarity index 55% rename from kernel/kernel-api/src/main/java/io/delta/kernel/internal/lang/CloseableIterable.java rename to kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/ConcurrentWriteException.java index f82f1aeecba..0b863d545b0 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/lang/CloseableIterable.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/ConcurrentWriteException.java @@ -13,11 +13,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package io.delta.kernel.exceptions; -package io.delta.kernel.internal.lang; +import io.delta.kernel.annotation.Evolving; -import io.delta.kernel.utils.CloseableIterator; - -public interface CloseableIterable { - CloseableIterator iterator(); +/** + * Thrown when a concurrent transaction has written data after the current transaction has started. + * + * @since 3.2.0 + */ +@Evolving +public class ConcurrentWriteException extends KernelException { + public ConcurrentWriteException() { + super("Transaction has encountered a conflict and can not be committed. " + + "Query needs to be re-executed using the latest version of the table."); + } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableAlreadyExistsException.java b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableAlreadyExistsException.java new file mode 100644 index 00000000000..d9d796c778e --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableAlreadyExistsException.java @@ -0,0 +1,48 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.exceptions; + +import java.util.Optional; + +import io.delta.kernel.annotation.Evolving; + +/** + * Thrown when trying to create a Delta table at a location where a Delta table already exists. + * + * @since 3.2.0 + */ +@Evolving +public class TableAlreadyExistsException extends KernelException { + private final String tablePath; + private final Optional context; + + public TableAlreadyExistsException(String tablePath, String context) { + this.tablePath = tablePath; + this.context = Optional.ofNullable(context); + } + + public TableAlreadyExistsException(String tablePath) { + this(tablePath, null); + } + + @Override + public String getMessage() { + return String.format( + "Delta table already exists at `%s`.%s", + tablePath, + context.map(c -> " Context: " + c).orElse("")); + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableNotFoundException.java b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableNotFoundException.java index feaeb78f728..cd1b67e52ab 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableNotFoundException.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/exceptions/TableNotFoundException.java @@ -13,7 +13,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package io.delta.kernel.exceptions; import io.delta.kernel.annotation.Evolving; @@ -32,8 +31,11 @@ public TableNotFoundException(String tablePath) { this(tablePath, null); } - public TableNotFoundException(String tablePath, Throwable cause) { - super(String.format("Delta table at path `%s` is not found", tablePath), cause); + public TableNotFoundException(String tablePath, String context) { + super(String.format( + "Delta table at path `%s` is not found.%s", + tablePath, + context == null ? "" : " Context: " + context)); this.tablePath = tablePath; } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DataWriteContextImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DataWriteContextImpl.java new file mode 100644 index 00000000000..02f5cebeeb9 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DataWriteContextImpl.java @@ -0,0 +1,88 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal; + +import java.util.List; +import java.util.Map; +import static java.util.Collections.unmodifiableList; +import static java.util.Collections.unmodifiableMap; + +import io.delta.kernel.DataWriteContext; +import io.delta.kernel.expressions.Column; +import io.delta.kernel.expressions.Literal; + +/** + * Implements the {@link DataWriteContext} interface. In addition to the data needed for the + * interface, it also contains the partition values of the targeted partition. In case of + * un-partitioned tables, the partition values will be empty. + */ +public class DataWriteContextImpl implements DataWriteContext { + private final String targetDirectory; + private final Map partitionValues; + private final List statsColumns; + + /** + * Creates a new instance of WriteContext. + * + * @param targetDirectory fully qualified path of the target directory + * @param partitionValues partition values for the data to be written. If the table is + * un-partitioned, this should be an empty map. + * @param statsColumns Set of columns that need statistics for the data to be written. + * The column can be a top-level column or a nested column. + * E.g. "a.b.c" is a nested column. "d" is a top-level column. + */ + public DataWriteContextImpl( + String targetDirectory, + Map partitionValues, + List statsColumns) { + this.targetDirectory = targetDirectory; + this.partitionValues = unmodifiableMap(partitionValues); + this.statsColumns = unmodifiableList(statsColumns); + } + + /** + * Returns the target directory where the data should be written. + * + * @return fully qualified path of the target directory + */ + public String getTargetDirectory() { + return targetDirectory; + } + + /** + * Returns the partition values for the data to be written. If the table is un-partitioned, + * this should be an empty map. + * + * @return partition values + */ + public Map getPartitionValues() { + return partitionValues; + } + + /** + * Returns the list of {@link Column} that the connector can optionally collect statistics. Each + * {@link Column} is a reference to a top-level or nested column in the table. + *

+ * Statistics collections can be skipped or collected for a partial list of the returned + * {@link Column}s. When stats are present in the written Delta log, they can be used to + * optimize query performance. + * + * @return schema of the statistics + */ + public List getStatisticsColumns() { + return statsColumns; + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java index 5c4eb9374f5..7e4dc4ab7fd 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java @@ -16,8 +16,12 @@ package io.delta.kernel.internal; import java.sql.Timestamp; +import java.util.List; +import static java.lang.String.format; -import io.delta.kernel.exceptions.KernelException; +import io.delta.kernel.exceptions.*; +import io.delta.kernel.types.DataType; +import io.delta.kernel.types.StructType; /** * Contains methods to create user-facing Delta exceptions. @@ -130,8 +134,45 @@ public static KernelException columnInvariantsNotSupported() { return new KernelException(message); } - /* ------------------------ HELPER METHODS ----------------------------- */ + public static KernelException unsupportedDataType(DataType dataType) { + return new KernelException("Kernel doesn't support writing data of type: " + dataType); + } + + public static KernelException unsupportedPartitionDataType(String colName, DataType dataType) { + String msgT = "Kernel doesn't support writing data with partition column (%s) of type: %s"; + return new KernelException(format(msgT, colName, dataType)); + } + public static KernelException duplicateColumnsInSchema( + StructType schema, + List duplicateColumns) { + String msg = format( + "Schema contains duplicate columns: %s.\nSchema: %s", + String.join(", ", duplicateColumns), + schema); + return new KernelException(msg); + } + + public static KernelException invalidColumnName( + String columnName, + String unsupportedChars) { + return new KernelException(format( + "Column name '%s' contains one of the unsupported (%s) characters.", + columnName, + unsupportedChars)); + } + + public static KernelException requiresSchemaForNewTable(String tablePath) { + return new TableNotFoundException( + tablePath, + "Must provide a new schema to write to a new table."); + } + + public static KernelException tableAlreadyExists(String tablePath, String message) { + return new TableAlreadyExistsException(tablePath, message); + } + + /* ------------------------ HELPER METHODS ----------------------------- */ private static String formatTimestamp(long millisSinceEpochUTC) { return new Timestamp(millisSinceEpochUTC).toInstant().toString(); } 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 431e13f4d27..8aff3b37776 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 @@ -28,13 +28,13 @@ import io.delta.kernel.internal.replay.CreateCheckpointIterator; import io.delta.kernel.internal.replay.LogReplay; import io.delta.kernel.internal.snapshot.LogSegment; -import io.delta.kernel.internal.snapshot.SnapshotHint; import static io.delta.kernel.internal.TableConfig.TOMBSTONE_RETENTION; /** * Implementation of {@link Snapshot}. */ public class SnapshotImpl implements Snapshot { + private final Path logPath; private final Path dataPath; private final long version; private final LogReplay logReplay; @@ -43,24 +43,18 @@ public class SnapshotImpl implements Snapshot { private final LogSegment logSegment; public SnapshotImpl( - Path logPath, Path dataPath, - long version, LogSegment logSegment, - Engine engine, - long timestamp, - Optional snapshotHint) { + LogReplay logReplay, + Protocol protocol, + Metadata metadata) { + this.logPath = new Path(dataPath, "_delta_log"); this.dataPath = dataPath; - this.version = version; + this.version = logSegment.version; this.logSegment = logSegment; - this.logReplay = new LogReplay( - logPath, - dataPath, - version, engine, - logSegment, - snapshotHint); - this.protocol = logReplay.getProtocol(); - this.metadata = logReplay.getMetadata(); + this.logReplay = logReplay; + this.protocol = protocol; + this.metadata = metadata; } @Override @@ -119,4 +113,12 @@ public Optional getLatestTransactionVersion(String applicationId) { public LogSegment getLogSegment() { return logSegment; } + + public Path getLogPath() { + return logPath; + } + + public Path getDataPath() { + return dataPath; + } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableConfig.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableConfig.java index bc3ed026ec6..736c3d76c33 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableConfig.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableConfig.java @@ -48,6 +48,18 @@ public class TableConfig { " and years are not accepted. You may specify '365 days' for a year instead." ); + /** + * How often to checkpoint the delta log? For every N (this config) commits to the log, we will + * suggest write out a checkpoint file that can speed up the Delta table state reconstruction. + */ + public static final TableConfig CHECKPOINT_INTERVAL = new TableConfig<>( + "delta.checkpointInterval", + "10", + Integer::valueOf, + value -> value > 0, + "needs to be a positive integer." + ); + private final String key; private final String defaultValue; private final Function fromString; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java index f022a3b99e9..235f48ee47a 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TableImpl.java @@ -72,4 +72,20 @@ public void checkpoint(Engine engine, long version) throws TableNotFoundException, CheckpointAlreadyExistsException, IOException { snapshotManager.checkpoint(engine, version); } + + @Override + public TransactionBuilder createTransactionBuilder( + Engine engine, + String engineInfo, + Operation operation) { + return new TransactionBuilderImpl(this, engineInfo, operation); + } + + protected Path getDataPath() { + return new Path(tablePath); + } + + protected Path getLogPath() { + return new Path(tablePath, "_delta_log"); + } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java new file mode 100644 index 00000000000..73b214fdd37 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java @@ -0,0 +1,189 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal; + +import java.util.*; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.delta.kernel.*; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.exceptions.TableNotFoundException; +import io.delta.kernel.types.StructType; + +import io.delta.kernel.internal.actions.*; +import io.delta.kernel.internal.fs.Path; +import io.delta.kernel.internal.replay.LogReplay; +import io.delta.kernel.internal.snapshot.LogSegment; +import io.delta.kernel.internal.snapshot.SnapshotHint; +import io.delta.kernel.internal.util.SchemaUtils; +import io.delta.kernel.internal.util.Tuple2; +import static io.delta.kernel.internal.DeltaErrors.requiresSchemaForNewTable; +import static io.delta.kernel.internal.DeltaErrors.tableAlreadyExists; +import static io.delta.kernel.internal.TransactionImpl.DEFAULT_READ_VERSION; +import static io.delta.kernel.internal.TransactionImpl.DEFAULT_WRITE_VERSION; +import static io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames; +import static io.delta.kernel.internal.util.VectorUtils.stringArrayValue; +import static io.delta.kernel.internal.util.VectorUtils.stringStringMapValue; + +public class TransactionBuilderImpl implements TransactionBuilder { + private static final Logger logger = LoggerFactory.getLogger(TransactionBuilderImpl.class); + + private final long currentTimeMillis = System.currentTimeMillis(); + private final TableImpl table; + private final String engineInfo; + private final Operation operation; + private Optional schema = Optional.empty(); + private Optional> partitionColumns = Optional.empty(); + + public TransactionBuilderImpl(TableImpl table, String engineInfo, Operation operation) { + this.table = table; + this.engineInfo = engineInfo; + this.operation = operation; + } + + @Override + public TransactionBuilder withSchema(Engine engine, StructType newSchema) { + this.schema = Optional.of(newSchema); // will be verified as part of the build() call + return this; + } + + @Override + public TransactionBuilder withPartitionColumns(Engine engine, List partitionColumns) { + if (!partitionColumns.isEmpty()) { + this.partitionColumns = Optional.of(partitionColumns); + } + return this; + } + + @Override + public Transaction build(Engine engine) { + SnapshotImpl snapshot; + try { + snapshot = (SnapshotImpl) table.getLatestSnapshot(engine); + } catch (TableNotFoundException tblf) { + String tablePath = table.getPath(engine); + logger.info("Table {} doesn't exist yet. Trying to create a new table.", tablePath); + schema.orElseThrow(() -> requiresSchemaForNewTable(tablePath)); + // Table doesn't exist yet. Create an initial snapshot with the new schema. + Metadata metadata = getInitialMetadata(); + Protocol protocol = getInitialProtocol(); + LogReplay logReplay = getEmptyLogReplay(engine, metadata, protocol); + snapshot = new InitialSnapshot(table.getDataPath(), logReplay, metadata, protocol); + } + + boolean isNewTable = snapshot.getVersion(engine) < 0; + validate(engine, snapshot, isNewTable); + + return new TransactionImpl( + isNewTable, + table.getDataPath(), + table.getLogPath(), + snapshot, + engineInfo, + operation, + snapshot.getProtocol(), + snapshot.getMetadata()); + } + + /** + * Validate the given parameters for the transaction. + */ + private void validate(Engine engine, SnapshotImpl snapshot, boolean isNewTable) { + String tablePath = table.getPath(engine); + // Validate the table has no features that Kernel doesn't yet support writing into it. + TableFeatures.validateWriteSupportedTable( + snapshot.getProtocol(), + snapshot.getMetadata(), + snapshot.getMetadata().getSchema(), + tablePath); + + if (!isNewTable) { + if (schema.isPresent()) { + throw tableAlreadyExists( + tablePath, + "Table already exists, but provided a new schema. " + + "Schema can only be set on a new table."); + } + if (partitionColumns.isPresent()) { + throw tableAlreadyExists( + tablePath, + "Table already exists, but provided new partition columns. " + + "Partition columns can only be set on a new table."); + } + } else { + // New table verify the given schema and partition columns + SchemaUtils.validateSchema(schema.get(), false /* isColumnMappingEnabled */); + SchemaUtils.validatePartitionColumns( + schema.get(), partitionColumns.orElse(Collections.emptyList())); + } + } + + private class InitialSnapshot extends SnapshotImpl { + InitialSnapshot(Path dataPath, LogReplay logReplay, Metadata metadata, Protocol protocol) { + super(dataPath, LogSegment.empty(table.getLogPath()), logReplay, protocol, metadata); + } + } + + private LogReplay getEmptyLogReplay(Engine engine, Metadata metadata, Protocol protocol) { + return new LogReplay( + table.getLogPath(), + table.getDataPath(), + -1, + engine, + LogSegment.empty(table.getLogPath()), + Optional.empty()) { + + @Override + protected Tuple2 loadTableProtocolAndMetadata( + Optional snapshotHint, long snapshotVersion) { + return new Tuple2<>(protocol, metadata); + } + + @Override + public Optional getLatestTransactionIdentifier(String applicationId) { + return Optional.empty(); + } + }; + } + + private Metadata getInitialMetadata() { + List partitionColumnsCasePreserving = casePreservingPartitionColNames( + schema.get(), + partitionColumns.orElse(Collections.emptyList())); + + return new Metadata( + java.util.UUID.randomUUID().toString(), /* id */ + Optional.empty(), /* name */ + Optional.empty(), /* description */ + new Format(), /* format */ + schema.get().toJson(), /* schemaString */ + schema.get(), /* schema */ + stringArrayValue(partitionColumnsCasePreserving), /* partitionColumns */ + Optional.of(currentTimeMillis), /* createdTime */ + stringStringMapValue(Collections.emptyMap()) /* configuration */ + ); + } + + private Protocol getInitialProtocol() { + return new Protocol( + DEFAULT_READ_VERSION, + DEFAULT_WRITE_VERSION, + null /* readerFeatures */, + null /* writerFeatures */); + } +} 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 new file mode 100644 index 00000000000..8bead35758f --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java @@ -0,0 +1,174 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal; + +import java.io.IOException; +import java.nio.file.FileAlreadyExistsException; +import java.util.*; +import java.util.stream.Collectors; + +import io.delta.kernel.*; +import io.delta.kernel.data.Row; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.exceptions.ConcurrentWriteException; +import io.delta.kernel.types.StructType; +import io.delta.kernel.utils.CloseableIterable; +import io.delta.kernel.utils.CloseableIterator; + +import io.delta.kernel.internal.actions.*; +import io.delta.kernel.internal.data.TransactionStateRow; +import io.delta.kernel.internal.fs.Path; +import io.delta.kernel.internal.util.FileNames; +import io.delta.kernel.internal.util.VectorUtils; +import static io.delta.kernel.internal.TableConfig.CHECKPOINT_INTERVAL; +import static io.delta.kernel.internal.actions.SingleAction.*; +import static io.delta.kernel.internal.util.Preconditions.checkState; +import static io.delta.kernel.internal.util.Utils.toCloseableIterator; + +public class TransactionImpl + implements Transaction { + public static final int DEFAULT_READ_VERSION = 1; + public static final int DEFAULT_WRITE_VERSION = 2; + + private final UUID txnId = UUID.randomUUID(); + + private final boolean isNewTable; // the transaction is creating a new table + private final String engineInfo; + private final Operation operation; + private final Path dataPath; + private final Path logPath; + private final Protocol protocol; + private final Metadata metadata; + private final SnapshotImpl readSnapshot; + + private boolean closed; // To avoid trying to commit the same transaction again. + + public TransactionImpl( + boolean isNewTable, + Path dataPath, + Path logPath, + SnapshotImpl readSnapshot, + String engineInfo, + Operation operation, + Protocol protocol, + Metadata metadata) { + this.isNewTable = isNewTable; + this.dataPath = dataPath; + this.logPath = logPath; + this.readSnapshot = readSnapshot; + this.engineInfo = engineInfo; + this.operation = operation; + this.protocol = protocol; + this.metadata = metadata; + } + + @Override + public Row getTransactionState(Engine engine) { + return TransactionStateRow.of(metadata, dataPath.toString()); + } + + @Override + public List getPartitionColumns(Engine engine) { + return VectorUtils.toJavaList(metadata.getPartitionColumns()); + } + + @Override + public StructType getSchema(Engine engine) { + return readSnapshot.getSchema(engine); + } + + @Override + public TransactionCommitResult commit( + Engine engine, + CloseableIterable dataActions) + throws ConcurrentWriteException { + checkState( + !closed, + "Transaction is already attempted to commit. Create a new transaction."); + List metadataActions = new ArrayList<>(); + metadataActions.add(createCommitInfoSingleAction(generateCommitAction())); + if (isNewTable) { + // In the future, we need to add metadata and action when there are any changes to them. + metadataActions.add(createMetadataSingleAction(metadata.toRow())); + metadataActions.add(createProtocolSingleAction(protocol.toRow())); + } + + try (CloseableIterator stageDataIter = dataActions.iterator()) { + // Create a new CloseableIterator that will return the metadata actions followed by the + // data actions. + CloseableIterator dataAndMetadataActions = + toCloseableIterator(metadataActions.iterator()).combine(stageDataIter); + + try { + long readVersion = readSnapshot.getVersion(engine); + if (readVersion == -1) { + // New table, create a delta log directory + if (!engine.getFileSystemClient().mkdirs(logPath.toString())) { + throw new RuntimeException( + "Failed to create delta log directory: " + logPath); + } + } + + long newVersion = readVersion + 1; + // Write the staged data to a delta file + engine.getJsonHandler().writeJsonFileAtomically( + FileNames.deltaFile(logPath, newVersion), + dataAndMetadataActions, + false /* overwrite */); + + return new TransactionCommitResult(newVersion, isReadyForCheckpoint(newVersion)); + } catch (FileAlreadyExistsException e) { + // TODO: Resolve conflicts and retry commit + throw new ConcurrentWriteException(); + } + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } finally { + closed = true; + } + } + + private Row generateCommitAction() { + return new CommitInfo( + System.currentTimeMillis(), /* timestamp */ + "Kernel-" + Meta.KERNEL_VERSION + "/" + engineInfo, /* engineInfo */ + operation.getDescription(), /* description */ + getOperationParameters(), /* operationParameters */ + isBlindAppend(), /* isBlindAppend */ + txnId.toString() /* txnId */ + ).toRow(); + } + + private boolean isReadyForCheckpoint(long newVersion) { + int checkpointInterval = CHECKPOINT_INTERVAL.fromMetadata(metadata); + return newVersion > 0 && newVersion % checkpointInterval == 0; + } + + private boolean isBlindAppend() { + return isNewTable; // Later can add more conditions to determine if it is a blind append + } + + private Map getOperationParameters() { + if (isNewTable) { + List partitionCols = VectorUtils.toJavaList(metadata.getPartitionColumns()); + String partitionBy = partitionCols.stream() + .map(col -> "\"" + col + "\"") + .collect(Collectors.joining(",", "[", "]")); + return Collections.singletonMap("partitionBy", partitionBy); + } + return Collections.emptyMap(); + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/CommitInfo.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/CommitInfo.java index bc42a9b5d23..aea3461bb41 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/CommitInfo.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/CommitInfo.java @@ -15,21 +15,28 @@ */ package io.delta.kernel.internal.actions; -import java.util.HashMap; -import java.util.Map; +import java.util.*; +import java.util.stream.IntStream; +import static java.util.stream.Collectors.toMap; import io.delta.kernel.data.Row; import io.delta.kernel.types.*; import io.delta.kernel.internal.data.GenericRow; +import static io.delta.kernel.internal.util.VectorUtils.stringStringMapValue; /** - * Delta log action representing a commit information action. According to the Delta protocol - * there isn't any specific schema for this action, but we use the following schema: + * Delta log action representing a commit information action. According to the Delta protocol there + * isn't any specific schema for this action, but we use the following schema: *

    *
  • timestamp: Long - Milliseconds since epoch UTC of when this commit happened
  • *
  • engineInfo: String - Engine that made this commit
  • *
  • operation: String - Operation (e.g. insert, delete, merge etc.)
  • + *
  • operationParameters: Map(String, String) - each operation depending upon the type + * may add zero or more parameters about the operation. E.g. when creating a table + * `partitionBy` key with list of partition columns is added.
  • + *
  • isBlindAppend: Boolean - Is this commit a blind append?
  • + *
  • txnId: String - a unique transaction id of this commit
  • *
* The Delta-Spark connector adds lot more fields to this action. We can add them as needed. */ @@ -38,16 +45,37 @@ public class CommitInfo { public static StructType FULL_SCHEMA = new StructType() .add("timestamp", LongType.LONG) .add("engineInfo", StringType.STRING) - .add("operation", StringType.STRING); + .add("operation", StringType.STRING) + .add("operationParameters", + new MapType(StringType.STRING, StringType.STRING, true /* nullable */)) + .add("isBlindAppend", BooleanType.BOOLEAN, true /* nullable */) + .add("txnId", StringType.STRING); + + private static final Map COL_NAME_TO_ORDINAL = + IntStream.range(0, FULL_SCHEMA.length()) + .boxed() + .collect(toMap(i -> FULL_SCHEMA.at(i).getName(), i -> i)); private final long timestamp; private final String engineInfo; private final String operation; + private final Map operationParameters; + private final boolean isBlindAppend; + private final String txnId; - public CommitInfo(long timestamp, String engineInfo, String operation) { + public CommitInfo( + long timestamp, + String engineInfo, + String operation, + Map operationParameters, + boolean isBlindAppend, + String txnId) { this.timestamp = timestamp; this.engineInfo = engineInfo; this.operation = operation; + this.operationParameters = Collections.unmodifiableMap(operationParameters); + this.isBlindAppend = isBlindAppend; + this.txnId = txnId; } public long getTimestamp() { @@ -69,9 +97,13 @@ public String getOperation() { */ public Row toRow() { Map commitInfo = new HashMap<>(); - commitInfo.put(0, timestamp); - commitInfo.put(1, engineInfo); - commitInfo.put(2, operation); + commitInfo.put(COL_NAME_TO_ORDINAL.get("timestamp"), timestamp); + commitInfo.put(COL_NAME_TO_ORDINAL.get("engineInfo"), engineInfo); + commitInfo.put(COL_NAME_TO_ORDINAL.get("operation"), operation); + commitInfo.put(COL_NAME_TO_ORDINAL.get("operationParameters"), + stringStringMapValue(operationParameters)); + commitInfo.put(COL_NAME_TO_ORDINAL.get("isBlindAppend"), isBlindAppend); + commitInfo.put(COL_NAME_TO_ORDINAL.get("txnId"), txnId); return new GenericRow(CommitInfo.FULL_SCHEMA, commitInfo); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java index c757428303b..2098092824f 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/Protocol.java @@ -17,8 +17,7 @@ import java.util.*; -import io.delta.kernel.data.ColumnVector; -import io.delta.kernel.data.Row; +import io.delta.kernel.data.*; import io.delta.kernel.types.ArrayType; import io.delta.kernel.types.IntegerType; import io.delta.kernel.types.StringType; @@ -103,8 +102,8 @@ public Row toRow() { Map protocolMap = new HashMap<>(); protocolMap.put(0, minReaderVersion); protocolMap.put(1, minWriterVersion); - protocolMap.put(2, readerFeatures == null ? null : stringArrayValue(readerFeatures)); - protocolMap.put(3, writerFeatures == null ? null : stringArrayValue(writerFeatures)); + protocolMap.put(2, stringArrayValue(readerFeatures)); + protocolMap.put(3, stringArrayValue(writerFeatures)); return new GenericRow(Protocol.FULL_SCHEMA, protocolMap); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java new file mode 100644 index 00000000000..80659c054af --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java @@ -0,0 +1,73 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.internal.data; + +import java.util.*; +import java.util.stream.IntStream; +import static java.util.stream.Collectors.toMap; + +import io.delta.kernel.Transaction; +import io.delta.kernel.data.Row; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.types.*; + +import io.delta.kernel.internal.actions.Metadata; +import io.delta.kernel.internal.util.VectorUtils; + +public class TransactionStateRow extends GenericRow { + private static final StructType SCHEMA = new StructType() + .add("partitionColumns", new ArrayType(StringType.STRING, false)) + .add("tablePath", StringType.STRING); + + private static final Map COL_NAME_TO_ORDINAL = + IntStream.range(0, SCHEMA.length()) + .boxed() + .collect(toMap(i -> SCHEMA.at(i).getName(), i -> i)); + + public static TransactionStateRow of(Metadata metadata, String tablePath) { + HashMap valueMap = new HashMap<>(); + valueMap.put(COL_NAME_TO_ORDINAL.get("partitionColumns"), metadata.getPartitionColumns()); + valueMap.put(COL_NAME_TO_ORDINAL.get("tablePath"), tablePath); + return new TransactionStateRow(valueMap); + } + + private TransactionStateRow(HashMap valueMap) { + super(SCHEMA, valueMap); + } + + /** + * Get the list of partition column names from the write state {@link Row} returned by + * {@link Transaction#getTransactionState(Engine)} + * + * @param transactionState Scan state {@link Row} + * @return List of partition column names according to the scan state. + */ + public static List getPartitionColumnsList(Row transactionState) { + return VectorUtils.toJavaList( + transactionState.getArray(COL_NAME_TO_ORDINAL.get("partitionColumns"))); + } + + /** + * Get the table path from scan state {@link Row} returned by + * {@link Transaction#getTransactionState(Engine)} + * + * @param transactionState Transaction state state {@link Row} + * @return Fully qualified path to the location of the table. + */ + public static String getTablePath(Row transactionState) { + return transactionState.getString(COL_NAME_TO_ORDINAL.get("tablePath")); + } +} 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 a3952bd5f76..c6a4c945f16 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 @@ -187,7 +187,7 @@ public CloseableIterator getAddFilesAsColumnarBatches( * delta files newer than the hint to search for any new P & M. If we don't find them, we can * just use the P and/or M from the hint. */ - private Tuple2 loadTableProtocolAndMetadata( + protected Tuple2 loadTableProtocolAndMetadata( Optional snapshotHint, long snapshotVersion) { diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java index 49695ae7bce..c4876008341 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java @@ -40,6 +40,7 @@ import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.lang.ListUtils; import io.delta.kernel.internal.replay.CreateCheckpointIterator; +import io.delta.kernel.internal.replay.LogReplay; import io.delta.kernel.internal.util.FileNames; import io.delta.kernel.internal.util.Tuple2; import static io.delta.kernel.internal.TableFeatures.validateWriteSupportedTable; @@ -365,18 +366,26 @@ private SnapshotImpl createSnapshot( .orElse("."); logger.info("{}: Loading version {} {}", tablePath, initSegment.version, startingFromStr); + + LogReplay logReplay = new LogReplay( + logPath, + tablePath, + initSegment.version, + engine, + initSegment, + Optional.ofNullable(latestSnapshotHint.get())); + long startTimeMillis = System.currentTimeMillis(); assertLogFilesBelongToTable(logPath, initSegment.allLogFilesUnsorted()); final SnapshotImpl snapshot = new SnapshotImpl( - logPath, tablePath, - initSegment.version, - initSegment, engine, - initSegment.lastCommitTimestamp, - Optional.ofNullable(latestSnapshotHint.get()) - ); + initSegment, + logReplay, + logReplay.getProtocol(), + logReplay.getMetadata()); + logger.info( "{}: Took {}ms to construct the snapshot (loading protocol and metadata) for {} {}", tablePath, diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java index aeb533434d1..e8178029f9f 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java @@ -17,10 +17,11 @@ import java.util.*; import java.util.stream.Collectors; -import static java.lang.String.format; import io.delta.kernel.types.*; +import io.delta.kernel.internal.DeltaErrors; +import static io.delta.kernel.internal.DeltaErrors.*; import static io.delta.kernel.internal.util.Preconditions.checkArgument; /** @@ -33,8 +34,8 @@ private SchemaUtils() { } /** - * Validate the schema. This method checks if the schema has no duplicate columns and the names - * contain only valid characters. + * Validate the schema. This method checks if the schema has no duplicate columns, the names + * contain only valid characters and the data types are supported. * * @param schema the schema to validate * @param isColumnMappingEnabled whether column mapping is enabled. When column mapping is @@ -44,6 +45,8 @@ private SchemaUtils() { * @throws IllegalArgumentException if the schema is invalid */ public static void validateSchema(StructType schema, boolean isColumnMappingEnabled) { + checkArgument(schema.length() > 0, "Schema should contain at least one column"); + List flattenColNames = flattenNestedFieldNames(schema); // check there are no duplicate column names in the schema @@ -53,12 +56,11 @@ public static void validateSchema(StructType schema, boolean isColumnMappingEnab if (uniqueColNames.size() != flattenColNames.size()) { Set uniqueCols = new HashSet<>(); - String duplicateColumns = flattenColNames.stream() + List duplicateColumns = flattenColNames.stream() .map(String::toLowerCase) .filter(n -> !uniqueCols.add(n)) - .collect(Collectors.joining(", ")); - throw new IllegalArgumentException( - "Schema contains duplicate columns: " + duplicateColumns); + .collect(Collectors.toList()); + throw DeltaErrors.duplicateColumnsInSchema(schema, duplicateColumns); } // Check the column names are valid @@ -66,10 +68,70 @@ public static void validateSchema(StructType schema, boolean isColumnMappingEnab validParquetColumnNames(flattenColNames); } else { // when column mapping is enabled, just check the name contains no new line in it. - flattenColNames.forEach(name -> checkArgument( - !name.contains("\\n"), - format("Attribute name '%s' contains invalid new line characters.", name))); + flattenColNames.forEach(name -> { + if (name.contains("\\n")) { + throw invalidColumnName(name, "\\n"); + } + }); } + + validateSupportedType(schema); + } + + /** + * Verify the partition columns exists in the table schema and a supported data type for a + * partition column. + * + * @param schema + * @param partitionCols + */ + public static void validatePartitionColumns(StructType schema, List partitionCols) { + // partition columns are always the top-level columns + Map columnNameToType = schema.fields().stream() + .collect(Collectors.toMap( + field -> field.getName().toLowerCase(Locale.ROOT), + StructField::getDataType)); + + partitionCols.stream().forEach(partitionCol -> { + DataType dataType = columnNameToType.get(partitionCol.toLowerCase(Locale.ROOT)); + checkArgument( + dataType != null, + "Partition column " + partitionCol + " not found in the schema"); + + if (!(dataType instanceof BooleanType || + dataType instanceof ByteType || + dataType instanceof ShortType || + dataType instanceof IntegerType || + dataType instanceof LongType || + dataType instanceof FloatType || + dataType instanceof DoubleType || + dataType instanceof DecimalType || + dataType instanceof StringType || + dataType instanceof BinaryType || + dataType instanceof DateType || + dataType instanceof TimestampType)) { + throw unsupportedPartitionDataType(partitionCol, dataType); + } + }); + } + + /** + * Delta expects partition column names to be same case preserving as the name in the schema. + * E.g: Schema: (a INT, B STRING) and partition columns: (b). In this case we store the + * schema as (a INT, B STRING) and partition columns as (B). + * + * This method expects the inputs are already validated (i.e. schema contains all the partition + * columns). + */ + public static List casePreservingPartitionColNames( + StructType tableSchema, + List partitionColumns) { + Map columnNameMap = new HashMap<>(); + tableSchema.fieldNames().forEach(colName -> + columnNameMap.put(colName.toLowerCase(Locale.ROOT), colName)); + return partitionColumns.stream() + .map(colName -> columnNameMap.get(colName.toLowerCase(Locale.ROOT))) + .collect(Collectors.toList()); } /** @@ -90,7 +152,8 @@ private static List flattenNestedFieldNames(StructType schema) { for (StructField field : schema.fields()) { String escapedName = escapeDots(field.getName()); fieldNames.add(escapedName); - fieldNames.addAll(flattenNestedFieldNamesRecursive(escapedName, field.getDataType())); + fieldNames.addAll( + flattenNestedFieldNamesRecursive(escapedName, field.getDataType())); } return fieldNames; } @@ -113,7 +176,8 @@ private static List flattenNestedFieldNamesRecursive(String prefix, Data fieldNames.addAll( flattenNestedFieldNamesRecursive(prefix + ".key", mapType.getKeyType())); fieldNames.addAll( - flattenNestedFieldNamesRecursive(prefix + ".value", mapType.getValueType())); + flattenNestedFieldNamesRecursive(prefix + ".value", + mapType.getValueType())); } return fieldNames; } @@ -125,10 +189,44 @@ private static String escapeDots(String name) { protected static void validParquetColumnNames(List columnNames) { for (String name : columnNames) { // ,;{}()\n\t= and space are special characters in Parquet schema - checkArgument( - !name.matches(".*[ ,;{}()\n\t=].*"), - format("Attribute name '%s' contains invalid character(s) among" + - " ' ,;{}()\\n\\t='.", name)); + if (name.matches(".*[ ,;{}()\n\t=].*")) { + throw invalidColumnName(name, "[ ,;{}()\\n\\t=]"); + } } } + + /** + * Validate the supported data types. Once we start supporting additional types, take input the + * protocol features and validate the schema. + * + * @param dataType the data type to validate + */ + protected static void validateSupportedType(DataType dataType) { + if (dataType instanceof BooleanType || + dataType instanceof ByteType || + dataType instanceof ShortType || + dataType instanceof IntegerType || + dataType instanceof LongType || + dataType instanceof FloatType || + dataType instanceof DoubleType || + dataType instanceof DecimalType || + dataType instanceof StringType || + dataType instanceof BinaryType || + dataType instanceof DateType || + dataType instanceof TimestampType) { + // supported types + return; + } else if (dataType instanceof StructType) { + ((StructType) dataType).fields() + .forEach(field -> validateSupportedType(field.getDataType())); + } else if (dataType instanceof ArrayType) { + validateSupportedType(((ArrayType) dataType).getElementType()); + } else if (dataType instanceof MapType) { + validateSupportedType(((MapType) dataType).getKeyType()); + validateSupportedType(((MapType) dataType).getValueType()); + } else { + throw unsupportedDataType(dataType); + } + } + } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/VectorUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/VectorUtils.java index 7a44836d29a..206321c0d3c 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/VectorUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/VectorUtils.java @@ -81,6 +81,9 @@ public static Map toJavaMap(MapValue mapValue) { * @return an {@link ArrayValue} with the given values of type {@link StringType} */ public static ArrayValue stringArrayValue(List values) { + if (values == null) { + return null; + } return new ArrayValue() { @Override public int getSize() { diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterable.java b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterable.java new file mode 100644 index 00000000000..d228c76b499 --- /dev/null +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterable.java @@ -0,0 +1,108 @@ +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.utils; + +import java.io.Closeable; +import java.io.IOException; +import java.util.*; +import java.util.function.Consumer; + +import static io.delta.kernel.internal.util.Utils.toCloseableIterator; + +/** + * Extend the Java {@link Iterable} interface to provide a way to close the iterator. + * + * @param the type of elements returned by this iterator + */ +public interface CloseableIterable extends Iterable, Closeable { + + /** + * Overrides the default iterator method to return a {@link CloseableIterator}. + * + * @return a {@link CloseableIterator} instance. + */ + @Override + CloseableIterator iterator(); + + @Override + default void forEach(Consumer action) { + try (CloseableIterator iterator = iterator()) { + iterator.forEachRemaining(action); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + default Spliterator spliterator() { + // We need a way to close the iterator and is not used in Kernel, so for now + // make the default implementation throw an exception. + throw new UnsupportedOperationException("spliterator is not supported"); + } + + /** + * Return an {@link CloseableIterable} object that is backed by an in-memory collection of given + * {@link CloseableIterator}. Users should aware that the returned {@link CloseableIterable} + * will hold the data in memory. + * + * @param iterator the iterator to be converted to a {@link CloseableIterable}. It will be + * closed by this callee. + * @param the type of elements returned by the iterator + * @return a {@link CloseableIterable} instance. + */ + static CloseableIterable inMemoryIterable(CloseableIterator iterator) { + final ArrayList elements = new ArrayList<>(); + try (CloseableIterator iter = iterator) { + while (iter.hasNext()) { + elements.add(iter.next()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + return new CloseableIterable() { + @Override + public void close() throws IOException { + // nothing to close + } + + @Override + public CloseableIterator iterator() { + return toCloseableIterator(elements.iterator()); + } + }; + } + + /** + * Return an {@link CloseableIterable} object for an empty collection. + * @return a {@link CloseableIterable} instance. + * @param the type of elements returned by the iterator + */ + static CloseableIterable emptyIterable() { + final CloseableIterator EMPTY_ITERATOR = + toCloseableIterator(Collections.emptyList().iterator()); + return new CloseableIterable() { + @Override + public void close() throws IOException { + // nothing to close + } + + @Override + public CloseableIterator iterator() { + return EMPTY_ITERATOR; + } + }; + } +} diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterator.java b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterator.java index 398364c5c3d..29a398c70fb 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterator.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterator.java @@ -25,6 +25,8 @@ import io.delta.kernel.annotation.Evolving; +import io.delta.kernel.internal.util.Utils; + /** * Closeable extension of {@link Iterator} * @@ -102,4 +104,36 @@ public void close() } }; } + + /** + * Combine the current iterator with another iterator. The resulting iterator will return all + * elements from the current iterator followed by all elements from the other iterator. + * + * @param other the other iterator to combine with + * @return a new iterator that combines the current iterator with the other iterator + */ + default CloseableIterator combine(CloseableIterator other) { + + CloseableIterator delegate = this; + return new CloseableIterator() { + @Override + public boolean hasNext() { + return delegate.hasNext() || other.hasNext(); + } + + @Override + public T next() { + if (delegate.hasNext()) { + return delegate.next(); + } else { + return other.next(); + } + } + + @Override + public void close() throws IOException { + Utils.closeCloseables(delegate, other); + } + }; + } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/SchemaUtilsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/SchemaUtilsSuite.scala index 6499343d4bb..6e9381eee03 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/SchemaUtilsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/SchemaUtilsSuite.scala @@ -15,6 +15,7 @@ */ package io.delta.kernel.internal.util +import io.delta.kernel.exceptions.KernelException import io.delta.kernel.internal.util.SchemaUtils.validateSchema import io.delta.kernel.types.IntegerType.INTEGER import io.delta.kernel.types.{ArrayType, MapType, StringType, StructType} @@ -24,7 +25,7 @@ import java.util.Locale class SchemaUtilsSuite extends AnyFunSuite { private def expectFailure(shouldContain: String*)(f: => Unit): Unit = { - val e = intercept[IllegalArgumentException] { + val e = intercept[KernelException] { f } val msg = e.getMessage.toLowerCase(Locale.ROOT) @@ -143,7 +144,7 @@ class SchemaUtilsSuite extends AnyFunSuite { new StructType().add("b", INTEGER).add("c", INTEGER).add("d", INTEGER) ) - val e = intercept[IllegalArgumentException] { + val e = intercept[KernelException] { validateSchema(schema, false /* isColumnMappingEnabled */) } assert(e.getMessage.contains("Schema contains duplicate columns: top, top.b, top.c")) @@ -250,7 +251,7 @@ class SchemaUtilsSuite extends AnyFunSuite { badCharacters.foreach { char => Seq(s"a${char}b", s"${char}ab", s"ab${char}", char.toString).foreach { name => val schema = new StructType().add(name, INTEGER) - val e = intercept[IllegalArgumentException] { + val e = intercept[KernelException] { validateSchema(schema, false /* isColumnMappingEnabled */) } @@ -259,7 +260,7 @@ class SchemaUtilsSuite extends AnyFunSuite { validateSchema(schema, true /* isColumnMappingEnabled */) } - assert(e.getMessage.contains("invalid character")) + assert(e.getMessage.contains("contains one of the unsupported")) } } diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/parquet/ParquetFileWriter.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/parquet/ParquetFileWriter.java index 9261cca7068..9302544ed2a 100644 --- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/parquet/ParquetFileWriter.java +++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/parquet/ParquetFileWriter.java @@ -31,6 +31,7 @@ import org.apache.parquet.schema.MessageType; import static org.apache.parquet.hadoop.ParquetOutputFormat.*; +import io.delta.kernel.Meta; import io.delta.kernel.data.*; import io.delta.kernel.expressions.Column; import io.delta.kernel.types.StructType; @@ -286,9 +287,9 @@ public String getName() { @Override public WriteContext init(Configuration configuration) { - // TODO: figure out a way to dynamically fetch the Kernel version. Map extraProps = Collections.singletonMap( - "io.delta.kernel.default-parquet-writer", "3.2.0-SNAPSHOT"); + "io.delta.kernel.default-parquet-writer", + "Kernel-Defaults-" + Meta.KERNEL_VERSION); return new WriteContext(parquetSchema, extraProps); } 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 new file mode 100644 index 00000000000..9c375a4da14 --- /dev/null +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala @@ -0,0 +1,296 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.kernel.defaults + +import com.fasterxml.jackson.databind.ObjectMapper +import io.delta.golden.GoldenTableUtils.goldenTablePath +import io.delta.kernel.Operation.CREATE_TABLE +import io.delta.kernel.defaults.internal.parquet.ParquetSuiteBase +import io.delta.kernel.defaults.utils.{TestRow, TestUtils} +import io.delta.kernel.engine.Engine +import io.delta.kernel.exceptions.{KernelException, TableAlreadyExistsException, TableNotFoundException} +import io.delta.kernel.types.DateType.DATE +import io.delta.kernel.types.IntegerType.INTEGER +import io.delta.kernel.types.TimestampNTZType.TIMESTAMP_NTZ +import io.delta.kernel.types._ +import io.delta.kernel.utils.CloseableIterable.emptyIterable +import io.delta.kernel.{Meta, Operation, Table} +import org.scalatest.funsuite.AnyFunSuite + +import scala.collection.JavaConverters._ + +class DeltaTableWritesSuite extends AnyFunSuite with TestUtils with ParquetSuiteBase { + val OBJ_MAPPER = new ObjectMapper() + val testEngineInfo = "test-engine" + + /** Test table schemas */ + val testSchema = new StructType().add("id", INTEGER) + val testPartitionColumns = Seq("part1", "part2") + val testPartitionSchema = new StructType() + .add("id", INTEGER) + .add("part1", INTEGER) // partition column + .add("part2", INTEGER) // partition column + + test("create table - provide no schema - expect failure") { + withTempDirAndEngine { (tablePath, engine) => + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + + val ex = intercept[TableNotFoundException] { + txnBuilder.build(engine) + } + assert(ex.getMessage.contains("Must provide a new schema to write to a new table")) + } + } + + test("create table - provide partition columns but no schema - expect failure") { + withTempDirAndEngine { (tablePath, engine) => + val table = Table.forPath(engine, tablePath) + val txnBuilder = table + .createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + .withPartitionColumns(engine, Seq("part1", "part2").asJava) + + val ex = intercept[TableNotFoundException] { + txnBuilder.build(engine) + } + assert(ex.getMessage.contains("Must provide a new schema to write to a new table")) + } + } + + test("create table - provide unsupported column types - expect failure") { + withTempDirAndEngine { (tablePath, engine) => + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + val ex = intercept[KernelException] { + txnBuilder + .withSchema(engine, new StructType().add("ts_ntz", TIMESTAMP_NTZ)) + .build(engine) + } + assert(ex.getMessage.contains("Kernel doesn't support writing data of type: timestamp_ntz")) + } + } + + test("create table - table already exists at the location") { + withTempDirAndEngine { (tablePath, engine) => + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + + val txn = txnBuilder.withSchema(engine, testSchema).build(engine) + txn.commit(engine, emptyIterable()) + + { + val ex = intercept[TableAlreadyExistsException] { + table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + .withSchema(engine, testSchema) + .build(engine) + } + assert(ex.getMessage.contains("Table already exists, but provided a new schema. " + + "Schema can only be set on a new table.")) + } + { + val ex = intercept[TableAlreadyExistsException] { + table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + .withPartitionColumns(engine, Seq("part1", "part2").asJava) + .build(engine) + } + assert(ex.getMessage.contains("Table already exists, but provided new partition columns." + + " Partition columns can only be set on a new table.")) + } + } + } + + test("create un-partitioned table") { + withTempDirAndEngine { (tablePath, engine) => + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + + val txn = txnBuilder + .withSchema(engine, testSchema) + .build(engine) + + assert(txn.getSchema(engine) === testSchema) + assert(txn.getPartitionColumns(engine) === Seq.empty.asJava) + val txnResult = txn.commit(engine, emptyIterable()) + + assert(txnResult.getVersion === 0) + assert(!txnResult.isReadyForCheckpoint) + + verifyCommitInfo(tablePath = tablePath, version = 0) + verifyWrittenContent(tablePath, testSchema, Seq.empty) + } + } + + test("create partitioned table - partition column is not part of the schema") { + withTempDirAndEngine { (tablePath, engine) => + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + + val ex = intercept[IllegalArgumentException] { + txnBuilder + .withSchema(engine, testPartitionSchema) + .withPartitionColumns(engine, Seq("PART1", "part3").asJava) + .build(engine) + } + assert(ex.getMessage.contains("Partition column part3 not found in the schema")) + } + } + + test("create partitioned table - partition column type is not supported") { + withTempDirAndEngine { (tablePath, engine) => + val table = Table.forPath(engine, tablePath) + val txnBuilder = table.createTransactionBuilder(engine, testEngineInfo, CREATE_TABLE) + + val schema = new StructType() + .add("p1", new ArrayType(INTEGER, true)) + .add("c1", DATE) + .add("c2", new DecimalType(14, 2)) + + val ex = intercept[KernelException] { + txnBuilder + .withSchema(engine, schema) + .withPartitionColumns(engine, Seq("p1", "c1").asJava) + .build(engine) + } + assert(ex.getMessage.contains( + "Kernel doesn't support writing data with partition column (p1) of type: array[integer]")) + } + } + + 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) + assert(!txnResult.isReadyForCheckpoint) + + verifyCommitInfo(tablePath, version = 0, Seq("Part1", "part2")) + verifyWrittenContent(tablePath, schema, Seq.empty) + } + } + + 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 = txn.commit(engine, emptyIterable()) + + assert(txnResult.getVersion === 0) + assert(!txnResult.isReadyForCheckpoint) + + verifyCommitInfo(tablePath, version = 0) + verifyWrittenContent(tablePath, schema, Seq.empty) + } + } + + def withTempDirAndEngine(f: (String, Engine) => Unit): Unit = { + withTempDir { dir => f(dir.getAbsolutePath, defaultEngine) } + } + + def verifyWrittenContent( + tablePath: String, + expSchema: StructType, + expData: Seq[TestRow]): Unit = { + val actSchema = tableSchema(tablePath) + assert(actSchema === expSchema) + + // verify data using Kernel reader + checkTable(tablePath, expData) + + // verify data using Spark reader + val resultSpark = spark.sql(s"SELECT * FROM delta.`$tablePath`").collect().map(TestRow(_)) + checkAnswer(resultSpark, expData) + } + + 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`") + .filter(s"version = $version") + .select( + "version", + "operationParameters.partitionBy", + "isBlindAppend", + "engineInfo", + "operation") + .collect().last + + assert(row.getAs[Long]("version") === version) + assert(row.getAs[Long]("partitionBy") === + (if (partitionCols == null) null else OBJ_MAPPER.writeValueAsString(partitionCols.asJava))) + assert(row.getAs[Boolean]("isBlindAppend") === isBlindAppend) + assert(row.getAs[Seq[String]]("engineInfo") === + "Kernel-" + Meta.KERNEL_VERSION + "/" + testEngineInfo) + assert(row.getAs[String]("operation") === operation.getDescription) + } + + def removeUnsupportedTypes(structType: StructType): StructType = { + def process(dataType: DataType): Option[DataType] = dataType match { + case a: ArrayType => + val newElementType = process(a.getElementType) + newElementType.map(new ArrayType(_, a.containsNull())) + case m: MapType => + val newKeyType = process(m.getKeyType) + val newValueType = process(m.getValueType) + (newKeyType, newValueType) match { + case (Some(newKeyType), Some(newValueType)) => + Some(new MapType(newKeyType, newValueType, m.isValueContainsNull)) + case _ => None + } + case _: TimestampNTZType => None // ignore + case s: StructType => + val newType = removeUnsupportedTypes(s); + if (newType.length() > 0) { + Some(newType) + } else { + None + } + case _ => Some(dataType) + } + + var newStructType = new StructType(); + structType.fields().forEach { field => + val newDataType = process(field.getDataType) + if (newDataType.isDefined) { + newStructType = newStructType.add(field.getName(), newDataType.get) + } + } + newStructType + } +} 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 ac231447a02..183e772e2d9 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 @@ -110,6 +110,12 @@ trait TestUtils extends Assertions with SQLHelper { } } + implicit class FilteredColumnarBatchOps(batch: FilteredColumnarBatch) { + def toTestRows: Seq[TestRow] = { + batch.getRows.toSeq.map(TestRow(_)) + } + } + implicit class ColumnOps(column: Column) { def toPath: String = column.getNames.mkString(".") } From 6453fe50b7e13c3d4bee1c6710e075186cdcda6a Mon Sep 17 00:00:00 2001 From: Venki Korukanti Date: Sun, 5 May 2024 16:56:04 -0700 Subject: [PATCH 09/11] [3.2][Kernel][Writes] Add support of inserting data into tables (#3030) (Split from #2944) Adds support for inserting data into the table. Tests for inserting into partitioned and unpartitioned tables with various combinations of the types, partition values etc. Also tests the checkpoint is ready to create. --- .../java/io/delta/kernel/Transaction.java | 84 ++- .../io/delta/kernel/engine/JsonHandler.java | 3 +- .../io/delta/kernel/internal/DeltaErrors.java | 16 + .../kernel/internal/TransactionImpl.java | 17 +- .../kernel/internal/actions/AddFile.java | 45 ++ .../internal/data/TransactionStateRow.java | 17 +- .../kernel/internal/util/InternalUtils.java | 28 + .../kernel/internal/util/PartitionUtils.java | 83 +++ .../kernel/internal/util/SchemaUtils.java | 45 ++ .../delta/kernel/utils/CloseableIterable.java | 10 +- .../kernel/utils/DataFileStatistics.java | 5 + .../delta/kernel/test/VectorTestUtils.scala | 12 + .../defaults/engine/DefaultJsonHandler.java | 3 +- .../engine/DefaultParquetHandler.java | 5 +- .../defaults/internal/json/JsonUtils.java | 4 +- .../defaults/CreateCheckpointSuite.scala | 49 +- .../defaults/DeltaTableWriteSuiteBase.scala | 111 ++++ .../defaults/DeltaTableWritesSuite.scala | 583 +++++++++++++++++- .../internal/json/JsonUtilsSuite.scala | 2 +- .../kernel/defaults/utils/TestUtils.scala | 49 +- 20 files changed, 1085 insertions(+), 86 deletions(-) create mode 100644 kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java b/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java index efea669dd7d..3c6e5af55e8 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/Transaction.java @@ -15,18 +15,31 @@ */ package io.delta.kernel; +import java.net.URI; import java.util.List; import java.util.Map; import io.delta.kernel.annotation.Evolving; -import io.delta.kernel.data.FilteredColumnarBatch; -import io.delta.kernel.data.Row; +import io.delta.kernel.data.*; import io.delta.kernel.engine.Engine; import io.delta.kernel.exceptions.ConcurrentWriteException; import io.delta.kernel.expressions.Literal; import io.delta.kernel.types.StructType; import io.delta.kernel.utils.*; +import io.delta.kernel.internal.DataWriteContextImpl; +import io.delta.kernel.internal.actions.AddFile; +import io.delta.kernel.internal.actions.SingleAction; +import io.delta.kernel.internal.fs.Path; +import static io.delta.kernel.internal.DeltaErrors.dataSchemaMismatch; +import static io.delta.kernel.internal.DeltaErrors.partitionColumnMissingInData; +import static io.delta.kernel.internal.TransactionImpl.getStatisticsColumns; +import static io.delta.kernel.internal.data.TransactionStateRow.*; +import static io.delta.kernel.internal.util.PartitionUtils.getTargetDirectory; +import static io.delta.kernel.internal.util.PartitionUtils.validateAndSanitizePartitionValues; +import static io.delta.kernel.internal.util.Preconditions.checkArgument; +import static io.delta.kernel.internal.util.SchemaUtils.findColIndex; + /** * Represents a transaction to mutate a Delta table. * @@ -104,7 +117,41 @@ static CloseableIterator transformLogicalData( Row transactionState, CloseableIterator dataIter, Map partitionValues) { - throw new UnsupportedOperationException("Not implemented yet"); + + // Note: `partitionValues` are not used as of now in this API, but taking the partition + // values as input forces the connector to not pass data from multiple partitions this + // API in a single call. + StructType tableSchema = getLogicalSchema(engine, transactionState); + List partitionColNames = getPartitionColumnsList(transactionState); + validateAndSanitizePartitionValues(tableSchema, partitionColNames, partitionValues); + + // TODO: add support for: + // - enforcing the constraints + // - generating the default value columns + // - generating the generated columns + + // Remove the partition columns from the data as they are already part of file metadata + // and are not needed in the data files. TODO: once we start supporting uniform complaint + // tables, we may conditionally skip this step. + + // TODO: set the correct schema once writing into column mapping enabled table is supported. + String tablePath = getTablePath(transactionState); + return dataIter.map( + filteredBatch -> { + ColumnarBatch data = filteredBatch.getData(); + if (!data.getSchema().equals(tableSchema)) { + throw dataSchemaMismatch(tablePath, tableSchema, data.getSchema()); + } + for (String partitionColName : partitionColNames) { + int partitionColIndex = findColIndex(data.getSchema(), partitionColName); + if (partitionColIndex < 0) { + throw partitionColumnMissingInData(tablePath, partitionColName); + } + data = data.withDeletedColumnAt(partitionColIndex); + } + return new FilteredColumnarBatch(data, filteredBatch.getSelectionVector()); + } + ); } /** @@ -124,7 +171,21 @@ static DataWriteContext getWriteContext( Engine engine, Row transactionState, Map partitionValues) { - throw new UnsupportedOperationException("Not implemented yet"); + StructType tableSchema = getLogicalSchema(engine, transactionState); + List partitionColNames = getPartitionColumnsList(transactionState); + + partitionValues = + validateAndSanitizePartitionValues(tableSchema, partitionColNames, partitionValues); + + String targetDirectory = getTargetDirectory( + getTablePath(transactionState), + partitionColNames, + partitionValues); + + return new DataWriteContextImpl( + targetDirectory, + partitionValues, + getStatisticsColumns(engine, transactionState)); } /** @@ -146,6 +207,19 @@ static CloseableIterator generateAppendActions( Row transactionState, CloseableIterator fileStatusIter, DataWriteContext dataWriteContext) { - throw new UnsupportedOperationException("Not implemented yet"); + checkArgument(dataWriteContext instanceof DataWriteContextImpl, + "DataWriteContext is not created by the `Transaction.getWriteContext()`"); + + URI tableRoot = new Path(getTablePath(transactionState)).toUri(); + return fileStatusIter.map( + dataFileStatus -> { + Row addFileRow = AddFile.convertDataFileStatus( + tableRoot, + dataFileStatus, + ((DataWriteContextImpl) dataWriteContext).getPartitionValues(), + true /* dataChange */); + return SingleAction.createAddFileSingleAction(addFileRow); + } + ); } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/engine/JsonHandler.java b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/JsonHandler.java index 9bbbb467663..0f23d104f76 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/engine/JsonHandler.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/engine/JsonHandler.java @@ -110,7 +110,8 @@ CloseableIterator readJsonFiles( *
    *
  • Primitive types: @code boolean, byte, short, int, long, float, double, string}
  • *
  • {@code struct}: any element whose value is null is not written to file
  • - *
  • {@code map}: only a {@code map} with {@code string} key type is supported
  • + *
  • {@code map}: only a {@code map} with {@code string} key type is supported. If an + * entry value is {@code null}, it should be written to the file.
  • *
  • {@code array}: {@code null} value elements are written to file
  • *
* diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java index 7e4dc4ab7fd..b22aa0dd4ab 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java @@ -172,6 +172,22 @@ public static KernelException tableAlreadyExists(String tablePath, String messag return new TableAlreadyExistsException(tablePath, message); } + public static KernelException dataSchemaMismatch( + String tablePath, + StructType tableSchema, + StructType dataSchema) { + String msgT = "The schema of the data to be written to the table doesn't match " + + "the table schema. \nTable: %s\nTable schema: %s, \nData schema: %s"; + return new KernelException(format(msgT, tablePath, tableSchema, dataSchema)); + } + + public static KernelException partitionColumnMissingInData( + String tablePath, + String partitionColumn) { + String msgT = "Missing partition column '%s' in the data to be written to the table '%s'."; + return new KernelException(format(msgT, partitionColumn, tablePath)); + } + /* ------------------------ HELPER METHODS ----------------------------- */ private static String formatTimestamp(long millisSinceEpochUTC) { return new Timestamp(millisSinceEpochUTC).toInstant().toString(); 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 8bead35758f..483f245855d 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 @@ -24,6 +24,7 @@ import io.delta.kernel.data.Row; import io.delta.kernel.engine.Engine; import io.delta.kernel.exceptions.ConcurrentWriteException; +import io.delta.kernel.expressions.Column; import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterable; import io.delta.kernel.utils.CloseableIterator; @@ -158,7 +159,9 @@ private boolean isReadyForCheckpoint(long newVersion) { } private boolean isBlindAppend() { - return isNewTable; // Later can add more conditions to determine if it is a blind append + // For now, Kernel just supports blind append. + // Change this when read-after-write is supported. + return true; } private Map getOperationParameters() { @@ -171,4 +174,16 @@ private Map getOperationParameters() { } return Collections.emptyMap(); } + + /** + * Get the part of the schema of the table that needs the statistics to be collected per file. + * + * @param engine {@link Engine} instance to use. + * @param transactionState State of the transaction + * @return + */ + public static List getStatisticsColumns(Engine engine, Row transactionState) { + // TODO: implement this once we start supporting collecting stats + return Collections.emptyList(); + } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/AddFile.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/AddFile.java index 66bf43cbce0..cfd2cbfcc0b 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/AddFile.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/actions/AddFile.java @@ -15,7 +15,21 @@ */ package io.delta.kernel.internal.actions; +import java.net.URI; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.IntStream; +import static java.util.stream.Collectors.toMap; + +import io.delta.kernel.data.Row; +import io.delta.kernel.expressions.Literal; import io.delta.kernel.types.*; +import io.delta.kernel.utils.DataFileStatus; + +import io.delta.kernel.internal.data.GenericRow; +import io.delta.kernel.internal.fs.Path; +import static io.delta.kernel.internal.util.InternalUtils.relativizePath; +import static io.delta.kernel.internal.util.PartitionUtils.serializePartitionMap; /** * Delta log action representing an `AddFile` @@ -57,4 +71,35 @@ public class AddFile { true /* nullable */); // There are more fields which are added when row-id tracking and clustering is enabled. // When Kernel starts supporting row-ids and clustering, we should add those fields here. + + private static final Map COL_NAME_TO_ORDINAL = + IntStream.range(0, FULL_SCHEMA.length()) + .boxed() + .collect(toMap(i -> FULL_SCHEMA.at(i).getName(), i -> i)); + + /** + * Utility to generate `AddFile` row from the given {@link DataFileStatus} and partition values. + */ + public static Row convertDataFileStatus( + URI tableRoot, + DataFileStatus dataFileStatus, + Map partitionValues, + boolean dataChange) { + Path filePath = new Path(dataFileStatus.getPath()); + Map valueMap = new HashMap<>(); + valueMap.put(COL_NAME_TO_ORDINAL.get("path"), + relativizePath(filePath, tableRoot).toString()); + valueMap.put(COL_NAME_TO_ORDINAL.get("partitionValues"), + serializePartitionMap(partitionValues)); + valueMap.put(COL_NAME_TO_ORDINAL.get("size"), dataFileStatus.getSize()); + valueMap.put(COL_NAME_TO_ORDINAL.get("modificationTime"), + dataFileStatus.getModificationTime()); + valueMap.put(COL_NAME_TO_ORDINAL.get("dataChange"), dataChange); + if (dataFileStatus.getStatistics().isPresent()) { + valueMap.put(COL_NAME_TO_ORDINAL.get("stats"), + dataFileStatus.getStatistics().get().serializeAsJson()); + } + // any fields not present in the valueMap are considered null + return new GenericRow(FULL_SCHEMA, valueMap); + } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java index 80659c054af..3aaa2ddc789 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/data/TransactionStateRow.java @@ -29,6 +29,7 @@ public class TransactionStateRow extends GenericRow { private static final StructType SCHEMA = new StructType() + .add("logicalSchemaString", StringType.STRING) .add("partitionColumns", new ArrayType(StringType.STRING, false)) .add("tablePath", StringType.STRING); @@ -39,6 +40,7 @@ public class TransactionStateRow extends GenericRow { public static TransactionStateRow of(Metadata metadata, String tablePath) { HashMap valueMap = new HashMap<>(); + valueMap.put(COL_NAME_TO_ORDINAL.get("logicalSchemaString"), metadata.getSchemaString()); valueMap.put(COL_NAME_TO_ORDINAL.get("partitionColumns"), metadata.getPartitionColumns()); valueMap.put(COL_NAME_TO_ORDINAL.get("tablePath"), tablePath); return new TransactionStateRow(valueMap); @@ -48,11 +50,24 @@ private TransactionStateRow(HashMap valueMap) { super(SCHEMA, valueMap); } + /** + * Get the logical schema of the table from the transaction state {@link Row} returned by + * {@link Transaction#getTransactionState(Engine)}} + * + * @param engine {@link Engine} instance to use for parsing the schema + * @param transactionState Transaction state state {@link Row} + * @return Logical schema of the table as {@link StructType} + */ + public static StructType getLogicalSchema(Engine engine, Row transactionState) { + return engine.getJsonHandler().deserializeStructType( + transactionState.getString(COL_NAME_TO_ORDINAL.get("logicalSchemaString"))); + } + /** * Get the list of partition column names from the write state {@link Row} returned by * {@link Transaction#getTransactionState(Engine)} * - * @param transactionState Scan state {@link Row} + * @param transactionState Transaction state state {@link Row} * @return List of partition column names according to the scan state. */ public static List getPartitionColumnsList(Row transactionState) { diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/InternalUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/InternalUtils.java index 3f798e84aff..e0161c072a0 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/InternalUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/InternalUtils.java @@ -16,13 +16,17 @@ package io.delta.kernel.internal.util; import java.io.IOException; +import java.net.URI; import java.sql.Date; import java.sql.Timestamp; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; +import java.util.Collection; import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; import io.delta.kernel.data.ColumnVector; import io.delta.kernel.data.ColumnarBatch; @@ -31,6 +35,8 @@ import io.delta.kernel.types.StringType; import io.delta.kernel.utils.CloseableIterator; +import io.delta.kernel.internal.fs.Path; + public class InternalUtils { private static final LocalDate EPOCH_DAY = LocalDate.ofEpochDay(0); private static final LocalDateTime EPOCH_DATETIME = @@ -152,4 +158,26 @@ public static ColumnVector requireNonNull(ColumnVector vector, int rowId, String } return vector; } + + /** + * Relativize the given child path with respect to the given root URI. If the child path is + * already a relative path, it is returned as is. + * + * @param child + * @param root Root directory as URI. Relativization is done with respect to this root. + * The relativize operation requires conversion to URI, so the caller is expected to + * convert the root directory to URI once and use it for relativizing for multiple + * child paths. + * @return + */ + public static Path relativizePath(Path child, URI root) { + if (child.isAbsolute()) { + return new Path(root.relativize(child.toUri())); + } + return child; + } + + public static Set toLowerCaseSet(Collection set) { + return set.stream().map(String::toLowerCase).collect(Collectors.toSet()); + } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/PartitionUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/PartitionUtils.java index 2409570767f..b0337565334 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/PartitionUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/PartitionUtils.java @@ -38,7 +38,9 @@ import io.delta.kernel.internal.InternalScanFileUtils; import io.delta.kernel.internal.fs.Path; +import static io.delta.kernel.internal.util.InternalUtils.toLowerCaseSet; import static io.delta.kernel.internal.util.Preconditions.checkArgument; +import static io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames; public class PartitionUtils { private static final DateTimeFormatter PARTITION_TIMESTAMP_FORMATTER = @@ -111,6 +113,87 @@ public static ColumnarBatch withPartitionColumns( return dataBatch; } + /** + * Convert the given partition values to a {@link MapValue} that can be serialized to a Delta + * commit file. + * + * @param partitionValueMap Expected the partition column names to be same case as in the + * schema. We want to preserve the case of the partition column names + * when serializing to the Delta commit file. + * @return {@link MapValue} representing the serialized partition values that can be written to + * a Delta commit file. + */ + public static MapValue serializePartitionMap(Map partitionValueMap) { + if (partitionValueMap == null || partitionValueMap.isEmpty()) { + return VectorUtils.stringStringMapValue(Collections.emptyMap()); + } + + Map serializedPartValues = new HashMap<>(); + for(Map.Entry entry : partitionValueMap.entrySet()) { + serializedPartValues.put( + entry.getKey(), // partition column name + serializePartitionValue(entry.getValue())); // serialized partition value as str + } + + return VectorUtils.stringStringMapValue(serializedPartValues); + } + + /** + * Validate {@code partitionValues} contains values for every partition column in the table + * and the type of the value is correct. Once validated the partition values are sanitized + * to match the case of the partition column names in the table schema and returned + * + * @param tableSchema Schema of the table. + * @param partitionColNames Partition column name. These should be from the table metadata + * that retain the same case as in the table schema. + * @param partitionValues Map of partition column to value map given by the connector + * @return Sanitized partition values. + */ + public static Map validateAndSanitizePartitionValues( + StructType tableSchema, + List partitionColNames, + Map partitionValues) { + + if (!toLowerCaseSet(partitionColNames) + .equals(toLowerCaseSet(partitionValues.keySet()))) { + throw new IllegalArgumentException( + String.format( + "Partition values provided are not matching the partition columns. " + + "Partition columns: %s, Partition values: %s", + partitionColNames, partitionValues)); + } + + // Convert the partition column names in given `partitionValues` to schema case. Schema + // case is the exact case the column name was given by the connector when creating the + // table. Comparing the column names is case-insensitive, but preserve the case as stored + // in the table metadata when writing the partition column name to DeltaLog + // (`partitionValues` in `AddFile`) or generating the target directory for writing the + // data belonging to a partition. + Map schemaCasePartitionValues = + casePreservingPartitionColNames(partitionColNames, partitionValues); + + // validate types are the same + schemaCasePartitionValues.entrySet().forEach(entry -> { + String partColName = entry.getKey(); + Literal partValue = entry.getValue(); + StructField partColField = tableSchema.get(partColName); + + // this shouldn't happen as we have already validated the partition column names + checkArgument( + partColField != null, + "Partition column " + partColName + " is not present in the table schema"); + DataType partColType = partColField.getDataType(); + + if (!partColType.equivalent(partValue.getDataType())) { + throw new IllegalArgumentException(String.format( + "Partition column %s is of type %s but the value provided is of type %s", + partColName, partColType, partValue.getDataType())); + } + }); + + return schemaCasePartitionValues; + } + /** * Split the given predicate into predicate on partition columns and predicate on data columns. * diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java index e8178029f9f..528bbbf84be 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/SchemaUtils.java @@ -18,6 +18,7 @@ import java.util.*; import java.util.stream.Collectors; +import io.delta.kernel.expressions.Literal; import io.delta.kernel.types.*; import io.delta.kernel.internal.DeltaErrors; @@ -134,6 +135,50 @@ public static List casePreservingPartitionColNames( .collect(Collectors.toList()); } + /** + * Convert the partition column names in {@code partitionValues} map into the same case as the + * column in the table metadata. Delta expects the partition column names to preserve the case + * same as the table schema. + * + * @param partitionColNames List of partition columns in the table metadata. The names preserve + * the case as given by the connector when the table is created. + * @param partitionValues Map of partition column name to partition value. Convert the + * partition column name to be same case preserving name as its + * equivalent column in the {@code partitionColName}. Column name + * comparison is case-insensitive. + * @return Rewritten {@code partitionValues} map with names case preserved. + */ + public static Map casePreservingPartitionColNames( + List partitionColNames, + Map partitionValues) { + Map partitionColNameMap = new HashMap<>(); + partitionColNames.forEach(colName -> + partitionColNameMap.put(colName.toLowerCase(Locale.ROOT), colName)); + + return partitionValues.entrySet().stream() + .collect(Collectors.toMap( + entry -> partitionColNameMap.get( + entry.getKey().toLowerCase(Locale.ROOT)), + Map.Entry::getValue)); + } + + /** + * Search (case-insensitive) for the given {@code colName} in the {@code schema} and return its + * position in the {@code schema}. + * + * @param schema {@link StructType} + * @param colName Name of the column whose index is needed. + * @return Valid index or -1 if not found. + */ + public static int findColIndex(StructType schema, String colName) { + for (int i = 0; i < schema.length(); i++) { + if (schema.at(i).getName().equalsIgnoreCase(colName)) { + return i; + } + } + return -1; + } + /** * Returns all column names in this schema as a flat list. For example, a schema like: *
diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterable.java b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterable.java
index d228c76b499..387d25f8fa2 100644
--- a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterable.java
+++ b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterable.java
@@ -20,6 +20,8 @@
 import java.util.*;
 import java.util.function.Consumer;
 
+import io.delta.kernel.exceptions.KernelException;
+
 import static io.delta.kernel.internal.util.Utils.toCloseableIterator;
 
 /**
@@ -70,7 +72,13 @@ static  CloseableIterable inMemoryIterable(CloseableIterator iterator)
                 elements.add(iter.next());
             }
         } catch (Exception e) {
-            throw new RuntimeException(e);
+            // TODO: we may need utility methods to throw the KernelException as is
+            // without wrapping in RuntimeException.
+            if (e instanceof KernelException) {
+                throw (KernelException) e;
+            } else {
+                throw new RuntimeException(e);
+            }
         }
         return new CloseableIterable() {
             @Override
diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/DataFileStatistics.java b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/DataFileStatistics.java
index 6596cd00372..80f914b8a8f 100644
--- a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/DataFileStatistics.java
+++ b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/DataFileStatistics.java
@@ -91,4 +91,9 @@ public Map getMaxValues() {
     public Map getNullCounts() {
         return nullCounts;
     }
+
+    public String serializeAsJson() {
+        // TODO: implement this
+        return "{}";
+    }
 }
diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/VectorTestUtils.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/VectorTestUtils.scala
index cfe2154460b..53ce2c46c1d 100644
--- a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/VectorTestUtils.scala
+++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/VectorTestUtils.scala
@@ -89,4 +89,16 @@ trait VectorTestUtils {
 
     override def getLong(rowId: Int): Long = values(rowId)
   }
+
+  def selectSingleElement(size: Int, selectRowId: Int): ColumnVector = new ColumnVector {
+    override def getDataType: DataType = BooleanType.BOOLEAN
+
+    override def getSize: Int = size
+
+    override def close(): Unit = {}
+
+    override def isNullAt(rowId: Int): Boolean = false
+
+    override def getBoolean(rowId: Int): Boolean = rowId == selectRowId
+  }
 }
diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultJsonHandler.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultJsonHandler.java
index ea793bcaa69..f8702f0ecb9 100644
--- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultJsonHandler.java
+++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultJsonHandler.java
@@ -16,6 +16,7 @@
 package io.delta.kernel.defaults.engine;
 
 import java.io.*;
+import java.net.URI;
 import java.nio.charset.StandardCharsets;
 import java.util.*;
 
@@ -186,7 +187,7 @@ public void writeJsonFileAtomically(
             String filePath,
             CloseableIterator data,
             boolean overwrite) throws IOException {
-        Path path = new Path(filePath);
+        Path path = new Path(URI.create(filePath));
         LogStore logStore = LogStoreProvider.getLogStore(hadoopConf, path.toUri().getScheme());
         try {
             logStore.write(
diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultParquetHandler.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultParquetHandler.java
index b39e3b4a4a3..dd1821fb047 100644
--- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultParquetHandler.java
+++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/engine/DefaultParquetHandler.java
@@ -17,6 +17,7 @@
 
 import java.io.IOException;
 import java.io.UncheckedIOException;
+import java.net.URI;
 import java.util.*;
 import static java.lang.String.format;
 
@@ -103,7 +104,7 @@ public CloseableIterator writeParquetFiles(
             CloseableIterator dataIter,
             List statsColumns) throws IOException {
         ParquetFileWriter batchWriter =
-            new ParquetFileWriter(hadoopConf, new Path(directoryPath), statsColumns);
+            new ParquetFileWriter(hadoopConf, new Path(URI.create(directoryPath)), statsColumns);
         return batchWriter.write(dataIter);
     }
 
@@ -120,7 +121,7 @@ public void writeParquetFileAtomically(
             String filePath,
             CloseableIterator data) throws IOException {
         try {
-            Path targetPath = new Path(filePath);
+            Path targetPath = new Path(URI.create(filePath));
             LogStore logStore =
                     LogStoreProvider.getLogStore(hadoopConf, targetPath.toUri().getScheme());
 
diff --git a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/json/JsonUtils.java b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/json/JsonUtils.java
index 136374b36f7..89061bbc123 100644
--- a/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/json/JsonUtils.java
+++ b/kernel/kernel-defaults/src/main/java/io/delta/kernel/defaults/internal/json/JsonUtils.java
@@ -145,9 +145,11 @@ private void writeMapValue(JsonGenerator gen, MapValue mapValue, MapType mapType
             ColumnVector keys = mapValue.getKeys();
             ColumnVector values = mapValue.getValues();
             for (int i = 0; i < mapValue.getSize(); i++) {
+                gen.writeFieldName(keys.getString(i));
                 if (!values.isNullAt(i)) {
-                    gen.writeFieldName(keys.getString(i));
                     writeValue(gen, values, i, mapType.getValueType());
+                } else {
+                    gen.writeNull();
                 }
             }
             gen.writeEndObject();
diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala
index eb4b8dd60a1..bdda713267f 100644
--- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala
+++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CreateCheckpointSuite.scala
@@ -37,7 +37,7 @@ import io.delta.kernel.exceptions.{CheckpointAlreadyExistsException, TableNotFou
 /**
  * Test suite for `io.delta.kernel.Table.checkpoint(engine, version)`
  */
-class CreateCheckpointSuite extends AnyFunSuite with TestUtils {
+class CreateCheckpointSuite extends DeltaTableWriteSuiteBase {
 
   ///////////
   // Tests //
@@ -403,51 +403,4 @@ class CreateCheckpointSuite extends AnyFunSuite with TestUtils {
     // verify using Kernel reader
     checkTable(tablePath, expResults)
   }
-
-  def verifyLastCheckpointMetadata(tablePath: String, checkpointAt: Long, expSize: Long): Unit = {
-    val filePath = f"$tablePath/_delta_log/_last_checkpoint"
-
-    val source = scala.io.Source.fromFile(filePath)
-    val result = try source.getLines().mkString(",") finally source.close()
-
-    assert(result === s"""{"version":$checkpointAt,"size":$expSize}""")
-  }
-
-  /** Helper method to remove the delta files before the given version, to make sure the read is
-   * using a checkpoint as base for state reconstruction.
-   */
-  def deleteDeltaFilesBefore(tablePath: String, beforeVersion: Long): Unit = {
-    Seq.range(0, beforeVersion).foreach { version =>
-      val filePath = new Path(f"$tablePath/_delta_log/$version%020d.json")
-      new Path(tablePath).getFileSystem(new Configuration()).delete(filePath, false /* recursive */)
-    }
-
-    // try to query a version < beforeVersion
-    val ex = intercept[VersionNotFoundException] {
-      spark.read.format("delta").option("versionAsOf", beforeVersion - 1).load(tablePath)
-    }
-    assert(ex.getMessage().contains(
-      s"Cannot time travel Delta table to version ${beforeVersion - 1}"))
-  }
-
-  def withTempDirAndEngine(f: (String, Engine) => Unit): Unit = {
-    val engine = DefaultEngine.create(new Configuration() {
-      {
-        // Set the batch sizes to small so that we get to test the multiple batch scenarios.
-        set("delta.kernel.default.parquet.reader.batch-size", "200");
-        set("delta.kernel.default.json.reader.batch-size", "200");
-      }
-    })
-    withTempDir { dir => f(dir.getAbsolutePath, engine) }
-  }
-
-  def checkpointFilePath(tablePath: String, checkpointVersion: Long): String = {
-    f"$tablePath/_delta_log/$checkpointVersion%020d.checkpoint.parquet"
-  }
-
-  def copyTable(goldenTableName: String, targetLocation: String): Unit = {
-    val source = new File(goldenTablePath(goldenTableName))
-    val target = new File(targetLocation)
-    FileUtils.copyDirectory(source, target)
-  }
 }
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
new file mode 100644
index 00000000000..99188078102
--- /dev/null
+++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWriteSuiteBase.scala
@@ -0,0 +1,111 @@
+/*
+ * Copyright (2024) The Delta Lake Project Authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.delta.kernel.defaults
+
+import io.delta.golden.GoldenTableUtils.goldenTablePath
+import io.delta.kernel.defaults.engine.DefaultEngine
+import io.delta.kernel.defaults.utils.TestUtils
+import io.delta.kernel.engine.Engine
+import io.delta.kernel.internal.util.FileNames.checkpointFileSingular
+import io.delta.kernel.{Table, TransactionCommitResult}
+import org.apache.commons.io.FileUtils
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.delta.VersionNotFoundException
+import org.scalatest.funsuite.AnyFunSuite
+
+import java.io.File
+import java.nio.file.{Files, Paths}
+import scala.collection.JavaConverters._
+
+/**
+ * Common utility methods for write test suites.
+ */
+trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils {
+  def withTempDirAndEngine(f: (String, Engine) => Unit): Unit = {
+    val engine = DefaultEngine.create(new Configuration() {
+      {
+        // Set the batch sizes to small so that we get to test the multiple batch/file scenarios.
+        set("delta.kernel.default.parquet.reader.batch-size", "20");
+        set("delta.kernel.default.json.reader.batch-size", "20");
+        set("delta.kernel.default.parquet.writer.targetMaxFileSize", "20");
+      }
+    })
+    withTempDir { dir => f(dir.getAbsolutePath, engine) }
+  }
+
+  def verifyLastCheckpointMetadata(tablePath: String, checkpointAt: Long, expSize: Long): Unit = {
+    val filePath = f"$tablePath/_delta_log/_last_checkpoint"
+
+    val source = scala.io.Source.fromFile(filePath)
+    val result = try source.getLines().mkString(",") finally source.close()
+
+    assert(result === s"""{"version":$checkpointAt,"size":$expSize}""")
+  }
+
+  /** Helper method to remove the delta files before the given version, to make sure the read is
+   * using a checkpoint as base for state reconstruction.
+   */
+  def deleteDeltaFilesBefore(tablePath: String, beforeVersion: Long): Unit = {
+    Seq.range(0, beforeVersion).foreach { version =>
+      val filePath = new Path(f"$tablePath/_delta_log/$version%020d.json")
+      new Path(tablePath).getFileSystem(new Configuration()).delete(filePath, false /* recursive */)
+    }
+
+    // try to query a version < beforeVersion
+    val ex = intercept[VersionNotFoundException] {
+      spark.read.format("delta").option("versionAsOf", beforeVersion - 1).load(tablePath)
+    }
+    assert(ex.getMessage().contains(
+      s"Cannot time travel Delta table to version ${beforeVersion - 1}"))
+  }
+
+  def setCheckpointInterval(tablePath: String, interval: Int): Unit = {
+    spark.sql(s"ALTER TABLE delta.`$tablePath` " +
+      s"SET TBLPROPERTIES ('delta.checkpointInterval' = '$interval')")
+  }
+
+  def dataFileCount(tablePath: String): Int = {
+    Files.walk(Paths.get(tablePath)).iterator().asScala
+      .count(path => path.toString.endsWith(".parquet") && !path.toString.contains("_delta_log"))
+  }
+
+  def checkpointFilePath(tablePath: String, checkpointVersion: Long): String = {
+    f"$tablePath/_delta_log/$checkpointVersion%020d.checkpoint.parquet"
+  }
+
+  def assertCheckpointExists(tablePath: String, atVersion: Long): Unit = {
+    val cpPath = checkpointFilePath(tablePath, checkpointVersion = atVersion)
+    assert(new File(cpPath).exists())
+  }
+
+  def copyTable(goldenTableName: String, targetLocation: String): Unit = {
+    val source = new File(goldenTablePath(goldenTableName))
+    val target = new File(targetLocation)
+    FileUtils.copyDirectory(source, target)
+  }
+
+  def checkpointIfReady(
+    engine: Engine,
+    tablePath: String,
+    result: TransactionCommitResult,
+    expSize: Long): Unit = {
+    if (result.isReadyForCheckpoint) {
+      Table.forPath(engine, tablePath).checkpoint(engine, result.getVersion)
+      verifyLastCheckpointMetadata(tablePath, checkpointAt = result.getVersion, expSize)
+    }
+  }
+}
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 9c375a4da14..e3602c9f399 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
@@ -17,33 +17,66 @@ package io.delta.kernel.defaults
 
 import com.fasterxml.jackson.databind.ObjectMapper
 import io.delta.golden.GoldenTableUtils.goldenTablePath
-import io.delta.kernel.Operation.CREATE_TABLE
+import io.delta.kernel.Operation.{CREATE_TABLE, WRITE}
+import io.delta.kernel._
+import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, Row}
+import io.delta.kernel.defaults.internal.data.DefaultColumnarBatch
 import io.delta.kernel.defaults.internal.parquet.ParquetSuiteBase
-import io.delta.kernel.defaults.utils.{TestRow, TestUtils}
+import io.delta.kernel.defaults.utils.TestRow
 import io.delta.kernel.engine.Engine
 import io.delta.kernel.exceptions.{KernelException, TableAlreadyExistsException, TableNotFoundException}
+import io.delta.kernel.expressions.Literal
+import io.delta.kernel.expressions.Literal._
+import io.delta.kernel.internal.checkpoints.CheckpointerSuite.selectSingleElement
+import io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames
+import io.delta.kernel.internal.util.Utils.toCloseableIterator
 import io.delta.kernel.types.DateType.DATE
+import io.delta.kernel.types.DoubleType.DOUBLE
 import io.delta.kernel.types.IntegerType.INTEGER
+import io.delta.kernel.types.StringType.STRING
 import io.delta.kernel.types.TimestampNTZType.TIMESTAMP_NTZ
+import io.delta.kernel.types.TimestampType.TIMESTAMP
 import io.delta.kernel.types._
-import io.delta.kernel.utils.CloseableIterable.emptyIterable
-import io.delta.kernel.{Meta, Operation, Table}
-import org.scalatest.funsuite.AnyFunSuite
+import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable}
+import io.delta.kernel.utils.CloseableIterator
 
+import java.util.Optional
 import scala.collection.JavaConverters._
+import scala.collection.immutable.Seq
 
-class DeltaTableWritesSuite extends AnyFunSuite with TestUtils with ParquetSuiteBase {
+class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase {
   val OBJ_MAPPER = new ObjectMapper()
   val testEngineInfo = "test-engine"
 
-  /** Test table schemas */
+  /** Test table schemas and test */
   val testSchema = new StructType().add("id", INTEGER)
+  val dataBatches1 = generateData(testSchema, Seq.empty, Map.empty, 200, 3)
+  val dataBatches2 = generateData(testSchema, Seq.empty, Map.empty, 400, 5)
+
   val testPartitionColumns = Seq("part1", "part2")
   val testPartitionSchema = new StructType()
     .add("id", INTEGER)
     .add("part1", INTEGER) // partition column
     .add("part2", INTEGER) // partition column
 
+  val dataPartitionBatches1 = generateData(
+    testPartitionSchema,
+    testPartitionColumns,
+    Map("part1" -> ofInt(1), "part2" -> ofInt(2)),
+    batchSize = 237,
+    numBatches = 3)
+
+  val dataPartitionBatches2 = generateData(
+    testPartitionSchema,
+    testPartitionColumns,
+    Map("part1" -> ofInt(4), "part2" -> ofInt(5)),
+    batchSize = 876,
+    numBatches = 7)
+
+  ///////////////////////////////////////////////////////////////////////////
+  // Create table tests
+  ///////////////////////////////////////////////////////////////////////////
+
   test("create table - provide no schema - expect failure") {
     withTempDirAndEngine { (tablePath, engine) =>
       val table = Table.forPath(engine, tablePath)
@@ -216,23 +249,444 @@ class DeltaTableWritesSuite extends AnyFunSuite with TestUtils with ParquetSuite
     }
   }
 
-  def withTempDirAndEngine(f: (String, Engine) => Unit): Unit = {
-    withTempDir { dir => f(dir.getAbsolutePath, defaultEngine) }
+  ///////////////////////////////////////////////////////////////////////////
+  // 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)
+    }
   }
 
-  def verifyWrittenContent(
-    tablePath: String,
-    expSchema: StructType,
-    expData: Seq[TestRow]): Unit = {
-    val actSchema = tableSchema(tablePath)
+  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)
+
+      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) =>
+      val table = Table.forPath(engine, tblPath)
+
+      val txn = createWriteTxnBuilder(table)
+        .withSchema(engine, testSchema)
+        .build(engine)
+
+      val txnState = txn.getTransactionState(engine)
+      val stagedFiles = stageData(txnState, Map.empty, dataBatches1)
+
+      val stagedActionsIterable = inMemoryIterable(stagedFiles)
+      val commitResult = txn.commit(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."))
+    }
+  }
+
+  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 { (tblPath, engine) =>
+      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)
+      }
+      {
+        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)
+      }
+    }
+  }
+
+  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(10, 2).foreach { checkpointInterval =>
+    test(s"insert into partitioned table - isReadyForCheckpoint(interval=$checkpointInterval)") {
+      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")
+        val partValues = Map("PART1" -> ofDouble(1.0), "pART2" -> ofTimestamp(1231212L))
+        val data = Seq(
+          partValues -> generateData(schema, partCols, partValues, batchSize = 200, numBatches = 3))
+
+        // create a table first
+        appendData(engine, tblPath, isNewTable = true, schema, partCols, data) // version 0
+        var expData = data.map(_._2).flatMap(_.flatMap(_.toTestRows))
+        var currentTableVersion = 0
+
+        if (checkpointInterval != 10) {
+          // If it is not the default interval alter the table using Spark to set a
+          // custom checkpoint interval
+          setCheckpointInterval(tblPath, interval = checkpointInterval) // version 1
+          currentTableVersion = 1
+        }
+
+        def isCheckpointExpected(version: Long): Boolean = {
+          version != 0 && version % checkpointInterval == 0
+        }
+
+        for (i <- currentTableVersion + 1 until 31) {
+          val commitResult = appendData(engine, tblPath, data = data)
+
+          val parquetFileCount = dataFileCount(tblPath)
+          assert(parquetFileCount > 0)
+          checkpointIfReady(engine, tblPath, commitResult, expSize = parquetFileCount)
+
+          verifyCommitResult(commitResult, expVersion = i, isCheckpointExpected(i))
+
+          expData = expData ++ data.map(_._2).flatMap(_.flatMap(_.toTestRows))
+        }
+
+        // expect the checkpoints created at expected versions
+        Seq.range(0, 31).filter(isCheckpointExpected(_)).foreach { version =>
+          assertCheckpointExists(tblPath, atVersion = version)
+        }
+
+        // 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)
+      }
+    }
+  }
+
+  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}")
+            }
+          }
+          (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))
+        }
+      }
+
+      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)
+
+        expData = expData ++ dataWithPartInfo.flatMap(_._2).flatMap(_.toTestRows)
+
+        val fileCount = dataFileCount(tblPath)
+        checkpointIfReady(engine, tblPath, commitResult, expSize = fileCount)
+
+        verifyCommitResult(commitResult, expVersion = i, i % checkpointInterval == 0)
+        verifyCommitInfo(tblPath, version = i, partitionCols = null, operation = WRITE)
+        verifyWrittenContent(tblPath, schema, expData)
+      }
+
+      assertCheckpointExists(tblPath, atVersion = checkpointInterval)
+    }
+  }
+
+  test("insert into table - given data schema mismatch") {
+    withTempDirAndEngine { (tblPath, engine) =>
+      val ex = intercept[KernelException] {
+        val data = Seq(Map.empty[String, Literal] -> dataPartitionBatches1) // data schema mismatch
+        appendData(engine, tblPath, isNewTable = true, testSchema, partCols = Seq.empty, data)
+      }
+      assert(ex.getMessage.contains("The schema of the data to be written to " +
+        "the table doesn't match the table schema"))
+    }
+  }
+
+  test("insert into table - missing partition column info") {
+    withTempDirAndEngine { (tblPath, engine) =>
+      val ex = intercept[IllegalArgumentException] {
+        appendData(engine,
+          tblPath,
+          isNewTable = true,
+          testPartitionSchema,
+          testPartitionColumns,
+          data = Seq(Map("part1" -> ofInt(1)) -> dataPartitionBatches1) // missing part2
+        )
+      }
+      assert(ex.getMessage.contains(
+        "Partition values provided are not matching the partition columns."))
+    }
+  }
+
+  test("insert into partitioned table - invalid type of partition value") {
+    withTempDirAndEngine { (tblPath, engine) =>
+      val ex = intercept[IllegalArgumentException] {
+        // part2 type should be int, be giving a string value
+        val data = Seq(Map("part1" -> ofInt(1), "part2" -> ofString("sdsd"))
+          -> dataPartitionBatches1)
+        appendData(engine,
+          tblPath,
+          isNewTable = true,
+          testPartitionSchema,
+          testPartitionColumns,
+          data)
+      }
+      assert(ex.getMessage.contains(
+        "Partition column part2 is of type integer but the value provided is of type string"))
+    }
+  }
+
+  def verifyWrittenContent(path: String, expSchema: StructType, expData: Seq[TestRow]): Unit = {
+    val actSchema = tableSchema(path)
     assert(actSchema === expSchema)
 
     // verify data using Kernel reader
-    checkTable(tablePath, expData)
-
-    // verify data using Spark reader
-    val resultSpark = spark.sql(s"SELECT * FROM delta.`$tablePath`").collect().map(TestRow(_))
-    checkAnswer(resultSpark, expData)
+    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`").collect().map(TestRow(_))
+      checkAnswer(resultSpark, expData)
+    }
   }
 
   def verifyCommitInfo(
@@ -260,6 +714,14 @@ class DeltaTableWritesSuite extends AnyFunSuite with TestUtils with ParquetSuite
     assert(row.getAs[String]("operation") === operation.getDescription)
   }
 
+  def verifyCommitResult(
+    result: TransactionCommitResult,
+    expVersion: Long,
+    expIsReadyForCheckpoint: Boolean): Unit = {
+    assert(result.getVersion === expVersion)
+    assert(result.isReadyForCheckpoint === expIsReadyForCheckpoint)
+  }
+
   def removeUnsupportedTypes(structType: StructType): StructType = {
     def process(dataType: DataType): Option[DataType] = dataType match {
       case a: ArrayType =>
@@ -293,4 +755,87 @@ class DeltaTableWritesSuite extends AnyFunSuite with TestUtils with ParquetSuite
     }
     newStructType
   }
+
+  def createWriteTxnBuilder(table: Table): TransactionBuilder = {
+    table.createTransactionBuilder(defaultEngine, testEngineInfo, Operation.WRITE)
+  }
+
+  def generateData(
+    schema: StructType,
+    partitionCols: Seq[String],
+    partitionValues: Map[String, Literal],
+    batchSize: Int,
+    numBatches: Int): Seq[FilteredColumnarBatch] = {
+    val partitionValuesSchemaCase =
+      casePreservingPartitionColNames(partitionCols.asJava, partitionValues.asJava)
+
+    var batches = Seq.empty[ColumnarBatch]
+    for (_ <- 0 until numBatches) {
+      var vectors = Seq.empty[ColumnVector]
+      schema.fields().forEach { field =>
+        val colType = field.getDataType
+        val partValue = partitionValuesSchemaCase.get(field.getName)
+        if (partValue != null) {
+          // handle the partition column by inserting a vector with single value
+          val vector = testSingleValueVector(colType, batchSize, partValue.getValue)
+          vectors = vectors :+ vector
+        } else {
+          // handle the regular columns
+          val vector = testColumnVector(batchSize, colType)
+          vectors = vectors :+ vector
+        }
+      }
+      batches = batches :+ new DefaultColumnarBatch(batchSize, schema, vectors.toArray)
+    }
+    batches.map(batch => new FilteredColumnarBatch(batch, Optional.empty()))
+  }
+
+  def stageData(
+    state: Row,
+    partitionValues: Map[String, Literal],
+    data: Seq[FilteredColumnarBatch])
+  : CloseableIterator[Row] = {
+    val physicalDataIter = Transaction.transformLogicalData(
+      defaultEngine,
+      state,
+      toCloseableIterator(data.toIterator.asJava),
+      partitionValues.asJava)
+
+    val writeContext = Transaction.getWriteContext(defaultEngine, state, partitionValues.asJava)
+
+    val writeResultIter = defaultEngine
+      .getParquetHandler
+      .writeParquetFiles(
+        writeContext.getTargetDirectory,
+        physicalDataIter,
+        writeContext.getStatisticsColumns)
+
+    Transaction.generateAppendActions(defaultEngine, state, writeResultIter, writeContext)
+  }
+
+  def appendData(
+    engine: Engine = defaultEngine,
+    tablePath: String,
+    isNewTable: Boolean = false,
+    schema: StructType = null,
+    partCols: Seq[String] = null,
+    data: Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])]): TransactionCommitResult = {
+
+    var txnBuilder = createWriteTxnBuilder(Table.forPath(engine, tablePath))
+
+    if (isNewTable) {
+      txnBuilder = txnBuilder.withSchema(engine, schema)
+        .withPartitionColumns(engine, partCols.asJava)
+    }
+
+    val txn = txnBuilder.build(engine)
+    val txnState = txn.getTransactionState(engine)
+
+    val actions = data.map { case (partValues, partData) =>
+      stageData(txnState, partValues, partData)
+    }
+
+    val combineActions = inMemoryIterable(actions.reduceLeft(_ combine _))
+    txn.commit(engine, combineActions)
+  }
 }
diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/json/JsonUtilsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/json/JsonUtilsSuite.scala
index d0284a26c18..4e2420d034d 100644
--- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/json/JsonUtilsSuite.scala
+++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/internal/json/JsonUtilsSuite.scala
@@ -122,7 +122,7 @@ class JsonUtilsSuite extends AnyFunSuite with TestUtils with VectorTestUtils {
       ),
       """{
         |"c0":{"24":200,"25":201},
-        |"c1":{"25":203},
+        |"c1":{"27":null,"25":203},
         |"c3":{}
         |}""".stripMargin
     ),
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 183e772e2d9..a97aee85a70 100644
--- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala
+++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala
@@ -19,26 +19,23 @@ import java.io.{File, FileNotFoundException}
 import java.math.{BigDecimal => BigDecimalJ}
 import java.nio.file.Files
 import java.util.{Optional, TimeZone, UUID}
-
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
-
 import io.delta.golden.GoldenTableUtils
 import io.delta.kernel.{Scan, Snapshot, Table}
-import io.delta.kernel.data.{ColumnarBatch, ColumnVector, FilteredColumnarBatch, MapValue, Row}
+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
 import io.delta.kernel.engine.Engine
 import io.delta.kernel.expressions.{Column, Predicate}
 import io.delta.kernel.internal.InternalScanFileUtils
 import io.delta.kernel.internal.data.ScanStateRow
-import io.delta.kernel.internal.util.ColumnMapping
+import io.delta.kernel.internal.util.{ColumnMapping, Utils}
 import io.delta.kernel.internal.util.Utils.singletonCloseableIterator
 import io.delta.kernel.types._
 import io.delta.kernel.utils.CloseableIterator
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.shaded.org.apache.commons.io.FileUtils
-
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.{types => sparktypes}
 import org.apache.spark.sql.catalyst.plans.SQLHelper
@@ -241,6 +238,38 @@ trait TestUtils extends Assertions with SQLHelper {
     result
   }
 
+  def readTableUsingKernel(
+    engine: Engine,
+    tablePath: String,
+    readSchema: StructType): Seq[FilteredColumnarBatch] = {
+    val scan = Table.forPath(engine, tablePath)
+      .getLatestSnapshot(engine)
+      .getScanBuilder(engine)
+      .withReadSchema(engine, readSchema)
+      .build()
+    val scanState = scan.getScanState(engine)
+
+    val physicalDataReadSchema = ScanStateRow.getPhysicalDataReadSchema(engine, scanState)
+    var result: Seq[FilteredColumnarBatch] = Nil
+    scan.getScanFiles(engine).forEach { fileColumnarBatch =>
+      fileColumnarBatch.getRows.forEach { scanFile =>
+        val fileStatus = InternalScanFileUtils.getAddFileStatus(scanFile)
+        val physicalDataIter = engine.getParquetHandler.readParquetFiles(
+          singletonCloseableIterator(fileStatus),
+          physicalDataReadSchema,
+          Optional.empty())
+        var dataBatches: CloseableIterator[FilteredColumnarBatch] = null
+        try {
+          dataBatches = Scan.transformPhysicalData(engine, scanState, scanFile, physicalDataIter)
+          dataBatches.forEach { dataBatch => result = result :+ dataBatch }
+        } finally {
+          Utils.closeCloseables(dataBatches)
+        }
+      }
+    }
+    result
+  }
+
   /**
    * Execute {@code f} with {@code TimeZone.getDefault()} set to the time zone provided.
    *
@@ -455,6 +484,16 @@ trait TestUtils extends Assertions with SQLHelper {
     }
   }
 
+  def withSparkTimeZone(timeZone: String)(fn: () => Unit): Unit = {
+    val prevTimeZone = spark.conf.get("spark.sql.session.timeZone")
+    try {
+      spark.conf.set("spark.sql.session.timeZone", timeZone)
+      fn()
+    } finally {
+      spark.conf.set("spark.sql.session.timeZone", prevTimeZone)
+    }
+  }
+
   /**
    * Builds a MapType ColumnVector from a sequence of maps.
    */

From 1ba483226363d8056ecb2ffb029ac63f4c1ac5a9 Mon Sep 17 00:00:00 2001
From: Dhruv Arya 
Date: Mon, 6 May 2024 09:01:19 -0700
Subject: [PATCH 10/11] [Spark][3.2] Fix CommitInfo.inCommitTimestamp
 deserialization for very small timestamps (#3045)



#### Which Delta project/connector is this regarding?


- [X] Spark
- [ ] Standalone
- [ ] Flink
- [ ] Kernel
- [ ] Other (fill in here)

## Description


Currently, if we deserialize a CommitInfo with a very small
inCommitTimestamp and then try to access this inCommitTimestamp, this
exception is thrown

```
java.lang.ClassCastException: class java.lang.Integer cannot be cast to class java.lang.Long
```
This PR fixes the CommitInfo so that the inCommitTimestamp field is
deserialized correctly.

## How was this patch tested?


Added a new test case that was failing before the fix.

## Does this PR introduce _any_ user-facing changes?


No
---
 .../spark/sql/delta/actions/actions.scala     |  1 +
 .../sql/delta/ActionSerializerSuite.scala     | 19 +++++++++++++++++++
 2 files changed, 20 insertions(+)

diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala b/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala
index ac341031579..e8eec00405e 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala
@@ -1077,6 +1077,7 @@ case class CommitInfo(
     // infer the commit version from the file name and fill in this field then.
     @JsonDeserialize(contentAs = classOf[java.lang.Long])
     version: Option[Long],
+    @JsonDeserialize(contentAs = classOf[java.lang.Long])
     inCommitTimestamp: Option[Long],
     timestamp: Timestamp,
     userId: Option[String],
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/ActionSerializerSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/ActionSerializerSuite.scala
index 605eb8ee77f..3fe73dbaea9 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/ActionSerializerSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/ActionSerializerSuite.scala
@@ -185,6 +185,25 @@ class ActionSerializerSuite extends QueryTest with SharedSparkSession with Delta
     assert(Action.fromJson(json1) === expectedCommitInfo)
   }
 
+  test("deserialization of CommitInfo with a very small ICT") {
+    val json1 =
+      """{"commitInfo":{"inCommitTimestamp":123,"timestamp":123,"operation":"CONVERT",""" +
+        """"operationParameters":{},"readVersion":23,""" +
+        """"isolationLevel":"SnapshotIsolation","isBlindAppend":true,""" +
+        """"operationMetrics":{"m1":"v1","m2":"v2"},"userMetadata":"123"}}""".stripMargin
+    assert(Action.fromJson(json1).asInstanceOf[CommitInfo].inCommitTimestamp.get == 123L)
+  }
+
+  test("deserialization of CommitInfo with missing ICT") {
+    val json1 =
+      """{"commitInfo":{"timestamp":123,"operation":"CONVERT",""" +
+        """"operationParameters":{},"readVersion":23,""" +
+        """"isolationLevel":"SnapshotIsolation","isBlindAppend":true,""" +
+        """"operationMetrics":{"m1":"v1","m2":"v2"},"userMetadata":"123"}}""".stripMargin
+    val ictOpt: Option[Long] = Action.fromJson(json1).asInstanceOf[CommitInfo].inCommitTimestamp
+    assert(ictOpt.isEmpty)
+  }
+
   testActionSerDe(
     "Protocol - json serialization/deserialization",
     Protocol(minReaderVersion = 1, minWriterVersion = 2),

From 8cb2e7826914033698b0cc7e4435c7c2065c75db Mon Sep 17 00:00:00 2001
From: Johan Lasperas 
Date: Mon, 6 May 2024 18:29:11 +0200
Subject: [PATCH 11/11] [Doc] Type Widening documentation (#3025)

## Description
Add a documentation page for the type widening table feature, in preview
in Delta. 3.2

## How was this patch tested?
N/A
---
 docs/source/delta-drop-feature.md  |  1 +
 docs/source/delta-type-widening.md | 79 ++++++++++++++++++++++++++++++
 docs/source/index.md               |  1 +
 docs/source/versioning.md          |  2 +
 4 files changed, 83 insertions(+)
 create mode 100644 docs/source/delta-type-widening.md

diff --git a/docs/source/delta-drop-feature.md b/docs/source/delta-drop-feature.md
index 2aa3dca2566..f27416fa2dc 100644
--- a/docs/source/delta-drop-feature.md
+++ b/docs/source/delta-drop-feature.md
@@ -25,6 +25,7 @@ To remove a Delta table feature, you run an `ALTER TABLE  DROP FEATU
 You can drop the following Delta table features:
 
 - `deletionVectors`. See [_](delta-deletion-vectors.md).
+- `typeWidening-preview`. See [_](delta-type-widening.md). Type widening is available in preview in  3.2.0 and above.
 - `v2Checkpoint`. See [V2 Checkpoint Spec](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#v2-spec). Drop support for V2 Checkpoints is available in  3.1.0 and above.
 
 You cannot drop other [Delta table features](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#valid-feature-names-in-table-features).
diff --git a/docs/source/delta-type-widening.md b/docs/source/delta-type-widening.md
new file mode 100644
index 00000000000..3b952dfebe8
--- /dev/null
+++ b/docs/source/delta-type-widening.md
@@ -0,0 +1,79 @@
+---
+description: Learn about type widening in Delta.
+---
+
+# Delta type widening
+
+.. note:: This feature is available in preview in  3.2.
+
+The type widening feature allows changing the type of columns in a Delta table to a wider type. This enables manual type changes using the `ALTER TABLE ALTER COLUMN` command and automatic type migration with schema evolution in `INSERT` and `MERGE INTO` commands.
+
+## Supported type changes
+
+The feature preview in  3.2 supports a limited set of type changes:
+- `BYTE` to `SHORT` and `INT`.
+- `SHORT` to `INT`
+
+Type changes are supported for top-level columns as well as fields nested inside structs, maps and arrays.
+
+## How to enable  type widening
+
+.. important::
+
+  Enabling type widening sets the Delta table feature `typeWidening-preview`, a reader/writer protocol feature. Only clients that support this table feature can read and write to the table once the table feature is set. You must use  3.2 or above to read and write to such Delta tables.
+
+You can enable type widening on an existing table by setting the `delta.enableTypeWidening` table property to `true`:
+
+  ```sql
+  ALTER TABLE  SET TBLPROPERTIES ('delta.enableTypeWidening' = 'true')
+  ```
+
+Alternatively, you can enable type widening during table creation:
+
+  ```sql
+  CREATE TABLE T(c1 INT) USING DELTA TBLPROPERTIES('delta.enableTypeWidening' = 'true')
+  ```
+
+To disable type widening:
+
+  ```sql
+  ALTER TABLE  SET TBLPROPERTIES ('delta.enableTypeWidening' = 'false')
+  ```
+
+Disabling type widening prevents future type changes from being applied to the table. It doesn't affect type changes previously applied and in particular, it doesn't remove the type widening table feature and doesn't allow clients that don't support the type widening table feature to read and write to the table.
+
+To remove the type widening table feature from the table and allow other clients that don't support this feature to read and write to the table, see [_](#removing-the-type-widening-table-feature).
+
+## Manually applying a type change
+
+When type widening is enabled on a Delta table, you can change the type of a column using the `ALTER COLUMN` command:
+
+```sql
+ALTER TABLE  ALTER COLUMN  TYPE 
+```
+
+The table schema is updated without rewriting the underlying Parquet files.
+
+## Type changes with automatic schema evolution
+Type changes are applied automatically during ingestion with `INSERT` and `MERGE INTO` commands when:
+- Type widening is enabled on the target table.
+- The command runs with [automatic schema evolution](delta-update.md#merge-schema-evolution) enabled.
+- The source column type is wider than the target column type.
+- Changing the target column type to the source type is a [supported type change](#supported-type-changes)
+
+When all conditions are satisfied, the target table schema is updated automatically to change the target column type to the source column type.
+
+## Removing the type widening table feature
+
+The type widening feature can be removed from a Delta table using the `DROP FEATURE` command:
+
+```sql
+ ALTER TABLE  DROP FEATURE 'typeWidening-preview' [TRUNCATE HISTORY]
+```
+
+See [_](delta-drop-feature.md) for more information on dropping Delta table features.
+
+When dropping the type widening feature, the underlying Parquet files are rewritten when necessary to ensure that the column types in the files match the column types in the Delta table schema.
+After the type widening feature is removed from the table, Delta clients that don't support the feature can read and write to the table.
+
+.. include:: /shared/replacements.md
diff --git a/docs/source/index.md b/docs/source/index.md
index 783301b6a03..efacd6fbb08 100644
--- a/docs/source/index.md
+++ b/docs/source/index.md
@@ -28,6 +28,7 @@ This is the documentation site for .
     delta-drop-feature
     delta-apidoc
     delta-storage
+    delta-type-widening
     delta-uniform
     delta-sharing
     concurrency-control
diff --git a/docs/source/versioning.md b/docs/source/versioning.md
index e7287163b2a..ff5f6026124 100644
--- a/docs/source/versioning.md
+++ b/docs/source/versioning.md
@@ -26,6 +26,7 @@ The following  features break forward compatibility. Features are enabled
    V2 Checkpoints, [Delta Lake 3.0.0](https://github.com/delta-io/delta/releases/tag/v3.0.0),[V2 Checkpoint Spec](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#v2-spec)
    Domain metadata, [Delta Lake 3.0.0](https://github.com/delta-io/delta/releases/tag/v3.0.0),[Domain Metadata Spec](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#domain-metadata)
    Clustering, [Delta Lake 3.1.0](https://github.com/delta-io/delta/releases/tag/v3.1.0),[_](/delta-clustering.md)
+   Type widening (Preview),[Delta Lake 3.2.0](https://github.com/delta-io/delta/releases/tag/v3.2.0),[_](/delta-type-widening.md)
 
 
 
@@ -106,6 +107,7 @@ The following table shows minimum protocol versions required for  feature
    Timestamp without Timezone,7,3,[TimestampNTZType](https://spark.apache.org/docs/latest/sql-ref-datatypes.html)
    Iceberg Compatibility V1,7,2,[IcebergCompatV1](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#iceberg-compatibility-v1)
    V2 Checkpoints,7,3,[V2 Checkpoint Spec](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#v2-spec)
+   Type widening (Preview),7,3,[_](/delta-type-widening.md)