From a1a0e0f75cca6936c392a25e70809be6fc452393 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Thu, 26 Sep 2024 13:53:08 -0700 Subject: [PATCH 1/4] feat: Add support for using JDBC Catalog for Iceberg testing Fixes #6029 --- extensions/iceberg/build.gradle | 18 ++- .../iceberg/layout/IcebergBaseLayout.java | 12 +- .../iceberg/relative/RelativeFileIO.java | 137 ++++++++++++++++++ .../iceberg/relative/RelativeInputFile.java | 39 +++++ .../iceberg/relative/RelativeOutputFile.java | 40 +++++ .../iceberg/util/IcebergCatalogAdapter.java | 2 +- .../deephaven/iceberg/CatalogAdapterTest.java | 49 +++++++ .../io/deephaven/iceberg/CatalogHelper.java | 59 ++++++++ .../io/deephaven/iceberg/PyIceberg1Test.java | 111 ++++++++++++++ .../iceberg/TestCatalog/IcebergTestTable.java | 41 +++--- .../iceberg/junit5/CatalogAdapterBase.java | 27 ++++ .../deephaven/iceberg/pyiceberg-1/README.md | 46 ++++++ .../iceberg/pyiceberg-1/dh-test-catalog.db | Bin 0 -> 20480 bytes ...acf870-0d6f-4d74-9cb7-d3be07a263eb.parquet | 3 + ...acdfcc-2afa-4020-af3c-4992ee16e9e8.parquet | 3 + ...-6384-4c13-8d84-74a9ffd2d158.metadata.json | 1 + ...-0a0e-4e20-a733-00ba6e10e517.metadata.json | 1 + ...-9adc-43dc-8e18-a7034a6a837a.metadata.json | 1 + ...-9990-4db5-9f8b-2bd8054c65ff.metadata.json | 1 + ...acf870-0d6f-4d74-9cb7-d3be07a263eb-m0.avro | Bin 0 -> 4405 bytes ...acdfcc-2afa-4020-af3c-4992ee16e9e8-m0.avro | Bin 0 -> 4408 bytes ...-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.avro | Bin 0 -> 1910 bytes ...-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro | Bin 0 -> 1776 bytes .../pyiceberg-1/generate-pyiceberg-1.py | 53 +++++++ gradle/libs.versions.toml | 3 + 25 files changed, 619 insertions(+), 28 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeFileIO.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeInputFile.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeOutputFile.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogHelper.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/README.md create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-catalog.db create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.parquet create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.parquet create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00000-cf59d01d-6384-4c13-8d84-74a9ffd2d158.metadata.json create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00001-6c47a69a-0a0e-4e20-a733-00ba6e10e517.metadata.json create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00002-4c26a33a-9adc-43dc-8e18-a7034a6a837a.metadata.json create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00003-669f91cb-9990-4db5-9f8b-2bd8054c65ff.metadata.json create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/78acf870-0d6f-4d74-9cb7-d3be07a263eb-m0.avro create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/a1acdfcc-2afa-4020-af3c-4992ee16e9e8-m0.avro create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/snap-7895378999715099054-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.avro create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/snap-8833084047573120270-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/generate-pyiceberg-1.py diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle index 2c30f242bfe..6f0b61b03a8 100644 --- a/extensions/iceberg/build.gradle +++ b/extensions/iceberg/build.gradle @@ -44,12 +44,20 @@ dependencies { testRuntimeOnly libs.junit.jupiter.engine testRuntimeOnly libs.junit.platform.launcher - testImplementation libs.testcontainers - testImplementation libs.testcontainers.junit.jupiter - testImplementation libs.testcontainers.localstack - testImplementation libs.testcontainers.minio - testRuntimeOnly project(':test-configs') testRuntimeOnly project(':log-to-slf4j') testRuntimeOnly libs.slf4j.simple + + testRuntimeOnly libs.xerial.sqlite.jdbc + + // NOTE: we need to pull down more hadoop dependencies, + // buildSrc/src/main/groovy/io.deephaven.hadoop-common-dependencies.gradle excludes some stuff we need to do + // Hadoop FileIO. + testRuntimeOnly libs.hadoop.common + + testImplementation project(':engine-test-utils') +} + +test { + useJUnitPlatform() } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index 7bf0f5222a2..cbf8b272376 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -9,11 +9,13 @@ import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; import io.deephaven.iceberg.location.IcebergTableParquetLocationKey; +import io.deephaven.iceberg.relative.RelativeFileIO; import io.deephaven.iceberg.util.IcebergInstructions; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -130,6 +132,14 @@ public IcebergBaseLayout( abstract IcebergTableLocationKey keyFromDataFile(DataFile df, URI fileUri); + private URI dataFileUri(DataFile df) { + String path = df.path().toString(); + if (fileIO instanceof RelativeFileIO) { + path = ((RelativeFileIO) fileIO).absoluteLocation(path); + } + return FileUtils.convertToURI(path, false); + } + @Override public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { try { @@ -144,7 +154,7 @@ public synchronized void findKeys(@NotNull final Consumer reader = ManifestFiles.read(manifestFile, fileIO)) { for (DataFile df : reader) { - final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); + final URI fileUri = dataFileUri(df); final IcebergTableLocationKey locationKey = cache.computeIfAbsent(fileUri, uri -> keyFromDataFile(df, fileUri)); if (locationKey != null) { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeFileIO.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeFileIO.java new file mode 100644 index 00000000000..921134053e5 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeFileIO.java @@ -0,0 +1,137 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.relative; + +import io.deephaven.util.annotations.VisibleForTesting; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hadoop.HadoopConfigurable; +import org.apache.iceberg.hadoop.SerializableConfiguration; +import org.apache.iceberg.io.BulkDeletionFailureException; +import org.apache.iceberg.io.DelegateFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileInfo; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.ResolvingFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.util.SerializableSupplier; + +import java.util.Map; +import java.util.function.Function; + +/** + * While this class is in the public source set, it is meant to support testing use cases only and should not be used in + * production. + * + * @see Support relative paths in Table Metadata + */ +@VisibleForTesting +public final class RelativeFileIO implements HadoopConfigurable, DelegateFileIO { + public static final String BASE_PATH = "relative.base-path"; + private static final String IO_IMPL = "relative.io-impl"; + private static final String IO_DEFAULT_IMPL = ResolvingFileIO.class.getName(); + + private String basePath; + + private DelegateFileIO io; + + private SerializableSupplier hadoopConf; + + public RelativeFileIO() {} + + public RelativeFileIO(Configuration hadoopConf) { + this(new SerializableConfiguration(hadoopConf)::get); + } + + public RelativeFileIO(SerializableSupplier hadoopConf) { + this.hadoopConf = hadoopConf; + } + + @Override + public Configuration getConf() { + return hadoopConf.get(); + } + + @Override + public void setConf(Configuration conf) { + this.hadoopConf = new SerializableConfiguration(conf)::get; + } + + @Override + public void serializeConfWith(Function> confSerializer) { + this.hadoopConf = confSerializer.apply(getConf()); + } + + public String absoluteLocation(String location) { + return basePath + location; + } + + private String relativeLocation(String location) { + if (!location.startsWith(basePath)) { + throw new IllegalStateException(); + } + return location.substring(basePath.length()); + } + + @Override + public void initialize(Map properties) { + this.basePath = StringUtils.appendIfMissing(properties.get(BASE_PATH), "/"); + String impl = properties.getOrDefault(IO_IMPL, IO_DEFAULT_IMPL); + FileIO fileIO = CatalogUtil.loadFileIO(impl, properties, hadoopConf.get()); + if (!(fileIO instanceof DelegateFileIO)) { + throw new IllegalArgumentException("filoIO must be DelegateFileIO, " + fileIO.getClass()); + } + this.io = (DelegateFileIO) fileIO; + } + + @Override + public Map properties() { + return io.properties(); + } + + @Override + public InputFile newInputFile(String path) { + return new RelativeInputFile(path, io.newInputFile(absoluteLocation(path))); + } + + @Override + public InputFile newInputFile(String path, long length) { + return new RelativeInputFile(path, io.newInputFile(absoluteLocation(path), length)); + } + + @Override + public OutputFile newOutputFile(String path) { + return new RelativeOutputFile(path, io.newOutputFile(absoluteLocation(path))); + } + + @Override + public void deleteFiles(Iterable iterable) throws BulkDeletionFailureException { + io.deleteFiles(Streams.stream(iterable).map(this::absoluteLocation)::iterator); + } + + @Override + public Iterable listPrefix(String s) { + return Streams.stream(io.listPrefix(absoluteLocation(s))) + .map(x -> new FileInfo(relativeLocation(x.location()), x.size(), x.createdAtMillis()))::iterator; + } + + @Override + public void deletePrefix(String s) { + io.deletePrefix(absoluteLocation(s)); + } + + @Override + public void deleteFile(String path) { + io.deleteFile(absoluteLocation(path)); + } + + @Override + public void close() { + if (io != null) { + io.close(); + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeInputFile.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeInputFile.java new file mode 100644 index 00000000000..81fc85ac540 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeInputFile.java @@ -0,0 +1,39 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.relative; + +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.SeekableInputStream; + +import java.util.Objects; + +final class RelativeInputFile implements InputFile { + private final String location; + private final InputFile file; + + public RelativeInputFile(String location, InputFile file) { + this.location = Objects.requireNonNull(location); + this.file = Objects.requireNonNull(file); + } + + @Override + public long getLength() { + return file.getLength(); + } + + @Override + public SeekableInputStream newStream() { + return file.newStream(); + } + + @Override + public String location() { + return location; + } + + @Override + public boolean exists() { + return file.exists(); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeOutputFile.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeOutputFile.java new file mode 100644 index 00000000000..638814b782d --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeOutputFile.java @@ -0,0 +1,40 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.relative; + +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.PositionOutputStream; + +import java.util.Objects; + +final class RelativeOutputFile implements OutputFile { + private final String location; + private final OutputFile file; + + public RelativeOutputFile(String location, OutputFile file) { + this.location = Objects.requireNonNull(location); + this.file = Objects.requireNonNull(file); + } + + @Override + public PositionOutputStream create() { + return file.create(); + } + + @Override + public PositionOutputStream createOrOverwrite() { + return file.createOrOverwrite(); + } + + @Override + public String location() { + return location; + } + + @Override + public InputFile toInputFile() { + return new RelativeInputFile(location, file.toInputFile()); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index b76a750602d..8e52593b1b7 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -770,7 +770,7 @@ private Table readTableInternal( // Do we want the latest or a specific snapshot? final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); - final Schema schema = table.schemas().get(snapshot.schemaId()); + final Schema schema = snapshot == null ? table.schema() : table.schemas().get(snapshot.schemaId()); // Load the partitioning schema. final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java new file mode 100644 index 00000000000..d08e1735db0 --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -0,0 +1,49 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg; + +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.iceberg.junit5.CatalogAdapterBase; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +public class CatalogAdapterTest extends CatalogAdapterBase { + @Test + void empty() { + assertThat(catalogAdapter.listNamespaces()).isEmpty(); + } + + @Test + void createEmptyTable() { + final Schema schema = new Schema( + Types.NestedField.required(1, "Foo", Types.StringType.get()), + Types.NestedField.required(2, "Bar", Types.IntegerType.get()), + Types.NestedField.optional(3, "Baz", Types.DoubleType.get())); + final Namespace myNamespace = Namespace.of("MyNamespace"); + final TableIdentifier myTableId = TableIdentifier.of(myNamespace, "MyTable"); + catalogAdapter.catalog().createTable(myTableId, schema); + + assertThat(catalogAdapter.listNamespaces()).containsExactly(myNamespace); + assertThat(catalogAdapter.listTables(myNamespace)).containsExactly(myTableId); + final Table table; + { + final TableDefinition expectedDefinition = TableDefinition.of( + ColumnDefinition.ofString("Foo"), + ColumnDefinition.ofInt("Bar"), + ColumnDefinition.ofDouble("Baz")); + assertThat(catalogAdapter.getTableDefinition(myTableId, null)).isEqualTo(expectedDefinition); + table = catalogAdapter.readTable(myTableId, null); + assertThat(table.getDefinition()).isEqualTo(expectedDefinition); + } + // Note: this is failing w/ NPE, assumes that Snapshot is non-null. + // assertThat(table.isEmpty()).isTrue(); + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogHelper.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogHelper.java new file mode 100644 index 00000000000..88cc29fb62a --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogHelper.java @@ -0,0 +1,59 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg; + +import io.deephaven.iceberg.relative.RelativeFileIO; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.jdbc.JdbcCatalog; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Stream; + +public class CatalogHelper { + + // Using specific names to make clear these aren't a standard / convention + public static final String CATALOG_NAME = "dh-test-catalog.db"; + public static final String WAREHOUSE_NAME = "dh-test-warehouse"; + + public static Catalog createJdbcCatalog(String name, Path path, boolean isNew) throws IOException { + if (!Files.isDirectory(path)) { + throw new IllegalArgumentException("Must provide directory that exists"); + } + if (isNew) { + try (final Stream list = Files.list(path).limit(1)) { + if (list.iterator().hasNext()) { + throw new IllegalArgumentException("Expected directory to be empty"); + } + } + } + final Path catalogFile = path.resolve(CATALOG_NAME); + final Path warehouseDir = path.resolve(WAREHOUSE_NAME); + if (isNew) { + Files.createDirectory(warehouseDir); + } else { + if (!Files.isDirectory(warehouseDir)) { + throw new IllegalStateException("Expected warehouse directory to already exist"); + } + } + final Map properties = new HashMap<>(); + properties.put(CatalogProperties.CATALOG_IMPL, JdbcCatalog.class.getName()); + properties.put(CatalogProperties.URI, String.format("jdbc:sqlite:%s", catalogFile)); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseDir.toString()); + if (!isNew) { + // When we are referring to a catalog that already exists in our unit testing filesystem, we need to make + // hook in relative file support. See https://github.com/apache/iceberg/issues/1617 + properties.put(CatalogProperties.FILE_IO_IMPL, RelativeFileIO.class.getName()); + properties.put(RelativeFileIO.BASE_PATH, path.toString()); + } + final Configuration hadoopConf = new Configuration(); + return CatalogUtil.buildIcebergCatalog(name, properties, hadoopConf); + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java new file mode 100644 index 00000000000..3b98858d84e --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java @@ -0,0 +1,111 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg; + +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.testutil.TstUtils; +import io.deephaven.engine.util.TableTools; +import io.deephaven.iceberg.util.IcebergCatalogAdapter; +import io.deephaven.iceberg.util.IcebergTools; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.nio.file.Path; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +public class PyIceberg1Test { + private static final Namespace NAMESPACE = Namespace.of("dh-default"); + private static final TableIdentifier CITIES_ID = TableIdentifier.of(NAMESPACE, "cities"); + + // This will need to be updated if the data is regenerated + private static final long SNAPSHOT_1_ID = 8833084047573120270L; + private static final long SNAPSHOT_2_ID = 7895378999715099054L; + + private static final TableDefinition CITIES_1_TD = TableDefinition.of( + ColumnDefinition.ofString("city"), + ColumnDefinition.ofDouble("latitude"), + ColumnDefinition.ofDouble("lon")); + + private static final TableDefinition CITIES_2_TD = TableDefinition.of( + ColumnDefinition.ofString("city"), + ColumnDefinition.ofDouble("latitude"), + ColumnDefinition.ofDouble("longitude")); + + private IcebergCatalogAdapter catalogAdapter; + + @BeforeEach + void setUp() throws IOException, URISyntaxException { + // Note: for some reason, the name must be the same name that was used to create the SqlCatalog. Potentially so + // a single DB file can support multiple catalogs? + catalogAdapter = IcebergTools.createAdapter(CatalogHelper.createJdbcCatalog( + "SqlCatalogName", + Path.of(PyIceberg1Test.class.getResource("pyiceberg-1").toURI()), + false)); + } + + @Test + void catalogInfo() { + assertThat(catalogAdapter.listNamespaces()).containsExactly(NAMESPACE); + assertThat(catalogAdapter.listTables(NAMESPACE)).containsExactly(CITIES_ID); + final List snapshots = catalogAdapter.listSnapshots(CITIES_ID); + assertThat(snapshots).hasSize(2); + { + final Snapshot snapshot = snapshots.get(0); + assertThat(snapshot.parentId()).isNull(); + assertThat(snapshot.schemaId()).isEqualTo(0); + assertThat(snapshot.sequenceNumber()).isEqualTo(1L); + assertThat(snapshot.snapshotId()).isEqualTo(SNAPSHOT_1_ID); + } + { + final Snapshot snapshot = snapshots.get(1); + assertThat(snapshot.parentId()).isEqualTo(SNAPSHOT_1_ID); + assertThat(snapshot.schemaId()).isEqualTo(1); + assertThat(snapshot.sequenceNumber()).isEqualTo(2L); + assertThat(snapshot.snapshotId()).isEqualTo(SNAPSHOT_2_ID); + } + } + + @Test + void cities1() { + final Table cities1; + { + final TableDefinition td = catalogAdapter.getTableDefinition(CITIES_ID.toString(), SNAPSHOT_1_ID, null); + assertThat(td).isEqualTo(CITIES_1_TD); + cities1 = catalogAdapter.readTable(CITIES_ID, SNAPSHOT_1_ID); + assertThat(cities1.getDefinition()).isEqualTo(CITIES_1_TD); + } + final Table expectedCities1 = TableTools.newTable(CITIES_1_TD, + TableTools.stringCol("city", "Amsterdam", "San Francisco", "Drachten", "Paris"), + TableTools.doubleCol("latitude", 52.371807, 37.773972, 53.11254, 48.864716), + TableTools.doubleCol("lon", 4.896029, -122.431297, 6.0989, 2.349014)); + TstUtils.assertTableEquals(expectedCities1, cities1); + } + + @Test + void cities2() { + final Table cities2; + { + final TableDefinition td = catalogAdapter.getTableDefinition(CITIES_ID.toString(), SNAPSHOT_2_ID, null); + assertThat(td).isEqualTo(CITIES_2_TD); + cities2 = catalogAdapter.readTable(CITIES_ID, SNAPSHOT_2_ID); + assertThat(cities2.getDefinition()).isEqualTo(CITIES_2_TD); + } + // TODO(deephaven-core#6118): Iceberg column rename handling + // final Table expectedCities2 = TableTools.newTable(CITIES_2_TD, + // TableTools.stringCol("city", "Amsterdam", "San Francisco", "Drachten", "Paris", "Minneapolis", "New York"), + // TableTools.doubleCol("latitude", 52.371807, 37.773972, 53.11254, 48.864716, 44.977479, 40.730610), + // TableTools.doubleCol("longitude", 4.896029, -122.431297, 6.0989, 2.349014, -93.264358, -73.935242) + // ); + // TstUtils.assertTableEquals(expectedCities2, cities2); + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestTable.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestTable.java index bcac783def0..1ae6894bd13 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestTable.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestTable.java @@ -10,7 +10,6 @@ import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.ResolvingFileIO; import org.jetbrains.annotations.NotNull; -import org.testcontainers.shaded.org.apache.commons.lang3.NotImplementedException; import java.io.File; import java.nio.file.Path; @@ -60,7 +59,7 @@ public void refresh() {} @Override public TableScan newScan() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override @@ -141,82 +140,82 @@ public Iterable snapshots() { @Override public List history() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public UpdateSchema updateSchema() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public UpdatePartitionSpec updateSpec() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public UpdateProperties updateProperties() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public ReplaceSortOrder replaceSortOrder() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public UpdateLocation updateLocation() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public AppendFiles newAppend() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public RewriteFiles newRewrite() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public RewriteManifests rewriteManifests() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public OverwriteFiles newOverwrite() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public RowDelta newRowDelta() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public ReplacePartitions newReplacePartitions() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public DeleteFiles newDelete() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public ExpireSnapshots expireSnapshots() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public ManageSnapshots manageSnapshots() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public Transaction newTransaction() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override @@ -229,12 +228,12 @@ public FileIO io() { @Override public EncryptionManager encryption() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public LocationProvider locationProvider() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override @@ -244,6 +243,6 @@ public List statisticsFiles() { @Override public Map refs() { - throw new NotImplementedException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java new file mode 100644 index 00000000000..7a0fd7d8eab --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java @@ -0,0 +1,27 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.junit5; + +import io.deephaven.iceberg.CatalogHelper; +import io.deephaven.iceberg.util.IcebergCatalogAdapter; +import io.deephaven.iceberg.util.IcebergTools; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestInfo; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.nio.file.Path; + + +public abstract class CatalogAdapterBase { + + protected IcebergCatalogAdapter catalogAdapter; + + @BeforeEach + void setUp(TestInfo testInfo, @TempDir Path catalogDir) throws IOException { + catalogAdapter = + IcebergTools + .createAdapter(CatalogHelper.createJdbcCatalog(testInfo.getDisplayName(), catalogDir, true)); + } +} diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/README.md b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/README.md new file mode 100644 index 00000000000..047fa91dd8a --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/README.md @@ -0,0 +1,46 @@ +# pyiceberg-1 + +This was the setup for generating the pyiceberg-1 data. + +```shell +pip install --only-binary=":all:" "pyiceberg[sql-sqlite, pyarrow]==0.7.1" +``` + +```shell +$ pip list +Package Version +------------------ ----------- +annotated-types 0.7.0 +certifi 2024.8.30 +charset-normalizer 3.3.2 +click 8.1.7 +fsspec 2024.9.0 +greenlet 3.1.1 +idna 3.10 +markdown-it-py 3.0.0 +mdurl 0.1.2 +mmh3 4.1.0 +numpy 1.26.4 +pip 23.3.2 +pyarrow 17.0.0 +pydantic 2.9.2 +pydantic_core 2.23.4 +Pygments 2.18.0 +pyiceberg 0.7.1 +pyparsing 3.1.4 +python-dateutil 2.9.0.post0 +requests 2.32.3 +rich 13.8.1 +setuptools 69.0.3 +six 1.16.0 +sortedcontainers 2.4.0 +SQLAlchemy 2.0.35 +strictyaml 1.7.3 +tenacity 8.5.0 +typing_extensions 4.12.2 +urllib3 2.2.3 +``` + +```shell +python generate-pyiceberg-1.py +``` \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-catalog.db b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-catalog.db new file mode 100644 index 0000000000000000000000000000000000000000..9a242fc758bc4b9509abbdaa6dcbea2e4adeb978 GIT binary patch literal 20480 zcmeI%!B5jr90%}M7z`F*Ud&?R;TaFoWZk-T8*4o1T$*T{paV4?Nb}mhNu{i#?Is){ zT>P8-Lr92!h)3U6w?TB3a6`ULyY;od=DmJ??cx2p{jTLkoOb-+fJIaz_lP8s=adpc z(qdZ>+jyB1JE?I){7aYiXYHm*VeoA=`;(+rF34Ix`+enN?cuCo9ESh|AOHafKmY;| zfB*y_a5I6^XY=WtVMu3E#4L}yHn({2+3d0d9`=~c&0gU5co4ZfOgfipwvEP)L3bK2 zS_Vyur0bbwn?=m?<6h{8M!Wg4(OxfCt1{i(+M%28Tdh3JEKkNTJLLD^j?I03e2u69 z^ZI-eS5j5=YL9Q*uiiA;pXh7j6J5VN$UL14CQmPSDre-6sm0vBArZIh@T2h16B9*~ z^&@}OHj^mKQqtLl)25Wp$+C3TxRRPtW0H)xHTPHk_}uluNAc6tPoVzD%rtOz%mI&> zBl2u|zL0eN?j&^NK1^^g-~-q1hvqe66H+rB&FG2E)9C?`o~Lt*xh?r$=^0%TP7@Nh z=fyjgC#U^FbM66gK009U<00Izz00bZa0SG_<0{^?f$v|3?A3ZqPJ=xtp^qO%M zXH%4loISBQh&xe-N=^fdB*`009U<00Izz00bZa0SG|g779FG5G9S}681^@s6 literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.parquet b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.parquet new file mode 100644 index 00000000000..ada32f6e0b2 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:0a2932c3351e5164283e7d76ed8439b08c420a1d6b039081121ed58eeefe2a4e +size 1660 diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.parquet b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.parquet new file mode 100644 index 00000000000..d142358a461 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:9cee42864f62bfb767514cac315aba2c6c7bc3cec7a37edd696ca01e690375ba +size 1616 diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00000-cf59d01d-6384-4c13-8d84-74a9ffd2d158.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00000-cf59d01d-6384-4c13-8d84-74a9ffd2d158.metadata.json new file mode 100644 index 00000000000..05c0007ce29 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00000-cf59d01d-6384-4c13-8d84-74a9ffd2d158.metadata.json @@ -0,0 +1 @@ +{"location":"dh-test-warehouse/dh-default.db/cities","table-uuid":"d514855a-3946-461e-a714-b606b4a08661","last-updated-ms":1727384248249,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]}],"current-schema-id":0,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"snapshots":[],"snapshot-log":[],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{},"format-version":2,"last-sequence-number":0} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00001-6c47a69a-0a0e-4e20-a733-00ba6e10e517.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00001-6c47a69a-0a0e-4e20-a733-00ba6e10e517.metadata.json new file mode 100644 index 00000000000..df06a8b3d42 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00001-6c47a69a-0a0e-4e20-a733-00ba6e10e517.metadata.json @@ -0,0 +1 @@ +{"location":"dh-test-warehouse/dh-default.db/cities","table-uuid":"d514855a-3946-461e-a714-b606b4a08661","last-updated-ms":1727384248322,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]}],"current-schema-id":0,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"current-snapshot-id":8833084047573120270,"snapshots":[{"snapshot-id":8833084047573120270,"sequence-number":1,"timestamp-ms":1727384248322,"manifest-list":"dh-test-warehouse/dh-default.db/cities/metadata/snap-8833084047573120270-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro","summary":{"operation":"append","added-files-size":"1660","added-data-files":"1","added-records":"4","total-data-files":"1","total-delete-files":"0","total-records":"4","total-files-size":"1660","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":0}],"snapshot-log":[{"snapshot-id":8833084047573120270,"timestamp-ms":1727384248322}],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{"main":{"snapshot-id":8833084047573120270,"type":"branch"}},"format-version":2,"last-sequence-number":1} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00002-4c26a33a-9adc-43dc-8e18-a7034a6a837a.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00002-4c26a33a-9adc-43dc-8e18-a7034a6a837a.metadata.json new file mode 100644 index 00000000000..09c47901c0c --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00002-4c26a33a-9adc-43dc-8e18-a7034a6a837a.metadata.json @@ -0,0 +1 @@ +{"location":"dh-test-warehouse/dh-default.db/cities","table-uuid":"d514855a-3946-461e-a714-b606b4a08661","last-updated-ms":1727384248331,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]},{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"longitude","type":"double","required":false}],"schema-id":1,"identifier-field-ids":[]}],"current-schema-id":1,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"current-snapshot-id":8833084047573120270,"snapshots":[{"snapshot-id":8833084047573120270,"sequence-number":1,"timestamp-ms":1727384248322,"manifest-list":"dh-test-warehouse/dh-default.db/cities/metadata/snap-8833084047573120270-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro","summary":{"operation":"append","added-files-size":"1660","added-data-files":"1","added-records":"4","total-data-files":"1","total-delete-files":"0","total-records":"4","total-files-size":"1660","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":0}],"snapshot-log":[{"snapshot-id":8833084047573120270,"timestamp-ms":1727384248322}],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{"main":{"snapshot-id":8833084047573120270,"type":"branch"}},"format-version":2,"last-sequence-number":1} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00003-669f91cb-9990-4db5-9f8b-2bd8054c65ff.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00003-669f91cb-9990-4db5-9f8b-2bd8054c65ff.metadata.json new file mode 100644 index 00000000000..6a2bea6d70e --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00003-669f91cb-9990-4db5-9f8b-2bd8054c65ff.metadata.json @@ -0,0 +1 @@ +{"location":"dh-test-warehouse/dh-default.db/cities","table-uuid":"d514855a-3946-461e-a714-b606b4a08661","last-updated-ms":1727384248366,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]},{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"longitude","type":"double","required":false}],"schema-id":1,"identifier-field-ids":[]}],"current-schema-id":1,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"current-snapshot-id":7895378999715099054,"snapshots":[{"snapshot-id":8833084047573120270,"sequence-number":1,"timestamp-ms":1727384248322,"manifest-list":"dh-test-warehouse/dh-default.db/cities/metadata/snap-8833084047573120270-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro","summary":{"operation":"append","added-files-size":"1660","added-data-files":"1","added-records":"4","total-data-files":"1","total-delete-files":"0","total-records":"4","total-files-size":"1660","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":0},{"snapshot-id":7895378999715099054,"parent-snapshot-id":8833084047573120270,"sequence-number":2,"timestamp-ms":1727384248366,"manifest-list":"dh-test-warehouse/dh-default.db/cities/metadata/snap-7895378999715099054-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.avro","summary":{"operation":"append","added-files-size":"1616","added-data-files":"1","added-records":"2","total-data-files":"2","total-delete-files":"0","total-records":"6","total-files-size":"3276","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":1}],"snapshot-log":[{"snapshot-id":8833084047573120270,"timestamp-ms":1727384248322},{"snapshot-id":7895378999715099054,"timestamp-ms":1727384248366}],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{"main":{"snapshot-id":7895378999715099054,"type":"branch"}},"format-version":2,"last-sequence-number":2} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/78acf870-0d6f-4d74-9cb7-d3be07a263eb-m0.avro b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/78acf870-0d6f-4d74-9cb7-d3be07a263eb-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..7ab66c8946d1921f1ae686747641867662533cfb GIT binary patch literal 4405 zcmb`LO>7%Q6o4H!3RZ=pfDi)I!)VWTvOj73(@P69sHlX}q^HPTjd#cKlHHwUb~cfz zd_tnP9FUOslX~Nb#G$AN303069dYEsl?zCfIP%`CckQ(kleo4bYdt&j-kb0J%$vO5 z`EY6V#n|bwfc|mmfhPMA(^^_AMdC=Us&zf)yRp{Vf1r7;)-tMENCTYVc(M=gNAtb# z0DcShQQ`>(Q@hlU*`S)PGxK#mm7YvoHd^cQq~qi7GuB$8wLCn>vUXKV_ftcQ^(q+7 zLg~RyL``i6#&JjO!Ig*#u*TzIEsmJ8yx-P}BQUe9U+HoYP+2=-B8I7DbJgLY1d$b& zO8P225_~;XcycWRNMJJ#7p_(%IYI&qQ+NdUQklg0TV*NHVRFioXkI9_L6wZH2x%1e zxU^wYnb35ZF*_%|PkGmW4P32Fn>o#{FQFuu1-MvMQJ=r7MyF^79E5?PApBgo#VJtE* zql+5m+D)DwQ@YOW0SwA44TPbaxSfpfoh%;-I-1BN@>wlm^5! z$|lGj8oB_KO+abtIiQQ9Y+f9tsV^F(%AG0%rsF3r)D3lb5E{BF;;~}S0&I*G zy5|RM8VJ~&AFyf60oYs|VB|#!tMZfXaRxbctJ9+E)uP|g>hQXm0Y+Gm|L^ao_gw- z^~JGOtcYS&gqTFp*$@|cFqv3Qt>>tw=KoaFQz2RlET@XqX%OL1wF4$8HadBb3>$iD ztZ~aLrk2{;h9EubXuKIZq93WY5yX>BJn8tnQ*>b*`JS|Sw;MA#l?c{YFPEzybr5-T z!F(3LH7GM0JFBeUNL#C!=lCv~ASqn^AbnFi3IPun2nsX$aN(J@@SLeSVd2)rEZoMs zFLW3&NwD*TKEq{_iNFgP5&W2B7GnRFKf9n)!ZiV!yLf#9Iv2oPxR5;4m$=Q-PDynE zndR|YZ$qyd2^QlG7mn!|RKQgPRNzGgI%O|brlc)Z*6|5wx|MITfq;fym&MWsF`};x zUQw4?hr7(F;CWU0?3X7Gzklb;CzoIN?$M9e-ukEf`=xUEo8KP&_3+onpFdkFl}b;S z++Gd(U|Bn+g7r8c+klZ_sII%64fv0N$KnkP_zfLDHN94EP^a6d!=t;^t!=vX&05px z)N8KQVS1gKTNdlA5ARj0cW&JM;NH!>%D=CdSIS?Mmru$oC*=}6SC%U)6?`ZymsUz8 zHTLq2AO>f-G+18ywEV*-Fa7!E^_8{O@lW@^`uhG;oczWv4av5kq2t94UtW9m^WTpT ZPTqxC`t5HoZ?o5~gQ#)^_BOfu{{SB(TKWJ0 literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/a1acdfcc-2afa-4020-af3c-4992ee16e9e8-m0.avro b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/a1acdfcc-2afa-4020-af3c-4992ee16e9e8-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..b00ba51684e3fab5934246d7237730ccefbf1570 GIT binary patch literal 4408 zcmb`LPjBNy6u^^gRQ0f`dO#~7Rxt7jq@<1Q(k4YnfOadb#FlQiD}+{&HL=IdP{(7& z<5r={1##xUC)f|bi33Me2?;J7*b5w%0|yWi>(L4_St~`b>q}f!-yH%MyzDwDx+ceoCRKNY(G9VxMyse4I`ujoZ)gcg!hyA zJUoQol6{hJ$zZBagP4sP={hT47f{7j;<3qEPb9qne_ycHo~#w&;bj+h8%DaT=GboT zu~2c?k8J8q!8q=#JGvcF2{v&NHsgr7^~YVKJOVRot=qnk1FD+GOvW&^X05m)R3LKM zql(_5$5O1N3cq+O!$@FDu8^M5AUQ|^BvXV0C{vZh`CDx%X~E=_DlvGdj8TIWR)jQ) z`$9ResZ40P%$zKHY=lV=;QIi88C`hrm`{^H8QTP3l|_QbSm-h*Oa?tBr^MRV73=eW zIq_WC8`qVM#2r*HWvn$3cv%;Woj4`PvY;k_&$2|AjnBnns~-8LmX8Z(ao64htOI?Y^y znOjDZrk>-91d<4ZXGsq}G+TwRPLpJ!eOIRe5k5&J#Du5Hka1asrXKT8nZrY;H&kpo zm(5l&j&wFo6dedf1A;OL%0M-#j8b~KB1i_IgEMBOw?W#tR^&-PiF_(0g&GZIM6m(X zI;bOAVVTsW%n3)Px#b+2W;;Ouul>@pu%FfkMTCGK+)Vb4uz;uJegSw#)4?;ta?9$ybTRc{rd4SDA zp?h(_mWhC^>jSpTYXG(u2N-z~%?n}}D^$sxkcM+%+FOf^R=W)_?auXK+U=6{mr8uQ zvq+fOxnMnjxJ;?$RXeQ^WxRnnVbbXVcW42DWU(MwZ!Qw8wTW@Ml`FY&A+R>*qdoW3 zv9=b+R z8!&VjF-frVgg(P#ipc>F8Ij_IWENuomOp!-Q^7R>n!9*y3c8TMTzZf^vzNGy^G->9 z0h#CVdv~B$jUk5y#%XnT@{<-;v_3hrz{J~dW_m2MX_G&-fsMUV`{)eYO{_@$E&#IM5<*e%U zo6rZV<_VRoF96v(jEqBd&Fihh{|z{c*D>JNTli_Vn$)DO=eusxqCRbIv@96%ZMV76 z=~#@Jo2%HF&C4<6k;s6TnRwp9DAT7O<$dS0!-bG=?)s^deYURkPCYWR5T zJ`Y1iBN6bpzWUAT?@vDO-g#$f_0LzHzWw*!hi5ovb(fuxkAys`uZG{fzWnNF2PVAs T?|ao}r~79hr?w0mo8J0=xfFKBpRgh(N=520ucjw%h&3S}tZ0rTC z0%^6e6Ra&1!A1})4A@Hw8+&K(ZZDH;vWd;Y&AjjX-uvdgclW}_OP5zgMuuX*C5qWi z*YkU}`uM(U_OS2co^xYJa>yjo!#vNnvFBjN?YXvTVavtW1w1Vvqkys^4Iyt^*CL*$ zM51lTMa;9dwMw>gzAoqhQu6NVjv>cGFm82>94OD3vDPu77!oGb+2)Rsk#u^Sk}Qru z$Y4n1U=gH}Rbf4h6{_M9GLN&ZGK6tj65r*NNG08Qc>hjk6w5(p<3VSt55{DzbAm@h9!CDN5 zr?P5G<{Yf0P_-iS9`=mM}O zxb$&%f9GKD*P9J;F?zkcf8oOt8zAMDL?iXzGT?=Pt~!~&#yShTRP9p`-74KKF$rZ6 zrN|-?K@PSsB9Tpz{KrbE&O>~AI8Zts5IJ9zQs{j0yF?8;R3xB>qF>!@8z literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/snap-8833084047573120270-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/snap-8833084047573120270-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro new file mode 100644 index 0000000000000000000000000000000000000000..e5ab26b754da32f87a8b6dcfdbe7270264ff95b2 GIT binary patch literal 1776 zcmb7FyKdA#6y4n_1!WW(ge(hDpg23;N9<5g0TL1d2}BhkYdmwEb+Bj5%s9&`+7>m|H;UdS%I{+b5&N{sfd?5Ii zBqPZ2`^$%}niSyPbX)<13+iqwZ;lsJEBsf z6UhQdq%*5Zsj}=@_>`$tIN?lbM^A$Mpb1pD9nIaYfrl2jo^hV)To?U1i($Kw0aFSz zs)pdSn)?^PCus&UBD}1Zj|LXL<{7+5rP9rgniy;kY77)IP`ZRB0^5Qb!Q27C$7dAa z+fA>xU|B%}L$ifd1q}qYhxHVeb1P4d53*)&*JWjv>4@Wn#X{hT!DtV2a|{;C80x1~ zL2fqqIJ$Yg&Gh4xNj-)$mH7>*7EDFx1xEf zKT=3=nJJyFQD0f=jE_M?u^!FZI{Vhqv3Ms6qs z`x#8492BNl;hI&J(f=vFyw7Hn-0Ax4$P-S1tQk{Vch>)Yx^?vZOZ;vB<)0gGUw3}B zKep+}({3qmtbZmWUP|cdmqj2`wyWs naBsU0v6rJQBmU!dtJUee`+D^A)#ndyez!ZV)|!54)iU`9KE*TK literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/generate-pyiceberg-1.py b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/generate-pyiceberg-1.py new file mode 100644 index 00000000000..0eacbb7a10f --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/generate-pyiceberg-1.py @@ -0,0 +1,53 @@ +from pyiceberg.schema import Schema +from pyiceberg.types import NestedField, StringType, DoubleType +from pyiceberg.catalog.sql import SqlCatalog + +import pyarrow as pa + +catalog = SqlCatalog( + "SqlCatalogName", + **{ + "uri": f"sqlite:///dh-test-catalog.db", + "warehouse": f"dh-test-warehouse", + }, +) + +original_schema = Schema( + NestedField(1, "city", StringType(), required=False), + NestedField(2, "latitude", DoubleType(), required=False), + NestedField(3, "lon", DoubleType(), required=False), +) + +# Using specific names to make clear these aren't a standard / convention +catalog.create_namespace("dh-default") + +table = catalog.create_table( + "dh-default.cities", + schema=original_schema, +) + +# Add some data +table.append( + pa.Table.from_pylist( + [ + {"city": "Amsterdam", "latitude": 52.371807, "lon": 4.896029}, + {"city": "San Francisco", "latitude": 37.773972, "lon": -122.431297}, + {"city": "Drachten", "latitude": 53.11254, "lon": 6.0989}, + {"city": "Paris", "latitude": 48.864716, "lon": 2.349014}, + ], + ) +) + +# Oops, we should be consistent with naming +with table.update_schema() as update: + update.rename_column("lon", "longitude") + +# Add some data. Note, to simplify ingestion, we are matching the latest column names +table.append( + pa.Table.from_pylist( + [ + {"city": "Minneapolis", "latitude": 44.977479, "longitude": -93.264358}, + {"city": "New York", "latitude": 40.730610, "longitude": -73.935242}, + ], + ) +) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index b4158ed3525..f342d603109 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -79,6 +79,7 @@ univocity = "2.6.0" vertispan-nio = "1.0-alpha-2" vertispan-flatbuffers-gwt = "1.12.0-1" vertispan-ts-defs = "1.0.0-RC4" +xerial = "3.46.1.3" # test versions assertj = "3.19.0" @@ -293,6 +294,8 @@ vertispan-flatbuffers-gwt = { module = "com.vertispan.flatbuffers:flatbuffers-gw vertispan-ts-defs-annotations = { module = "com.vertispan.tsdefs:jsinterop-ts-defs-annotations", version.ref = "vertispan-ts-defs" } vertispan-ts-defs-doclet = { module = "com.vertispan.tsdefs:jsinterop-ts-defs-doclet", version.ref = "vertispan-ts-defs" } +xerial-sqlite-jdbc = { module = "org.xerial:sqlite-jdbc", version.ref = "xerial" } + # test libraries assertj = { module = "org.assertj:assertj-core", version.ref = "assertj" } From d755759ea88e34727e6de2c41942d9e88fc3d3b6 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Fri, 27 Sep 2024 10:05:55 -0700 Subject: [PATCH 2/4] Refactor conventions --- extensions/iceberg/TESTING.md | 125 ++++++++++++++++++ .../io/deephaven/iceberg/CatalogHelper.java | 59 --------- .../io/deephaven/iceberg/PyIceberg1Test.java | 20 ++- .../iceberg/junit5/CatalogAdapterBase.java | 11 +- .../deephaven/iceberg/sqlite/DbResource.java | 20 +++ .../iceberg/sqlite/SqliteHelper.java | 73 ++++++++++ .../deephaven/iceberg/pyiceberg-1/README.md | 46 ------- ...-6384-4c13-8d84-74a9ffd2d158.metadata.json | 1 - ...-0a0e-4e20-a733-00ba6e10e517.metadata.json | 1 - ...-9adc-43dc-8e18-a7034a6a837a.metadata.json | 1 - ...-9990-4db5-9f8b-2bd8054c65ff.metadata.json | 1 - ...6b5d5-cc59-4f48-bcc5-3a4bf37e5139.parquet} | 0 ...68bfb-107c-4122-8d64-9a078f2f45a9.parquet} | 0 ...-5bde-4642-bac7-cc7573558c7e.metadata.json | 1 + ...-934f-483e-b2b0-4c08baebc5b2.metadata.json | 1 + ...-5f0d-478c-aba2-a05842613e29.metadata.json | 1 + ...-2052-4569-b9f1-1b45ab0c2be0.metadata.json | 1 + ...6b5d5-cc59-4f48-bcc5-3a4bf37e5139-m0.avro} | Bin 4408 -> 4411 bytes ...68bfb-107c-4122-8d64-9a078f2f45a9-m0.avro} | Bin 4405 -> 4407 bytes ...cc668bfb-107c-4122-8d64-9a078f2f45a9.avro} | Bin 1776 -> 1778 bytes ...6976b5d5-cc59-4f48-bcc5-3a4bf37e5139.avro} | Bin 1910 -> 1915 bytes .../db_resource/dh-iceberg-test.db} | Bin 20480 -> 20480 bytes .../db_resource}/generate-pyiceberg-1.py | 6 +- 23 files changed, 240 insertions(+), 128 deletions(-) create mode 100644 extensions/iceberg/TESTING.md delete mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogHelper.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/sqlite/DbResource.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/sqlite/SqliteHelper.java delete mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/README.md delete mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00000-cf59d01d-6384-4c13-8d84-74a9ffd2d158.metadata.json delete mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00001-6c47a69a-0a0e-4e20-a733-00ba6e10e517.metadata.json delete mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00002-4c26a33a-9adc-43dc-8e18-a7034a6a837a.metadata.json delete mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00003-669f91cb-9990-4db5-9f8b-2bd8054c65ff.metadata.json rename extensions/iceberg/src/test/resources/io/deephaven/iceberg/{pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.parquet => sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/data/00000-0-6976b5d5-cc59-4f48-bcc5-3a4bf37e5139.parquet} (100%) rename extensions/iceberg/src/test/resources/io/deephaven/iceberg/{pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.parquet => sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/data/00000-0-cc668bfb-107c-4122-8d64-9a078f2f45a9.parquet} (100%) create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00000-4aec1d46-5bde-4642-bac7-cc7573558c7e.metadata.json create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00001-3128d2b2-934f-483e-b2b0-4c08baebc5b2.metadata.json create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00002-262f9917-5f0d-478c-aba2-a05842613e29.metadata.json create mode 100644 extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00003-979e64ae-2052-4569-b9f1-1b45ab0c2be0.metadata.json rename extensions/iceberg/src/test/resources/io/deephaven/iceberg/{pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/a1acdfcc-2afa-4020-af3c-4992ee16e9e8-m0.avro => sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/6976b5d5-cc59-4f48-bcc5-3a4bf37e5139-m0.avro} (92%) rename extensions/iceberg/src/test/resources/io/deephaven/iceberg/{pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/78acf870-0d6f-4d74-9cb7-d3be07a263eb-m0.avro => sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/cc668bfb-107c-4122-8d64-9a078f2f45a9-m0.avro} (92%) rename extensions/iceberg/src/test/resources/io/deephaven/iceberg/{pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/snap-8833084047573120270-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro => sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/snap-1743193108934390753-0-cc668bfb-107c-4122-8d64-9a078f2f45a9.avro} (86%) rename extensions/iceberg/src/test/resources/io/deephaven/iceberg/{pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/snap-7895378999715099054-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.avro => sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/snap-4630159959461529013-0-6976b5d5-cc59-4f48-bcc5-3a4bf37e5139.avro} (76%) rename extensions/iceberg/src/test/resources/io/deephaven/iceberg/{pyiceberg-1/dh-test-catalog.db => sqlite/db_resource/dh-iceberg-test.db} (96%) rename extensions/iceberg/src/test/resources/io/deephaven/iceberg/{pyiceberg-1 => sqlite/db_resource}/generate-pyiceberg-1.py (92%) diff --git a/extensions/iceberg/TESTING.md b/extensions/iceberg/TESTING.md new file mode 100644 index 00000000000..7019f41e727 --- /dev/null +++ b/extensions/iceberg/TESTING.md @@ -0,0 +1,125 @@ +# sqlite catalogs + +The sqlite JDBC catalog is able to support multiple catalogs through a single database file. +As such, the following convention has been established for testing purposes: + +* database file: `/dh-iceberg-test.db` +* warehouse directory: `/catalogs//` + +Both writers and readers of this catalog need to be setup to support relative metadata locations to ensure portability. + +A root directory for extension-iceberg testing has been established at `extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource`. + +## Usage + +### Java + +```java +import org.apache.iceberg.catalog.Catalog; +import io.deephaven.iceberg.sqlite.DbResource; + +Catalog catalog = DbResource.openCatalog(""); +``` + +### pyiceberg + +To setup in [pyiceberg](https://py.iceberg.apache.org/): + +```python +from pyiceberg.catalog.sql import SqlCatalog + +catalog = SqlCatalog( + "", + **{ + "uri": f"sqlite:///dh-iceberg-test.db", + "warehouse": f"catalogs/", + }, +) +``` + +## Generating data + +Note that any scripts that write data should be run relative to +[db_resource](src/test/resources/io/deephaven/iceberg/sqlite/db_resource) working directory to ensure unit testability. + +### pyiceberg-1 + +Here's an example of what was needed to generate this data: + +```shell +$ cd extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource + +# Note: 3.10 explicitly chosen b/c it doesn't seem like pyiceberg produces 3.12 wheels yet +$ python3.10 -m venv /tmp/iceberg + +$ source /tmp/iceberg/bin/activate + +$ pip install --only-binary=":all:" "pyiceberg[sql-sqlite, pyarrow]" + +$ pip freeze +annotated-types==0.7.0 +certifi==2024.8.30 +charset-normalizer==3.3.2 +click==8.1.7 +fsspec==2024.9.0 +greenlet==3.1.1 +idna==3.10 +markdown-it-py==3.0.0 +mdurl==0.1.2 +mmh3==4.1.0 +numpy==1.26.4 +pyarrow==17.0.0 +pydantic==2.9.2 +pydantic_core==2.23.4 +Pygments==2.18.0 +pyiceberg==0.7.1 +pyparsing==3.1.4 +python-dateutil==2.9.0.post0 +requests==2.32.3 +rich==13.8.1 +six==1.16.0 +sortedcontainers==2.4.0 +SQLAlchemy==2.0.35 +strictyaml==1.7.3 +tenacity==8.5.0 +typing_extensions==4.12.2 +urllib3==2.2.3 + +$ python generate-pyiceberg-1.py + +$ sqlite3 dh-iceberg-test.db +SQLite version 3.45.1 2024-01-30 16:01:20 +Enter ".help" for usage hints. +sqlite> .dump +PRAGMA foreign_keys=OFF; +BEGIN TRANSACTION; +CREATE TABLE iceberg_tables ( + catalog_name VARCHAR(255) NOT NULL, + table_namespace VARCHAR(255) NOT NULL, + table_name VARCHAR(255) NOT NULL, + metadata_location VARCHAR(1000), + previous_metadata_location VARCHAR(1000), + PRIMARY KEY (catalog_name, table_namespace, table_name) +); +INSERT INTO iceberg_tables VALUES('pyiceberg-1','dh-default','cities','catalogs/pyiceberg-1/dh-default.db/cities/metadata/00003-68091f71-d3c5-42bb-8161-e2e187dece14.metadata.json','catalogs/pyiceberg-1/dh-default.db/cities/metadata/00002-106b37f8-8818-439d-87c5-3cae608d1972.metadata.json'); +CREATE TABLE iceberg_namespace_properties ( + catalog_name VARCHAR(255) NOT NULL, + namespace VARCHAR(255) NOT NULL, + property_key VARCHAR(255) NOT NULL, + property_value VARCHAR(1000) NOT NULL, + PRIMARY KEY (catalog_name, namespace, property_key) +); +INSERT INTO iceberg_namespace_properties VALUES('pyiceberg-1','dh-default','exists','true'); +COMMIT; +sqlite> +``` + +### sqlite + +If we add a lot of catalogs to the database, we may want to look into vacuuming the database to keep the file size small. + +`sqlite3 dh-iceberg-test.db 'VACUUM;'` + +Ideally, the sqlite database can be a small collection of catalogs that were created via external tooling to very that +we can integrate with them successfully. + diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogHelper.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogHelper.java deleted file mode 100644 index 88cc29fb62a..00000000000 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogHelper.java +++ /dev/null @@ -1,59 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.iceberg; - -import io.deephaven.iceberg.relative.RelativeFileIO; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.jdbc.JdbcCatalog; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.HashMap; -import java.util.Map; -import java.util.stream.Stream; - -public class CatalogHelper { - - // Using specific names to make clear these aren't a standard / convention - public static final String CATALOG_NAME = "dh-test-catalog.db"; - public static final String WAREHOUSE_NAME = "dh-test-warehouse"; - - public static Catalog createJdbcCatalog(String name, Path path, boolean isNew) throws IOException { - if (!Files.isDirectory(path)) { - throw new IllegalArgumentException("Must provide directory that exists"); - } - if (isNew) { - try (final Stream list = Files.list(path).limit(1)) { - if (list.iterator().hasNext()) { - throw new IllegalArgumentException("Expected directory to be empty"); - } - } - } - final Path catalogFile = path.resolve(CATALOG_NAME); - final Path warehouseDir = path.resolve(WAREHOUSE_NAME); - if (isNew) { - Files.createDirectory(warehouseDir); - } else { - if (!Files.isDirectory(warehouseDir)) { - throw new IllegalStateException("Expected warehouse directory to already exist"); - } - } - final Map properties = new HashMap<>(); - properties.put(CatalogProperties.CATALOG_IMPL, JdbcCatalog.class.getName()); - properties.put(CatalogProperties.URI, String.format("jdbc:sqlite:%s", catalogFile)); - properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseDir.toString()); - if (!isNew) { - // When we are referring to a catalog that already exists in our unit testing filesystem, we need to make - // hook in relative file support. See https://github.com/apache/iceberg/issues/1617 - properties.put(CatalogProperties.FILE_IO_IMPL, RelativeFileIO.class.getName()); - properties.put(RelativeFileIO.BASE_PATH, path.toString()); - } - final Configuration hadoopConf = new Configuration(); - return CatalogUtil.buildIcebergCatalog(name, properties, hadoopConf); - } -} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java index 3b98858d84e..4f59adae18c 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.testutil.TstUtils; import io.deephaven.engine.util.TableTools; +import io.deephaven.iceberg.sqlite.DbResource; import io.deephaven.iceberg.util.IcebergCatalogAdapter; import io.deephaven.iceberg.util.IcebergTools; import org.apache.iceberg.Snapshot; @@ -16,20 +17,22 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.io.IOException; import java.net.URISyntaxException; -import java.nio.file.Path; import java.util.List; import static org.assertj.core.api.Assertions.assertThat; +/** + * This test shows that we can integrate with data written by pyiceberg. + * See TESTING.md and generate-pyiceberg-1.py for more details. + */ public class PyIceberg1Test { private static final Namespace NAMESPACE = Namespace.of("dh-default"); private static final TableIdentifier CITIES_ID = TableIdentifier.of(NAMESPACE, "cities"); // This will need to be updated if the data is regenerated - private static final long SNAPSHOT_1_ID = 8833084047573120270L; - private static final long SNAPSHOT_2_ID = 7895378999715099054L; + private static final long SNAPSHOT_1_ID = 1743193108934390753L; + private static final long SNAPSHOT_2_ID = 4630159959461529013L; private static final TableDefinition CITIES_1_TD = TableDefinition.of( ColumnDefinition.ofString("city"), @@ -44,13 +47,8 @@ public class PyIceberg1Test { private IcebergCatalogAdapter catalogAdapter; @BeforeEach - void setUp() throws IOException, URISyntaxException { - // Note: for some reason, the name must be the same name that was used to create the SqlCatalog. Potentially so - // a single DB file can support multiple catalogs? - catalogAdapter = IcebergTools.createAdapter(CatalogHelper.createJdbcCatalog( - "SqlCatalogName", - Path.of(PyIceberg1Test.class.getResource("pyiceberg-1").toURI()), - false)); + void setUp() throws URISyntaxException { + catalogAdapter = IcebergTools.createAdapter(DbResource.openCatalog("pyiceberg-1")); } @Test diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java index 7a0fd7d8eab..c045a093e4a 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java @@ -3,9 +3,10 @@ // package io.deephaven.iceberg.junit5; -import io.deephaven.iceberg.CatalogHelper; +import io.deephaven.iceberg.sqlite.SqliteHelper; import io.deephaven.iceberg.util.IcebergCatalogAdapter; import io.deephaven.iceberg.util.IcebergTools; +import org.apache.iceberg.catalog.Catalog; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.io.TempDir; @@ -19,9 +20,9 @@ public abstract class CatalogAdapterBase { protected IcebergCatalogAdapter catalogAdapter; @BeforeEach - void setUp(TestInfo testInfo, @TempDir Path catalogDir) throws IOException { - catalogAdapter = - IcebergTools - .createAdapter(CatalogHelper.createJdbcCatalog(testInfo.getDisplayName(), catalogDir, true)); + void setUp(TestInfo testInfo, @TempDir Path rootDir) throws IOException { + SqliteHelper.createJdbcDatabase(rootDir); + final Catalog catalog = SqliteHelper.createJdbcCatalog(rootDir, testInfo.getDisplayName(), false); + catalogAdapter = IcebergTools.createAdapter(catalog); } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/sqlite/DbResource.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/sqlite/DbResource.java new file mode 100644 index 00000000000..10c2de49b63 --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/sqlite/DbResource.java @@ -0,0 +1,20 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.sqlite; + +import org.apache.iceberg.catalog.Catalog; + +import java.net.URISyntaxException; +import java.nio.file.Path; + +public class DbResource { + public static Catalog openCatalog(String catalogName) throws URISyntaxException { + return SqliteHelper.openJdbcCatalog( + // Note: we are using a resource path that is not shared with our build classes to ensure we are + // resolving against the actual resource directory + Path.of(DbResource.class.getResource("db_resource").toURI()), + catalogName, + true); + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/sqlite/SqliteHelper.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/sqlite/SqliteHelper.java new file mode 100644 index 00000000000..51759f57522 --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/sqlite/SqliteHelper.java @@ -0,0 +1,73 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.sqlite; + +import io.deephaven.iceberg.relative.RelativeFileIO; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.jdbc.JdbcCatalog; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Stream; + +public class SqliteHelper { + + private static final String DB_FILE = "dh-iceberg-test.db"; + private static final String CATALOGS_DIR = "catalogs"; + + public static void createJdbcDatabase(Path rootDir) throws IOException { + if (!Files.isDirectory(rootDir)) { + throw new IllegalArgumentException("Must provide rootDir that exists: " + rootDir); + } + try (final Stream list = Files.list(rootDir).limit(1)) { + if (list.iterator().hasNext()) { + throw new IllegalArgumentException("Expected rootDir to be empty: " + rootDir); + } + } + Files.createFile(rootDir.resolve(DB_FILE)); + Files.createDirectory(rootDir.resolve("catalogs")); + } + + public static Catalog createJdbcCatalog(Path rootDir, String catalogName, boolean relativeSupport) + throws IOException { + if (!Files.isDirectory(rootDir)) { + throw new IllegalArgumentException("Must provide rootDir that exists: " + rootDir); + } + if (!Files.isRegularFile(rootDir.resolve(DB_FILE))) { + throw new IllegalArgumentException("Must create jdbc database first: " + rootDir); + } + if (!Files.isDirectory(rootDir.resolve(CATALOGS_DIR))) { + throw new IllegalArgumentException("Must create jdbc database first: " + rootDir); + } + Files.createDirectory(rootDir.resolve(CATALOGS_DIR).resolve(catalogName)); + return openJdbcCatalog(rootDir, catalogName, relativeSupport); + } + + public static Catalog openJdbcCatalog(Path rootDir, String catalogName, boolean relativeSupport) { + final Path warehouseDir = rootDir.resolve(CATALOGS_DIR).resolve(catalogName); + if (!Files.isDirectory(warehouseDir)) { + throw new IllegalArgumentException("Expected warehouse to already exist: " + warehouseDir); + } + final Map properties = new HashMap<>(); + properties.put(CatalogProperties.CATALOG_IMPL, JdbcCatalog.class.getName()); + properties.put(CatalogProperties.URI, String.format("jdbc:sqlite:%s", rootDir.resolve(DB_FILE))); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseDir.toString()); + if (relativeSupport) { + // When we are referring to a catalog that already exists in our unit testing filesystem, we need to make + // hook in relative file support. See https://github.com/apache/iceberg/issues/1617 + properties.put(CatalogProperties.FILE_IO_IMPL, RelativeFileIO.class.getName()); + properties.put(RelativeFileIO.BASE_PATH, rootDir.toString()); + } + final Configuration hadoopConf = new Configuration(); + // Note: the catalogName is very important here, the JDBC catalog uses it for lookups. In this way, a single + // dbFile can be used for multiple catalogs. + return CatalogUtil.buildIcebergCatalog(catalogName, properties, hadoopConf); + } +} diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/README.md b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/README.md deleted file mode 100644 index 047fa91dd8a..00000000000 --- a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/README.md +++ /dev/null @@ -1,46 +0,0 @@ -# pyiceberg-1 - -This was the setup for generating the pyiceberg-1 data. - -```shell -pip install --only-binary=":all:" "pyiceberg[sql-sqlite, pyarrow]==0.7.1" -``` - -```shell -$ pip list -Package Version ------------------- ----------- -annotated-types 0.7.0 -certifi 2024.8.30 -charset-normalizer 3.3.2 -click 8.1.7 -fsspec 2024.9.0 -greenlet 3.1.1 -idna 3.10 -markdown-it-py 3.0.0 -mdurl 0.1.2 -mmh3 4.1.0 -numpy 1.26.4 -pip 23.3.2 -pyarrow 17.0.0 -pydantic 2.9.2 -pydantic_core 2.23.4 -Pygments 2.18.0 -pyiceberg 0.7.1 -pyparsing 3.1.4 -python-dateutil 2.9.0.post0 -requests 2.32.3 -rich 13.8.1 -setuptools 69.0.3 -six 1.16.0 -sortedcontainers 2.4.0 -SQLAlchemy 2.0.35 -strictyaml 1.7.3 -tenacity 8.5.0 -typing_extensions 4.12.2 -urllib3 2.2.3 -``` - -```shell -python generate-pyiceberg-1.py -``` \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00000-cf59d01d-6384-4c13-8d84-74a9ffd2d158.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00000-cf59d01d-6384-4c13-8d84-74a9ffd2d158.metadata.json deleted file mode 100644 index 05c0007ce29..00000000000 --- a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00000-cf59d01d-6384-4c13-8d84-74a9ffd2d158.metadata.json +++ /dev/null @@ -1 +0,0 @@ -{"location":"dh-test-warehouse/dh-default.db/cities","table-uuid":"d514855a-3946-461e-a714-b606b4a08661","last-updated-ms":1727384248249,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]}],"current-schema-id":0,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"snapshots":[],"snapshot-log":[],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{},"format-version":2,"last-sequence-number":0} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00001-6c47a69a-0a0e-4e20-a733-00ba6e10e517.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00001-6c47a69a-0a0e-4e20-a733-00ba6e10e517.metadata.json deleted file mode 100644 index df06a8b3d42..00000000000 --- a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00001-6c47a69a-0a0e-4e20-a733-00ba6e10e517.metadata.json +++ /dev/null @@ -1 +0,0 @@ -{"location":"dh-test-warehouse/dh-default.db/cities","table-uuid":"d514855a-3946-461e-a714-b606b4a08661","last-updated-ms":1727384248322,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]}],"current-schema-id":0,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"current-snapshot-id":8833084047573120270,"snapshots":[{"snapshot-id":8833084047573120270,"sequence-number":1,"timestamp-ms":1727384248322,"manifest-list":"dh-test-warehouse/dh-default.db/cities/metadata/snap-8833084047573120270-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro","summary":{"operation":"append","added-files-size":"1660","added-data-files":"1","added-records":"4","total-data-files":"1","total-delete-files":"0","total-records":"4","total-files-size":"1660","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":0}],"snapshot-log":[{"snapshot-id":8833084047573120270,"timestamp-ms":1727384248322}],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{"main":{"snapshot-id":8833084047573120270,"type":"branch"}},"format-version":2,"last-sequence-number":1} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00002-4c26a33a-9adc-43dc-8e18-a7034a6a837a.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00002-4c26a33a-9adc-43dc-8e18-a7034a6a837a.metadata.json deleted file mode 100644 index 09c47901c0c..00000000000 --- a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00002-4c26a33a-9adc-43dc-8e18-a7034a6a837a.metadata.json +++ /dev/null @@ -1 +0,0 @@ -{"location":"dh-test-warehouse/dh-default.db/cities","table-uuid":"d514855a-3946-461e-a714-b606b4a08661","last-updated-ms":1727384248331,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]},{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"longitude","type":"double","required":false}],"schema-id":1,"identifier-field-ids":[]}],"current-schema-id":1,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"current-snapshot-id":8833084047573120270,"snapshots":[{"snapshot-id":8833084047573120270,"sequence-number":1,"timestamp-ms":1727384248322,"manifest-list":"dh-test-warehouse/dh-default.db/cities/metadata/snap-8833084047573120270-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro","summary":{"operation":"append","added-files-size":"1660","added-data-files":"1","added-records":"4","total-data-files":"1","total-delete-files":"0","total-records":"4","total-files-size":"1660","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":0}],"snapshot-log":[{"snapshot-id":8833084047573120270,"timestamp-ms":1727384248322}],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{"main":{"snapshot-id":8833084047573120270,"type":"branch"}},"format-version":2,"last-sequence-number":1} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00003-669f91cb-9990-4db5-9f8b-2bd8054c65ff.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00003-669f91cb-9990-4db5-9f8b-2bd8054c65ff.metadata.json deleted file mode 100644 index 6a2bea6d70e..00000000000 --- a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/00003-669f91cb-9990-4db5-9f8b-2bd8054c65ff.metadata.json +++ /dev/null @@ -1 +0,0 @@ -{"location":"dh-test-warehouse/dh-default.db/cities","table-uuid":"d514855a-3946-461e-a714-b606b4a08661","last-updated-ms":1727384248366,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]},{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"longitude","type":"double","required":false}],"schema-id":1,"identifier-field-ids":[]}],"current-schema-id":1,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"current-snapshot-id":7895378999715099054,"snapshots":[{"snapshot-id":8833084047573120270,"sequence-number":1,"timestamp-ms":1727384248322,"manifest-list":"dh-test-warehouse/dh-default.db/cities/metadata/snap-8833084047573120270-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.avro","summary":{"operation":"append","added-files-size":"1660","added-data-files":"1","added-records":"4","total-data-files":"1","total-delete-files":"0","total-records":"4","total-files-size":"1660","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":0},{"snapshot-id":7895378999715099054,"parent-snapshot-id":8833084047573120270,"sequence-number":2,"timestamp-ms":1727384248366,"manifest-list":"dh-test-warehouse/dh-default.db/cities/metadata/snap-7895378999715099054-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.avro","summary":{"operation":"append","added-files-size":"1616","added-data-files":"1","added-records":"2","total-data-files":"2","total-delete-files":"0","total-records":"6","total-files-size":"3276","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":1}],"snapshot-log":[{"snapshot-id":8833084047573120270,"timestamp-ms":1727384248322},{"snapshot-id":7895378999715099054,"timestamp-ms":1727384248366}],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{"main":{"snapshot-id":7895378999715099054,"type":"branch"}},"format-version":2,"last-sequence-number":2} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.parquet b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/data/00000-0-6976b5d5-cc59-4f48-bcc5-3a4bf37e5139.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-a1acdfcc-2afa-4020-af3c-4992ee16e9e8.parquet rename to extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/data/00000-0-6976b5d5-cc59-4f48-bcc5-3a4bf37e5139.parquet diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.parquet b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/data/00000-0-cc668bfb-107c-4122-8d64-9a078f2f45a9.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/data/00000-0-78acf870-0d6f-4d74-9cb7-d3be07a263eb.parquet rename to extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/data/00000-0-cc668bfb-107c-4122-8d64-9a078f2f45a9.parquet diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00000-4aec1d46-5bde-4642-bac7-cc7573558c7e.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00000-4aec1d46-5bde-4642-bac7-cc7573558c7e.metadata.json new file mode 100644 index 00000000000..90a76f4767a --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00000-4aec1d46-5bde-4642-bac7-cc7573558c7e.metadata.json @@ -0,0 +1 @@ +{"location":"catalogs/pyiceberg-1/dh-default.db/cities","table-uuid":"5f168e92-7a3c-4277-aa9d-0bf8425e0e05","last-updated-ms":1727458044013,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]}],"current-schema-id":0,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"snapshots":[],"snapshot-log":[],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{},"format-version":2,"last-sequence-number":0} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00001-3128d2b2-934f-483e-b2b0-4c08baebc5b2.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00001-3128d2b2-934f-483e-b2b0-4c08baebc5b2.metadata.json new file mode 100644 index 00000000000..dd42fa006b9 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00001-3128d2b2-934f-483e-b2b0-4c08baebc5b2.metadata.json @@ -0,0 +1 @@ +{"location":"catalogs/pyiceberg-1/dh-default.db/cities","table-uuid":"5f168e92-7a3c-4277-aa9d-0bf8425e0e05","last-updated-ms":1727458044082,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]}],"current-schema-id":0,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"current-snapshot-id":1743193108934390753,"snapshots":[{"snapshot-id":1743193108934390753,"sequence-number":1,"timestamp-ms":1727458044082,"manifest-list":"catalogs/pyiceberg-1/dh-default.db/cities/metadata/snap-1743193108934390753-0-cc668bfb-107c-4122-8d64-9a078f2f45a9.avro","summary":{"operation":"append","added-files-size":"1660","added-data-files":"1","added-records":"4","total-data-files":"1","total-delete-files":"0","total-records":"4","total-files-size":"1660","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":0}],"snapshot-log":[{"snapshot-id":1743193108934390753,"timestamp-ms":1727458044082}],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{"main":{"snapshot-id":1743193108934390753,"type":"branch"}},"format-version":2,"last-sequence-number":1} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00002-262f9917-5f0d-478c-aba2-a05842613e29.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00002-262f9917-5f0d-478c-aba2-a05842613e29.metadata.json new file mode 100644 index 00000000000..01108ba0fe1 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00002-262f9917-5f0d-478c-aba2-a05842613e29.metadata.json @@ -0,0 +1 @@ +{"location":"catalogs/pyiceberg-1/dh-default.db/cities","table-uuid":"5f168e92-7a3c-4277-aa9d-0bf8425e0e05","last-updated-ms":1727458044091,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]},{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"longitude","type":"double","required":false}],"schema-id":1,"identifier-field-ids":[]}],"current-schema-id":1,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"current-snapshot-id":1743193108934390753,"snapshots":[{"snapshot-id":1743193108934390753,"sequence-number":1,"timestamp-ms":1727458044082,"manifest-list":"catalogs/pyiceberg-1/dh-default.db/cities/metadata/snap-1743193108934390753-0-cc668bfb-107c-4122-8d64-9a078f2f45a9.avro","summary":{"operation":"append","added-files-size":"1660","added-data-files":"1","added-records":"4","total-data-files":"1","total-delete-files":"0","total-records":"4","total-files-size":"1660","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":0}],"snapshot-log":[{"snapshot-id":1743193108934390753,"timestamp-ms":1727458044082}],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{"main":{"snapshot-id":1743193108934390753,"type":"branch"}},"format-version":2,"last-sequence-number":1} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00003-979e64ae-2052-4569-b9f1-1b45ab0c2be0.metadata.json b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00003-979e64ae-2052-4569-b9f1-1b45ab0c2be0.metadata.json new file mode 100644 index 00000000000..48001734dc6 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/00003-979e64ae-2052-4569-b9f1-1b45ab0c2be0.metadata.json @@ -0,0 +1 @@ +{"location":"catalogs/pyiceberg-1/dh-default.db/cities","table-uuid":"5f168e92-7a3c-4277-aa9d-0bf8425e0e05","last-updated-ms":1727458044105,"last-column-id":3,"schemas":[{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"lon","type":"double","required":false}],"schema-id":0,"identifier-field-ids":[]},{"type":"struct","fields":[{"id":1,"name":"city","type":"string","required":false},{"id":2,"name":"latitude","type":"double","required":false},{"id":3,"name":"longitude","type":"double","required":false}],"schema-id":1,"identifier-field-ids":[]}],"current-schema-id":1,"partition-specs":[{"spec-id":0,"fields":[]}],"default-spec-id":0,"last-partition-id":999,"properties":{},"current-snapshot-id":4630159959461529013,"snapshots":[{"snapshot-id":1743193108934390753,"sequence-number":1,"timestamp-ms":1727458044082,"manifest-list":"catalogs/pyiceberg-1/dh-default.db/cities/metadata/snap-1743193108934390753-0-cc668bfb-107c-4122-8d64-9a078f2f45a9.avro","summary":{"operation":"append","added-files-size":"1660","added-data-files":"1","added-records":"4","total-data-files":"1","total-delete-files":"0","total-records":"4","total-files-size":"1660","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":0},{"snapshot-id":4630159959461529013,"parent-snapshot-id":1743193108934390753,"sequence-number":2,"timestamp-ms":1727458044105,"manifest-list":"catalogs/pyiceberg-1/dh-default.db/cities/metadata/snap-4630159959461529013-0-6976b5d5-cc59-4f48-bcc5-3a4bf37e5139.avro","summary":{"operation":"append","added-files-size":"1616","added-data-files":"1","added-records":"2","total-data-files":"2","total-delete-files":"0","total-records":"6","total-files-size":"3276","total-position-deletes":"0","total-equality-deletes":"0"},"schema-id":1}],"snapshot-log":[{"snapshot-id":1743193108934390753,"timestamp-ms":1727458044082},{"snapshot-id":4630159959461529013,"timestamp-ms":1727458044105}],"metadata-log":[],"sort-orders":[{"order-id":0,"fields":[]}],"default-sort-order-id":0,"refs":{"main":{"snapshot-id":4630159959461529013,"type":"branch"}},"format-version":2,"last-sequence-number":2} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/a1acdfcc-2afa-4020-af3c-4992ee16e9e8-m0.avro b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/6976b5d5-cc59-4f48-bcc5-3a4bf37e5139-m0.avro similarity index 92% rename from extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/a1acdfcc-2afa-4020-af3c-4992ee16e9e8-m0.avro rename to extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/6976b5d5-cc59-4f48-bcc5-3a4bf37e5139-m0.avro index b00ba51684e3fab5934246d7237730ccefbf1570..37bf6fa460960528b6d4d6b25f8c14420e08586a 100644 GIT binary patch delta 126 zcmdm?v|DL|u7E|fbB3DPEwzUhk}L@}OwZplU1MfqdNr-%{HH$`4mL0{FfbfrOinCG z%*jtL)-R~cOioQoElSrloSY<}tYT(qZkA-4Vyc^*Y-*`%l4fF|n*?O)8Yh}0r5T&2 Qni?8gPF^WcFMw_f08(!&wg3PC delta 123 zcmdn3v_olwu7K%h9!;Zt0oO7E*KA74ewq@%bb*

