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") + ) }