CUm^7tdU8nsl3yfq~%wV@if@ zNosM4Zh2x+YDRu(aq8q$0cDj$!^Gs2wB%%6qr|jCT@wQ%1Kq?l<78bEOG~5FR70~= P%T$ZWYXs^A(5(OfyG}0p diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/78acf870-0d6f-4d74-9cb7-d3be07a263eb-m0.avro b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/cc668bfb-107c-4122-8d64-9a078f2f45a9-m0.avro similarity index 92% rename from extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-warehouse/dh-default.db/cities/metadata/78acf870-0d6f-4d74-9cb7-d3be07a263eb-m0.avro rename to extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/catalogs/pyiceberg-1/dh-default.db/cities/metadata/cc668bfb-107c-4122-8d64-9a078f2f45a9-m0.avro index 7ab66c8946d1921f1ae686747641867662533cfb..651f0a05ccb9711276af8f0f704e83465ba5016e 100644 GIT binary patch delta 125 zcmdn0v|VX~x`6qnmiyQGGS~g?G-N&Y>y?!e(=BEurbFku`#0_E*l57Oz;KK)Ik6-$ zCqKPdzo0TRIW;M@C|%cZa*TkoN^-K9nMG1slCGhFd9tpFp^=fUMT(h;u4STuxkZ{$ Rnu%$m<>bWzO#m>cLC zq?o1YnxvSU=vpQxnd_z)C#4#gCmNXjSfwDbC^fG{ z7g?F1xrwo%rLm!bg{85Hv892zsqsc1NjA;mWjDfhpLe&5E(+%2&GxWhdB(htF*&g$ zF(*I0Sihh$GdVRWwJ2TJaB?@Be6*RRxml8Fim7gLvZzWuE8R=T2n3?EWCK{Mqq#31|m?px_e+M+5iRsY!?*2_XJ2n~s4dwtFkM1J?eLP`m delta 307 zcmey(_l?iTKPiimN31w6v7k63zeG1PMa|s8($pA8SX!DJni^PI8km}>6eJd<=9TCo zE3>dLHa4&@F)%SVH8(aiGB7eX*ytn4rna(}>(HD#)onKq5(4)YGilnmXH z)Z!A|a-caG`K86Fll$1@BN7c0lT*@?lXZ;}(-L(}42%qP6Vr^7bxkZSjZ#w$%~CB> zEp&4Y^b*U8@;|sRFtD)PId=Twnd?oHZZk3gxeN?I5{KcT<`#*`X%^-Nx&|p`X}TsU x<|ew9$w}tADaJ{u2Ih%IX2z*WFvDK~4QFDSb7lUU_LEzuegzrM0X7`nGXOLgW>NqE diff --git a/extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-catalog.db b/extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/dh-iceberg-test.db similarity index 96% rename from extensions/iceberg/src/test/resources/io/deephaven/iceberg/pyiceberg-1/dh-test-catalog.db rename to extensions/iceberg/src/test/resources/io/deephaven/iceberg/sqlite/db_resource/dh-iceberg-test.db index 9a242fc758bc4b9509abbdaa6dcbea2e4adeb978..a59334f571f1aef6644091b28392a625ba2ddb40 100644 GIT binary patch delta 333 zcmZozz}T>WaRZwG3kw53%Vt4^Uwl@Lm5dz9ijs}VjmZU-naQb1sYU6!hAA1kDXD3R zr8y=fArka@~rs^6Qm>TJtn3`GYCRwH#>KZ1Q zm?kC}BpW5A8tCPwmL#SCP0-6K&de8_C8}Xu3=9m6{P!97?*n~vntx&dm!JYG&^w&S?);!H#&U~+{}xmU ftB?vKNR246nm6`REXc+vViWaRZwG3p)cp`({CfzkF7WWsDq}ijs}7jj_RnInIeCi8=Y{eu=rMDH*yc zscDI&IVH)NC7G$kKz>PTafxnuVo_>Fera*)WIK6PXEQU)G)u$eBwb5OO9Ne#lq6GK z%QTB5U8AHF3j Date: Fri, 27 Sep 2024 12:07:13 -0700 Subject: [PATCH 3/4] Review response --- extensions/iceberg/TESTING.md | 2 +- .../deephaven/iceberg/layout/IcebergBaseLayout.java | 4 ++-- .../deephaven/iceberg/relative/RelativeFileIO.java | 6 +++--- .../io/deephaven/iceberg/CatalogAdapterTest.java | 1 + .../deephaven/iceberg/junit5/CatalogAdapterBase.java | 12 ++++++++++-- 5 files changed, 17 insertions(+), 8 deletions(-) diff --git a/extensions/iceberg/TESTING.md b/extensions/iceberg/TESTING.md index 7019f41e727..ed638f84419 100644 --- a/extensions/iceberg/TESTING.md +++ b/extensions/iceberg/TESTING.md @@ -120,6 +120,6 @@ If we add a lot of catalogs to the database, we may want to look into vacuuming `sqlite3 dh-iceberg-test.db 'VACUUM;'` -Ideally, the sqlite database can be a small collection of catalogs that were created via external tooling to very that +Ideally, the sqlite database can be a small collection of catalogs that were created via external tooling to verify that we can integrate with them successfully. diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index cbf8b272376..c2a3bfc7a67 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -15,7 +15,6 @@ import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.InputFile; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -132,7 +131,8 @@ public IcebergBaseLayout( abstract IcebergTableLocationKey keyFromDataFile(DataFile df, URI fileUri); - private URI dataFileUri(DataFile df) { + @NotNull + private URI dataFileUri(@NotNull DataFile df) { String path = df.path().toString(); if (fileIO instanceof RelativeFileIO) { path = ((RelativeFileIO) fileIO).absoluteLocation(path); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeFileIO.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeFileIO.java index 921134053e5..ef1ffcda24b 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeFileIO.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/relative/RelativeFileIO.java @@ -31,7 +31,6 @@ @VisibleForTesting public final class RelativeFileIO implements HadoopConfigurable, DelegateFileIO { public static final String BASE_PATH = "relative.base-path"; - private static final String IO_IMPL = "relative.io-impl"; private static final String IO_DEFAULT_IMPL = ResolvingFileIO.class.getName(); private String basePath; @@ -79,8 +78,9 @@ private String relativeLocation(String location) { @Override public void initialize(Map properties) { this.basePath = StringUtils.appendIfMissing(properties.get(BASE_PATH), "/"); - String impl = properties.getOrDefault(IO_IMPL, IO_DEFAULT_IMPL); - FileIO fileIO = CatalogUtil.loadFileIO(impl, properties, hadoopConf.get()); + // We can add a property here later if we need to override the default + // final String impl = properties.getOrDefault(IO_IMPL, IO_DEFAULT_IMPL); + final FileIO fileIO = CatalogUtil.loadFileIO(IO_DEFAULT_IMPL, properties, hadoopConf.get()); if (!(fileIO instanceof DelegateFileIO)) { throw new IllegalArgumentException("filoIO must be DelegateFileIO, " + fileIO.getClass()); } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index d08e1735db0..3c102fc2621 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -6,6 +6,7 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.util.TableTools; import io.deephaven.iceberg.junit5.CatalogAdapterBase; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java index c045a093e4a..eb60c17055c 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/CatalogAdapterBase.java @@ -3,26 +3,34 @@ // package io.deephaven.iceberg.junit5; +import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.iceberg.sqlite.SqliteHelper; import io.deephaven.iceberg.util.IcebergCatalogAdapter; import io.deephaven.iceberg.util.IcebergTools; import org.apache.iceberg.catalog.Catalog; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.io.TempDir; -import java.io.IOException; import java.nio.file.Path; public abstract class CatalogAdapterBase { protected IcebergCatalogAdapter catalogAdapter; + private EngineCleanup engineCleanup = new EngineCleanup(); @BeforeEach - void setUp(TestInfo testInfo, @TempDir Path rootDir) throws IOException { + void setUp(TestInfo testInfo, @TempDir Path rootDir) throws Exception { + engineCleanup.setUp(); SqliteHelper.createJdbcDatabase(rootDir); final Catalog catalog = SqliteHelper.createJdbcCatalog(rootDir, testInfo.getDisplayName(), false); catalogAdapter = IcebergTools.createAdapter(catalog); } + + @AfterEach + void tearDown() throws Exception { + engineCleanup.tearDown(); + } } From cff8c541c38168781fc176fbcd8c89cf2dd286d7 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Fri, 27 Sep 2024 12:13:28 -0700 Subject: [PATCH 4/4] oops --- .../src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index 3c102fc2621..d08e1735db0 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -6,7 +6,6 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.util.TableTools; import io.deephaven.iceberg.junit5.CatalogAdapterBase; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace;