From 54e0a805d4cec2e0e83df41e021f40344e1b1806 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 22 Mar 2024 09:08:17 -0700 Subject: [PATCH 01/25] Initial commit of Iceberg integration. --- extensions/iceberg/build.gradle | 74 ++ extensions/iceberg/gradle.properties | 1 + .../iceberg/layout/IcebergFlatLayout.java | 78 ++ .../layout/IcebergPartitionedLayout.java | 93 ++ .../location/IcebergColumnLocation.java | 822 ++++++++++++++++++ .../location/IcebergTableLocation.java | 179 ++++ .../location/IcebergTableLocationFactory.java | 43 + .../location/IcebergTableLocationKey.java | 190 ++++ .../iceberg/util/IcebergCatalog.java | 250 ++++++ .../iceberg/util/IcebergInstructions.java | 145 +++ .../deephaven/iceberg/util/IcebergTools.java | 83 ++ .../IcebergToolsTest.java | 54 ++ server/build.gradle | 1 + settings.gradle | 3 + 14 files changed, 2016 insertions(+) create mode 100644 extensions/iceberg/build.gradle create mode 100644 extensions/iceberg/gradle.properties create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java create mode 100644 extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle new file mode 100644 index 00000000000..b10d538a345 --- /dev/null +++ b/extensions/iceberg/build.gradle @@ -0,0 +1,74 @@ +plugins { + id 'java-library' + id 'io.deephaven.project.register' +} + +description 'Iceberg: Support to read iceberg catalogs.' + +ext { + icebergVersion = '1.4.3' + hadoopVersion = '3.3.6' +} + +dependencies { + api project(':engine-api') + api project(':engine-stringset') + api project(':engine-table') + + implementation project(':engine-table') + implementation project(':engine-base') + implementation project(':log-factory') + implementation project(':Configuration') + implementation depTrove3 + + compileOnly depAnnotations + + Classpaths.inheritAutoService(project) + Classpaths.inheritImmutables(project) + + Classpaths.inheritJUnitPlatform(project) + Classpaths.inheritAssertJ(project) + + Classpaths.inheritParquetHadoop(project) + implementation project(':extensions-parquet-base') + implementation project(':extensions-parquet-table') + implementation project(':extensions-s3') + + implementation "org.apache.iceberg:iceberg-api:${icebergVersion}" + implementation "org.apache.iceberg:iceberg-core:${icebergVersion}" + implementation "org.apache.iceberg:iceberg-core:${icebergVersion}:tests" + implementation "org.apache.iceberg:iceberg-aws:${icebergVersion}" + implementation "org.apache.iceberg:iceberg-azure:${icebergVersion}" + implementation "org.apache.iceberg:iceberg-gcp:${icebergVersion}" + implementation "org.apache.iceberg:iceberg-bundled-guava:${icebergVersion}" + + implementation "org.apache.hadoop:hadoop-common:${hadoopVersion}" + implementation "org.apache.hadoop:hadoop-hdfs-client:${hadoopVersion}" + + runtimeOnly "org.apache.iceberg:iceberg-aws-bundle:${icebergVersion}" + runtimeOnly "org.apache.iceberg:iceberg-azure-bundle:${icebergVersion}" + runtimeOnly "org.apache.iceberg:iceberg-gcp-bundle:${icebergVersion}" + + implementation 'org.apache.httpcomponents.client5:httpclient5:5.3.1' + implementation 'org.apache.httpcomponents.core5:httpcore5:5.2.4' + + implementation 'com.google.guava:guava:33.0.0-jre' + + implementation 'org.xerial:sqlite-jdbc:3.45.1.0' + implementation 'org.postgresql:postgresql:42.7.1' + + + testImplementation depCommonsCompress + testImplementation depCommonsIo + testImplementation project(':engine-test-utils'), + TestTools.projectDependency(project, 'engine-base') + + Classpaths.inheritJUnitClassic(project, 'testImplementation') + + testRuntimeOnly project(':log-to-slf4j'), + project(path: ':configs'), + project(path: ':test-configs') + Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly') +} + +TestTools.addEngineOutOfBandTest(project) diff --git a/extensions/iceberg/gradle.properties b/extensions/iceberg/gradle.properties new file mode 100644 index 00000000000..c186bbfdde1 --- /dev/null +++ b/extensions/iceberg/gradle.properties @@ -0,0 +1 @@ +io.deephaven.project.ProjectType=JAVA_PUBLIC diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java new file mode 100644 index 00000000000..5fb2d1b0373 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -0,0 +1,78 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.iceberg.layout; + +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import io.deephaven.parquet.table.ParquetInstructions; +import org.apache.iceberg.*; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.jetbrains.annotations.NotNull; + +import java.net.URI; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; + +/** + * Iceberg {@link TableLocationKeyFinder location finder} for tables without partitions that will discover data files + * from a {@link org.apache.iceberg.Snapshot} + */ +public final class IcebergFlatLayout implements TableLocationKeyFinder { + private final Snapshot tableSnapshot; + private final FileIO fileIO; + private final Map cache; + private final ParquetInstructions readInstructions; + + private static IcebergTableLocationKey locationKey(final URI fileUri, @NotNull final ParquetInstructions readInstructions) { + return new IcebergTableLocationKey(fileUri, 0, null, readInstructions); + } + + /** + * @param tableSnapshot The {@link Snapshot} from which to discover data files. + * @param readInstructions the instructions for customizations while reading. + */ + public IcebergFlatLayout( + @NotNull final Snapshot tableSnapshot, + @NotNull final FileIO fileIO, + @NotNull final ParquetInstructions readInstructions) { + this.tableSnapshot = tableSnapshot; + this.fileIO = fileIO; + this.readInstructions = readInstructions; + + this.cache = new HashMap<>(); + } + + public String toString() { + return IcebergFlatLayout.class.getSimpleName() + '[' + tableSnapshot + ']'; + } + + @Override + public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { + try { + // Retrieve the manifest files from the snapshot + final List manifestFiles = tableSnapshot.allManifests(fileIO); + for (final ManifestFile manifestFile : manifestFiles) { + final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO); + for (DataFile df : reader) { + final URI fileUri = URI.create(df.path().toString()); + IcebergTableLocationKey locationKey = cache.get(fileUri); + if (locationKey == null) { + locationKey = locationKey(fileUri, readInstructions); + if (!locationKey.verifyFileReader()) { + continue; + } + cache.put(fileUri, locationKey); + } + locationKeyObserver.accept(locationKey); + } + } + } catch (final Exception e) { + throw new TableDataException("Error finding Iceberg locations under " + tableSnapshot, e); + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java new file mode 100644 index 00000000000..cd8baf5a1b9 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java @@ -0,0 +1,93 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.iceberg.layout; + +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import io.deephaven.parquet.table.ParquetInstructions; +import org.apache.iceberg.*; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.jetbrains.annotations.NotNull; + +import java.net.URI; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; + +/** + * Iceberg {@link TableLocationKeyFinder location finder} for tables without partitions that will discover data files + * from a {@link org.apache.iceberg.Snapshot} + */ +public final class IcebergPartitionedLayout implements TableLocationKeyFinder { + private final Snapshot tableSnapshot; + private final FileIO fileIO; + private final String[] partitionColumns; + private final ParquetInstructions readInstructions; + private final Map cache; + + private static IcebergTableLocationKey locationKey( + final URI fileUri, + final Map> partitions, + @NotNull final ParquetInstructions readInstructions) { + return new IcebergTableLocationKey(fileUri, 0, partitions, readInstructions); + } + + /** + * @param tableSnapshot The {@link Snapshot} from which to discover data files. + * @param fileIO The file IO to use for reading manifest data files. + * @param partitionColumns The columns to use for partitioning. + * @param readInstructions The instructions for customizations while reading. + */ + public IcebergPartitionedLayout( + @NotNull final Snapshot tableSnapshot, + @NotNull final FileIO fileIO, + @NotNull final String[] partitionColumns, + @NotNull final ParquetInstructions readInstructions) { + this.tableSnapshot = tableSnapshot; + this.fileIO = fileIO; + this.partitionColumns = partitionColumns; + this.readInstructions = readInstructions; + + this.cache = new HashMap<>(); + } + + public String toString() { + return IcebergFlatLayout.class.getSimpleName() + '[' + tableSnapshot + ']'; + } + + @Override + public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { + final Map> partitions = new LinkedHashMap<>(); + try { + // Retrieve the manifest files from the snapshot + final List manifestFiles = tableSnapshot.allManifests(fileIO); + for (final ManifestFile manifestFile : manifestFiles) { + final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO); + for (DataFile df : reader) { + final URI fileUri = URI.create(df.path().toString()); + + IcebergTableLocationKey locationKey = cache.get(fileUri); + if (locationKey == null) { + final PartitionData partitionData = (PartitionData) df.partition(); + for (int ii = 0; ii < partitionColumns.length; ++ii) { + partitions.put(partitionColumns[ii], (Comparable) partitionData.get(ii)); + } + locationKey = locationKey(fileUri, partitions, readInstructions); + if (!locationKey.verifyFileReader()) { + continue; + } + cache.put(fileUri, locationKey); + } + locationKeyObserver.accept(locationKey); + } + } + } catch (final Exception e) { + throw new TableDataException("Error finding Iceberg locations under " + tableSnapshot, e); + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java new file mode 100644 index 00000000000..036edcf1278 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java @@ -0,0 +1,822 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.iceberg.location; + +import io.deephaven.base.verify.Assert; +import io.deephaven.base.verify.Require; +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.configuration.Configuration; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSequenceFactory; +import io.deephaven.engine.rowset.chunkattributes.UnorderedRowKeys; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.Context; +import io.deephaven.engine.table.impl.CodecLookup; +import io.deephaven.engine.table.impl.chunkattributes.DictionaryKeys; +import io.deephaven.engine.table.impl.chunkboxer.ChunkBoxer; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.impl.AbstractColumnLocation; +import io.deephaven.engine.table.impl.sources.regioned.*; +import io.deephaven.internal.log.LoggerFactory; +import io.deephaven.io.logger.Logger; +import io.deephaven.parquet.base.ColumnChunkReader; +import io.deephaven.parquet.base.ParquetFileReader; +import io.deephaven.parquet.base.RowGroupReader; +import io.deephaven.parquet.table.*; +import io.deephaven.parquet.table.metadata.CodecInfo; +import io.deephaven.parquet.table.metadata.ColumnTypeInfo; +import io.deephaven.parquet.table.metadata.GroupingColumnInfo; +import io.deephaven.parquet.table.metadata.TableInfo; +import io.deephaven.parquet.table.pagestore.ColumnChunkPageStore; +import io.deephaven.parquet.table.pagestore.PageCache; +import io.deephaven.parquet.table.pagestore.topage.*; +import io.deephaven.parquet.table.region.*; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; +import io.deephaven.util.codec.CodecCache; +import io.deephaven.util.codec.ObjectCodec; +import io.deephaven.util.codec.SimpleByteArrayCodec; +import io.deephaven.vector.Vector; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.URI; +import java.util.*; +import java.util.function.Function; +import java.util.function.LongFunction; +import java.util.function.Supplier; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK; +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; +import static io.deephaven.parquet.table.ParquetTableWriter.*; + +final class IcebergColumnLocation extends AbstractColumnLocation { + + private static final String IMPLEMENTATION_NAME = IcebergColumnLocation.class.getSimpleName(); + + private static final int CHUNK_SIZE = Configuration.getInstance() + .getIntegerForClassWithDefault(IcebergColumnLocation.class, "chunkSize", 4096); + private static final int INITIAL_PAGE_CACHE_SIZE = Configuration.getInstance() + .getIntegerForClassWithDefault(IcebergColumnLocation.class, "initialPageCacheSize", 128); + private static final int MAX_PAGE_CACHE_SIZE = Configuration.getInstance() + .getIntegerForClassWithDefault(IcebergColumnLocation.class, "maxPageCacheSize", 8192); + + private static final Logger log = LoggerFactory.getLogger(IcebergColumnLocation.class); + + private final String parquetColumnName; + /** + * Factory object needed for deferred initialization of the remaining fields. Reference serves as a barrier to + * ensure visibility of the derived fields. + */ + private volatile ColumnChunkReader[] columnChunkReaders; + private final boolean hasGroupingTable; + + // We should consider moving this to column level if needed. Column-location level likely allows more parallelism. + private volatile PageCache pageCache; + + private ColumnChunkPageStore[] pageStores; + private Supplier>[] dictionaryChunkSuppliers; + private ColumnChunkPageStore[] dictionaryKeysPageStores; + + /** + * Construct a new {@link IcebergColumnLocation} for the specified {@link IcebergTableLocation} and column name. + * + * @param tableLocation The table location enclosing this column location + * @param parquetColumnName The Parquet file column name + * @param columnChunkReaders The {@link ColumnChunkReader column chunk readers} for this location + * @param hasGroupingTable Whether this column has an associated grouping table file + */ + IcebergColumnLocation(@NotNull final IcebergTableLocation tableLocation, + @NotNull final String columnName, + @NotNull final String parquetColumnName, + @Nullable final ColumnChunkReader[] columnChunkReaders, + final boolean hasGroupingTable) { + super(tableLocation, columnName); + this.parquetColumnName = parquetColumnName; + this.columnChunkReaders = columnChunkReaders; + this.hasGroupingTable = hasGroupingTable; + } + + private PageCache ensurePageCache() { + PageCache localPageCache; + if ((localPageCache = pageCache) != null) { + return localPageCache; + } + + synchronized (this) { + if ((localPageCache = pageCache) != null) { + return localPageCache; + } + return pageCache = new PageCache<>(INITIAL_PAGE_CACHE_SIZE, MAX_PAGE_CACHE_SIZE); + } + } + + // ----------------------------------------------------------------------------------------------------------------- + // AbstractColumnLocation implementation + // ----------------------------------------------------------------------------------------------------------------- + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + @Override + public boolean exists() { + // If we see a null columnChunkReaders array, either we don't exist or we are guaranteed to + // see a non-null + // pageStores array + return columnChunkReaders != null || pageStores != null; + } + + private IcebergTableLocation tl() { + return (IcebergTableLocation) getTableLocation(); + } + + private static final ColumnDefinition FIRST_KEY_COL_DEF = + ColumnDefinition.ofLong("__firstKey__"); + private static final ColumnDefinition LAST_KEY_COL_DEF = + ColumnDefinition.ofLong("__lastKey__"); + + /** + * Helper method for logging a warning on failure in reading an index file + */ + private void logWarnFailedToRead(final String indexFilePath) { + log.warn().append("Failed to read expected index file ").append(indexFilePath) + .append(" for table location ").append(tl()).append(", column ") + .append(getName()) + .endl(); + } + + @Override + @Nullable + public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition columnDefinition) { + if (!hasGroupingTable) { + return null; + } + final URI parquetFileURI = tl().getParquetKey().getURI(); + Assert.assertion(FILE_URI_SCHEME.equals(parquetFileURI.getScheme()), + "Expected a file uri, got " + parquetFileURI); + final File parquetFile = new File(parquetFileURI); + try { + ParquetFileReader parquetFileReader; + final String indexFilePath; + final GroupingColumnInfo groupingColumnInfo = tl().getGroupingColumns().get(parquetColumnName); + final SeekableChannelsProvider channelsProvider = tl().getChannelProvider(); + if (groupingColumnInfo != null) { + final String indexFileRelativePath = groupingColumnInfo.groupingTablePath(); + indexFilePath = parquetFile.toPath().getParent().resolve(indexFileRelativePath).toString(); + try { + parquetFileReader = new ParquetFileReader(indexFilePath, channelsProvider); + } catch (final RuntimeException e) { + logWarnFailedToRead(indexFilePath); + return null; + } + } else { + final String relativeIndexFilePath = + ParquetTools.getRelativeIndexFilePath(parquetFile, parquetColumnName); + indexFilePath = parquetFile.toPath().getParent().resolve(relativeIndexFilePath).toString(); + try { + parquetFileReader = new ParquetFileReader(indexFilePath, channelsProvider); + } catch (final RuntimeException e1) { + // Retry with legacy grouping file path + final String legacyGroupingFileName = + ParquetTools.legacyGroupingFileName(parquetFile, parquetColumnName); + final File legacyGroupingFile = new File(parquetFile.getParent(), legacyGroupingFileName); + try { + parquetFileReader = + new ParquetFileReader(legacyGroupingFile.getAbsolutePath(), channelsProvider); + } catch (final RuntimeException e2) { + logWarnFailedToRead(indexFilePath); + return null; + } + } + } + final Optional tableInfo = ParquetSchemaReader.parseMetadata( + new ParquetMetadataConverter().fromParquetMetadata(parquetFileReader.fileMetaData) + .getFileMetaData().getKeyValueMetaData()); + final Map columnTypes = + tableInfo.map(TableInfo::columnTypeMap).orElse(Collections.emptyMap()); + final String version = tableInfo.map(TableInfo::version).orElse(null); + + final RowGroupReader rowGroupReader = parquetFileReader.getRowGroup(0, version); + final ColumnChunkReader groupingKeyReader, beginPosReader, endPosReader; + try (final SeekableChannelContext channelContext = channelsProvider.makeSingleUseContext()) { + groupingKeyReader = + rowGroupReader.getColumnChunk(Collections.singletonList(GROUPING_KEY), channelContext); + beginPosReader = rowGroupReader.getColumnChunk(Collections.singletonList(BEGIN_POS), channelContext); + endPosReader = rowGroupReader.getColumnChunk(Collections.singletonList(END_POS), channelContext); + } + if (groupingKeyReader == null || beginPosReader == null || endPosReader == null) { + log.warn().append("Index file ").append(indexFilePath) + .append(" is missing one or more expected columns for table location ") + .append(tl()).append(", column ").append(getName()).endl(); + return null; + } + + final PageCache localPageCache = ensurePageCache(); + + // noinspection unchecked + return (METADATA_TYPE) new MetaDataTableFactory( + ColumnChunkPageStore.create( + localPageCache.castAttr(), groupingKeyReader, + ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, + makeToPage(columnTypes.get(GROUPING_KEY), ParquetInstructions.EMPTY, + GROUPING_KEY, groupingKeyReader, columnDefinition), + columnDefinition).pageStore, + ColumnChunkPageStore.create( + localPageCache.castAttr(), beginPosReader, + ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, + makeToPage(columnTypes.get(BEGIN_POS), ParquetInstructions.EMPTY, BEGIN_POS, + beginPosReader, FIRST_KEY_COL_DEF), + columnDefinition).pageStore, + ColumnChunkPageStore.create( + localPageCache.castAttr(), endPosReader, + ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, + makeToPage(columnTypes.get(END_POS), ParquetInstructions.EMPTY, END_POS, + endPosReader, LAST_KEY_COL_DEF), + columnDefinition).pageStore) + .get(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private REGION_TYPE makeColumnRegion( + @NotNull final Function, SOURCE[]> sourceArrayFactory, + @NotNull final ColumnDefinition columnDefinition, + @NotNull final LongFunction nullRegionFactory, + @NotNull final Function singleRegionFactory, + @NotNull final Function, REGION_TYPE> multiRegionFactory) { + final SOURCE[] sources = sourceArrayFactory.apply(columnDefinition); + return sources.length == 1 + ? makeSingleColumnRegion(sources[0], nullRegionFactory, singleRegionFactory) + : multiRegionFactory.apply(Arrays.stream(sources).map( + source -> makeSingleColumnRegion(source, nullRegionFactory, singleRegionFactory))); + } + + private REGION_TYPE makeSingleColumnRegion(final SOURCE source, + @NotNull final LongFunction nullRegionFactory, + @NotNull final Function singleRegionFactory) { + return source == null ? nullRegionFactory.apply(tl().getRegionParameters().regionMask) + : singleRegionFactory.apply(source); + } + + @Override + public ColumnRegionChar makeColumnRegionChar( + @NotNull final ColumnDefinition columnDefinition) { + // noinspection unchecked + return (ColumnRegionChar) makeColumnRegion(this::getPageStores, columnDefinition, + ColumnRegionChar::createNull, ParquetColumnRegionChar::new, + rs -> new ColumnRegionChar.StaticPageStore<>(tl().getRegionParameters(), + rs.toArray(ColumnRegionChar[]::new))); + } + + @Override + public ColumnRegionByte makeColumnRegionByte( + @NotNull final ColumnDefinition columnDefinition) { + // noinspection unchecked + return (ColumnRegionByte) makeColumnRegion(this::getPageStores, columnDefinition, + ColumnRegionByte::createNull, ParquetColumnRegionByte::new, + rs -> new ColumnRegionByte.StaticPageStore<>(tl().getRegionParameters(), + rs.toArray(ColumnRegionByte[]::new))); + } + + @Override + public ColumnRegionShort makeColumnRegionShort( + @NotNull final ColumnDefinition columnDefinition) { + // noinspection unchecked + return (ColumnRegionShort) makeColumnRegion(this::getPageStores, columnDefinition, + ColumnRegionShort::createNull, ParquetColumnRegionShort::new, + rs -> new ColumnRegionShort.StaticPageStore<>(tl().getRegionParameters(), + rs.toArray(ColumnRegionShort[]::new))); + } + + @Override + public ColumnRegionInt makeColumnRegionInt( + @NotNull final ColumnDefinition columnDefinition) { + // noinspection unchecked + return (ColumnRegionInt) makeColumnRegion(this::getPageStores, columnDefinition, + ColumnRegionInt::createNull, ParquetColumnRegionInt::new, + rs -> new ColumnRegionInt.StaticPageStore<>(tl().getRegionParameters(), + rs.toArray(ColumnRegionInt[]::new))); + } + + @Override + public ColumnRegionLong makeColumnRegionLong( + @NotNull final ColumnDefinition columnDefinition) { + // noinspection unchecked + return (ColumnRegionLong) makeColumnRegion(this::getPageStores, columnDefinition, + ColumnRegionLong::createNull, ParquetColumnRegionLong::new, + rs -> new ColumnRegionLong.StaticPageStore<>(tl().getRegionParameters(), + rs.toArray(ColumnRegionLong[]::new))); + } + + @Override + public ColumnRegionFloat makeColumnRegionFloat( + @NotNull final ColumnDefinition columnDefinition) { + // noinspection unchecked + return (ColumnRegionFloat) makeColumnRegion(this::getPageStores, columnDefinition, + ColumnRegionFloat::createNull, ParquetColumnRegionFloat::new, + rs -> new ColumnRegionFloat.StaticPageStore<>(tl().getRegionParameters(), + rs.toArray(ColumnRegionFloat[]::new))); + } + + @Override + public ColumnRegionDouble makeColumnRegionDouble( + @NotNull final ColumnDefinition columnDefinition) { + // noinspection unchecked + return (ColumnRegionDouble) makeColumnRegion(this::getPageStores, columnDefinition, + ColumnRegionDouble::createNull, ParquetColumnRegionDouble::new, + rs -> new ColumnRegionDouble.StaticPageStore<>(tl().getRegionParameters(), + rs.toArray(ColumnRegionDouble[]::new))); + } + + @Override + public ColumnRegionObject makeColumnRegionObject( + @NotNull final ColumnDefinition columnDefinition) { + final Class dataType = columnDefinition.getDataType(); + final ColumnChunkPageStore[] sources = getPageStores(columnDefinition); + final ColumnChunkPageStore[] dictKeySources = + getDictionaryKeysPageStores(columnDefinition); + final Supplier>[] dictionaryChunkSuppliers = + getDictionaryChunkSuppliers(columnDefinition); + if (sources.length == 1) { + // noinspection unchecked + return (ColumnRegionObject) makeSingleColumnRegionObject(dataType, + sources[0], dictKeySources[0], dictionaryChunkSuppliers[0]); + } + // noinspection unchecked + return (ColumnRegionObject) new ColumnRegionObject.StaticPageStore( + tl().getRegionParameters(), + IntStream.range(0, sources.length) + .mapToObj(ri -> makeSingleColumnRegionObject(dataType, sources[ri], + dictKeySources[ri], dictionaryChunkSuppliers[ri])) + .toArray(ColumnRegionObject[]::new)); + } + + private ColumnRegionObject makeSingleColumnRegionObject( + @NotNull final Class dataType, + @Nullable final ColumnChunkPageStore source, + @Nullable final ColumnChunkPageStore dictKeySource, + @Nullable final Supplier> dictValuesSupplier) { + if (source == null) { + return ColumnRegionObject.createNull(tl().getRegionParameters().regionMask); + } + return new ParquetColumnRegionObject<>(source, + () -> new ParquetColumnRegionLong<>(Require.neqNull(dictKeySource, "dictKeySource")), + () -> ColumnRegionChunkDictionary.create(tl().getRegionParameters().regionMask, + dataType, Require.neqNull(dictValuesSupplier, "dictValuesSupplier"))); + } + + /** + * Get the {@link ColumnChunkPageStore page stores} backing this column location. + * + * @param columnDefinition The {@link ColumnDefinition} used to lookup type information + * @return The page stores + */ + @NotNull + public ColumnChunkPageStore[] getPageStores( + @NotNull final ColumnDefinition columnDefinition) { + fetchValues(columnDefinition); + return pageStores; + } + + /** + * Get suppliers to access the {@link Chunk dictionary chunks} backing this column location. + * + * @param columnDefinition The {@link ColumnDefinition} used to lookup type information + * @return The dictionary values chunk suppliers, or null if none exist + */ + public Supplier>[] getDictionaryChunkSuppliers( + @NotNull final ColumnDefinition columnDefinition) { + fetchValues(columnDefinition); + return dictionaryChunkSuppliers; + } + + /** + * Get the {@link ColumnChunkPageStore page stores} backing the indices for this column location. Only usable when + * there are dictionaries. + * + * @param columnDefinition The {@link ColumnDefinition} used to lookup type information + * @return The page stores + */ + private ColumnChunkPageStore[] getDictionaryKeysPageStores( + @NotNull final ColumnDefinition columnDefinition) { + fetchValues(columnDefinition); + return dictionaryKeysPageStores; + } + + @SuppressWarnings("unchecked") + private void fetchValues(@NotNull final ColumnDefinition columnDefinition) { + if (columnChunkReaders == null) { + return; + } + synchronized (this) { + if (columnChunkReaders == null) { + return; + } + + final int pageStoreCount = columnChunkReaders.length; + pageStores = new ColumnChunkPageStore[pageStoreCount]; + dictionaryChunkSuppliers = new Supplier[pageStoreCount]; + dictionaryKeysPageStores = new ColumnChunkPageStore[pageStoreCount]; + for (int psi = 0; psi < pageStoreCount; ++psi) { + final ColumnChunkReader columnChunkReader = columnChunkReaders[psi]; + try { + final ColumnChunkPageStore.CreatorResult creatorResult = + ColumnChunkPageStore.create( + ensurePageCache(), + columnChunkReader, + tl().getRegionParameters().regionMask, + makeToPage(tl().getColumnTypes().get(parquetColumnName), + tl().getReadInstructions(), parquetColumnName, columnChunkReader, + columnDefinition), + columnDefinition); + pageStores[psi] = creatorResult.pageStore; + dictionaryChunkSuppliers[psi] = creatorResult.dictionaryChunkSupplier; + dictionaryKeysPageStores[psi] = creatorResult.dictionaryKeysPageStore; + } catch (IOException e) { + throw new TableDataException( + "Failed to read parquet file for " + this + ", row group " + psi, e); + } + } + + columnChunkReaders = null; + } + } + + private static final class MetaDataTableFactory { + + private final ColumnChunkPageStore keyColumn; + private final ColumnChunkPageStore firstColumn; + private final ColumnChunkPageStore lastColumn; + + private volatile Object metaData; + + private MetaDataTableFactory(@NotNull final ColumnChunkPageStore keyColumn, + @NotNull final ColumnChunkPageStore firstColumn, + @NotNull final ColumnChunkPageStore lastColumn) { + this.keyColumn = Require.neqNull(keyColumn, "keyColumn"); + this.firstColumn = Require.neqNull(firstColumn, "firstColumn"); + this.lastColumn = Require.neqNull(lastColumn, "lastColumn"); + } + + public Object get() { + if (metaData != null) { + return metaData; + } + synchronized (this) { + if (metaData != null) { + return metaData; + } + final int numRows = (int) keyColumn.numRows(); + + try ( + final ChunkBoxer.BoxerKernel boxerKernel = + ChunkBoxer.getBoxer(keyColumn.getChunkType(), CHUNK_SIZE); + final BuildGrouping buildGrouping = + BuildGrouping.builder(firstColumn.getChunkType(), numRows); + final ChunkSource.GetContext keyContext = keyColumn.makeGetContext(CHUNK_SIZE); + final ChunkSource.GetContext firstContext = + firstColumn.makeGetContext(CHUNK_SIZE); + final ChunkSource.GetContext lastContext = + lastColumn.makeGetContext(CHUNK_SIZE); + final RowSequence rows = RowSequenceFactory.forRange(0, numRows - 1); + final RowSequence.Iterator rowsIterator = rows.getRowSequenceIterator()) { + + while (rowsIterator.hasMore()) { + final RowSequence chunkRows = + rowsIterator.getNextRowSequenceWithLength(CHUNK_SIZE); + + buildGrouping.build( + boxerKernel.box(keyColumn.getChunk(keyContext, chunkRows)), + firstColumn.getChunk(firstContext, chunkRows), + lastColumn.getChunk(lastContext, chunkRows)); + } + + metaData = buildGrouping.getGrouping(); + } + } + return metaData; + } + + private interface BuildGrouping extends Context { + void build(@NotNull ObjectChunk keyChunk, + @NotNull Chunk firstChunk, + @NotNull Chunk lastChunk); + + Object getGrouping(); + + static BuildGrouping builder(@NotNull final ChunkType chunkType, final int numRows) { + switch (chunkType) { + case Int: + return new IntBuildGrouping(numRows); + case Long: + return new LongBuildGrouping(numRows); + default: + throw new IllegalArgumentException( + "Unknown type for a rowSet: " + chunkType); + } + } + + final class IntBuildGrouping implements BuildGrouping { + + private final Map grouping; + + IntBuildGrouping(final int numRows) { + grouping = new LinkedHashMap<>(numRows); + } + + @Override + public void build(@NotNull final ObjectChunk keyChunk, + @NotNull final Chunk firstChunk, + @NotNull final Chunk lastChunk) { + final IntChunk firstIntChunk = + firstChunk.asIntChunk(); + final IntChunk lastIntChunk = + lastChunk.asIntChunk(); + + for (int ki = 0; ki < keyChunk.size(); ++ki) { + final int[] range = new int[2]; + + range[0] = firstIntChunk.get(ki); + range[1] = lastIntChunk.get(ki); + + grouping.put(keyChunk.get(ki), range); + } + } + + @Override + public Object getGrouping() { + return grouping; + } + } + + final class LongBuildGrouping implements BuildGrouping { + + private final Map grouping; + + LongBuildGrouping(final int numRows) { + grouping = new LinkedHashMap<>(numRows); + } + + @Override + public void build(@NotNull final ObjectChunk keyChunk, + @NotNull final Chunk firstChunk, + @NotNull final Chunk lastChunk) { + final LongChunk firstLongChunk = + firstChunk.asLongChunk(); + final LongChunk lastLongChunk = + lastChunk.asLongChunk(); + + for (int ki = 0; ki < keyChunk.size(); ++ki) { + final long[] range = new long[2]; + + range[0] = firstLongChunk.get(ki); + range[1] = lastLongChunk.get(ki); + + grouping.put(keyChunk.get(ki), range); + } + } + + @Override + public Object getGrouping() { + return grouping; + } + } + } + } + + private static ToPage makeToPage( + @Nullable final ColumnTypeInfo columnTypeInfo, + @NotNull final ParquetInstructions readInstructions, + @NotNull final String parquetColumnName, + @NotNull final ColumnChunkReader columnChunkReader, + @NotNull final ColumnDefinition columnDefinition) { + final PrimitiveType type = columnChunkReader.getType(); + final LogicalTypeAnnotation logicalTypeAnnotation = type.getLogicalTypeAnnotation(); + final String codecFromInstructions = + readInstructions.getCodecName(columnDefinition.getName()); + final String codecName = (codecFromInstructions != null) + ? codecFromInstructions + : columnTypeInfo == null ? null + : columnTypeInfo.codec().map(CodecInfo::codecName).orElse(null); + final ColumnTypeInfo.SpecialType specialTypeName = + columnTypeInfo == null ? null : columnTypeInfo.specialType().orElse(null); + + final boolean isArray = columnChunkReader.getMaxRl() > 0; + final boolean isCodec = CodecLookup.explicitCodecPresent(codecName); + + if (isArray && columnChunkReader.getMaxRl() > 1) { + throw new TableDataException("No support for nested repeated parquet columns."); + } + + try { + // Note that componentType is null for a StringSet. ToStringSetPage.create specifically + // doesn't take this parameter. + final Class dataType = columnDefinition.getDataType(); + final Class componentType = columnDefinition.getComponentType(); + final Class pageType = isArray ? componentType : dataType; + + ToPage toPage = null; + + if (!isCodec && logicalTypeAnnotation != null) { + toPage = logicalTypeAnnotation.accept( + new LogicalTypeVisitor(parquetColumnName, columnChunkReader, pageType)) + .orElse(null); + } + + if (toPage == null) { + final PrimitiveType.PrimitiveTypeName typeName = type.getPrimitiveTypeName(); + switch (typeName) { + case BOOLEAN: + toPage = ToBooleanAsBytePage.create(pageType); + break; + case INT32: + toPage = ToIntPage.create(pageType); + break; + case INT64: + toPage = ToLongPage.create(pageType); + break; + case INT96: + toPage = ToInstantPageFromInt96.create(pageType); + break; + case DOUBLE: + toPage = ToDoublePage.create(pageType); + break; + case FLOAT: + toPage = ToFloatPage.create(pageType); + break; + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + // noinspection rawtypes + final ObjectCodec codec; + if (isCodec) { + final String codecArgs = codecFromInstructions != null + ? readInstructions.getCodecArgs(columnDefinition.getName()) + : columnTypeInfo.codec().flatMap(CodecInfo::codecArg).orElse(null); + codec = CodecCache.DEFAULT.getCodec(codecName, codecArgs); + } else { + final String codecArgs = + (typeName == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) + ? Integer.toString(type.getTypeLength()) + : null; + codec = CodecCache.DEFAULT + .getCodec(SimpleByteArrayCodec.class.getName(), codecArgs); + } + // noinspection unchecked + toPage = ToObjectPage.create(dataType, codec, + columnChunkReader.getDictionarySupplier()); + break; + default: + } + } + + if (toPage == null) { + throw new TableDataException( + "Unsupported parquet column type " + type.getPrimitiveTypeName() + + " with logical type " + logicalTypeAnnotation); + } + + if (specialTypeName == ColumnTypeInfo.SpecialType.StringSet) { + Assert.assertion(isArray, "isArray"); + toPage = ToStringSetPage.create(dataType, toPage); + } else if (isArray) { + Assert.assertion(!isCodec, "!isCodec"); + if (Vector.class.isAssignableFrom(dataType)) { + toPage = ToVectorPage.create(dataType, componentType, toPage); + } else if (dataType.isArray()) { + toPage = ToArrayPage.create(dataType, componentType, toPage); + } + } + + // noinspection unchecked + return (ToPage) toPage; + + } catch (RuntimeException except) { + throw new TableDataException( + "Unexpected exception accessing column " + parquetColumnName, except); + } + } + + private static class LogicalTypeVisitor + implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { + + private final String name; + private final ColumnChunkReader columnChunkReader; + private final Class componentType; + + LogicalTypeVisitor(@NotNull final String name, @NotNull final ColumnChunkReader columnChunkReader, + final Class componentType) { + this.name = name; + this.columnChunkReader = columnChunkReader; + this.componentType = componentType; + } + + @Override + public Optional> visit( + final LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + return Optional + .of(ToStringPage.create(componentType, columnChunkReader.getDictionarySupplier())); + } + + @Override + public Optional> visit( + final LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + if (timestampLogicalType.isAdjustedToUTC()) { + return Optional.of(ToInstantPage.create(componentType, timestampLogicalType.getUnit())); + } + return Optional.of(ToLocalDateTimePage.create(componentType, timestampLogicalType.getUnit())); + } + + @Override + public Optional> visit(final LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { + + if (intLogicalType.isSigned()) { + switch (intLogicalType.getBitWidth()) { + case 8: + return Optional.of(ToBytePageFromInt.create(componentType)); + case 16: + return Optional.of(ToShortPageFromInt.create(componentType)); + case 32: + return Optional.of(ToIntPage.create(componentType)); + case 64: + return Optional.of(ToLongPage.create(componentType)); + } + } else { + switch (intLogicalType.getBitWidth()) { + case 8: + case 16: + return Optional.of(ToCharPageFromInt.create(componentType)); + case 32: + return Optional.of(ToLongPage.create(componentType)); + } + } + + return Optional.empty(); + } + + @Override + public Optional> visit(final LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return Optional.of(ToDatePageFromInt.create(componentType)); + } + + @Override + public Optional> visit(final LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + return Optional + .of(ToTimePage.create(componentType, timeLogicalType.getUnit(), timeLogicalType.isAdjustedToUTC())); + } + + @Override + public Optional> visit( + final LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) { + final PrimitiveType type = columnChunkReader.getType(); + final PrimitiveType.PrimitiveTypeName typeName = type.getPrimitiveTypeName(); + switch (typeName) { + case INT32: + return Optional.of(ToBigDecimalFromIntPage.create( + componentType, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + case INT64: + return Optional.of(ToBigDecimalFromLongPage.create( + componentType, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + final int encodedSizeInBytes = + (typeName == PrimitiveType.PrimitiveTypeName.BINARY) ? -1 : type.getTypeLength(); + if (BigDecimal.class.equals(componentType)) { + return Optional.of( + ToObjectPage.create( + BigDecimal.class, + new BigDecimalParquetBytesCodec( + decimalLogicalType.getPrecision(), decimalLogicalType.getScale(), + encodedSizeInBytes), + columnChunkReader.getDictionarySupplier())); + } else if (BigInteger.class.equals(componentType)) { + return Optional.of( + ToObjectPage.create( + BigInteger.class, + new BigIntegerParquetBytesCodec(encodedSizeInBytes), + columnChunkReader.getDictionarySupplier())); + } + + // We won't blow up here, Maybe someone will provide us a codec instead. + default: + return Optional.empty(); + } + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java new file mode 100644 index 00000000000..cc3d72276d8 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java @@ -0,0 +1,179 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.iceberg.location; + +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetBuilderSequential; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.impl.locations.TableKey; +import io.deephaven.engine.table.impl.locations.TableLocationState; +import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation; +import io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource; +import io.deephaven.engine.table.impl.sources.regioned.RegionedPageStore; +import io.deephaven.parquet.base.ColumnChunkReader; +import io.deephaven.parquet.base.ParquetFileReader; +import io.deephaven.parquet.base.RowGroupReader; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.ParquetSchemaReader; +import io.deephaven.parquet.table.metadata.ColumnTypeInfo; +import io.deephaven.parquet.table.metadata.GroupingColumnInfo; +import io.deephaven.parquet.table.metadata.TableInfo; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.format.RowGroup; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.jetbrains.annotations.NotNull; + +import java.io.File; +import java.util.*; +import java.util.stream.IntStream; + +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; + +public class IcebergTableLocation extends AbstractTableLocation { + + private static final String IMPLEMENTATION_NAME = IcebergColumnLocation.class.getSimpleName(); + + private final ParquetInstructions readInstructions; + private final ParquetFileReader parquetFileReader; + private final int[] rowGroupIndices; + + private final RowGroup[] rowGroups; + private final RegionedPageStore.Parameters regionParameters; + private final Map parquetColumnNameToPath; + private final Map groupingColumns; + private final Map columnTypes; + private final String version; + + private volatile RowGroupReader[] rowGroupReaders; + + public IcebergTableLocation(@NotNull final TableKey tableKey, + @NotNull final IcebergTableLocationKey tableLocationKey, + @NotNull final ParquetInstructions readInstructions) { + super(tableKey, tableLocationKey, false); + this.readInstructions = readInstructions; + final ParquetMetadata parquetMetadata; + // noinspection SynchronizationOnLocalVariableOrMethodParameter + synchronized (tableLocationKey) { + parquetFileReader = tableLocationKey.getFileReader(); + parquetMetadata = tableLocationKey.getMetadata(); + rowGroupIndices = tableLocationKey.getRowGroupIndices(); + } + + final int rowGroupCount = rowGroupIndices.length; + rowGroups = IntStream.of(rowGroupIndices) + .mapToObj(rgi -> parquetFileReader.fileMetaData.getRow_groups().get(rgi)) + .sorted(Comparator.comparingInt(RowGroup::getOrdinal)) + .toArray(RowGroup[]::new); + final long maxRowCount = Arrays.stream(rowGroups).mapToLong(RowGroup::getNum_rows).max().orElse(0L); + regionParameters = new RegionedPageStore.Parameters( + RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, rowGroupCount, maxRowCount); + + parquetColumnNameToPath = new HashMap<>(); + for (final ColumnDescriptor column : parquetFileReader.getSchema().getColumns()) { + final String[] path = column.getPath(); + if (path.length > 1) { + parquetColumnNameToPath.put(path[0], path); + } + } + + // TODO (https://github.com/deephaven/deephaven-core/issues/958): + // When/if we support _metadata files for Deephaven-written Parquet tables, we may need to revise this + // in order to read *this* file's metadata, rather than inheriting file metadata from the _metadata file. + // Obvious issues included grouping table paths, codecs, etc. + // Presumably, we could store per-file instances of the metadata in the _metadata file's map. + final Optional tableInfo = + ParquetSchemaReader.parseMetadata(parquetMetadata.getFileMetaData().getKeyValueMetaData()); + groupingColumns = tableInfo.map(TableInfo::groupingColumnMap).orElse(Collections.emptyMap()); + columnTypes = tableInfo.map(TableInfo::columnTypeMap).orElse(Collections.emptyMap()); + version = tableInfo.map(TableInfo::version).orElse(null); + + if (!FILE_URI_SCHEME.equals(tableLocationKey.getURI().getScheme())) { + // We do not have the last modified time for non-file URIs + handleUpdate(computeIndex(), TableLocationState.NULL_TIME); + } else { + handleUpdate(computeIndex(), new File(tableLocationKey.getURI()).lastModified()); + } + } + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + @Override + public void refresh() {} + + IcebergTableLocationKey getParquetKey() { + return (IcebergTableLocationKey) getKey(); + } + + ParquetInstructions getReadInstructions() { + return readInstructions; + } + + SeekableChannelsProvider getChannelProvider() { + return parquetFileReader.getChannelsProvider(); + } + + RegionedPageStore.Parameters getRegionParameters() { + return regionParameters; + } + + public Map getGroupingColumns() { + return groupingColumns; + } + + public Map getColumnTypes() { + return columnTypes; + } + + private RowGroupReader[] getRowGroupReaders() { + RowGroupReader[] local; + if ((local = rowGroupReaders) != null) { + return local; + } + synchronized (this) { + if ((local = rowGroupReaders) != null) { + return local; + } + return rowGroupReaders = IntStream.of(rowGroupIndices) + .mapToObj(idx -> parquetFileReader.getRowGroup(idx, version)) + .sorted(Comparator.comparingInt(rgr -> rgr.getRowGroup().getOrdinal())) + .toArray(RowGroupReader[]::new); + } + } + + @NotNull + @Override + protected IcebergColumnLocation makeColumnLocation(@NotNull final String columnName) { + final String parquetColumnName = readInstructions.getParquetColumnNameFromColumnNameOrDefault(columnName); + final String[] columnPath = parquetColumnNameToPath.get(parquetColumnName); + final List nameList = + columnPath == null ? Collections.singletonList(parquetColumnName) : Arrays.asList(columnPath); + final ColumnChunkReader[] columnChunkReaders; + try (final SeekableChannelContext channelContext = getChannelProvider().makeSingleUseContext()) { + columnChunkReaders = Arrays.stream(getRowGroupReaders()) + .map(rgr -> rgr.getColumnChunk(nameList, channelContext)).toArray(ColumnChunkReader[]::new); + } + final boolean exists = Arrays.stream(columnChunkReaders).anyMatch(ccr -> ccr != null && ccr.numRows() > 0); + return new IcebergColumnLocation<>(this, columnName, parquetColumnName, + exists ? columnChunkReaders : null, + exists && groupingColumns.containsKey(parquetColumnName)); + } + + private RowSet computeIndex() { + final RowSetBuilderSequential sequentialBuilder = RowSetFactory.builderSequential(); + + for (int rgi = 0; rgi < rowGroups.length; ++rgi) { + final long subRegionSize = rowGroups[rgi].getNum_rows(); + final long subRegionFirstKey = (long) rgi << regionParameters.regionMaskNumBits; + final long subRegionLastKey = subRegionFirstKey + subRegionSize - 1; + sequentialBuilder.appendRange(subRegionFirstKey, subRegionLastKey); + } + return sequentialBuilder.build(); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java new file mode 100644 index 00000000000..638c75181fb --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java @@ -0,0 +1,43 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.iceberg.location; + +import io.deephaven.engine.table.impl.locations.TableKey; +import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.engine.table.impl.locations.impl.NonexistentTableLocation; +import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.parquet.table.ParquetInstructions; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.File; +import java.net.URI; + +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; + +/** + * {@link TableLocationFactory} for {@link IcebergTableLocation}s. + */ +public final class IcebergTableLocationFactory implements TableLocationFactory { + + private final ParquetInstructions readInstructions; + + public IcebergTableLocationFactory(@NotNull final ParquetInstructions readInstructions) { + this.readInstructions = readInstructions; + } + + @Override + @NotNull + public TableLocation makeLocation(@NotNull final TableKey tableKey, + @NotNull final IcebergTableLocationKey locationKey, + @Nullable final TableDataRefreshService refreshService) { + final URI parquetFileURI = locationKey.getURI(); + if (!FILE_URI_SCHEME.equals(parquetFileURI.getScheme()) || new File(parquetFileURI).exists()) { + return new IcebergTableLocation(tableKey, locationKey, readInstructions); + } else { + return new NonexistentTableLocation(tableKey, locationKey); + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java new file mode 100644 index 00000000000..67783085777 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java @@ -0,0 +1,190 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.iceberg.location; + +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.TableLocationKey; +import io.deephaven.engine.table.impl.locations.local.URITableLocationKey; +import io.deephaven.parquet.base.ParquetFileReader; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.ParquetTableWriter; +import io.deephaven.parquet.table.ParquetTools; +import org.apache.parquet.format.RowGroup; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.stream.IntStream; + +/** + * {@link TableLocationKey} implementation for use with data stored in the parquet format. + */ +public class IcebergTableLocationKey extends URITableLocationKey { + + private static final String IMPLEMENTATION_NAME = IcebergTableLocationKey.class.getSimpleName(); + + private ParquetFileReader fileReader; + private ParquetMetadata metadata; + private int[] rowGroupIndices; + private final ParquetInstructions readInstructions; + + /** + * Construct a new IcebergTableLocationKey for the supplied {@code fileUri} and {@code partitions}. + * + * @param fileUri The file that backs the keyed location. + * @param order Explicit ordering index, taking precedence over other fields + * @param partitions The table partitions enclosing the table location keyed by {@code this}. Note that if this + * parameter is {@code null}, the location will be a member of no partitions. An ordered copy of the map will + * be made, so the calling code is free to mutate the map after this call + * @param readInstructions the instructions for customizations while reading + */ + public IcebergTableLocationKey( + @NotNull final URI fileUri, + final int order, + @Nullable final Map> partitions, + @NotNull final ParquetInstructions readInstructions) { + super(validateIcebergDataFile(fileUri), order, partitions); + this.readInstructions = readInstructions; + } + + private static URI validateIcebergDataFile(@NotNull final URI fileUri) { + // TODO: when/if ORC format supported, need to accept and validate it. + + // TODO: this file extension check is probably not needed (assuming we read the path from the manifest file) + // correctly). + + if (!fileUri.getRawPath().endsWith(ParquetTableWriter.PARQUET_FILE_EXTENSION)) { + throw new IllegalArgumentException("Iceberg data file must end in " + ParquetTableWriter.PARQUET_FILE_EXTENSION); + } + return fileUri; + } + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + /** + * Returns {@code true} if a previous {@link ParquetFileReader} has been created, or if one was successfully created + * on-demand. + * + *

+ * When {@code false}, this may mean that the file: + *

    + *
  1. does not exist, or is otherwise inaccessible
  2. + *
  3. is in the process of being written, and is not yet a valid parquet file
  4. + *
  5. is _not_ a parquet file
  6. + *
  7. is a corrupt parquet file
  8. + *
+ * + * Callers wishing to handle these cases more explicit may call + * {@link ParquetTools#getParquetFileReaderChecked(URI, ParquetInstructions)}. + * + * @return true if the file reader exists or was successfully created + */ + public synchronized boolean verifyFileReader() { + if (fileReader != null) { + return true; + } + try { + fileReader = ParquetTools.getParquetFileReaderChecked(uri, readInstructions); + } catch (IOException e) { + return false; + } + return true; + } + + /** + * Get a previously-{@link #setFileReader(ParquetFileReader) set} or on-demand created {@link ParquetFileReader} for + * this location key's {@code file}. + * + * @return A {@link ParquetFileReader} for this location key's {@code file}. + */ + public synchronized ParquetFileReader getFileReader() { + if (fileReader != null) { + return fileReader; + } + return fileReader = ParquetTools.getParquetFileReader(uri, readInstructions); + } + + /** + * Set the {@link ParquetFileReader} that will be returned by {@link #getFileReader()}. Pass {@code null} to force + * on-demand construction at the next invocation. Always clears cached {@link ParquetMetadata} and {@link RowGroup} + * indices. + * + * @param fileReader The new {@link ParquetFileReader} + */ + public synchronized void setFileReader(final ParquetFileReader fileReader) { + this.fileReader = fileReader; + this.metadata = null; + this.rowGroupIndices = null; + } + + /** + * Get a previously-{@link #setMetadata(ParquetMetadata) set} or on-demand created {@link ParquetMetadata} for this + * location key's {@code file}. + * + * @return A {@link ParquetMetadata} for this location key's {@code file}. + */ + public synchronized ParquetMetadata getMetadata() { + if (metadata != null) { + return metadata; + } + try { + return metadata = new ParquetMetadataConverter().fromParquetMetadata(getFileReader().fileMetaData); + } catch (IOException e) { + throw new TableDataException("Failed to convert Parquet file metadata: " + getURI(), e); + } + } + + /** + * Set the {@link ParquetMetadata} that will be returned by {@link #getMetadata()} ()}. Pass {@code null} to force + * on-demand construction at the next invocation. + * + * @param metadata The new {@link ParquetMetadata} + */ + public synchronized void setMetadata(final ParquetMetadata metadata) { + this.metadata = metadata; + } + + /** + * Get previously-{@link #setRowGroupIndices(int[]) set} or on-demand created {@link RowGroup} indices for this + * location key's current {@link ParquetFileReader}. + * + * @return {@link RowGroup} indices for this location key's current {@link ParquetFileReader}. + */ + public synchronized int[] getRowGroupIndices() { + if (rowGroupIndices != null) { + return rowGroupIndices; + } + final List rowGroups = getFileReader().fileMetaData.getRow_groups(); + return rowGroupIndices = IntStream.range(0, rowGroups.size()).filter(rgi -> { + // 1. We can safely assume there's always at least one column. Our tools will refuse to write a + // column-less table, and other readers we've tested fail catastrophically. + // 2. null file path means the column is local to the file the metadata was read from (which had + // better be this file, in that case). + // 3. We're assuming row groups are contained within a single file. + // While it seems that row group *could* have column chunks splayed out into multiple files, + // we're not expecting that in this code path. To support it, discovery tools should figure out + // the row groups for a partition themselves and call setRowGroupReaders. + final String filePath = rowGroups.get(rgi).getColumns().get(0).getFile_path(); + return filePath == null || new File(filePath).getAbsoluteFile().toURI().equals(uri); + }).toArray(); + } + + /** + * Set the {@link RowGroup} indices that will be returned by {@link #getRowGroupIndices()} + * + * @param rowGroupIndices The new {@link RowGroup} indices + */ + public synchronized void setRowGroupIndices(final int[] rowGroupIndices) { + this.rowGroupIndices = rowGroupIndices; + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java new file mode 100644 index 00000000000..6a9b4bc2acb --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java @@ -0,0 +1,250 @@ +package io.deephaven.iceberg.util; + +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.PartitionAwareSourceTable; +import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder; +import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; +import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; +import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +import io.deephaven.extensions.s3.Credentials; +import io.deephaven.extensions.s3.S3Instructions; +import io.deephaven.iceberg.layout.IcebergFlatLayout; +import io.deephaven.iceberg.layout.IcebergPartitionedLayout; +import io.deephaven.iceberg.location.IcebergTableLocationFactory; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import io.deephaven.parquet.table.ParquetInstructions; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.*; +import org.apache.iceberg.aws.AwsClientProperties; +import org.apache.iceberg.aws.s3.S3FileIOProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.rest.RESTCatalog; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class IcebergCatalog { + private final IcebergInstructions instructions; + private final Catalog catalog; + private final FileIO fileIO; + + private final S3Instructions s3Instructions; + + /** + * Construct an IcebergCatalog given a set of configurable instructions.. + * + * @param name The optional service name + */ + IcebergCatalog(final @Nullable String name, final IcebergInstructions instructions) { + this.instructions = instructions; + + // Set up the properties map for the Iceberg catalog + Map properties = new HashMap<>(); + + final Configuration conf = new Configuration(); + + properties.put(CatalogProperties.CATALOG_IMPL, instructions.catalogImpl().value); + if (instructions.catalogImpl() == IcebergInstructions.CATALOG_IMPL.RESTCatalog) { + final RESTCatalog restCatalog = new RESTCatalog(); + restCatalog.setConf(conf); + catalog = restCatalog; + } else { + throw new UnsupportedOperationException( + "Unsupported catalog implementation: " + instructions.catalogImpl()); + } + + properties.put(CatalogProperties.URI, instructions.catalogURI()); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, instructions.warehouseLocation()); + + properties.put(CatalogProperties.FILE_IO_IMPL, instructions.fileIOImpl().value); + if (instructions.fileIOImpl() == IcebergInstructions.FILEIO_IMPL.S3) { + final S3Instructions.Builder builder = S3Instructions.builder() + .credentials(Credentials.basic(instructions.s3AccessKeyId(), instructions.s3SecretAccessKey())) + .regionName(instructions.s3Region()); + + properties.put(AwsClientProperties.CLIENT_REGION, instructions.s3Region()); + properties.put(S3FileIOProperties.ACCESS_KEY_ID, instructions.s3AccessKeyId()); + properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, instructions.s3SecretAccessKey()); + if (instructions.s3EndpointOverride().isPresent()) { + properties.put(S3FileIOProperties.ENDPOINT, instructions.s3EndpointOverride().get()); + builder.endpointOverride(instructions.s3EndpointOverride().get()); + } + s3Instructions = builder.build(); + // TODO: create a FileIO interface wrapping the Deephaven S3SeekableByteChannel/Provider + fileIO = CatalogUtil.loadFileIO(instructions.fileIOImpl().value, properties, conf); + } else { + throw new UnsupportedOperationException("Unsupported file IO implementation: " + instructions.fileIOImpl()); + } + + final String catalogName = name != null ? name : "IcebergTableDataService-" + instructions.catalogURI(); + catalog.initialize(catalogName, properties); + } + + public List listTables(final Namespace namespace) { + // TODO: have this return a Deephaven Table of table identifiers + return catalog.listTables(namespace); + } + + /** + * Read a static snapshot of a table from the Iceberg catalog. + * + * @param tableIdentifier The table identifier to load + * @param snapshotId The snapshot ID to load + * @return The loaded table + */ + public Table readTable( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final String snapshotId) { + return readTableInternal(tableIdentifier, snapshotId, false); + } + + /** + * Read the latest static snapshot of a table from the Iceberg catalog. + * + * @param tableIdentifier The table identifier to load + * @return The loaded table + */ + public Table readTable(@NotNull final TableIdentifier tableIdentifier) { + return readTableInternal(tableIdentifier, null, false); + } + + /** + * Subscribe to a table from the Iceberg catalog. Intially the latest snapshot will be loaded, but the output table + * will be updated as new snapshots are added to the table. + * + * @param tableIdentifier The table identifier to load + * @return The loaded table + */ + public Table subscribeTable(@NotNull final TableIdentifier tableIdentifier) { + return readTableInternal(tableIdentifier, null, true); + } + + private static KnownLocationKeyFinder toKnownKeys( + TableLocationKeyFinder keyFinder) { + return keyFinder instanceof KnownLocationKeyFinder + ? (KnownLocationKeyFinder) keyFinder + : KnownLocationKeyFinder.copyFrom(keyFinder, Comparator.naturalOrder()); + } + + private Table readTableInternal( + @NotNull final TableIdentifier tableIdentifier, + @Nullable final String snapshotId, + final boolean isRefreshing) { + // Validate that the user is not trying to subscribe to a snapshot. + Assert.eqFalse(isRefreshing && (snapshotId != null), "Must not specify a snapshot ID when subscribing to a table."); + + // Load the table from the catalog + final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); + + final Snapshot snapshot; + final Schema schema; + // Do we want the latest or a specific snapshot? + if (snapshotId == null) { + snapshot = table.currentSnapshot(); + schema = table.schema(); + } else { + // Load the specific snapshot and retrieve the schema for that snapshot + snapshot = table.snapshot(snapshotId); + schema = table.schemas().get(snapshot.schemaId()); + } + + // Load the partitioning schema + final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); + + // Convert the Iceberg schema to a Deephaven TableDefinition + final TableDefinition tableDefinition = IcebergTools.fromSchema(schema, partitionSpec); + + // Build a parquet instructions object + final ParquetInstructions instructions = ParquetInstructions.builder() + .setSpecialInstructions(s3Instructions) + .build(); + + final String description; + final TableLocationKeyFinder keyFinder; + final TableDataRefreshService refreshService; + final UpdateSourceRegistrar updateSourceRegistrar; + + if (partitionSpec.isUnpartitioned()) { + // Create the flat layout location key finder + final TableLocationKeyFinder locationKeyFinder + = new IcebergFlatLayout(snapshot, fileIO, instructions); + + if (isRefreshing) { + keyFinder = locationKeyFinder; + description = "Read refreshing iceberg table with " + keyFinder; + refreshService = TableDataRefreshService.getSharedRefreshService(); + updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + } else { + keyFinder = toKnownKeys(locationKeyFinder); + description = "Read static iceberg table with " + keyFinder; + refreshService = null; + updateSourceRegistrar = null; + } + + return new PartitionAwareSourceTable( + tableDefinition, + description, + RegionedTableComponentFactoryImpl.INSTANCE, + new PollingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(instructions), + refreshService), + updateSourceRegistrar); + } else { + final String[] partitionColumns = + partitionSpec.fields().stream().map(PartitionField::name).toArray(String[]::new); + + // Create the partitioning column location key finder + final TableLocationKeyFinder locationKeyFinder = new IcebergPartitionedLayout( + snapshot, + fileIO, + partitionColumns, + instructions); + + if (isRefreshing) { + keyFinder = locationKeyFinder; + description = "Read refreshing iceberg table with " + keyFinder; + refreshService = TableDataRefreshService.getSharedRefreshService(); + updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + } else { + keyFinder = toKnownKeys(locationKeyFinder); + description = "Read static iceberg table with " + keyFinder; + refreshService = null; + updateSourceRegistrar = null; + } + + return new PartitionAwareSourceTable( + tableDefinition, + description, + RegionedTableComponentFactoryImpl.INSTANCE, + new PollingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(instructions), + refreshService), + updateSourceRegistrar); + } + } + + + /** + * Return the internal Iceberg catalog. + */ + public Catalog catalog() { + return catalog; + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java new file mode 100644 index 00000000000..f7961e3e4db --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -0,0 +1,145 @@ +/** + * Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.iceberg.util; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value.Check; +import org.immutables.value.Value.Default; +import org.immutables.value.Value.Immutable; + +import java.util.Optional; + +/** + * This class provides instructions intended for reading Iceberg catalogs and tables. The default values documented in + * this class may change in the future. As such, callers may wish to explicitly set the values. + */ +@Immutable +@BuildableStyle +public abstract class IcebergInstructions { + public static Builder builder() { + return ImmutableIcebergInstructions.builder(); + } + + public enum CATALOG_IMPL { + RESTCatalog("org.apache.iceberg.rest.RESTCatalog"); + + public final String value; + + CATALOG_IMPL(String label) { + this.value = label; + } + } + + public enum FILEIO_IMPL { + S3("org.apache.iceberg.aws.s3.S3FileIO"); + + public final String value; + + FILEIO_IMPL(String label) { + this.value = label; + } + } + + private final static CATALOG_IMPL DEFAULT_CATALOG_IMPL = CATALOG_IMPL.RESTCatalog; + private final static FILEIO_IMPL DEFAULT_FILEIO_IMPL = FILEIO_IMPL.S3; + + /** + * A custom Catalog implementation to use by an engine. + */ + @Default + public CATALOG_IMPL catalogImpl() { + return DEFAULT_CATALOG_IMPL; + } + + /** + * The URI string for the catalog. + */ + public abstract String catalogURI(); + + /** + * The root path of the data warehouse, where the manifest and data files are stored. + */ + public abstract String warehouseLocation(); + + /** + * The custom FileIO implementation to use in the catalog. + */ + @Default + public FILEIO_IMPL fileIOImpl() { + return DEFAULT_FILEIO_IMPL; + } + + /** + * The endpoint to connect to. Callers connecting to AWS do not typically need to set this; it is most useful when + * connecting to non-AWS, S3-compatible APIs. + * + * @see Amazon Simple Storage Service endpoints + */ + public abstract Optional s3EndpointOverride(); + + /** + * The AWS access key, used to identify the user interacting with services. + */ + public abstract String s3AccessKeyId(); + + /** + * The AWS secret access key, used to authenticate the user interacting with services. + */ + public abstract String s3SecretAccessKey(); + + /** + * The AWS region to use for this connection. + */ + public abstract String s3Region(); + + public interface Builder { + Builder catalogImpl(CATALOG_IMPL catalogImpl); + + Builder catalogURI(String catalogURI); + + Builder warehouseLocation(String warehouseLocation); + + Builder fileIOImpl(FILEIO_IMPL fileIOImpl); + + Builder s3EndpointOverride(String s3EndpointOverride); + + Builder s3AccessKeyId(String s3AccessKeyId); + + Builder s3SecretAccessKey(String s3SecretAccessKey); + + Builder s3Region(String s3Region); + + IcebergInstructions build(); + } + + @Check + final void checkCatalogURI() { + if (catalogURI() == null || catalogURI().isEmpty()) { + throw new IllegalArgumentException("catalogURI must be provided"); + } + } + + @Check + final void checkWarehouseLocation() { + if (warehouseLocation() == null || warehouseLocation().isEmpty()) { + throw new IllegalArgumentException("warehouseLocation must be provided"); + } + } + + @Check + final void checkS3Fields() { + if (fileIOImpl() == FILEIO_IMPL.S3) { + if (s3AccessKeyId() == null || s3AccessKeyId().isEmpty()) { + throw new IllegalArgumentException("When using S3 FileIO, s3AccessKeyId must be provided"); + } + if (s3SecretAccessKey() == null || s3SecretAccessKey().isEmpty()) { + throw new IllegalArgumentException("When using S3 FileIO, s3SecretAccessKey must be provided"); + } + if (s3Region() == null || s3Region().isEmpty()) { + throw new IllegalArgumentException("When using S3 FileIO, s3Region must be provided"); + } + } + } + +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java new file mode 100644 index 00000000000..f28bb59fcbf --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java @@ -0,0 +1,83 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.iceberg.util; + +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.TableDefinition; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +import java.time.LocalDateTime; +import java.util.*; +import java.util.stream.Collectors; + +/** + * Tools for managing and manipulating tables on disk in parquet format. + */ +public class IcebergTools { + + public static IcebergCatalog createCatalog(final String name, final IcebergInstructions instructions) { + return new IcebergCatalog(name, instructions); + } + + public static TableDefinition fromSchema(final Schema schema, PartitionSpec partitionSpec) { + final Set partitionNames = + partitionSpec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); + + final List> columns = new ArrayList<>(); + + for (final Types.NestedField field : schema.columns()) { + final String name = field.name(); + final Type type = field.type(); + final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); + final ColumnDefinition column; + if (partitionNames.contains(name)) { + column = ColumnDefinition.of(name, qstType).withPartitioning(); + } else { + column = ColumnDefinition.of(name, qstType); + } + columns.add(column); + } + + return TableDefinition.of(columns); + } + + public static io.deephaven.qst.type.Type convertPrimitiveType(final Type icebergType) { + final Type.TypeID typeId = icebergType.typeId(); + if (icebergType.isPrimitiveType()) { + if (typeId == Type.TypeID.BINARY || typeId == Type.TypeID.BOOLEAN) { + return io.deephaven.qst.type.Type.booleanType(); + } else if (typeId == Type.TypeID.DOUBLE) { + return io.deephaven.qst.type.Type.doubleType(); + } else if (typeId == Type.TypeID.FLOAT) { + return io.deephaven.qst.type.Type.floatType(); + } else if (typeId == Type.TypeID.INTEGER) { + return io.deephaven.qst.type.Type.intType(); + } else if (typeId == Type.TypeID.LONG) { + return io.deephaven.qst.type.Type.longType(); + } else if (typeId == Type.TypeID.STRING) { + return io.deephaven.qst.type.Type.stringType(); + } else if (typeId == Type.TypeID.TIMESTAMP) { + final Types.TimestampType timestampType = (Types.TimestampType) icebergType; + return timestampType.shouldAdjustToUTC() + ? io.deephaven.qst.type.Type.instantType() + : io.deephaven.qst.type.Type.ofCustom(LocalDateTime.class); + } else if (typeId == Type.TypeID.DATE) { + // Not sure what type for this + return io.deephaven.qst.type.Type.intType(); + } else if (typeId == Type.TypeID.DECIMAL) { + // BigDecimal?? + return io.deephaven.qst.type.Type.doubleType(); + } else { + throw new UnsupportedOperationException("Unsupported type: " + typeId); + } + } + throw new UnsupportedOperationException("Unsupported type: " + typeId); + } + + private IcebergTools() {} +} diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java new file mode 100644 index 00000000000..13be402d406 --- /dev/null +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -0,0 +1,54 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.iceberg; + +import io.deephaven.engine.util.TableTools; +import io.deephaven.iceberg.util.IcebergCatalog; +import io.deephaven.iceberg.util.IcebergInstructions; +import io.deephaven.iceberg.util.IcebergTools; +import io.deephaven.time.DateTimeUtils; +import junit.framework.TestCase; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.Test; + +import java.util.List; + +public class IcebergToolsTest extends TestCase { + IcebergInstructions instructions; + + @Override + public void setUp() { + instructions = IcebergInstructions.builder() + .catalogImpl(IcebergInstructions.CATALOG_IMPL.RESTCatalog) + .catalogURI("http://rest:8181") + .warehouseLocation("s3a://warehouse/wh") + .fileIOImpl(IcebergInstructions.FILEIO_IMPL.S3) + .s3AccessKeyId("admin") + .s3SecretAccessKey("password") + .s3EndpointOverride("http://minio:9000") + .s3Region("us-east-1") + .build(); + } + + @Test + public void testListTables() { + final IcebergCatalog catalog = IcebergTools.createCatalog("minio-iceberg", instructions); + + final Namespace ns = Namespace.of("nyc"); + final List tables = catalog.listTables(ns); + } + + @Test + public void testOpenTable() { + final IcebergCatalog catalog = IcebergTools.createCatalog("minio-iceberg", instructions); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + io.deephaven.engine.table.Table table = catalog.readTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + +} diff --git a/server/build.gradle b/server/build.gradle index 539b7996ea4..c92c6f676ae 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -9,6 +9,7 @@ dependencies { implementation project(':engine-table') implementation project(':extensions-csv') implementation project(':extensions-arrow') + implementation project(':extensions-iceberg') implementation project(':extensions-parquet-table') implementation project(':extensions-performance') implementation project(':extensions-jdbc') diff --git a/settings.gradle b/settings.gradle index 52a854f3027..c9a2ce302b5 100644 --- a/settings.gradle +++ b/settings.gradle @@ -266,6 +266,9 @@ project(':extensions-trackedfile').projectDir = file('extensions/trackedfile') include(':extensions-s3') project(':extensions-s3').projectDir = file('extensions/s3') +include(':extensions-iceberg') +project(':extensions-iceberg').projectDir = file('extensions/iceberg') + include(':plugin') include(':plugin-dagger') From 780f2b1fd73f534463211049314d40387cdf3a40 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 22 Mar 2024 09:14:33 -0700 Subject: [PATCH 02/25] Ready for auto-spotless anytime! --- .../io/deephaven/iceberg/layout/IcebergFlatLayout.java | 3 ++- .../deephaven/iceberg/location/IcebergColumnLocation.java | 8 ++++---- .../deephaven/iceberg/location/IcebergTableLocation.java | 4 ++-- .../iceberg/location/IcebergTableLocationKey.java | 5 +++-- .../java/io/deephaven/iceberg/util/IcebergCatalog.java | 7 ++++--- 5 files changed, 15 insertions(+), 12 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java index 5fb2d1b0373..072a5c2d35d 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -28,7 +28,8 @@ public final class IcebergFlatLayout implements TableLocationKeyFinder cache; private final ParquetInstructions readInstructions; - private static IcebergTableLocationKey locationKey(final URI fileUri, @NotNull final ParquetInstructions readInstructions) { + private static IcebergTableLocationKey locationKey(final URI fileUri, + @NotNull final ParquetInstructions readInstructions) { return new IcebergTableLocationKey(fileUri, 0, null, readInstructions); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java index 036edcf1278..4f1979bcb05 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java @@ -101,10 +101,10 @@ final class IcebergColumnLocation extends AbstractColumnLoc * @param hasGroupingTable Whether this column has an associated grouping table file */ IcebergColumnLocation(@NotNull final IcebergTableLocation tableLocation, - @NotNull final String columnName, - @NotNull final String parquetColumnName, - @Nullable final ColumnChunkReader[] columnChunkReaders, - final boolean hasGroupingTable) { + @NotNull final String columnName, + @NotNull final String parquetColumnName, + @Nullable final ColumnChunkReader[] columnChunkReaders, + final boolean hasGroupingTable) { super(tableLocation, columnName); this.parquetColumnName = parquetColumnName; this.columnChunkReaders = columnChunkReaders; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java index cc3d72276d8..36a6c1d39fe 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java @@ -51,8 +51,8 @@ public class IcebergTableLocation extends AbstractTableLocation { private volatile RowGroupReader[] rowGroupReaders; public IcebergTableLocation(@NotNull final TableKey tableKey, - @NotNull final IcebergTableLocationKey tableLocationKey, - @NotNull final ParquetInstructions readInstructions) { + @NotNull final IcebergTableLocationKey tableLocationKey, + @NotNull final ParquetInstructions readInstructions) { super(tableKey, tableLocationKey, false); this.readInstructions = readInstructions; final ParquetMetadata parquetMetadata; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java index 67783085777..e5546cf2638 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java @@ -58,10 +58,11 @@ private static URI validateIcebergDataFile(@NotNull final URI fileUri) { // TODO: when/if ORC format supported, need to accept and validate it. // TODO: this file extension check is probably not needed (assuming we read the path from the manifest file) - // correctly). + // correctly). if (!fileUri.getRawPath().endsWith(ParquetTableWriter.PARQUET_FILE_EXTENSION)) { - throw new IllegalArgumentException("Iceberg data file must end in " + ParquetTableWriter.PARQUET_FILE_EXTENSION); + throw new IllegalArgumentException( + "Iceberg data file must end in " + ParquetTableWriter.PARQUET_FILE_EXTENSION); } return fileUri; } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java index 6a9b4bc2acb..a9977c13927 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java @@ -144,7 +144,8 @@ private Table readTableInternal( @Nullable final String snapshotId, final boolean isRefreshing) { // Validate that the user is not trying to subscribe to a snapshot. - Assert.eqFalse(isRefreshing && (snapshotId != null), "Must not specify a snapshot ID when subscribing to a table."); + Assert.eqFalse(isRefreshing && (snapshotId != null), + "Must not specify a snapshot ID when subscribing to a table."); // Load the table from the catalog final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); @@ -179,8 +180,8 @@ private Table readTableInternal( if (partitionSpec.isUnpartitioned()) { // Create the flat layout location key finder - final TableLocationKeyFinder locationKeyFinder - = new IcebergFlatLayout(snapshot, fileIO, instructions); + final TableLocationKeyFinder locationKeyFinder = + new IcebergFlatLayout(snapshot, fileIO, instructions); if (isRefreshing) { keyFinder = locationKeyFinder; From 585a26e8a856add3d86c194aef76ee7af8b94218 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 2 Apr 2024 10:27:36 -0700 Subject: [PATCH 03/25] Refactored to use internal parquet objects instead of re-implementing functionality. --- .../iceberg/layout/IcebergFlatLayout.java | 16 +- .../layout/IcebergPartitionedLayout.java | 13 +- .../location/IcebergColumnLocation.java | 822 ------------------ .../location/IcebergTableLocation.java | 185 ++-- .../location/IcebergTableLocationFactory.java | 6 +- .../location/IcebergTableLocationKey.java | 150 +--- .../iceberg/util/IcebergCatalog.java | 89 +- .../deephaven/iceberg/util/IcebergTools.java | 46 +- .../IcebergToolsTest.java | 48 +- 9 files changed, 175 insertions(+), 1200 deletions(-) delete mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java index 072a5c2d35d..207e94665a8 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -6,9 +6,7 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; -import io.deephaven.parquet.table.ParquetInstructions; import org.apache.iceberg.*; -import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.FileIO; import org.jetbrains.annotations.NotNull; @@ -26,11 +24,13 @@ public final class IcebergFlatLayout implements TableLocationKeyFinder cache; - private final ParquetInstructions readInstructions; + private final Object readInstructions; - private static IcebergTableLocationKey locationKey(final URI fileUri, - @NotNull final ParquetInstructions readInstructions) { - return new IcebergTableLocationKey(fileUri, 0, null, readInstructions); + private static IcebergTableLocationKey locationKey( + final FileFormat format, + final URI fileUri, + @NotNull final Object readInstructions) { + return new IcebergTableLocationKey(format, fileUri, 0, null, readInstructions); } /** @@ -40,7 +40,7 @@ private static IcebergTableLocationKey locationKey(final URI fileUri, public IcebergFlatLayout( @NotNull final Snapshot tableSnapshot, @NotNull final FileIO fileIO, - @NotNull final ParquetInstructions readInstructions) { + @NotNull final Object readInstructions) { this.tableSnapshot = tableSnapshot; this.fileIO = fileIO; this.readInstructions = readInstructions; @@ -63,7 +63,7 @@ public synchronized void findKeys(@NotNull final Consumer cache; private static IcebergTableLocationKey locationKey( + final FileFormat format, final URI fileUri, final Map> partitions, - @NotNull final ParquetInstructions readInstructions) { - return new IcebergTableLocationKey(fileUri, 0, partitions, readInstructions); + @NotNull final Object readInstructions) { + return new IcebergTableLocationKey(format, fileUri, 0, partitions, readInstructions); } /** @@ -47,7 +46,7 @@ public IcebergPartitionedLayout( @NotNull final Snapshot tableSnapshot, @NotNull final FileIO fileIO, @NotNull final String[] partitionColumns, - @NotNull final ParquetInstructions readInstructions) { + @NotNull final Object readInstructions) { this.tableSnapshot = tableSnapshot; this.fileIO = fileIO; this.partitionColumns = partitionColumns; @@ -77,7 +76,7 @@ public synchronized void findKeys(@NotNull final Consumer) partitionData.get(ii)); } - locationKey = locationKey(fileUri, partitions, readInstructions); + locationKey = locationKey(df.format(), fileUri, partitions, readInstructions); if (!locationKey.verifyFileReader()) { continue; } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java deleted file mode 100644 index 4f1979bcb05..00000000000 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergColumnLocation.java +++ /dev/null @@ -1,822 +0,0 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ -package io.deephaven.iceberg.location; - -import io.deephaven.base.verify.Assert; -import io.deephaven.base.verify.Require; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.Any; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.configuration.Configuration; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSequenceFactory; -import io.deephaven.engine.rowset.chunkattributes.UnorderedRowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.Context; -import io.deephaven.engine.table.impl.CodecLookup; -import io.deephaven.engine.table.impl.chunkattributes.DictionaryKeys; -import io.deephaven.engine.table.impl.chunkboxer.ChunkBoxer; -import io.deephaven.engine.table.impl.locations.TableDataException; -import io.deephaven.engine.table.impl.locations.impl.AbstractColumnLocation; -import io.deephaven.engine.table.impl.sources.regioned.*; -import io.deephaven.internal.log.LoggerFactory; -import io.deephaven.io.logger.Logger; -import io.deephaven.parquet.base.ColumnChunkReader; -import io.deephaven.parquet.base.ParquetFileReader; -import io.deephaven.parquet.base.RowGroupReader; -import io.deephaven.parquet.table.*; -import io.deephaven.parquet.table.metadata.CodecInfo; -import io.deephaven.parquet.table.metadata.ColumnTypeInfo; -import io.deephaven.parquet.table.metadata.GroupingColumnInfo; -import io.deephaven.parquet.table.metadata.TableInfo; -import io.deephaven.parquet.table.pagestore.ColumnChunkPageStore; -import io.deephaven.parquet.table.pagestore.PageCache; -import io.deephaven.parquet.table.pagestore.topage.*; -import io.deephaven.parquet.table.region.*; -import io.deephaven.util.channel.SeekableChannelContext; -import io.deephaven.util.channel.SeekableChannelsProvider; -import io.deephaven.util.codec.CodecCache; -import io.deephaven.util.codec.ObjectCodec; -import io.deephaven.util.codec.SimpleByteArrayCodec; -import io.deephaven.vector.Vector; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import org.apache.parquet.schema.LogicalTypeAnnotation; -import org.apache.parquet.schema.PrimitiveType; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.io.File; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.net.URI; -import java.util.*; -import java.util.function.Function; -import java.util.function.LongFunction; -import java.util.function.Supplier; -import java.util.stream.IntStream; -import java.util.stream.Stream; - -import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK; -import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; -import static io.deephaven.parquet.table.ParquetTableWriter.*; - -final class IcebergColumnLocation extends AbstractColumnLocation { - - private static final String IMPLEMENTATION_NAME = IcebergColumnLocation.class.getSimpleName(); - - private static final int CHUNK_SIZE = Configuration.getInstance() - .getIntegerForClassWithDefault(IcebergColumnLocation.class, "chunkSize", 4096); - private static final int INITIAL_PAGE_CACHE_SIZE = Configuration.getInstance() - .getIntegerForClassWithDefault(IcebergColumnLocation.class, "initialPageCacheSize", 128); - private static final int MAX_PAGE_CACHE_SIZE = Configuration.getInstance() - .getIntegerForClassWithDefault(IcebergColumnLocation.class, "maxPageCacheSize", 8192); - - private static final Logger log = LoggerFactory.getLogger(IcebergColumnLocation.class); - - private final String parquetColumnName; - /** - * Factory object needed for deferred initialization of the remaining fields. Reference serves as a barrier to - * ensure visibility of the derived fields. - */ - private volatile ColumnChunkReader[] columnChunkReaders; - private final boolean hasGroupingTable; - - // We should consider moving this to column level if needed. Column-location level likely allows more parallelism. - private volatile PageCache pageCache; - - private ColumnChunkPageStore[] pageStores; - private Supplier>[] dictionaryChunkSuppliers; - private ColumnChunkPageStore[] dictionaryKeysPageStores; - - /** - * Construct a new {@link IcebergColumnLocation} for the specified {@link IcebergTableLocation} and column name. - * - * @param tableLocation The table location enclosing this column location - * @param parquetColumnName The Parquet file column name - * @param columnChunkReaders The {@link ColumnChunkReader column chunk readers} for this location - * @param hasGroupingTable Whether this column has an associated grouping table file - */ - IcebergColumnLocation(@NotNull final IcebergTableLocation tableLocation, - @NotNull final String columnName, - @NotNull final String parquetColumnName, - @Nullable final ColumnChunkReader[] columnChunkReaders, - final boolean hasGroupingTable) { - super(tableLocation, columnName); - this.parquetColumnName = parquetColumnName; - this.columnChunkReaders = columnChunkReaders; - this.hasGroupingTable = hasGroupingTable; - } - - private PageCache ensurePageCache() { - PageCache localPageCache; - if ((localPageCache = pageCache) != null) { - return localPageCache; - } - - synchronized (this) { - if ((localPageCache = pageCache) != null) { - return localPageCache; - } - return pageCache = new PageCache<>(INITIAL_PAGE_CACHE_SIZE, MAX_PAGE_CACHE_SIZE); - } - } - - // ----------------------------------------------------------------------------------------------------------------- - // AbstractColumnLocation implementation - // ----------------------------------------------------------------------------------------------------------------- - - @Override - public String getImplementationName() { - return IMPLEMENTATION_NAME; - } - - @Override - public boolean exists() { - // If we see a null columnChunkReaders array, either we don't exist or we are guaranteed to - // see a non-null - // pageStores array - return columnChunkReaders != null || pageStores != null; - } - - private IcebergTableLocation tl() { - return (IcebergTableLocation) getTableLocation(); - } - - private static final ColumnDefinition FIRST_KEY_COL_DEF = - ColumnDefinition.ofLong("__firstKey__"); - private static final ColumnDefinition LAST_KEY_COL_DEF = - ColumnDefinition.ofLong("__lastKey__"); - - /** - * Helper method for logging a warning on failure in reading an index file - */ - private void logWarnFailedToRead(final String indexFilePath) { - log.warn().append("Failed to read expected index file ").append(indexFilePath) - .append(" for table location ").append(tl()).append(", column ") - .append(getName()) - .endl(); - } - - @Override - @Nullable - public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition columnDefinition) { - if (!hasGroupingTable) { - return null; - } - final URI parquetFileURI = tl().getParquetKey().getURI(); - Assert.assertion(FILE_URI_SCHEME.equals(parquetFileURI.getScheme()), - "Expected a file uri, got " + parquetFileURI); - final File parquetFile = new File(parquetFileURI); - try { - ParquetFileReader parquetFileReader; - final String indexFilePath; - final GroupingColumnInfo groupingColumnInfo = tl().getGroupingColumns().get(parquetColumnName); - final SeekableChannelsProvider channelsProvider = tl().getChannelProvider(); - if (groupingColumnInfo != null) { - final String indexFileRelativePath = groupingColumnInfo.groupingTablePath(); - indexFilePath = parquetFile.toPath().getParent().resolve(indexFileRelativePath).toString(); - try { - parquetFileReader = new ParquetFileReader(indexFilePath, channelsProvider); - } catch (final RuntimeException e) { - logWarnFailedToRead(indexFilePath); - return null; - } - } else { - final String relativeIndexFilePath = - ParquetTools.getRelativeIndexFilePath(parquetFile, parquetColumnName); - indexFilePath = parquetFile.toPath().getParent().resolve(relativeIndexFilePath).toString(); - try { - parquetFileReader = new ParquetFileReader(indexFilePath, channelsProvider); - } catch (final RuntimeException e1) { - // Retry with legacy grouping file path - final String legacyGroupingFileName = - ParquetTools.legacyGroupingFileName(parquetFile, parquetColumnName); - final File legacyGroupingFile = new File(parquetFile.getParent(), legacyGroupingFileName); - try { - parquetFileReader = - new ParquetFileReader(legacyGroupingFile.getAbsolutePath(), channelsProvider); - } catch (final RuntimeException e2) { - logWarnFailedToRead(indexFilePath); - return null; - } - } - } - final Optional tableInfo = ParquetSchemaReader.parseMetadata( - new ParquetMetadataConverter().fromParquetMetadata(parquetFileReader.fileMetaData) - .getFileMetaData().getKeyValueMetaData()); - final Map columnTypes = - tableInfo.map(TableInfo::columnTypeMap).orElse(Collections.emptyMap()); - final String version = tableInfo.map(TableInfo::version).orElse(null); - - final RowGroupReader rowGroupReader = parquetFileReader.getRowGroup(0, version); - final ColumnChunkReader groupingKeyReader, beginPosReader, endPosReader; - try (final SeekableChannelContext channelContext = channelsProvider.makeSingleUseContext()) { - groupingKeyReader = - rowGroupReader.getColumnChunk(Collections.singletonList(GROUPING_KEY), channelContext); - beginPosReader = rowGroupReader.getColumnChunk(Collections.singletonList(BEGIN_POS), channelContext); - endPosReader = rowGroupReader.getColumnChunk(Collections.singletonList(END_POS), channelContext); - } - if (groupingKeyReader == null || beginPosReader == null || endPosReader == null) { - log.warn().append("Index file ").append(indexFilePath) - .append(" is missing one or more expected columns for table location ") - .append(tl()).append(", column ").append(getName()).endl(); - return null; - } - - final PageCache localPageCache = ensurePageCache(); - - // noinspection unchecked - return (METADATA_TYPE) new MetaDataTableFactory( - ColumnChunkPageStore.create( - localPageCache.castAttr(), groupingKeyReader, - ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, - makeToPage(columnTypes.get(GROUPING_KEY), ParquetInstructions.EMPTY, - GROUPING_KEY, groupingKeyReader, columnDefinition), - columnDefinition).pageStore, - ColumnChunkPageStore.create( - localPageCache.castAttr(), beginPosReader, - ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, - makeToPage(columnTypes.get(BEGIN_POS), ParquetInstructions.EMPTY, BEGIN_POS, - beginPosReader, FIRST_KEY_COL_DEF), - columnDefinition).pageStore, - ColumnChunkPageStore.create( - localPageCache.castAttr(), endPosReader, - ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, - makeToPage(columnTypes.get(END_POS), ParquetInstructions.EMPTY, END_POS, - endPosReader, LAST_KEY_COL_DEF), - columnDefinition).pageStore) - .get(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - private REGION_TYPE makeColumnRegion( - @NotNull final Function, SOURCE[]> sourceArrayFactory, - @NotNull final ColumnDefinition columnDefinition, - @NotNull final LongFunction nullRegionFactory, - @NotNull final Function singleRegionFactory, - @NotNull final Function, REGION_TYPE> multiRegionFactory) { - final SOURCE[] sources = sourceArrayFactory.apply(columnDefinition); - return sources.length == 1 - ? makeSingleColumnRegion(sources[0], nullRegionFactory, singleRegionFactory) - : multiRegionFactory.apply(Arrays.stream(sources).map( - source -> makeSingleColumnRegion(source, nullRegionFactory, singleRegionFactory))); - } - - private REGION_TYPE makeSingleColumnRegion(final SOURCE source, - @NotNull final LongFunction nullRegionFactory, - @NotNull final Function singleRegionFactory) { - return source == null ? nullRegionFactory.apply(tl().getRegionParameters().regionMask) - : singleRegionFactory.apply(source); - } - - @Override - public ColumnRegionChar makeColumnRegionChar( - @NotNull final ColumnDefinition columnDefinition) { - // noinspection unchecked - return (ColumnRegionChar) makeColumnRegion(this::getPageStores, columnDefinition, - ColumnRegionChar::createNull, ParquetColumnRegionChar::new, - rs -> new ColumnRegionChar.StaticPageStore<>(tl().getRegionParameters(), - rs.toArray(ColumnRegionChar[]::new))); - } - - @Override - public ColumnRegionByte makeColumnRegionByte( - @NotNull final ColumnDefinition columnDefinition) { - // noinspection unchecked - return (ColumnRegionByte) makeColumnRegion(this::getPageStores, columnDefinition, - ColumnRegionByte::createNull, ParquetColumnRegionByte::new, - rs -> new ColumnRegionByte.StaticPageStore<>(tl().getRegionParameters(), - rs.toArray(ColumnRegionByte[]::new))); - } - - @Override - public ColumnRegionShort makeColumnRegionShort( - @NotNull final ColumnDefinition columnDefinition) { - // noinspection unchecked - return (ColumnRegionShort) makeColumnRegion(this::getPageStores, columnDefinition, - ColumnRegionShort::createNull, ParquetColumnRegionShort::new, - rs -> new ColumnRegionShort.StaticPageStore<>(tl().getRegionParameters(), - rs.toArray(ColumnRegionShort[]::new))); - } - - @Override - public ColumnRegionInt makeColumnRegionInt( - @NotNull final ColumnDefinition columnDefinition) { - // noinspection unchecked - return (ColumnRegionInt) makeColumnRegion(this::getPageStores, columnDefinition, - ColumnRegionInt::createNull, ParquetColumnRegionInt::new, - rs -> new ColumnRegionInt.StaticPageStore<>(tl().getRegionParameters(), - rs.toArray(ColumnRegionInt[]::new))); - } - - @Override - public ColumnRegionLong makeColumnRegionLong( - @NotNull final ColumnDefinition columnDefinition) { - // noinspection unchecked - return (ColumnRegionLong) makeColumnRegion(this::getPageStores, columnDefinition, - ColumnRegionLong::createNull, ParquetColumnRegionLong::new, - rs -> new ColumnRegionLong.StaticPageStore<>(tl().getRegionParameters(), - rs.toArray(ColumnRegionLong[]::new))); - } - - @Override - public ColumnRegionFloat makeColumnRegionFloat( - @NotNull final ColumnDefinition columnDefinition) { - // noinspection unchecked - return (ColumnRegionFloat) makeColumnRegion(this::getPageStores, columnDefinition, - ColumnRegionFloat::createNull, ParquetColumnRegionFloat::new, - rs -> new ColumnRegionFloat.StaticPageStore<>(tl().getRegionParameters(), - rs.toArray(ColumnRegionFloat[]::new))); - } - - @Override - public ColumnRegionDouble makeColumnRegionDouble( - @NotNull final ColumnDefinition columnDefinition) { - // noinspection unchecked - return (ColumnRegionDouble) makeColumnRegion(this::getPageStores, columnDefinition, - ColumnRegionDouble::createNull, ParquetColumnRegionDouble::new, - rs -> new ColumnRegionDouble.StaticPageStore<>(tl().getRegionParameters(), - rs.toArray(ColumnRegionDouble[]::new))); - } - - @Override - public ColumnRegionObject makeColumnRegionObject( - @NotNull final ColumnDefinition columnDefinition) { - final Class dataType = columnDefinition.getDataType(); - final ColumnChunkPageStore[] sources = getPageStores(columnDefinition); - final ColumnChunkPageStore[] dictKeySources = - getDictionaryKeysPageStores(columnDefinition); - final Supplier>[] dictionaryChunkSuppliers = - getDictionaryChunkSuppliers(columnDefinition); - if (sources.length == 1) { - // noinspection unchecked - return (ColumnRegionObject) makeSingleColumnRegionObject(dataType, - sources[0], dictKeySources[0], dictionaryChunkSuppliers[0]); - } - // noinspection unchecked - return (ColumnRegionObject) new ColumnRegionObject.StaticPageStore( - tl().getRegionParameters(), - IntStream.range(0, sources.length) - .mapToObj(ri -> makeSingleColumnRegionObject(dataType, sources[ri], - dictKeySources[ri], dictionaryChunkSuppliers[ri])) - .toArray(ColumnRegionObject[]::new)); - } - - private ColumnRegionObject makeSingleColumnRegionObject( - @NotNull final Class dataType, - @Nullable final ColumnChunkPageStore source, - @Nullable final ColumnChunkPageStore dictKeySource, - @Nullable final Supplier> dictValuesSupplier) { - if (source == null) { - return ColumnRegionObject.createNull(tl().getRegionParameters().regionMask); - } - return new ParquetColumnRegionObject<>(source, - () -> new ParquetColumnRegionLong<>(Require.neqNull(dictKeySource, "dictKeySource")), - () -> ColumnRegionChunkDictionary.create(tl().getRegionParameters().regionMask, - dataType, Require.neqNull(dictValuesSupplier, "dictValuesSupplier"))); - } - - /** - * Get the {@link ColumnChunkPageStore page stores} backing this column location. - * - * @param columnDefinition The {@link ColumnDefinition} used to lookup type information - * @return The page stores - */ - @NotNull - public ColumnChunkPageStore[] getPageStores( - @NotNull final ColumnDefinition columnDefinition) { - fetchValues(columnDefinition); - return pageStores; - } - - /** - * Get suppliers to access the {@link Chunk dictionary chunks} backing this column location. - * - * @param columnDefinition The {@link ColumnDefinition} used to lookup type information - * @return The dictionary values chunk suppliers, or null if none exist - */ - public Supplier>[] getDictionaryChunkSuppliers( - @NotNull final ColumnDefinition columnDefinition) { - fetchValues(columnDefinition); - return dictionaryChunkSuppliers; - } - - /** - * Get the {@link ColumnChunkPageStore page stores} backing the indices for this column location. Only usable when - * there are dictionaries. - * - * @param columnDefinition The {@link ColumnDefinition} used to lookup type information - * @return The page stores - */ - private ColumnChunkPageStore[] getDictionaryKeysPageStores( - @NotNull final ColumnDefinition columnDefinition) { - fetchValues(columnDefinition); - return dictionaryKeysPageStores; - } - - @SuppressWarnings("unchecked") - private void fetchValues(@NotNull final ColumnDefinition columnDefinition) { - if (columnChunkReaders == null) { - return; - } - synchronized (this) { - if (columnChunkReaders == null) { - return; - } - - final int pageStoreCount = columnChunkReaders.length; - pageStores = new ColumnChunkPageStore[pageStoreCount]; - dictionaryChunkSuppliers = new Supplier[pageStoreCount]; - dictionaryKeysPageStores = new ColumnChunkPageStore[pageStoreCount]; - for (int psi = 0; psi < pageStoreCount; ++psi) { - final ColumnChunkReader columnChunkReader = columnChunkReaders[psi]; - try { - final ColumnChunkPageStore.CreatorResult creatorResult = - ColumnChunkPageStore.create( - ensurePageCache(), - columnChunkReader, - tl().getRegionParameters().regionMask, - makeToPage(tl().getColumnTypes().get(parquetColumnName), - tl().getReadInstructions(), parquetColumnName, columnChunkReader, - columnDefinition), - columnDefinition); - pageStores[psi] = creatorResult.pageStore; - dictionaryChunkSuppliers[psi] = creatorResult.dictionaryChunkSupplier; - dictionaryKeysPageStores[psi] = creatorResult.dictionaryKeysPageStore; - } catch (IOException e) { - throw new TableDataException( - "Failed to read parquet file for " + this + ", row group " + psi, e); - } - } - - columnChunkReaders = null; - } - } - - private static final class MetaDataTableFactory { - - private final ColumnChunkPageStore keyColumn; - private final ColumnChunkPageStore firstColumn; - private final ColumnChunkPageStore lastColumn; - - private volatile Object metaData; - - private MetaDataTableFactory(@NotNull final ColumnChunkPageStore keyColumn, - @NotNull final ColumnChunkPageStore firstColumn, - @NotNull final ColumnChunkPageStore lastColumn) { - this.keyColumn = Require.neqNull(keyColumn, "keyColumn"); - this.firstColumn = Require.neqNull(firstColumn, "firstColumn"); - this.lastColumn = Require.neqNull(lastColumn, "lastColumn"); - } - - public Object get() { - if (metaData != null) { - return metaData; - } - synchronized (this) { - if (metaData != null) { - return metaData; - } - final int numRows = (int) keyColumn.numRows(); - - try ( - final ChunkBoxer.BoxerKernel boxerKernel = - ChunkBoxer.getBoxer(keyColumn.getChunkType(), CHUNK_SIZE); - final BuildGrouping buildGrouping = - BuildGrouping.builder(firstColumn.getChunkType(), numRows); - final ChunkSource.GetContext keyContext = keyColumn.makeGetContext(CHUNK_SIZE); - final ChunkSource.GetContext firstContext = - firstColumn.makeGetContext(CHUNK_SIZE); - final ChunkSource.GetContext lastContext = - lastColumn.makeGetContext(CHUNK_SIZE); - final RowSequence rows = RowSequenceFactory.forRange(0, numRows - 1); - final RowSequence.Iterator rowsIterator = rows.getRowSequenceIterator()) { - - while (rowsIterator.hasMore()) { - final RowSequence chunkRows = - rowsIterator.getNextRowSequenceWithLength(CHUNK_SIZE); - - buildGrouping.build( - boxerKernel.box(keyColumn.getChunk(keyContext, chunkRows)), - firstColumn.getChunk(firstContext, chunkRows), - lastColumn.getChunk(lastContext, chunkRows)); - } - - metaData = buildGrouping.getGrouping(); - } - } - return metaData; - } - - private interface BuildGrouping extends Context { - void build(@NotNull ObjectChunk keyChunk, - @NotNull Chunk firstChunk, - @NotNull Chunk lastChunk); - - Object getGrouping(); - - static BuildGrouping builder(@NotNull final ChunkType chunkType, final int numRows) { - switch (chunkType) { - case Int: - return new IntBuildGrouping(numRows); - case Long: - return new LongBuildGrouping(numRows); - default: - throw new IllegalArgumentException( - "Unknown type for a rowSet: " + chunkType); - } - } - - final class IntBuildGrouping implements BuildGrouping { - - private final Map grouping; - - IntBuildGrouping(final int numRows) { - grouping = new LinkedHashMap<>(numRows); - } - - @Override - public void build(@NotNull final ObjectChunk keyChunk, - @NotNull final Chunk firstChunk, - @NotNull final Chunk lastChunk) { - final IntChunk firstIntChunk = - firstChunk.asIntChunk(); - final IntChunk lastIntChunk = - lastChunk.asIntChunk(); - - for (int ki = 0; ki < keyChunk.size(); ++ki) { - final int[] range = new int[2]; - - range[0] = firstIntChunk.get(ki); - range[1] = lastIntChunk.get(ki); - - grouping.put(keyChunk.get(ki), range); - } - } - - @Override - public Object getGrouping() { - return grouping; - } - } - - final class LongBuildGrouping implements BuildGrouping { - - private final Map grouping; - - LongBuildGrouping(final int numRows) { - grouping = new LinkedHashMap<>(numRows); - } - - @Override - public void build(@NotNull final ObjectChunk keyChunk, - @NotNull final Chunk firstChunk, - @NotNull final Chunk lastChunk) { - final LongChunk firstLongChunk = - firstChunk.asLongChunk(); - final LongChunk lastLongChunk = - lastChunk.asLongChunk(); - - for (int ki = 0; ki < keyChunk.size(); ++ki) { - final long[] range = new long[2]; - - range[0] = firstLongChunk.get(ki); - range[1] = lastLongChunk.get(ki); - - grouping.put(keyChunk.get(ki), range); - } - } - - @Override - public Object getGrouping() { - return grouping; - } - } - } - } - - private static ToPage makeToPage( - @Nullable final ColumnTypeInfo columnTypeInfo, - @NotNull final ParquetInstructions readInstructions, - @NotNull final String parquetColumnName, - @NotNull final ColumnChunkReader columnChunkReader, - @NotNull final ColumnDefinition columnDefinition) { - final PrimitiveType type = columnChunkReader.getType(); - final LogicalTypeAnnotation logicalTypeAnnotation = type.getLogicalTypeAnnotation(); - final String codecFromInstructions = - readInstructions.getCodecName(columnDefinition.getName()); - final String codecName = (codecFromInstructions != null) - ? codecFromInstructions - : columnTypeInfo == null ? null - : columnTypeInfo.codec().map(CodecInfo::codecName).orElse(null); - final ColumnTypeInfo.SpecialType specialTypeName = - columnTypeInfo == null ? null : columnTypeInfo.specialType().orElse(null); - - final boolean isArray = columnChunkReader.getMaxRl() > 0; - final boolean isCodec = CodecLookup.explicitCodecPresent(codecName); - - if (isArray && columnChunkReader.getMaxRl() > 1) { - throw new TableDataException("No support for nested repeated parquet columns."); - } - - try { - // Note that componentType is null for a StringSet. ToStringSetPage.create specifically - // doesn't take this parameter. - final Class dataType = columnDefinition.getDataType(); - final Class componentType = columnDefinition.getComponentType(); - final Class pageType = isArray ? componentType : dataType; - - ToPage toPage = null; - - if (!isCodec && logicalTypeAnnotation != null) { - toPage = logicalTypeAnnotation.accept( - new LogicalTypeVisitor(parquetColumnName, columnChunkReader, pageType)) - .orElse(null); - } - - if (toPage == null) { - final PrimitiveType.PrimitiveTypeName typeName = type.getPrimitiveTypeName(); - switch (typeName) { - case BOOLEAN: - toPage = ToBooleanAsBytePage.create(pageType); - break; - case INT32: - toPage = ToIntPage.create(pageType); - break; - case INT64: - toPage = ToLongPage.create(pageType); - break; - case INT96: - toPage = ToInstantPageFromInt96.create(pageType); - break; - case DOUBLE: - toPage = ToDoublePage.create(pageType); - break; - case FLOAT: - toPage = ToFloatPage.create(pageType); - break; - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - // noinspection rawtypes - final ObjectCodec codec; - if (isCodec) { - final String codecArgs = codecFromInstructions != null - ? readInstructions.getCodecArgs(columnDefinition.getName()) - : columnTypeInfo.codec().flatMap(CodecInfo::codecArg).orElse(null); - codec = CodecCache.DEFAULT.getCodec(codecName, codecArgs); - } else { - final String codecArgs = - (typeName == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) - ? Integer.toString(type.getTypeLength()) - : null; - codec = CodecCache.DEFAULT - .getCodec(SimpleByteArrayCodec.class.getName(), codecArgs); - } - // noinspection unchecked - toPage = ToObjectPage.create(dataType, codec, - columnChunkReader.getDictionarySupplier()); - break; - default: - } - } - - if (toPage == null) { - throw new TableDataException( - "Unsupported parquet column type " + type.getPrimitiveTypeName() + - " with logical type " + logicalTypeAnnotation); - } - - if (specialTypeName == ColumnTypeInfo.SpecialType.StringSet) { - Assert.assertion(isArray, "isArray"); - toPage = ToStringSetPage.create(dataType, toPage); - } else if (isArray) { - Assert.assertion(!isCodec, "!isCodec"); - if (Vector.class.isAssignableFrom(dataType)) { - toPage = ToVectorPage.create(dataType, componentType, toPage); - } else if (dataType.isArray()) { - toPage = ToArrayPage.create(dataType, componentType, toPage); - } - } - - // noinspection unchecked - return (ToPage) toPage; - - } catch (RuntimeException except) { - throw new TableDataException( - "Unexpected exception accessing column " + parquetColumnName, except); - } - } - - private static class LogicalTypeVisitor - implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { - - private final String name; - private final ColumnChunkReader columnChunkReader; - private final Class componentType; - - LogicalTypeVisitor(@NotNull final String name, @NotNull final ColumnChunkReader columnChunkReader, - final Class componentType) { - this.name = name; - this.columnChunkReader = columnChunkReader; - this.componentType = componentType; - } - - @Override - public Optional> visit( - final LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { - return Optional - .of(ToStringPage.create(componentType, columnChunkReader.getDictionarySupplier())); - } - - @Override - public Optional> visit( - final LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { - if (timestampLogicalType.isAdjustedToUTC()) { - return Optional.of(ToInstantPage.create(componentType, timestampLogicalType.getUnit())); - } - return Optional.of(ToLocalDateTimePage.create(componentType, timestampLogicalType.getUnit())); - } - - @Override - public Optional> visit(final LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { - - if (intLogicalType.isSigned()) { - switch (intLogicalType.getBitWidth()) { - case 8: - return Optional.of(ToBytePageFromInt.create(componentType)); - case 16: - return Optional.of(ToShortPageFromInt.create(componentType)); - case 32: - return Optional.of(ToIntPage.create(componentType)); - case 64: - return Optional.of(ToLongPage.create(componentType)); - } - } else { - switch (intLogicalType.getBitWidth()) { - case 8: - case 16: - return Optional.of(ToCharPageFromInt.create(componentType)); - case 32: - return Optional.of(ToLongPage.create(componentType)); - } - } - - return Optional.empty(); - } - - @Override - public Optional> visit(final LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { - return Optional.of(ToDatePageFromInt.create(componentType)); - } - - @Override - public Optional> visit(final LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { - return Optional - .of(ToTimePage.create(componentType, timeLogicalType.getUnit(), timeLogicalType.isAdjustedToUTC())); - } - - @Override - public Optional> visit( - final LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) { - final PrimitiveType type = columnChunkReader.getType(); - final PrimitiveType.PrimitiveTypeName typeName = type.getPrimitiveTypeName(); - switch (typeName) { - case INT32: - return Optional.of(ToBigDecimalFromIntPage.create( - componentType, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); - case INT64: - return Optional.of(ToBigDecimalFromLongPage.create( - componentType, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - final int encodedSizeInBytes = - (typeName == PrimitiveType.PrimitiveTypeName.BINARY) ? -1 : type.getTypeLength(); - if (BigDecimal.class.equals(componentType)) { - return Optional.of( - ToObjectPage.create( - BigDecimal.class, - new BigDecimalParquetBytesCodec( - decimalLogicalType.getPrecision(), decimalLogicalType.getScale(), - encodedSizeInBytes), - columnChunkReader.getDictionarySupplier())); - } else if (BigInteger.class.equals(componentType)) { - return Optional.of( - ToObjectPage.create( - BigInteger.class, - new BigIntegerParquetBytesCodec(encodedSizeInBytes), - columnChunkReader.getDictionarySupplier())); - } - - // We won't blow up here, Maybe someone will provide us a codec instead. - default: - return Optional.empty(); - } - } - } -} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java index 36a6c1d39fe..63f45c51a2f 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java @@ -3,99 +3,37 @@ */ package io.deephaven.iceberg.location; -import io.deephaven.chunk.attributes.Values; +import io.deephaven.base.verify.Require; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderSequential; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.table.impl.locations.TableKey; -import io.deephaven.engine.table.impl.locations.TableLocationState; +import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation; -import io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource; -import io.deephaven.engine.table.impl.sources.regioned.RegionedPageStore; -import io.deephaven.parquet.base.ColumnChunkReader; -import io.deephaven.parquet.base.ParquetFileReader; -import io.deephaven.parquet.base.RowGroupReader; import io.deephaven.parquet.table.ParquetInstructions; -import io.deephaven.parquet.table.ParquetSchemaReader; -import io.deephaven.parquet.table.metadata.ColumnTypeInfo; -import io.deephaven.parquet.table.metadata.GroupingColumnInfo; -import io.deephaven.parquet.table.metadata.TableInfo; -import io.deephaven.util.channel.SeekableChannelContext; -import io.deephaven.util.channel.SeekableChannelsProvider; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.format.RowGroup; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import io.deephaven.parquet.table.location.ParquetTableLocation; +import io.deephaven.parquet.table.location.ParquetTableLocationKey; +import org.apache.iceberg.FileFormat; import org.jetbrains.annotations.NotNull; -import java.io.File; -import java.util.*; -import java.util.stream.IntStream; +public class IcebergTableLocation implements TableLocation { -import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; + private static final String IMPLEMENTATION_NAME = IcebergTableLocation.class.getSimpleName(); -public class IcebergTableLocation extends AbstractTableLocation { + private final ImmutableTableKey tableKey; + private final ImmutableTableLocationKey tableLocationKey; - private static final String IMPLEMENTATION_NAME = IcebergColumnLocation.class.getSimpleName(); - - private final ParquetInstructions readInstructions; - private final ParquetFileReader parquetFileReader; - private final int[] rowGroupIndices; - - private final RowGroup[] rowGroups; - private final RegionedPageStore.Parameters regionParameters; - private final Map parquetColumnNameToPath; - private final Map groupingColumns; - private final Map columnTypes; - private final String version; - - private volatile RowGroupReader[] rowGroupReaders; + AbstractTableLocation internalTableLocation; public IcebergTableLocation(@NotNull final TableKey tableKey, @NotNull final IcebergTableLocationKey tableLocationKey, - @NotNull final ParquetInstructions readInstructions) { - super(tableKey, tableLocationKey, false); - this.readInstructions = readInstructions; - final ParquetMetadata parquetMetadata; - // noinspection SynchronizationOnLocalVariableOrMethodParameter - synchronized (tableLocationKey) { - parquetFileReader = tableLocationKey.getFileReader(); - parquetMetadata = tableLocationKey.getMetadata(); - rowGroupIndices = tableLocationKey.getRowGroupIndices(); - } - - final int rowGroupCount = rowGroupIndices.length; - rowGroups = IntStream.of(rowGroupIndices) - .mapToObj(rgi -> parquetFileReader.fileMetaData.getRow_groups().get(rgi)) - .sorted(Comparator.comparingInt(RowGroup::getOrdinal)) - .toArray(RowGroup[]::new); - final long maxRowCount = Arrays.stream(rowGroups).mapToLong(RowGroup::getNum_rows).max().orElse(0L); - regionParameters = new RegionedPageStore.Parameters( - RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, rowGroupCount, maxRowCount); - - parquetColumnNameToPath = new HashMap<>(); - for (final ColumnDescriptor column : parquetFileReader.getSchema().getColumns()) { - final String[] path = column.getPath(); - if (path.length > 1) { - parquetColumnNameToPath.put(path[0], path); - } - } - - // TODO (https://github.com/deephaven/deephaven-core/issues/958): - // When/if we support _metadata files for Deephaven-written Parquet tables, we may need to revise this - // in order to read *this* file's metadata, rather than inheriting file metadata from the _metadata file. - // Obvious issues included grouping table paths, codecs, etc. - // Presumably, we could store per-file instances of the metadata in the _metadata file's map. - final Optional tableInfo = - ParquetSchemaReader.parseMetadata(parquetMetadata.getFileMetaData().getKeyValueMetaData()); - groupingColumns = tableInfo.map(TableInfo::groupingColumnMap).orElse(Collections.emptyMap()); - columnTypes = tableInfo.map(TableInfo::columnTypeMap).orElse(Collections.emptyMap()); - version = tableInfo.map(TableInfo::version).orElse(null); - - if (!FILE_URI_SCHEME.equals(tableLocationKey.getURI().getScheme())) { - // We do not have the last modified time for non-file URIs - handleUpdate(computeIndex(), TableLocationState.NULL_TIME); + @NotNull final Object readInstructions) { + this.tableKey = Require.neqNull(tableKey, "tableKey").makeImmutable(); + this.tableLocationKey = Require.neqNull(tableLocationKey, "tableLocationKey").makeImmutable(); + + if (tableLocationKey.format == FileFormat.PARQUET) { + this.internalTableLocation = new ParquetTableLocation(tableKey, + (ParquetTableLocationKey) tableLocationKey.internalTableLocationKey, + (ParquetInstructions) readInstructions); } else { - handleUpdate(computeIndex(), new File(tableLocationKey.getURI()).lastModified()); + throw new IllegalArgumentException("Unsupported file format: " + tableLocationKey.format); } } @@ -105,75 +43,58 @@ public String getImplementationName() { } @Override - public void refresh() {} + @NotNull + public ImmutableTableKey getTableKey() { + return tableKey; + } - IcebergTableLocationKey getParquetKey() { - return (IcebergTableLocationKey) getKey(); + @Override + public @NotNull ImmutableTableLocationKey getKey() { + return tableLocationKey; } - ParquetInstructions getReadInstructions() { - return readInstructions; + @Override + public boolean supportsSubscriptions() { + return internalTableLocation.supportsSubscriptions(); } - SeekableChannelsProvider getChannelProvider() { - return parquetFileReader.getChannelsProvider(); + @Override + public void subscribe(@NotNull Listener listener) { + internalTableLocation.subscribe(listener); } - RegionedPageStore.Parameters getRegionParameters() { - return regionParameters; + @Override + public void unsubscribe(@NotNull Listener listener) { + internalTableLocation.unsubscribe(listener); } - public Map getGroupingColumns() { - return groupingColumns; + @Override + public void refresh() { + internalTableLocation.refresh(); } - public Map getColumnTypes() { - return columnTypes; + @Override + public @NotNull ColumnLocation getColumnLocation(@NotNull CharSequence name) { + return internalTableLocation.getColumnLocation(name); } - private RowGroupReader[] getRowGroupReaders() { - RowGroupReader[] local; - if ((local = rowGroupReaders) != null) { - return local; - } - synchronized (this) { - if ((local = rowGroupReaders) != null) { - return local; - } - return rowGroupReaders = IntStream.of(rowGroupIndices) - .mapToObj(idx -> parquetFileReader.getRowGroup(idx, version)) - .sorted(Comparator.comparingInt(rgr -> rgr.getRowGroup().getOrdinal())) - .toArray(RowGroupReader[]::new); - } + @Override + public @NotNull Object getStateLock() { + return internalTableLocation.getStateLock(); } - @NotNull @Override - protected IcebergColumnLocation makeColumnLocation(@NotNull final String columnName) { - final String parquetColumnName = readInstructions.getParquetColumnNameFromColumnNameOrDefault(columnName); - final String[] columnPath = parquetColumnNameToPath.get(parquetColumnName); - final List nameList = - columnPath == null ? Collections.singletonList(parquetColumnName) : Arrays.asList(columnPath); - final ColumnChunkReader[] columnChunkReaders; - try (final SeekableChannelContext channelContext = getChannelProvider().makeSingleUseContext()) { - columnChunkReaders = Arrays.stream(getRowGroupReaders()) - .map(rgr -> rgr.getColumnChunk(nameList, channelContext)).toArray(ColumnChunkReader[]::new); - } - final boolean exists = Arrays.stream(columnChunkReaders).anyMatch(ccr -> ccr != null && ccr.numRows() > 0); - return new IcebergColumnLocation<>(this, columnName, parquetColumnName, - exists ? columnChunkReaders : null, - exists && groupingColumns.containsKey(parquetColumnName)); + public RowSet getRowSet() { + return internalTableLocation.getRowSet(); } - private RowSet computeIndex() { - final RowSetBuilderSequential sequentialBuilder = RowSetFactory.builderSequential(); + @Override + public long getSize() { + return internalTableLocation.getSize(); + } - for (int rgi = 0; rgi < rowGroups.length; ++rgi) { - final long subRegionSize = rowGroups[rgi].getNum_rows(); - final long subRegionFirstKey = (long) rgi << regionParameters.regionMaskNumBits; - final long subRegionLastKey = subRegionFirstKey + subRegionSize - 1; - sequentialBuilder.appendRange(subRegionFirstKey, subRegionLastKey); - } - return sequentialBuilder.build(); + @Override + public long getLastModifiedTimeMillis() { + return internalTableLocation.getLastModifiedTimeMillis(); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java index 638c75181fb..14d617afaae 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java @@ -8,7 +8,6 @@ import io.deephaven.engine.table.impl.locations.impl.NonexistentTableLocation; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; -import io.deephaven.parquet.table.ParquetInstructions; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -21,10 +20,9 @@ * {@link TableLocationFactory} for {@link IcebergTableLocation}s. */ public final class IcebergTableLocationFactory implements TableLocationFactory { + private final Object readInstructions; - private final ParquetInstructions readInstructions; - - public IcebergTableLocationFactory(@NotNull final ParquetInstructions readInstructions) { + public IcebergTableLocationFactory(@NotNull final Object readInstructions) { this.readInstructions = readInstructions; } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java index e5546cf2638..03b2ae21f16 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java @@ -3,25 +3,16 @@ */ package io.deephaven.iceberg.location; -import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableLocationKey; import io.deephaven.engine.table.impl.locations.local.URITableLocationKey; -import io.deephaven.parquet.base.ParquetFileReader; import io.deephaven.parquet.table.ParquetInstructions; -import io.deephaven.parquet.table.ParquetTableWriter; -import io.deephaven.parquet.table.ParquetTools; -import org.apache.parquet.format.RowGroup; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import io.deephaven.parquet.table.location.ParquetTableLocationKey; +import org.apache.iceberg.FileFormat; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.io.File; -import java.io.IOException; import java.net.URI; -import java.util.List; import java.util.Map; -import java.util.stream.IntStream; /** * {@link TableLocationKey} implementation for use with data stored in the parquet format. @@ -30,10 +21,9 @@ public class IcebergTableLocationKey extends URITableLocationKey { private static final String IMPLEMENTATION_NAME = IcebergTableLocationKey.class.getSimpleName(); - private ParquetFileReader fileReader; - private ParquetMetadata metadata; - private int[] rowGroupIndices; - private final ParquetInstructions readInstructions; + final FileFormat format; + + final URITableLocationKey internalTableLocationKey; /** * Construct a new IcebergTableLocationKey for the supplied {@code fileUri} and {@code partitions}. @@ -46,25 +36,21 @@ public class IcebergTableLocationKey extends URITableLocationKey { * @param readInstructions the instructions for customizations while reading */ public IcebergTableLocationKey( + final FileFormat format, @NotNull final URI fileUri, final int order, @Nullable final Map> partitions, - @NotNull final ParquetInstructions readInstructions) { - super(validateIcebergDataFile(fileUri), order, partitions); - this.readInstructions = readInstructions; - } - - private static URI validateIcebergDataFile(@NotNull final URI fileUri) { - // TODO: when/if ORC format supported, need to accept and validate it. - - // TODO: this file extension check is probably not needed (assuming we read the path from the manifest file) - // correctly). - - if (!fileUri.getRawPath().endsWith(ParquetTableWriter.PARQUET_FILE_EXTENSION)) { - throw new IllegalArgumentException( - "Iceberg data file must end in " + ParquetTableWriter.PARQUET_FILE_EXTENSION); + @NotNull final Object readInstructions) { + super(fileUri, order, partitions); + this.format = format; + if (format == FileFormat.PARQUET) { + // This constructor will perform validation of the parquet file + final ParquetInstructions parquetInstructions = (ParquetInstructions) readInstructions; + this.internalTableLocationKey = + new ParquetTableLocationKey(fileUri, order, partitions, parquetInstructions); + } else { + throw new IllegalArgumentException("Unsupported file format: " + format); } - return fileUri; } @Override @@ -73,8 +59,7 @@ public String getImplementationName() { } /** - * Returns {@code true} if a previous {@link ParquetFileReader} has been created, or if one was successfully created - * on-demand. + * Returns {@code true} if a previous file reader has been created, or if one was successfully created on-demand. * *

* When {@code false}, this may mean that the file: @@ -85,107 +70,14 @@ public String getImplementationName() { *

  • is a corrupt parquet file
  • * * - * Callers wishing to handle these cases more explicit may call - * {@link ParquetTools#getParquetFileReaderChecked(URI, ParquetInstructions)}. * * @return true if the file reader exists or was successfully created */ public synchronized boolean verifyFileReader() { - if (fileReader != null) { - return true; - } - try { - fileReader = ParquetTools.getParquetFileReaderChecked(uri, readInstructions); - } catch (IOException e) { - return false; - } - return true; - } - - /** - * Get a previously-{@link #setFileReader(ParquetFileReader) set} or on-demand created {@link ParquetFileReader} for - * this location key's {@code file}. - * - * @return A {@link ParquetFileReader} for this location key's {@code file}. - */ - public synchronized ParquetFileReader getFileReader() { - if (fileReader != null) { - return fileReader; - } - return fileReader = ParquetTools.getParquetFileReader(uri, readInstructions); - } - - /** - * Set the {@link ParquetFileReader} that will be returned by {@link #getFileReader()}. Pass {@code null} to force - * on-demand construction at the next invocation. Always clears cached {@link ParquetMetadata} and {@link RowGroup} - * indices. - * - * @param fileReader The new {@link ParquetFileReader} - */ - public synchronized void setFileReader(final ParquetFileReader fileReader) { - this.fileReader = fileReader; - this.metadata = null; - this.rowGroupIndices = null; - } - - /** - * Get a previously-{@link #setMetadata(ParquetMetadata) set} or on-demand created {@link ParquetMetadata} for this - * location key's {@code file}. - * - * @return A {@link ParquetMetadata} for this location key's {@code file}. - */ - public synchronized ParquetMetadata getMetadata() { - if (metadata != null) { - return metadata; - } - try { - return metadata = new ParquetMetadataConverter().fromParquetMetadata(getFileReader().fileMetaData); - } catch (IOException e) { - throw new TableDataException("Failed to convert Parquet file metadata: " + getURI(), e); - } - } - - /** - * Set the {@link ParquetMetadata} that will be returned by {@link #getMetadata()} ()}. Pass {@code null} to force - * on-demand construction at the next invocation. - * - * @param metadata The new {@link ParquetMetadata} - */ - public synchronized void setMetadata(final ParquetMetadata metadata) { - this.metadata = metadata; - } - - /** - * Get previously-{@link #setRowGroupIndices(int[]) set} or on-demand created {@link RowGroup} indices for this - * location key's current {@link ParquetFileReader}. - * - * @return {@link RowGroup} indices for this location key's current {@link ParquetFileReader}. - */ - public synchronized int[] getRowGroupIndices() { - if (rowGroupIndices != null) { - return rowGroupIndices; + if (format == FileFormat.PARQUET) { + return ((ParquetTableLocationKey) internalTableLocationKey).verifyFileReader(); + } else { + throw new IllegalArgumentException("Unsupported file format: " + format); } - final List rowGroups = getFileReader().fileMetaData.getRow_groups(); - return rowGroupIndices = IntStream.range(0, rowGroups.size()).filter(rgi -> { - // 1. We can safely assume there's always at least one column. Our tools will refuse to write a - // column-less table, and other readers we've tested fail catastrophically. - // 2. null file path means the column is local to the file the metadata was read from (which had - // better be this file, in that case). - // 3. We're assuming row groups are contained within a single file. - // While it seems that row group *could* have column chunks splayed out into multiple files, - // we're not expecting that in this code path. To support it, discovery tools should figure out - // the row groups for a partition themselves and call setRowGroupReaders. - final String filePath = rowGroups.get(rgi).getColumns().get(0).getFile_path(); - return filePath == null || new File(filePath).getAbsoluteFile().toURI().equals(uri); - }).toArray(); - } - - /** - * Set the {@link RowGroup} indices that will be returned by {@link #getRowGroupIndices()} - * - * @param rowGroupIndices The new {@link RowGroup} indices - */ - public synchronized void setRowGroupIndices(final int[] rowGroupIndices) { - this.rowGroupIndices = rowGroupIndices; } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java index a9977c13927..a1e32f3f3c4 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java @@ -5,7 +5,6 @@ import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.PartitionAwareSourceTable; -import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder; import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; @@ -31,13 +30,11 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; public class IcebergCatalog { - private final IcebergInstructions instructions; private final Catalog catalog; private final FileIO fileIO; @@ -49,8 +46,6 @@ public class IcebergCatalog { * @param name The optional service name */ IcebergCatalog(final @Nullable String name, final IcebergInstructions instructions) { - this.instructions = instructions; - // Set up the properties map for the Iceberg catalog Map properties = new HashMap<>(); @@ -93,6 +88,7 @@ public class IcebergCatalog { catalog.initialize(catalogName, properties); } + @SuppressWarnings("unused") public List listTables(final Namespace namespace) { // TODO: have this return a Deephaven Table of table identifiers return catalog.listTables(namespace); @@ -105,6 +101,7 @@ public List listTables(final Namespace namespace) { * @param snapshotId The snapshot ID to load * @return The loaded table */ + @SuppressWarnings("unused") public Table readTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final String snapshotId) { @@ -117,28 +114,23 @@ public Table readTable( * @param tableIdentifier The table identifier to load * @return The loaded table */ + @SuppressWarnings("unused") public Table readTable(@NotNull final TableIdentifier tableIdentifier) { return readTableInternal(tableIdentifier, null, false); } /** - * Subscribe to a table from the Iceberg catalog. Intially the latest snapshot will be loaded, but the output table + * Subscribe to a table from the Iceberg catalog. Initially the latest snapshot will be loaded, but the output table * will be updated as new snapshots are added to the table. * * @param tableIdentifier The table identifier to load * @return The loaded table */ + @SuppressWarnings("unused") public Table subscribeTable(@NotNull final TableIdentifier tableIdentifier) { return readTableInternal(tableIdentifier, null, true); } - private static KnownLocationKeyFinder toKnownKeys( - TableLocationKeyFinder keyFinder) { - return keyFinder instanceof KnownLocationKeyFinder - ? (KnownLocationKeyFinder) keyFinder - : KnownLocationKeyFinder.copyFrom(keyFinder, Comparator.naturalOrder()); - } - private Table readTableInternal( @NotNull final TableIdentifier tableIdentifier, @Nullable final String snapshotId, @@ -180,71 +172,46 @@ private Table readTableInternal( if (partitionSpec.isUnpartitioned()) { // Create the flat layout location key finder - final TableLocationKeyFinder locationKeyFinder = - new IcebergFlatLayout(snapshot, fileIO, instructions); - - if (isRefreshing) { - keyFinder = locationKeyFinder; - description = "Read refreshing iceberg table with " + keyFinder; - refreshService = TableDataRefreshService.getSharedRefreshService(); - updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); - } else { - keyFinder = toKnownKeys(locationKeyFinder); - description = "Read static iceberg table with " + keyFinder; - refreshService = null; - updateSourceRegistrar = null; - } - - return new PartitionAwareSourceTable( - tableDefinition, - description, - RegionedTableComponentFactoryImpl.INSTANCE, - new PollingTableLocationProvider<>( - StandaloneTableKey.getInstance(), - keyFinder, - new IcebergTableLocationFactory(instructions), - refreshService), - updateSourceRegistrar); + keyFinder = new IcebergFlatLayout(snapshot, fileIO, instructions); } else { final String[] partitionColumns = partitionSpec.fields().stream().map(PartitionField::name).toArray(String[]::new); // Create the partitioning column location key finder - final TableLocationKeyFinder locationKeyFinder = new IcebergPartitionedLayout( + keyFinder = new IcebergPartitionedLayout( snapshot, fileIO, partitionColumns, instructions); + } - if (isRefreshing) { - keyFinder = locationKeyFinder; - description = "Read refreshing iceberg table with " + keyFinder; - refreshService = TableDataRefreshService.getSharedRefreshService(); - updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); - } else { - keyFinder = toKnownKeys(locationKeyFinder); - description = "Read static iceberg table with " + keyFinder; - refreshService = null; - updateSourceRegistrar = null; - } - - return new PartitionAwareSourceTable( - tableDefinition, - description, - RegionedTableComponentFactoryImpl.INSTANCE, - new PollingTableLocationProvider<>( - StandaloneTableKey.getInstance(), - keyFinder, - new IcebergTableLocationFactory(instructions), - refreshService), - updateSourceRegistrar); + if (isRefreshing) { + refreshService = TableDataRefreshService.getSharedRefreshService(); + updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + description = "Read refreshing iceberg table with " + keyFinder; + } else { + refreshService = null; + updateSourceRegistrar = null; + description = "Read static iceberg table with " + keyFinder; } + + return new PartitionAwareSourceTable( + tableDefinition, + description, + RegionedTableComponentFactoryImpl.INSTANCE, + new PollingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(instructions), + refreshService), + updateSourceRegistrar); } /** * Return the internal Iceberg catalog. */ + @SuppressWarnings("unused") public Catalog catalog() { return catalog; } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java index f28bb59fcbf..d0b54e1e0a8 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java @@ -11,6 +11,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import java.time.Instant; import java.time.LocalDateTime; import java.util.*; import java.util.stream.Collectors; @@ -20,11 +21,12 @@ */ public class IcebergTools { - public static IcebergCatalog createCatalog(final String name, final IcebergInstructions instructions) { + @SuppressWarnings("unused") + public static IcebergCatalog loadCatalog(final String name, final IcebergInstructions instructions) { return new IcebergCatalog(name, instructions); } - public static TableDefinition fromSchema(final Schema schema, PartitionSpec partitionSpec) { + static TableDefinition fromSchema(final Schema schema, PartitionSpec partitionSpec) { final Set partitionNames = partitionSpec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); @@ -33,23 +35,29 @@ public static TableDefinition fromSchema(final Schema schema, PartitionSpec part for (final Types.NestedField field : schema.columns()) { final String name = field.name(); final Type type = field.type(); - final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); - final ColumnDefinition column; - if (partitionNames.contains(name)) { - column = ColumnDefinition.of(name, qstType).withPartitioning(); - } else { - column = ColumnDefinition.of(name, qstType); + try { + final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); + final ColumnDefinition column; + if (partitionNames.contains(name)) { + column = ColumnDefinition.of(name, qstType).withPartitioning(); + } else { + column = ColumnDefinition.of(name, qstType); + } + columns.add(column); + } catch (UnsupportedOperationException e) { + // TODO: Currently will silently skip the column. Would it be better to skip and warn the user or + // break and declare failure? We don't have a mechanism for skipping columns, do we need an overload + // with a supplied table definition? } - columns.add(column); } return TableDefinition.of(columns); } - public static io.deephaven.qst.type.Type convertPrimitiveType(final Type icebergType) { + static io.deephaven.qst.type.Type convertPrimitiveType(final Type icebergType) { final Type.TypeID typeId = icebergType.typeId(); if (icebergType.isPrimitiveType()) { - if (typeId == Type.TypeID.BINARY || typeId == Type.TypeID.BOOLEAN) { + if (typeId == Type.TypeID.BOOLEAN) { return io.deephaven.qst.type.Type.booleanType(); } else if (typeId == Type.TypeID.DOUBLE) { return io.deephaven.qst.type.Type.doubleType(); @@ -64,16 +72,16 @@ public static io.deephaven.qst.type.Type convertPrimitiveType(final Type iceb } else if (typeId == Type.TypeID.TIMESTAMP) { final Types.TimestampType timestampType = (Types.TimestampType) icebergType; return timestampType.shouldAdjustToUTC() - ? io.deephaven.qst.type.Type.instantType() - : io.deephaven.qst.type.Type.ofCustom(LocalDateTime.class); + ? io.deephaven.qst.type.Type.find(Instant.class) + : io.deephaven.qst.type.Type.find(LocalDateTime.class); } else if (typeId == Type.TypeID.DATE) { - // Not sure what type for this - return io.deephaven.qst.type.Type.intType(); + return io.deephaven.qst.type.Type.find(java.time.LocalDate.class); + } else if (typeId == Type.TypeID.TIME) { + return io.deephaven.qst.type.Type.find(java.time.LocalTime.class); } else if (typeId == Type.TypeID.DECIMAL) { - // BigDecimal?? - return io.deephaven.qst.type.Type.doubleType(); - } else { - throw new UnsupportedOperationException("Unsupported type: " + typeId); + return io.deephaven.qst.type.Type.find(java.math.BigDecimal.class); + } else if (typeId == Type.TypeID.FIXED || typeId == Type.TypeID.BINARY) { + return io.deephaven.qst.type.Type.find(byte[].class); } } throw new UnsupportedOperationException("Unsupported type: " + typeId); diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index 13be402d406..28b42cd4525 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -32,23 +32,35 @@ public void setUp() { .build(); } - @Test - public void testListTables() { - final IcebergCatalog catalog = IcebergTools.createCatalog("minio-iceberg", instructions); - - final Namespace ns = Namespace.of("nyc"); - final List tables = catalog.listTables(ns); - } - - @Test - public void testOpenTable() { - final IcebergCatalog catalog = IcebergTools.createCatalog("minio-iceberg", instructions); - - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); - io.deephaven.engine.table.Table table = catalog.readTable(tableId); - - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); - } + // TODO: discuss how to perform tests since they require a full MiniIO + Iceberg setup + // @Test + // public void testListTables() { + // final IcebergCatalog catalog = IcebergTools.loadCatalog("minio-iceberg", instructions); + // + // final Namespace ns = Namespace.of("nyc"); + // final List tables = catalog.listTables(ns); + // } + // + // @Test + // public void testOpenTable() { + // final IcebergCatalog catalog = IcebergTools.loadCatalog("minio-iceberg", instructions); + // + // final Namespace ns = Namespace.of("nyc"); + // final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + // io.deephaven.engine.table.Table table = catalog.readTable(tableId); + // + // TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + // } + // + // @Test + // public void testOpenAllTypesTable() { + // final IcebergCatalog catalog = IcebergTools.loadCatalog("minio-iceberg", instructions); + // + // final Namespace ns = Namespace.of("sample"); + // final TableIdentifier tableId = TableIdentifier.of(ns, "all_types"); + // io.deephaven.engine.table.Table table = catalog.readTable(tableId); + // + // TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + // } } From c2323d245f8527884132ecc9b8c84b40966215b7 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 2 Apr 2024 11:11:11 -0700 Subject: [PATCH 04/25] Rebased to main. --- .../iceberg/layout/IcebergFlatLayout.java | 6 ++-- .../layout/IcebergPartitionedLayout.java | 6 ++-- .../location/IcebergTableLocation.java | 31 +++++++++++++++++-- .../location/IcebergTableLocationFactory.java | 6 ++-- .../location/IcebergTableLocationKey.java | 6 ++-- .../iceberg/util/IcebergCatalog.java | 3 ++ .../iceberg/util/IcebergInstructions.java | 6 ++-- .../deephaven/iceberg/util/IcebergTools.java | 6 ++-- .../IcebergToolsTest.java | 6 ++-- 9 files changed, 52 insertions(+), 24 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java index 207e94665a8..56d1ad2d9ab 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -1,6 +1,6 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.layout; import io.deephaven.engine.table.impl.locations.TableDataException; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java index 3121775a175..3cfaf549f7c 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java @@ -1,6 +1,6 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.layout; import io.deephaven.engine.table.impl.locations.TableDataException; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java index 63f45c51a2f..eb66b98b16e 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java @@ -1,10 +1,12 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.location; +import io.deephaven.api.SortColumn; import io.deephaven.base.verify.Require; import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.BasicDataIndex; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation; import io.deephaven.parquet.table.ParquetInstructions; @@ -12,6 +14,9 @@ import io.deephaven.parquet.table.location.ParquetTableLocationKey; import org.apache.iceberg.FileFormat; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.List; public class IcebergTableLocation implements TableLocation { @@ -73,6 +78,26 @@ public void refresh() { internalTableLocation.refresh(); } + @Override + public @NotNull List getSortedColumns() { + return internalTableLocation.getSortedColumns(); + } + + @Override + public @NotNull List getDataIndexColumns() { + return internalTableLocation.getDataIndexColumns(); + } + + @Override + public boolean hasDataIndex(@NotNull String... columns) { + return internalTableLocation.hasDataIndex(columns); + } + + @Override + public @Nullable BasicDataIndex getDataIndex(@NotNull String... columns) { + return internalTableLocation.getDataIndex(columns); + } + @Override public @NotNull ColumnLocation getColumnLocation(@NotNull CharSequence name) { return internalTableLocation.getColumnLocation(name); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java index 14d617afaae..81e85c0da73 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java @@ -1,6 +1,6 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.location; import io.deephaven.engine.table.impl.locations.TableKey; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java index 03b2ae21f16..4713b7998ca 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java @@ -1,6 +1,6 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.location; import io.deephaven.engine.table.impl.locations.TableLocationKey; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java index a1e32f3f3c4..a3c8a8bcc59 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java @@ -1,3 +1,6 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.util; import io.deephaven.base.verify.Assert; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index f7961e3e4db..915b0b33bbc 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -1,6 +1,6 @@ -/** - * Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending - */ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.util; import io.deephaven.annotations.BuildableStyle; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java index d0b54e1e0a8..ec9791af1fc 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java @@ -1,6 +1,6 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.util; import io.deephaven.engine.table.ColumnDefinition; diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index 28b42cd4525..fcff074f523 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -1,6 +1,6 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg; import io.deephaven.engine.util.TableTools; From 62952f45ab26d4c9a964566dc5978f6178501d74 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 2 Apr 2024 11:34:39 -0700 Subject: [PATCH 05/25] update to use faster URI creation --- .../java/io/deephaven/iceberg/layout/IcebergFlatLayout.java | 3 ++- .../deephaven/iceberg/layout/IcebergPartitionedLayout.java | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java index 56d1ad2d9ab..fa7bce7862a 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -3,6 +3,7 @@ // package io.deephaven.iceberg.layout; +import io.deephaven.base.FileUtils; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; @@ -60,7 +61,7 @@ public synchronized void findKeys(@NotNull final Consumer reader = ManifestFiles.read(manifestFile, fileIO); for (DataFile df : reader) { - final URI fileUri = URI.create(df.path().toString()); + final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); IcebergTableLocationKey locationKey = cache.get(fileUri); if (locationKey == null) { locationKey = locationKey(df.format(), fileUri, readInstructions); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java index 3cfaf549f7c..33363d739d5 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java @@ -3,9 +3,11 @@ // package io.deephaven.iceberg.layout; +import io.deephaven.base.FileUtils; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; +import io.deephaven.util.channel.SeekableChannelsProvider; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; import org.jetbrains.annotations.NotNull; @@ -68,8 +70,7 @@ public synchronized void findKeys(@NotNull final Consumer reader = ManifestFiles.read(manifestFile, fileIO); for (DataFile df : reader) { - final URI fileUri = URI.create(df.path().toString()); - + final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); IcebergTableLocationKey locationKey = cache.get(fileUri); if (locationKey == null) { final PartitionData partitionData = (PartitionData) df.partition(); From e070b9f5cd3d7bca373569a09ac0cb413a8fb43f Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 2 Apr 2024 13:16:50 -0700 Subject: [PATCH 06/25] Address PR comments. --- ... => IcebergKeyValuePartitionedLayout.java} | 5 ++-- .../location/IcebergTableLocation.java | 20 +++++++++------ .../location/IcebergTableLocationFactory.java | 4 +-- .../location/IcebergTableLocationKey.java | 15 +---------- .../iceberg/util/IcebergCatalog.java | 8 +++--- .../deephaven/iceberg/util/IcebergTools.java | 25 ++++++++----------- .../IcebergToolsTest.java | 5 ++++ 7 files changed, 38 insertions(+), 44 deletions(-) rename extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/{IcebergPartitionedLayout.java => IcebergKeyValuePartitionedLayout.java} (95%) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java similarity index 95% rename from extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java rename to extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java index 33363d739d5..305cca303f7 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergPartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java @@ -7,7 +7,6 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; -import io.deephaven.util.channel.SeekableChannelsProvider; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; import org.jetbrains.annotations.NotNull; @@ -23,7 +22,7 @@ * Iceberg {@link TableLocationKeyFinder location finder} for tables without partitions that will discover data files * from a {@link org.apache.iceberg.Snapshot} */ -public final class IcebergPartitionedLayout implements TableLocationKeyFinder { +public final class IcebergKeyValuePartitionedLayout implements TableLocationKeyFinder { private final Snapshot tableSnapshot; private final FileIO fileIO; private final String[] partitionColumns; @@ -44,7 +43,7 @@ private static IcebergTableLocationKey locationKey( * @param partitionColumns The columns to use for partitioning. * @param readInstructions The instructions for customizations while reading. */ - public IcebergPartitionedLayout( + public IcebergKeyValuePartitionedLayout( @NotNull final Snapshot tableSnapshot, @NotNull final FileIO fileIO, @NotNull final String[] partitionColumns, diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java index eb66b98b16e..60857794ef7 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java @@ -25,7 +25,7 @@ public class IcebergTableLocation implements TableLocation { private final ImmutableTableKey tableKey; private final ImmutableTableLocationKey tableLocationKey; - AbstractTableLocation internalTableLocation; + private final AbstractTableLocation internalTableLocation; public IcebergTableLocation(@NotNull final TableKey tableKey, @NotNull final IcebergTableLocationKey tableLocationKey, @@ -54,7 +54,8 @@ public ImmutableTableKey getTableKey() { } @Override - public @NotNull ImmutableTableLocationKey getKey() { + @NotNull + public ImmutableTableLocationKey getKey() { return tableLocationKey; } @@ -79,12 +80,14 @@ public void refresh() { } @Override - public @NotNull List getSortedColumns() { + @NotNull + public List getSortedColumns() { return internalTableLocation.getSortedColumns(); } @Override - public @NotNull List getDataIndexColumns() { + @NotNull + public List getDataIndexColumns() { return internalTableLocation.getDataIndexColumns(); } @@ -94,17 +97,20 @@ public boolean hasDataIndex(@NotNull String... columns) { } @Override - public @Nullable BasicDataIndex getDataIndex(@NotNull String... columns) { + @Nullable + public BasicDataIndex getDataIndex(@NotNull String... columns) { return internalTableLocation.getDataIndex(columns); } @Override - public @NotNull ColumnLocation getColumnLocation(@NotNull CharSequence name) { + @NotNull + public ColumnLocation getColumnLocation(@NotNull CharSequence name) { return internalTableLocation.getColumnLocation(name); } @Override - public @NotNull Object getStateLock() { + @NotNull + public Object getStateLock() { return internalTableLocation.getStateLock(); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java index 81e85c0da73..b8b5a8cf609 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java @@ -31,8 +31,8 @@ public IcebergTableLocationFactory(@NotNull final Object readInstructions) { public TableLocation makeLocation(@NotNull final TableKey tableKey, @NotNull final IcebergTableLocationKey locationKey, @Nullable final TableDataRefreshService refreshService) { - final URI parquetFileURI = locationKey.getURI(); - if (!FILE_URI_SCHEME.equals(parquetFileURI.getScheme()) || new File(parquetFileURI).exists()) { + final URI fileURI = locationKey.getURI(); + if (!FILE_URI_SCHEME.equals(fileURI.getScheme()) || new File(fileURI).exists()) { return new IcebergTableLocation(tableKey, locationKey, readInstructions); } else { return new NonexistentTableLocation(tableKey, locationKey); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java index 4713b7998ca..28c3b688f27 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java @@ -22,7 +22,6 @@ public class IcebergTableLocationKey extends URITableLocationKey { private static final String IMPLEMENTATION_NAME = IcebergTableLocationKey.class.getSimpleName(); final FileFormat format; - final URITableLocationKey internalTableLocationKey; /** @@ -59,19 +58,7 @@ public String getImplementationName() { } /** - * Returns {@code true} if a previous file reader has been created, or if one was successfully created on-demand. - * - *

    - * When {@code false}, this may mean that the file: - *

      - *
    1. does not exist, or is otherwise inaccessible
    2. - *
    3. is in the process of being written, and is not yet a valid parquet file
    4. - *
    5. is _not_ a parquet file
    6. - *
    7. is a corrupt parquet file
    8. - *
    - * - * - * @return true if the file reader exists or was successfully created + * See {@link ParquetTableLocationKey#verifyFileReader()}. */ public synchronized boolean verifyFileReader() { if (format == FileFormat.PARQUET) { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java index a3c8a8bcc59..440f605113e 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java @@ -17,7 +17,7 @@ import io.deephaven.extensions.s3.Credentials; import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.layout.IcebergFlatLayout; -import io.deephaven.iceberg.layout.IcebergPartitionedLayout; +import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout; import io.deephaven.iceberg.location.IcebergTableLocationFactory; import io.deephaven.iceberg.location.IcebergTableLocationKey; import io.deephaven.parquet.table.ParquetInstructions; @@ -44,13 +44,13 @@ public class IcebergCatalog { private final S3Instructions s3Instructions; /** - * Construct an IcebergCatalog given a set of configurable instructions.. + * Construct an IcebergCatalog given a set of configurable instructions. * * @param name The optional service name */ IcebergCatalog(final @Nullable String name, final IcebergInstructions instructions) { // Set up the properties map for the Iceberg catalog - Map properties = new HashMap<>(); + final Map properties = new HashMap<>(); final Configuration conf = new Configuration(); @@ -181,7 +181,7 @@ private Table readTableInternal( partitionSpec.fields().stream().map(PartitionField::name).toArray(String[]::new); // Create the partitioning column location key finder - keyFinder = new IcebergPartitionedLayout( + keyFinder = new IcebergKeyValuePartitionedLayout( snapshot, fileIO, partitionColumns, diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java index ec9791af1fc..d361be2bb07 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java @@ -5,6 +5,7 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.locations.TableDataException; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -35,20 +36,14 @@ static TableDefinition fromSchema(final Schema schema, PartitionSpec partitionSp for (final Types.NestedField field : schema.columns()) { final String name = field.name(); final Type type = field.type(); - try { - final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); - final ColumnDefinition column; - if (partitionNames.contains(name)) { - column = ColumnDefinition.of(name, qstType).withPartitioning(); - } else { - column = ColumnDefinition.of(name, qstType); - } - columns.add(column); - } catch (UnsupportedOperationException e) { - // TODO: Currently will silently skip the column. Would it be better to skip and warn the user or - // break and declare failure? We don't have a mechanism for skipping columns, do we need an overload - // with a supplied table definition? + final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); + final ColumnDefinition column; + if (partitionNames.contains(name)) { + column = ColumnDefinition.of(name, qstType).withPartitioning(); + } else { + column = ColumnDefinition.of(name, qstType); } + columns.add(column); } return TableDefinition.of(columns); @@ -84,7 +79,9 @@ static io.deephaven.qst.type.Type convertPrimitiveType(final Type icebergType return io.deephaven.qst.type.Type.find(byte[].class); } } - throw new UnsupportedOperationException("Unsupported type: " + typeId); + throw new TableDataException( + "Unsupported iceberg column type " + typeId.name() + + " with logical type " + typeId.javaClass()); } private IcebergTools() {} diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index fcff074f523..a1d47f5aacc 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -32,6 +32,11 @@ public void setUp() { .build(); } + @Test + public void testNothing() { + // Dummy to prevent JUnit from complaining about no tests + } + // TODO: discuss how to perform tests since they require a full MiniIO + Iceberg setup // @Test From c7487fef3660e0e9fb470cba2f41f615351bc307 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 2 Apr 2024 14:50:11 -0700 Subject: [PATCH 07/25] Fix gradle broken-ness. --- extensions/iceberg/build.gradle | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle index b10d538a345..3e5617e2017 100644 --- a/extensions/iceberg/build.gradle +++ b/extensions/iceberg/build.gradle @@ -38,25 +38,16 @@ dependencies { implementation "org.apache.iceberg:iceberg-core:${icebergVersion}" implementation "org.apache.iceberg:iceberg-core:${icebergVersion}:tests" implementation "org.apache.iceberg:iceberg-aws:${icebergVersion}" - implementation "org.apache.iceberg:iceberg-azure:${icebergVersion}" - implementation "org.apache.iceberg:iceberg-gcp:${icebergVersion}" + // implementation "org.apache.iceberg:iceberg-azure:${icebergVersion}" + // implementation "org.apache.iceberg:iceberg-gcp:${icebergVersion}" implementation "org.apache.iceberg:iceberg-bundled-guava:${icebergVersion}" implementation "org.apache.hadoop:hadoop-common:${hadoopVersion}" implementation "org.apache.hadoop:hadoop-hdfs-client:${hadoopVersion}" runtimeOnly "org.apache.iceberg:iceberg-aws-bundle:${icebergVersion}" - runtimeOnly "org.apache.iceberg:iceberg-azure-bundle:${icebergVersion}" - runtimeOnly "org.apache.iceberg:iceberg-gcp-bundle:${icebergVersion}" - - implementation 'org.apache.httpcomponents.client5:httpclient5:5.3.1' - implementation 'org.apache.httpcomponents.core5:httpcore5:5.2.4' - - implementation 'com.google.guava:guava:33.0.0-jre' - - implementation 'org.xerial:sqlite-jdbc:3.45.1.0' - implementation 'org.postgresql:postgresql:42.7.1' - + // runtimeOnly "org.apache.iceberg:iceberg-azure-bundle:${icebergVersion}" + // runtimeOnly "org.apache.iceberg:iceberg-gcp-bundle:${icebergVersion}" testImplementation depCommonsCompress testImplementation depCommonsIo From d4a80b8cc4b882433917d129f97a84601f2baf14 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 2 Apr 2024 16:12:27 -0700 Subject: [PATCH 08/25] Gradle comments, a few changes to IcebergInstructions --- extensions/iceberg/build.gradle | 6 ++- .../location/IcebergTableLocationFactory.java | 13 +----- .../iceberg/util/IcebergCatalog.java | 40 ++++++++++++++----- .../iceberg/util/IcebergInstructions.java | 30 +++++--------- 4 files changed, 47 insertions(+), 42 deletions(-) diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle index 3e5617e2017..63d59f65bf1 100644 --- a/extensions/iceberg/build.gradle +++ b/extensions/iceberg/build.gradle @@ -38,6 +38,7 @@ dependencies { implementation "org.apache.iceberg:iceberg-core:${icebergVersion}" implementation "org.apache.iceberg:iceberg-core:${icebergVersion}:tests" implementation "org.apache.iceberg:iceberg-aws:${icebergVersion}" + // The following lines can be uncommented when we enable support for Azure and GCP // implementation "org.apache.iceberg:iceberg-azure:${icebergVersion}" // implementation "org.apache.iceberg:iceberg-gcp:${icebergVersion}" implementation "org.apache.iceberg:iceberg-bundled-guava:${icebergVersion}" @@ -45,9 +46,10 @@ dependencies { implementation "org.apache.hadoop:hadoop-common:${hadoopVersion}" implementation "org.apache.hadoop:hadoop-hdfs-client:${hadoopVersion}" - runtimeOnly "org.apache.iceberg:iceberg-aws-bundle:${icebergVersion}" + // The following lines can be uncommented when we enable support for Azure and GCP + // runtimeOnly "org.apache.iceberg:iceberg-aws-bundle:${icebergVersion}" // runtimeOnly "org.apache.iceberg:iceberg-azure-bundle:${icebergVersion}" - // runtimeOnly "org.apache.iceberg:iceberg-gcp-bundle:${icebergVersion}" + runtimeOnly "org.apache.iceberg:iceberg-gcp-bundle:${icebergVersion}" testImplementation depCommonsCompress testImplementation depCommonsIo diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java index b8b5a8cf609..a8889d5e6ee 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java @@ -5,17 +5,11 @@ import io.deephaven.engine.table.impl.locations.TableKey; import io.deephaven.engine.table.impl.locations.TableLocation; -import io.deephaven.engine.table.impl.locations.impl.NonexistentTableLocation; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.io.File; -import java.net.URI; - -import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; - /** * {@link TableLocationFactory} for {@link IcebergTableLocation}s. */ @@ -31,11 +25,6 @@ public IcebergTableLocationFactory(@NotNull final Object readInstructions) { public TableLocation makeLocation(@NotNull final TableKey tableKey, @NotNull final IcebergTableLocationKey locationKey, @Nullable final TableDataRefreshService refreshService) { - final URI fileURI = locationKey.getURI(); - if (!FILE_URI_SCHEME.equals(fileURI.getScheme()) || new File(fileURI).exists()) { - return new IcebergTableLocation(tableKey, locationKey, readInstructions); - } else { - return new NonexistentTableLocation(tableKey, locationKey); - } + return new IcebergTableLocation(tableKey, locationKey, readInstructions); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java index 440f605113e..5c6cb24d460 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java @@ -69,18 +69,24 @@ public class IcebergCatalog { properties.put(CatalogProperties.FILE_IO_IMPL, instructions.fileIOImpl().value); if (instructions.fileIOImpl() == IcebergInstructions.FILEIO_IMPL.S3) { - final S3Instructions.Builder builder = S3Instructions.builder() - .credentials(Credentials.basic(instructions.s3AccessKeyId(), instructions.s3SecretAccessKey())) - .regionName(instructions.s3Region()); - - properties.put(AwsClientProperties.CLIENT_REGION, instructions.s3Region()); - properties.put(S3FileIOProperties.ACCESS_KEY_ID, instructions.s3AccessKeyId()); - properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, instructions.s3SecretAccessKey()); + // Configure the properties map from the Iceberg instructions. + if (instructions.s3AccessKeyId().isPresent() && instructions.s3SecretAccessKey().isPresent()) { + properties.put(S3FileIOProperties.ACCESS_KEY_ID, instructions.s3AccessKeyId().get()); + properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, instructions.s3SecretAccessKey().get()); + } + if (instructions.s3Region().isPresent()) { + properties.put(AwsClientProperties.CLIENT_REGION, instructions.s3Region().get()); + } if (instructions.s3EndpointOverride().isPresent()) { properties.put(S3FileIOProperties.ENDPOINT, instructions.s3EndpointOverride().get()); - builder.endpointOverride(instructions.s3EndpointOverride().get()); } - s3Instructions = builder.build(); + + // The user may have provided readInstructions. If they did, we'll use them for the data file access. + // Otherwise we need to build one from the properties. + s3Instructions = instructions.readInstructions().isPresent() + ? (S3Instructions) instructions.readInstructions().get() + : buildS3Instructions(properties); + // TODO: create a FileIO interface wrapping the Deephaven S3SeekableByteChannel/Provider fileIO = CatalogUtil.loadFileIO(instructions.fileIOImpl().value, properties, conf); } else { @@ -91,6 +97,22 @@ public class IcebergCatalog { catalog.initialize(catalogName, properties); } + private static S3Instructions buildS3Instructions(final Map properties) { + final S3Instructions.Builder builder = S3Instructions.builder(); + if (properties.containsKey(S3FileIOProperties.ACCESS_KEY_ID) + && properties.containsKey(S3FileIOProperties.SECRET_ACCESS_KEY)) { + builder.credentials(Credentials.basic(properties.get(S3FileIOProperties.ACCESS_KEY_ID), + properties.get(S3FileIOProperties.SECRET_ACCESS_KEY))); + } + if (properties.containsKey(AwsClientProperties.CLIENT_REGION)) { + builder.regionName(properties.get(AwsClientProperties.CLIENT_REGION)); + } + if (properties.containsKey(S3FileIOProperties.ENDPOINT)) { + builder.endpointOverride(properties.get(S3FileIOProperties.ENDPOINT)); + } + return builder.build(); + } + @SuppressWarnings("unused") public List listTables(final Namespace namespace) { // TODO: have this return a Deephaven Table of table identifiers diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 915b0b33bbc..78e84302f85 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -81,17 +81,22 @@ public FILEIO_IMPL fileIOImpl() { /** * The AWS access key, used to identify the user interacting with services. */ - public abstract String s3AccessKeyId(); + public abstract Optional s3AccessKeyId(); /** * The AWS secret access key, used to authenticate the user interacting with services. */ - public abstract String s3SecretAccessKey(); + public abstract Optional s3SecretAccessKey(); /** * The AWS region to use for this connection. */ - public abstract String s3Region(); + public abstract Optional s3Region(); + + /** + * The {@link io.deephaven.extensions.s3.S3Instructions} to use for reading the Iceberg data files. + */ + public abstract Optional readInstructions(); public interface Builder { Builder catalogImpl(CATALOG_IMPL catalogImpl); @@ -110,6 +115,9 @@ public interface Builder { Builder s3Region(String s3Region); + @SuppressWarnings("unused") + Builder readInstructions(Object readInstructions); + IcebergInstructions build(); } @@ -126,20 +134,4 @@ final void checkWarehouseLocation() { throw new IllegalArgumentException("warehouseLocation must be provided"); } } - - @Check - final void checkS3Fields() { - if (fileIOImpl() == FILEIO_IMPL.S3) { - if (s3AccessKeyId() == null || s3AccessKeyId().isEmpty()) { - throw new IllegalArgumentException("When using S3 FileIO, s3AccessKeyId must be provided"); - } - if (s3SecretAccessKey() == null || s3SecretAccessKey().isEmpty()) { - throw new IllegalArgumentException("When using S3 FileIO, s3SecretAccessKey must be provided"); - } - if (s3Region() == null || s3Region().isEmpty()) { - throw new IllegalArgumentException("When using S3 FileIO, s3Region must be provided"); - } - } - } - } From 4f2ba2840dded39c59613ece90279af0d3a176d4 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 2 Apr 2024 17:13:24 -0700 Subject: [PATCH 09/25] Final gradle fix (uncomment the correct lines) --- extensions/iceberg/build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle index 63d59f65bf1..0644b887935 100644 --- a/extensions/iceberg/build.gradle +++ b/extensions/iceberg/build.gradle @@ -46,10 +46,10 @@ dependencies { implementation "org.apache.hadoop:hadoop-common:${hadoopVersion}" implementation "org.apache.hadoop:hadoop-hdfs-client:${hadoopVersion}" + runtimeOnly "org.apache.iceberg:iceberg-aws-bundle:${icebergVersion}" // The following lines can be uncommented when we enable support for Azure and GCP - // runtimeOnly "org.apache.iceberg:iceberg-aws-bundle:${icebergVersion}" // runtimeOnly "org.apache.iceberg:iceberg-azure-bundle:${icebergVersion}" - runtimeOnly "org.apache.iceberg:iceberg-gcp-bundle:${icebergVersion}" + // runtimeOnly "org.apache.iceberg:iceberg-gcp-bundle:${icebergVersion}" testImplementation depCommonsCompress testImplementation depCommonsIo From 3bd98f939a0567094a0eafa3b6736f563d523539 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 18 Apr 2024 08:37:54 -0700 Subject: [PATCH 10/25] Addressed PR comments, more testing needed. --- .../iceberg/layout/IcebergFlatLayout.java | 62 +++-- .../IcebergKeyValuePartitionedLayout.java | 93 +++++-- .../location/IcebergTableLocation.java | 131 ---------- .../location/IcebergTableLocationFactory.java | 13 +- .../location/IcebergTableLocationKey.java | 61 +---- .../location/IcebergTableParquetLocation.java | 29 +++ .../IcebergTableParquetLocationKey.java | 51 ++++ .../iceberg/util/IcebergCatalog.java | 243 ------------------ .../iceberg/util/IcebergCatalogAdapter.java | 230 +++++++++++++++++ .../iceberg/util/IcebergInstructions.java | 112 ++------ .../deephaven/iceberg/util/IcebergTools.java | 151 ++++++----- .../IcebergToolsTest.java | 155 ++++++++--- 12 files changed, 670 insertions(+), 661 deletions(-) delete mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocation.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java delete mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java index fa7bce7862a..5e6036da5da 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -4,9 +4,12 @@ package io.deephaven.iceberg.layout; import io.deephaven.base.FileUtils; +import io.deephaven.base.verify.Assert; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; +import io.deephaven.iceberg.location.IcebergTableParquetLocationKey; +import io.deephaven.parquet.table.ParquetInstructions; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; import org.jetbrains.annotations.NotNull; @@ -22,27 +25,54 @@ * from a {@link org.apache.iceberg.Snapshot} */ public final class IcebergFlatLayout implements TableLocationKeyFinder { - private final Snapshot tableSnapshot; + /** + * The Iceberg {@link Table} to discover locations for. + */ + private final Table table; + + /** + * The {@link FileIO} to use for passing to the catalog reading manifest data files. + */ private final FileIO fileIO; + + /** + * A cache of {@link IcebergTableLocationKey}s keyed by the URI of the file they represent. + */ private final Map cache; + + /** + * The instructions for customizations while reading. Could be a {@link ParquetInstructions} or similar. + */ private final Object readInstructions; + /** + * The current {@link Snapshot} to discover locations for. + */ + private Snapshot currentSnapshot; + private static IcebergTableLocationKey locationKey( final FileFormat format, final URI fileUri, @NotNull final Object readInstructions) { - return new IcebergTableLocationKey(format, fileUri, 0, null, readInstructions); + if (format == FileFormat.PARQUET) { + return new IcebergTableParquetLocationKey(fileUri, 0, null, (ParquetInstructions) readInstructions); + } + throw new UnsupportedOperationException("Unsupported file format: " + format); } /** + * @param table The {@link Table} to discover locations for. * @param tableSnapshot The {@link Snapshot} from which to discover data files. - * @param readInstructions the instructions for customizations while reading. + * @param fileIO The file IO to use for reading manifest data files. + * @param readInstructions The instructions for customizations while reading. */ public IcebergFlatLayout( + @NotNull final Table table, @NotNull final Snapshot tableSnapshot, @NotNull final FileIO fileIO, @NotNull final Object readInstructions) { - this.tableSnapshot = tableSnapshot; + this.table = table; + this.currentSnapshot = tableSnapshot; this.fileIO = fileIO; this.readInstructions = readInstructions; @@ -50,31 +80,33 @@ public IcebergFlatLayout( } public String toString() { - return IcebergFlatLayout.class.getSimpleName() + '[' + tableSnapshot + ']'; + return IcebergFlatLayout.class.getSimpleName() + '[' + table.name() + ']'; } @Override public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { try { // Retrieve the manifest files from the snapshot - final List manifestFiles = tableSnapshot.allManifests(fileIO); + final List manifestFiles = currentSnapshot.allManifests(fileIO); for (final ManifestFile manifestFile : manifestFiles) { + // Currently only can process manifest files with DATA content type. + Assert.eq(manifestFile.content(), "manifestFile.content()", + ManifestContent.DATA, "ManifestContent.DATA"); final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO); for (DataFile df : reader) { final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); - IcebergTableLocationKey locationKey = cache.get(fileUri); - if (locationKey == null) { - locationKey = locationKey(df.format(), fileUri, readInstructions); - if (!locationKey.verifyFileReader()) { - continue; - } - cache.put(fileUri, locationKey); + final IcebergTableLocationKey locationKey = cache.computeIfAbsent(fileUri, uri -> { + final IcebergTableLocationKey key = locationKey(df.format(), fileUri, readInstructions); + // Verify before caching. + return key.verifyFileReader() ? key : null; + }); + if (locationKey != null) { + locationKeyObserver.accept(locationKey); } - locationKeyObserver.accept(locationKey); } } } catch (final Exception e) { - throw new TableDataException("Error finding Iceberg locations under " + tableSnapshot, e); + throw new TableDataException("Error finding Iceberg locations under " + currentSnapshot, e); } } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java index 305cca303f7..e4290e402ef 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java @@ -4,9 +4,13 @@ package io.deephaven.iceberg.layout; import io.deephaven.base.FileUtils; +import io.deephaven.base.verify.Assert; import io.deephaven.engine.table.impl.locations.TableDataException; 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.util.IcebergInstructions; +import io.deephaven.parquet.table.ParquetInstructions; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; import org.jetbrains.annotations.NotNull; @@ -19,45 +23,89 @@ import java.util.function.Consumer; /** - * Iceberg {@link TableLocationKeyFinder location finder} for tables without partitions that will discover data files - * from a {@link org.apache.iceberg.Snapshot} + * Iceberg {@link TableLocationKeyFinder location finder} for tables with partitions that will discover data files from + * a {@link org.apache.iceberg.Snapshot} */ public final class IcebergKeyValuePartitionedLayout implements TableLocationKeyFinder { - private final Snapshot tableSnapshot; + /** + * The Iceberg {@link Table} to discover locations for. + */ + private final Table table; + + /** + * The {@link FileIO} to use for passing to the catalog reading manifest data files. + */ private final FileIO fileIO; + + /** + * The columns to use for partitioning. + */ private final String[] partitionColumns; - private final Object readInstructions; + + /** + * A cache of {@link IcebergTableLocationKey}s keyed by the URI of the file they represent. + */ private final Map cache; + /** + * The instructions for customizations while reading. Could be a {@link ParquetInstructions} or similar. + */ + private final IcebergInstructions instructions; + + /** + * The current {@link Snapshot} to discover locations for. + */ + private Snapshot currentSnapshot; + private static IcebergTableLocationKey locationKey( final FileFormat format, final URI fileUri, final Map> partitions, - @NotNull final Object readInstructions) { - return new IcebergTableLocationKey(format, fileUri, 0, partitions, readInstructions); + @NotNull final IcebergInstructions instructions) { + + if (format == FileFormat.PARQUET) { + final ParquetInstructions parquetInstructions; + if (instructions.parquetInstructions().isPresent()) { + // Accept the user supplied instructions without change + parquetInstructions = instructions.parquetInstructions().get(); + } else if (instructions.s3Instructions().isPresent()) { + // Create a default Parquet instruction object from the S3 instructions + parquetInstructions = ParquetInstructions.builder() + .setSpecialInstructions(instructions.s3Instructions().get()) + .build(); + } else { + // Create a default Parquet instruction object + parquetInstructions = ParquetInstructions.builder().build(); + } + return new IcebergTableParquetLocationKey(fileUri, 0, partitions, parquetInstructions); + } + throw new UnsupportedOperationException("Unsupported file format: " + format); } /** + * @param table The {@link Table} to discover locations for. * @param tableSnapshot The {@link Snapshot} from which to discover data files. * @param fileIO The file IO to use for reading manifest data files. * @param partitionColumns The columns to use for partitioning. - * @param readInstructions The instructions for customizations while reading. + * @param instructions The instructions for customizations while reading. */ public IcebergKeyValuePartitionedLayout( + @NotNull final Table table, @NotNull final Snapshot tableSnapshot, @NotNull final FileIO fileIO, @NotNull final String[] partitionColumns, - @NotNull final Object readInstructions) { - this.tableSnapshot = tableSnapshot; + @NotNull final IcebergInstructions instructions) { + this.table = table; + this.currentSnapshot = tableSnapshot; this.fileIO = fileIO; this.partitionColumns = partitionColumns; - this.readInstructions = readInstructions; + this.instructions = instructions; this.cache = new HashMap<>(); } public String toString() { - return IcebergFlatLayout.class.getSimpleName() + '[' + tableSnapshot + ']'; + return IcebergFlatLayout.class.getSimpleName() + '[' + table.name() + ']'; } @Override @@ -65,28 +113,31 @@ public synchronized void findKeys(@NotNull final Consumer> partitions = new LinkedHashMap<>(); try { // Retrieve the manifest files from the snapshot - final List manifestFiles = tableSnapshot.allManifests(fileIO); + final List manifestFiles = currentSnapshot.allManifests(fileIO); for (final ManifestFile manifestFile : manifestFiles) { + // Currently only can process manifest files with DATA content type. + Assert.eq(manifestFile.content(), "manifestFile.content()", + ManifestContent.DATA, "ManifestContent.DATA"); final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO); for (DataFile df : reader) { final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); - IcebergTableLocationKey locationKey = cache.get(fileUri); - if (locationKey == null) { + final IcebergTableLocationKey locationKey = cache.computeIfAbsent(fileUri, uri -> { final PartitionData partitionData = (PartitionData) df.partition(); for (int ii = 0; ii < partitionColumns.length; ++ii) { partitions.put(partitionColumns[ii], (Comparable) partitionData.get(ii)); } - locationKey = locationKey(df.format(), fileUri, partitions, readInstructions); - if (!locationKey.verifyFileReader()) { - continue; - } - cache.put(fileUri, locationKey); + final IcebergTableLocationKey key = + locationKey(df.format(), fileUri, partitions, instructions); + // Verify before caching. + return key.verifyFileReader() ? key : null; + }); + if (locationKey != null) { + locationKeyObserver.accept(locationKey); } - locationKeyObserver.accept(locationKey); } } } catch (final Exception e) { - throw new TableDataException("Error finding Iceberg locations under " + tableSnapshot, e); + throw new TableDataException("Error finding Iceberg locations under " + currentSnapshot, e); } } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java deleted file mode 100644 index 60857794ef7..00000000000 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocation.java +++ /dev/null @@ -1,131 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.iceberg.location; - -import io.deephaven.api.SortColumn; -import io.deephaven.base.verify.Require; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.BasicDataIndex; -import io.deephaven.engine.table.impl.locations.*; -import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation; -import io.deephaven.parquet.table.ParquetInstructions; -import io.deephaven.parquet.table.location.ParquetTableLocation; -import io.deephaven.parquet.table.location.ParquetTableLocationKey; -import org.apache.iceberg.FileFormat; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.List; - -public class IcebergTableLocation implements TableLocation { - - private static final String IMPLEMENTATION_NAME = IcebergTableLocation.class.getSimpleName(); - - private final ImmutableTableKey tableKey; - private final ImmutableTableLocationKey tableLocationKey; - - private final AbstractTableLocation internalTableLocation; - - public IcebergTableLocation(@NotNull final TableKey tableKey, - @NotNull final IcebergTableLocationKey tableLocationKey, - @NotNull final Object readInstructions) { - this.tableKey = Require.neqNull(tableKey, "tableKey").makeImmutable(); - this.tableLocationKey = Require.neqNull(tableLocationKey, "tableLocationKey").makeImmutable(); - - if (tableLocationKey.format == FileFormat.PARQUET) { - this.internalTableLocation = new ParquetTableLocation(tableKey, - (ParquetTableLocationKey) tableLocationKey.internalTableLocationKey, - (ParquetInstructions) readInstructions); - } else { - throw new IllegalArgumentException("Unsupported file format: " + tableLocationKey.format); - } - } - - @Override - public String getImplementationName() { - return IMPLEMENTATION_NAME; - } - - @Override - @NotNull - public ImmutableTableKey getTableKey() { - return tableKey; - } - - @Override - @NotNull - public ImmutableTableLocationKey getKey() { - return tableLocationKey; - } - - @Override - public boolean supportsSubscriptions() { - return internalTableLocation.supportsSubscriptions(); - } - - @Override - public void subscribe(@NotNull Listener listener) { - internalTableLocation.subscribe(listener); - } - - @Override - public void unsubscribe(@NotNull Listener listener) { - internalTableLocation.unsubscribe(listener); - } - - @Override - public void refresh() { - internalTableLocation.refresh(); - } - - @Override - @NotNull - public List getSortedColumns() { - return internalTableLocation.getSortedColumns(); - } - - @Override - @NotNull - public List getDataIndexColumns() { - return internalTableLocation.getDataIndexColumns(); - } - - @Override - public boolean hasDataIndex(@NotNull String... columns) { - return internalTableLocation.hasDataIndex(columns); - } - - @Override - @Nullable - public BasicDataIndex getDataIndex(@NotNull String... columns) { - return internalTableLocation.getDataIndex(columns); - } - - @Override - @NotNull - public ColumnLocation getColumnLocation(@NotNull CharSequence name) { - return internalTableLocation.getColumnLocation(name); - } - - @Override - @NotNull - public Object getStateLock() { - return internalTableLocation.getStateLock(); - } - - @Override - public RowSet getRowSet() { - return internalTableLocation.getRowSet(); - } - - @Override - public long getSize() { - return internalTableLocation.getSize(); - } - - @Override - public long getLastModifiedTimeMillis() { - return internalTableLocation.getLastModifiedTimeMillis(); - } -} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java index a8889d5e6ee..773df6371de 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java @@ -11,20 +11,19 @@ import org.jetbrains.annotations.Nullable; /** - * {@link TableLocationFactory} for {@link IcebergTableLocation}s. + * {@link TableLocationFactory} for Iceberg {@link TableLocation}s. */ public final class IcebergTableLocationFactory implements TableLocationFactory { - private final Object readInstructions; - - public IcebergTableLocationFactory(@NotNull final Object readInstructions) { - this.readInstructions = readInstructions; - } + public IcebergTableLocationFactory() {} @Override @NotNull public TableLocation makeLocation(@NotNull final TableKey tableKey, @NotNull final IcebergTableLocationKey locationKey, @Nullable final TableDataRefreshService refreshService) { - return new IcebergTableLocation(tableKey, locationKey, readInstructions); + if (locationKey instanceof IcebergTableParquetLocationKey) { + return new IcebergTableParquetLocation(tableKey, locationKey); + } + throw new UnsupportedOperationException("Unsupported location key type: " + locationKey.getClass()); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java index 28c3b688f27..42226bfa4ea 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java @@ -4,67 +4,22 @@ package io.deephaven.iceberg.location; import io.deephaven.engine.table.impl.locations.TableLocationKey; -import io.deephaven.engine.table.impl.locations.local.URITableLocationKey; -import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.location.ParquetTableLocationKey; -import org.apache.iceberg.FileFormat; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.net.URI; -import java.util.Map; /** - * {@link TableLocationKey} implementation for use with data stored in the parquet format. + * {@link TableLocationKey} implementation for use with data stored in Iceberg tables. */ -public class IcebergTableLocationKey extends URITableLocationKey { - - private static final String IMPLEMENTATION_NAME = IcebergTableLocationKey.class.getSimpleName(); - - final FileFormat format; - final URITableLocationKey internalTableLocationKey; - +public interface IcebergTableLocationKey extends TableLocationKey { /** - * Construct a new IcebergTableLocationKey for the supplied {@code fileUri} and {@code partitions}. + * Get the read instructions for the location. * - * @param fileUri The file that backs the keyed location. - * @param order Explicit ordering index, taking precedence over other fields - * @param partitions The table partitions enclosing the table location keyed by {@code this}. Note that if this - * parameter is {@code null}, the location will be a member of no partitions. An ordered copy of the map will - * be made, so the calling code is free to mutate the map after this call - * @param readInstructions the instructions for customizations while reading + * @return the read instructions */ - public IcebergTableLocationKey( - final FileFormat format, - @NotNull final URI fileUri, - final int order, - @Nullable final Map> partitions, - @NotNull final Object readInstructions) { - super(fileUri, order, partitions); - this.format = format; - if (format == FileFormat.PARQUET) { - // This constructor will perform validation of the parquet file - final ParquetInstructions parquetInstructions = (ParquetInstructions) readInstructions; - this.internalTableLocationKey = - new ParquetTableLocationKey(fileUri, order, partitions, parquetInstructions); - } else { - throw new IllegalArgumentException("Unsupported file format: " + format); - } - } - - @Override - public String getImplementationName() { - return IMPLEMENTATION_NAME; - } + Object readInstructions(); /** - * See {@link ParquetTableLocationKey#verifyFileReader()}. + * Verify that a reader for the file can be created successfully. Delegates to + * {@link ParquetTableLocationKey#verifyFileReader()} for parquet files. */ - public synchronized boolean verifyFileReader() { - if (format == FileFormat.PARQUET) { - return ((ParquetTableLocationKey) internalTableLocationKey).verifyFileReader(); - } else { - throw new IllegalArgumentException("Unsupported file format: " + format); - } - } + boolean verifyFileReader(); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocation.java new file mode 100644 index 00000000000..9575c3bd42c --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocation.java @@ -0,0 +1,29 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.location; + +import io.deephaven.engine.table.impl.locations.TableKey; +import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.location.ParquetTableLocation; +import io.deephaven.parquet.table.location.ParquetTableLocationKey; +import org.jetbrains.annotations.NotNull; + +/** + * {@link TableLocation} implementation for use with data stored in Iceberg tables in the parquet format. + */ +public class IcebergTableParquetLocation extends ParquetTableLocation { + private static final String IMPLEMENTATION_NAME = IcebergTableParquetLocation.class.getSimpleName(); + + public IcebergTableParquetLocation(@NotNull final TableKey tableKey, + @NotNull final IcebergTableLocationKey tableLocationKey) { + super(tableKey, (ParquetTableLocationKey) tableLocationKey, + (ParquetInstructions) tableLocationKey.readInstructions()); + } + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java new file mode 100644 index 00000000000..40b7d680314 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java @@ -0,0 +1,51 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.location; + +import io.deephaven.engine.table.impl.locations.TableLocationKey; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.location.ParquetTableLocationKey; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.net.URI; +import java.util.Map; + +/** + * {@link TableLocationKey} implementation for use with data stored in Iceberg tables in the parquet format. + */ +public class IcebergTableParquetLocationKey extends ParquetTableLocationKey implements IcebergTableLocationKey { + + private static final String IMPLEMENTATION_NAME = IcebergTableParquetLocationKey.class.getSimpleName(); + private final ParquetInstructions readInstructions; + + /** + * Construct a new IcebergTableParquetLocationKey for the supplied {@code fileUri} and {@code partitions}. + * + * @param fileUri The file that backs the keyed location + * @param order Explicit ordering index, taking precedence over other fields + * @param partitions The table partitions enclosing the table location keyed by {@code this}. Note that if this + * parameter is {@code null}, the location will be a member of no partitions. An ordered copy of the map will + * be made, so the calling code is free to mutate the map after this call + * @param readInstructions the instructions for customizations while reading + */ + public IcebergTableParquetLocationKey( + @NotNull final URI fileUri, + final int order, + @Nullable final Map> partitions, + @NotNull final ParquetInstructions readInstructions) { + super(fileUri, order, partitions, readInstructions); + this.readInstructions = readInstructions; + } + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + @Override + public Object readInstructions() { + return readInstructions; + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java deleted file mode 100644 index 5c6cb24d460..00000000000 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalog.java +++ /dev/null @@ -1,243 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.iceberg.util; - -import io.deephaven.base.verify.Assert; -import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.table.impl.PartitionAwareSourceTable; -import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; -import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; -import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; -import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; -import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; -import io.deephaven.engine.updategraph.UpdateSourceRegistrar; -import io.deephaven.extensions.s3.Credentials; -import io.deephaven.extensions.s3.S3Instructions; -import io.deephaven.iceberg.layout.IcebergFlatLayout; -import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout; -import io.deephaven.iceberg.location.IcebergTableLocationFactory; -import io.deephaven.iceberg.location.IcebergTableLocationKey; -import io.deephaven.parquet.table.ParquetInstructions; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.*; -import org.apache.iceberg.aws.AwsClientProperties; -import org.apache.iceberg.aws.s3.S3FileIOProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.rest.RESTCatalog; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class IcebergCatalog { - private final Catalog catalog; - private final FileIO fileIO; - - private final S3Instructions s3Instructions; - - /** - * Construct an IcebergCatalog given a set of configurable instructions. - * - * @param name The optional service name - */ - IcebergCatalog(final @Nullable String name, final IcebergInstructions instructions) { - // Set up the properties map for the Iceberg catalog - final Map properties = new HashMap<>(); - - final Configuration conf = new Configuration(); - - properties.put(CatalogProperties.CATALOG_IMPL, instructions.catalogImpl().value); - if (instructions.catalogImpl() == IcebergInstructions.CATALOG_IMPL.RESTCatalog) { - final RESTCatalog restCatalog = new RESTCatalog(); - restCatalog.setConf(conf); - catalog = restCatalog; - } else { - throw new UnsupportedOperationException( - "Unsupported catalog implementation: " + instructions.catalogImpl()); - } - - properties.put(CatalogProperties.URI, instructions.catalogURI()); - properties.put(CatalogProperties.WAREHOUSE_LOCATION, instructions.warehouseLocation()); - - properties.put(CatalogProperties.FILE_IO_IMPL, instructions.fileIOImpl().value); - if (instructions.fileIOImpl() == IcebergInstructions.FILEIO_IMPL.S3) { - // Configure the properties map from the Iceberg instructions. - if (instructions.s3AccessKeyId().isPresent() && instructions.s3SecretAccessKey().isPresent()) { - properties.put(S3FileIOProperties.ACCESS_KEY_ID, instructions.s3AccessKeyId().get()); - properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, instructions.s3SecretAccessKey().get()); - } - if (instructions.s3Region().isPresent()) { - properties.put(AwsClientProperties.CLIENT_REGION, instructions.s3Region().get()); - } - if (instructions.s3EndpointOverride().isPresent()) { - properties.put(S3FileIOProperties.ENDPOINT, instructions.s3EndpointOverride().get()); - } - - // The user may have provided readInstructions. If they did, we'll use them for the data file access. - // Otherwise we need to build one from the properties. - s3Instructions = instructions.readInstructions().isPresent() - ? (S3Instructions) instructions.readInstructions().get() - : buildS3Instructions(properties); - - // TODO: create a FileIO interface wrapping the Deephaven S3SeekableByteChannel/Provider - fileIO = CatalogUtil.loadFileIO(instructions.fileIOImpl().value, properties, conf); - } else { - throw new UnsupportedOperationException("Unsupported file IO implementation: " + instructions.fileIOImpl()); - } - - final String catalogName = name != null ? name : "IcebergTableDataService-" + instructions.catalogURI(); - catalog.initialize(catalogName, properties); - } - - private static S3Instructions buildS3Instructions(final Map properties) { - final S3Instructions.Builder builder = S3Instructions.builder(); - if (properties.containsKey(S3FileIOProperties.ACCESS_KEY_ID) - && properties.containsKey(S3FileIOProperties.SECRET_ACCESS_KEY)) { - builder.credentials(Credentials.basic(properties.get(S3FileIOProperties.ACCESS_KEY_ID), - properties.get(S3FileIOProperties.SECRET_ACCESS_KEY))); - } - if (properties.containsKey(AwsClientProperties.CLIENT_REGION)) { - builder.regionName(properties.get(AwsClientProperties.CLIENT_REGION)); - } - if (properties.containsKey(S3FileIOProperties.ENDPOINT)) { - builder.endpointOverride(properties.get(S3FileIOProperties.ENDPOINT)); - } - return builder.build(); - } - - @SuppressWarnings("unused") - public List listTables(final Namespace namespace) { - // TODO: have this return a Deephaven Table of table identifiers - return catalog.listTables(namespace); - } - - /** - * Read a static snapshot of a table from the Iceberg catalog. - * - * @param tableIdentifier The table identifier to load - * @param snapshotId The snapshot ID to load - * @return The loaded table - */ - @SuppressWarnings("unused") - public Table readTable( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final String snapshotId) { - return readTableInternal(tableIdentifier, snapshotId, false); - } - - /** - * Read the latest static snapshot of a table from the Iceberg catalog. - * - * @param tableIdentifier The table identifier to load - * @return The loaded table - */ - @SuppressWarnings("unused") - public Table readTable(@NotNull final TableIdentifier tableIdentifier) { - return readTableInternal(tableIdentifier, null, false); - } - - /** - * Subscribe to a table from the Iceberg catalog. Initially the latest snapshot will be loaded, but the output table - * will be updated as new snapshots are added to the table. - * - * @param tableIdentifier The table identifier to load - * @return The loaded table - */ - @SuppressWarnings("unused") - public Table subscribeTable(@NotNull final TableIdentifier tableIdentifier) { - return readTableInternal(tableIdentifier, null, true); - } - - private Table readTableInternal( - @NotNull final TableIdentifier tableIdentifier, - @Nullable final String snapshotId, - final boolean isRefreshing) { - // Validate that the user is not trying to subscribe to a snapshot. - Assert.eqFalse(isRefreshing && (snapshotId != null), - "Must not specify a snapshot ID when subscribing to a table."); - - // Load the table from the catalog - final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); - - final Snapshot snapshot; - final Schema schema; - // Do we want the latest or a specific snapshot? - if (snapshotId == null) { - snapshot = table.currentSnapshot(); - schema = table.schema(); - } else { - // Load the specific snapshot and retrieve the schema for that snapshot - snapshot = table.snapshot(snapshotId); - schema = table.schemas().get(snapshot.schemaId()); - } - - // Load the partitioning schema - final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); - - // Convert the Iceberg schema to a Deephaven TableDefinition - final TableDefinition tableDefinition = IcebergTools.fromSchema(schema, partitionSpec); - - // Build a parquet instructions object - final ParquetInstructions instructions = ParquetInstructions.builder() - .setSpecialInstructions(s3Instructions) - .build(); - - final String description; - final TableLocationKeyFinder keyFinder; - final TableDataRefreshService refreshService; - final UpdateSourceRegistrar updateSourceRegistrar; - - if (partitionSpec.isUnpartitioned()) { - // Create the flat layout location key finder - keyFinder = new IcebergFlatLayout(snapshot, fileIO, instructions); - } else { - final String[] partitionColumns = - partitionSpec.fields().stream().map(PartitionField::name).toArray(String[]::new); - - // Create the partitioning column location key finder - keyFinder = new IcebergKeyValuePartitionedLayout( - snapshot, - fileIO, - partitionColumns, - instructions); - } - - if (isRefreshing) { - refreshService = TableDataRefreshService.getSharedRefreshService(); - updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); - description = "Read refreshing iceberg table with " + keyFinder; - } else { - refreshService = null; - updateSourceRegistrar = null; - description = "Read static iceberg table with " + keyFinder; - } - - return new PartitionAwareSourceTable( - tableDefinition, - description, - RegionedTableComponentFactoryImpl.INSTANCE, - new PollingTableLocationProvider<>( - StandaloneTableKey.getInstance(), - keyFinder, - new IcebergTableLocationFactory(instructions), - refreshService), - updateSourceRegistrar); - } - - - /** - * Return the internal Iceberg catalog. - */ - @SuppressWarnings("unused") - public Catalog catalog() { - return catalog; - } -} 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 new file mode 100644 index 00000000000..79107b3543b --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -0,0 +1,230 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.PartitionAwareSourceTable; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; +import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; +import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +import io.deephaven.iceberg.layout.IcebergFlatLayout; +import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout; +import io.deephaven.iceberg.location.IcebergTableLocationFactory; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.jetbrains.annotations.NotNull; + +import java.time.Instant; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +public class IcebergCatalogAdapter { + private final Catalog catalog; + private final FileIO fileIO; + private final IcebergInstructions instructions; + + /** + * Construct an IcebergCatalogAdapter given a set of configurable instructions. + */ + @SuppressWarnings("unused") + IcebergCatalogAdapter( + @NotNull final Catalog catalog, + @NotNull final FileIO fileIO) { + this(catalog, fileIO, IcebergInstructions.builder().build()); + } + + /** + * Construct an IcebergCatalogAdapter given a set of configurable instructions. + */ + IcebergCatalogAdapter( + @NotNull final Catalog catalog, + @NotNull final FileIO fileIO, + @NotNull final IcebergInstructions instructions) { + this.catalog = catalog; + this.fileIO = fileIO; + this.instructions = instructions; + } + + static TableDefinition fromSchema(final Schema schema, final PartitionSpec partitionSpec) { + final Set partitionNames = + partitionSpec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); + + final List> columns = new ArrayList<>(); + + for (final Types.NestedField field : schema.columns()) { + final String name = field.name(); + final Type type = field.type(); + final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); + final ColumnDefinition column; + if (partitionNames.contains(name)) { + column = ColumnDefinition.of(name, qstType).withPartitioning(); + } else { + column = ColumnDefinition.of(name, qstType); + } + columns.add(column); + } + + return TableDefinition.of(columns); + } + + static io.deephaven.qst.type.Type convertPrimitiveType(final Type icebergType) { + final Type.TypeID typeId = icebergType.typeId(); + switch (typeId) { + case BOOLEAN: + return io.deephaven.qst.type.Type.booleanType(); + case DOUBLE: + return io.deephaven.qst.type.Type.doubleType(); + case FLOAT: + return io.deephaven.qst.type.Type.floatType(); + case INTEGER: + return io.deephaven.qst.type.Type.intType(); + case LONG: + return io.deephaven.qst.type.Type.longType(); + case STRING: + return io.deephaven.qst.type.Type.stringType(); + case TIMESTAMP: + final Types.TimestampType timestampType = (Types.TimestampType) icebergType; + return timestampType.shouldAdjustToUTC() + ? io.deephaven.qst.type.Type.find(Instant.class) + : io.deephaven.qst.type.Type.find(LocalDateTime.class); + case DATE: + return io.deephaven.qst.type.Type.find(java.time.LocalDate.class); + case TIME: + return io.deephaven.qst.type.Type.find(java.time.LocalTime.class); + case DECIMAL: + return io.deephaven.qst.type.Type.find(java.math.BigDecimal.class); + case FIXED: // Fall through + case BINARY: + return io.deephaven.qst.type.Type.find(byte[].class); + default: + throw new TableDataException("Unsupported iceberg column type " + typeId.name()); + } + } + + @SuppressWarnings("unused") + public List listTables(final Namespace namespace) { + // TODO: have this return a Deephaven Table of table identifiers + return catalog.listTables(namespace); + } + + public List listTableSnapshots(@NotNull final TableIdentifier tableIdentifier) { + final ArrayList snapshotIds = new ArrayList<>(); + catalog.loadTable(tableIdentifier).snapshots().forEach(snapshot -> { + snapshotIds.add(snapshot.snapshotId()); + }); + return snapshotIds; + } + + /** + * Read the latest static snapshot of a table from the Iceberg catalog. + * + * @param tableIdentifier The table identifier to load + * @return The loaded table + */ + @SuppressWarnings("unused") + public Table snapshotTable(@NotNull final TableIdentifier tableIdentifier) { + return readTableInternal(tableIdentifier, -1, false); + } + + /** + * Read a static snapshot of a table from the Iceberg catalog. + * + * @param tableIdentifier The table identifier to load + * @param snapshotId The snapshot ID to load + * @return The loaded table + */ + @SuppressWarnings("unused") + public Table snapshotTable( + @NotNull final TableIdentifier tableIdentifier, + final long snapshotId) { + return readTableInternal(tableIdentifier, snapshotId, false); + } + + private Table readTableInternal( + @NotNull final TableIdentifier tableIdentifier, + final long snapshotId, + final boolean isRefreshing) { + + // Load the table from the catalog + final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); + + // Do we want the latest or a specific snapshot? + final Snapshot snapshot; + if (snapshotId < 0) { + snapshot = table.currentSnapshot(); + } else { + snapshot = table.snapshot(snapshotId); + } + final Schema schema = table.schemas().get(snapshot.schemaId()); + + // Load the partitioning schema + final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); + + // Convert the Iceberg schema to a Deephaven TableDefinition + final TableDefinition schemaTableDef = fromSchema(schema, partitionSpec); + final TableDefinition tableDef = instructions.tableDefinition().isPresent() ? + instructions.tableDefinition().get() : schemaTableDef; + + final String description; + final TableLocationKeyFinder keyFinder; + final TableDataRefreshService refreshService; + final UpdateSourceRegistrar updateSourceRegistrar; + + if (partitionSpec.isUnpartitioned()) { + // Create the flat layout location key finder + keyFinder = new IcebergFlatLayout(table, snapshot, fileIO, instructions); + } else { + final String[] partitionColumns = + partitionSpec.fields().stream().map(PartitionField::name).toArray(String[]::new); + + // Create the partitioning column location key finder + keyFinder = new IcebergKeyValuePartitionedLayout( + table, + snapshot, + fileIO, + partitionColumns, + instructions); + } + + if (isRefreshing) { + refreshService = TableDataRefreshService.getSharedRefreshService(); + updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + description = "Read refreshing iceberg table with " + keyFinder; + } else { + refreshService = null; + updateSourceRegistrar = null; + description = "Read static iceberg table with " + keyFinder; + } + + return new PartitionAwareSourceTable( + tableDef, + description, + RegionedTableComponentFactoryImpl.INSTANCE, + new PollingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + refreshService), + updateSourceRegistrar); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 78e84302f85..15a459ae10a 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -4,8 +4,10 @@ package io.deephaven.iceberg.util; import io.deephaven.annotations.BuildableStyle; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.extensions.s3.S3Instructions; +import io.deephaven.parquet.table.ParquetInstructions; import org.immutables.value.Value.Check; -import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; import java.util.Optional; @@ -21,117 +23,39 @@ public static Builder builder() { return ImmutableIcebergInstructions.builder(); } - public enum CATALOG_IMPL { - RESTCatalog("org.apache.iceberg.rest.RESTCatalog"); - - public final String value; - - CATALOG_IMPL(String label) { - this.value = label; - } - } - - public enum FILEIO_IMPL { - S3("org.apache.iceberg.aws.s3.S3FileIO"); - - public final String value; - - FILEIO_IMPL(String label) { - this.value = label; - } - } - - private final static CATALOG_IMPL DEFAULT_CATALOG_IMPL = CATALOG_IMPL.RESTCatalog; - private final static FILEIO_IMPL DEFAULT_FILEIO_IMPL = FILEIO_IMPL.S3; - /** - * A custom Catalog implementation to use by an engine. + * The {@link TableDefinition} to use when reading Iceberg data files. */ - @Default - public CATALOG_IMPL catalogImpl() { - return DEFAULT_CATALOG_IMPL; - } + public abstract Optional tableDefinition(); /** - * The URI string for the catalog. + * The {@link S3Instructions} to use for reading the Iceberg data files. This is mutually exclusive with + * {@link #parquetInstructions()} which can be used to provide custom instructions for reading the data files. */ - public abstract String catalogURI(); + public abstract Optional s3Instructions(); /** - * The root path of the data warehouse, where the manifest and data files are stored. + * The {@link ParquetInstructions} to use for reading the Iceberg data files. */ - public abstract String warehouseLocation(); + public abstract Optional parquetInstructions(); - /** - * The custom FileIO implementation to use in the catalog. - */ - @Default - public FILEIO_IMPL fileIOImpl() { - return DEFAULT_FILEIO_IMPL; - } - - /** - * The endpoint to connect to. Callers connecting to AWS do not typically need to set this; it is most useful when - * connecting to non-AWS, S3-compatible APIs. - * - * @see Amazon Simple Storage Service endpoints - */ - public abstract Optional s3EndpointOverride(); - - /** - * The AWS access key, used to identify the user interacting with services. - */ - public abstract Optional s3AccessKeyId(); - - /** - * The AWS secret access key, used to authenticate the user interacting with services. - */ - public abstract Optional s3SecretAccessKey(); - - /** - * The AWS region to use for this connection. - */ - public abstract Optional s3Region(); - - /** - * The {@link io.deephaven.extensions.s3.S3Instructions} to use for reading the Iceberg data files. - */ - public abstract Optional readInstructions(); public interface Builder { - Builder catalogImpl(CATALOG_IMPL catalogImpl); - - Builder catalogURI(String catalogURI); - - Builder warehouseLocation(String warehouseLocation); - - Builder fileIOImpl(FILEIO_IMPL fileIOImpl); - - Builder s3EndpointOverride(String s3EndpointOverride); - - Builder s3AccessKeyId(String s3AccessKeyId); - - Builder s3SecretAccessKey(String s3SecretAccessKey); - - Builder s3Region(String s3Region); + @SuppressWarnings("unused") + Builder tableDefinition(TableDefinition tableDefinition); @SuppressWarnings("unused") - Builder readInstructions(Object readInstructions); + Builder s3Instructions(S3Instructions s3Instructions); - IcebergInstructions build(); - } + Builder parquetInstructions(ParquetInstructions parquetInstructions); - @Check - final void checkCatalogURI() { - if (catalogURI() == null || catalogURI().isEmpty()) { - throw new IllegalArgumentException("catalogURI must be provided"); - } + IcebergInstructions build(); } @Check - final void checkWarehouseLocation() { - if (warehouseLocation() == null || warehouseLocation().isEmpty()) { - throw new IllegalArgumentException("warehouseLocation must be provided"); + final void checkInstructions() { + if (s3Instructions().isPresent() && parquetInstructions().isPresent()) { + throw new IllegalArgumentException("Only one of s3Instructions or parquetInstructions may be provided"); } } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java index d361be2bb07..a3bb264ed26 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java @@ -3,86 +3,101 @@ // package io.deephaven.iceberg.util; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.table.impl.locations.TableDataException; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -import java.time.Instant; -import java.time.LocalDateTime; -import java.util.*; -import java.util.stream.Collectors; +import io.deephaven.extensions.s3.Credentials; +import io.deephaven.extensions.s3.S3Instructions; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.aws.AwsClientProperties; +import org.apache.iceberg.aws.s3.S3FileIOProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.parquet.Strings; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.HashMap; +import java.util.Map; /** - * Tools for managing and manipulating tables on disk in parquet format. + * Tools for accessing tables in the Iceberg table format. */ public class IcebergTools { @SuppressWarnings("unused") - public static IcebergCatalog loadCatalog(final String name, final IcebergInstructions instructions) { - return new IcebergCatalog(name, instructions); + public static IcebergCatalogAdapter createAdapter( + final Catalog catalog, + final FileIO fileIO, + final IcebergInstructions instructions) { + return new IcebergCatalogAdapter(catalog, fileIO, instructions); } - static TableDefinition fromSchema(final Schema schema, PartitionSpec partitionSpec) { - final Set partitionNames = - partitionSpec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); - - final List> columns = new ArrayList<>(); - - for (final Types.NestedField field : schema.columns()) { - final String name = field.name(); - final Type type = field.type(); - final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); - final ColumnDefinition column; - if (partitionNames.contains(name)) { - column = ColumnDefinition.of(name, qstType).withPartitioning(); - } else { - column = ColumnDefinition.of(name, qstType); - } - columns.add(column); + private IcebergTools() {} + + + public static IcebergCatalogAdapter createS3Rest( + @Nullable final String name, + @NotNull final String catalogURI, + @NotNull final String warehouseLocation, + @Nullable final String region, + @Nullable final String accessKeyId, + @Nullable final String secretAccessKey, + @Nullable final String endpointOverride, + @Nullable final IcebergInstructions specialInstructions) { + + + // Set up the properties map for the Iceberg catalog + final Map properties = new HashMap<>(); + + final RESTCatalog catalog = new RESTCatalog(); + + properties.put(CatalogProperties.CATALOG_IMPL, "org.apache.iceberg.rest.RESTCatalog"); + properties.put(CatalogProperties.URI, catalogURI); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + + properties.put(CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.aws.s3.S3FileIO"); + + // Configure the properties map from the Iceberg instructions. + if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) { + properties.put(S3FileIOProperties.ACCESS_KEY_ID, accessKeyId); + properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, secretAccessKey); } + if (!Strings.isNullOrEmpty(region)) { + properties.put(AwsClientProperties.CLIENT_REGION, region); + } + if (!Strings.isNullOrEmpty(endpointOverride)) { + properties.put(S3FileIOProperties.ENDPOINT, endpointOverride); + } + + // TODO: create a FileIO interface wrapping the Deephaven S3SeekableByteChannel/Provider + final FileIO fileIO = CatalogUtil.loadFileIO("org.apache.iceberg.aws.s3.S3FileIO", properties, null); + + final String catalogName = name != null ? name : "IcebergTableDataService-" + catalogURI; + catalog.initialize(catalogName, properties); + + // If the user did not supply custom read instructions, let's create some defaults. + final IcebergInstructions instructions = specialInstructions != null + ? specialInstructions + : buildInstructions(properties); - return TableDefinition.of(columns); + return new IcebergCatalogAdapter(catalog, fileIO, instructions); } - static io.deephaven.qst.type.Type convertPrimitiveType(final Type icebergType) { - final Type.TypeID typeId = icebergType.typeId(); - if (icebergType.isPrimitiveType()) { - if (typeId == Type.TypeID.BOOLEAN) { - return io.deephaven.qst.type.Type.booleanType(); - } else if (typeId == Type.TypeID.DOUBLE) { - return io.deephaven.qst.type.Type.doubleType(); - } else if (typeId == Type.TypeID.FLOAT) { - return io.deephaven.qst.type.Type.floatType(); - } else if (typeId == Type.TypeID.INTEGER) { - return io.deephaven.qst.type.Type.intType(); - } else if (typeId == Type.TypeID.LONG) { - return io.deephaven.qst.type.Type.longType(); - } else if (typeId == Type.TypeID.STRING) { - return io.deephaven.qst.type.Type.stringType(); - } else if (typeId == Type.TypeID.TIMESTAMP) { - final Types.TimestampType timestampType = (Types.TimestampType) icebergType; - return timestampType.shouldAdjustToUTC() - ? io.deephaven.qst.type.Type.find(Instant.class) - : io.deephaven.qst.type.Type.find(LocalDateTime.class); - } else if (typeId == Type.TypeID.DATE) { - return io.deephaven.qst.type.Type.find(java.time.LocalDate.class); - } else if (typeId == Type.TypeID.TIME) { - return io.deephaven.qst.type.Type.find(java.time.LocalTime.class); - } else if (typeId == Type.TypeID.DECIMAL) { - return io.deephaven.qst.type.Type.find(java.math.BigDecimal.class); - } else if (typeId == Type.TypeID.FIXED || typeId == Type.TypeID.BINARY) { - return io.deephaven.qst.type.Type.find(byte[].class); - } + private static IcebergInstructions buildInstructions(final Map properties) { + final S3Instructions.Builder builder = S3Instructions.builder(); + if (properties.containsKey(S3FileIOProperties.ACCESS_KEY_ID) + && properties.containsKey(S3FileIOProperties.SECRET_ACCESS_KEY)) { + builder.credentials(Credentials.basic(properties.get(S3FileIOProperties.ACCESS_KEY_ID), + properties.get(S3FileIOProperties.SECRET_ACCESS_KEY))); } - throw new TableDataException( - "Unsupported iceberg column type " + typeId.name() + - " with logical type " + typeId.javaClass()); + if (properties.containsKey(AwsClientProperties.CLIENT_REGION)) { + builder.regionName(properties.get(AwsClientProperties.CLIENT_REGION)); + } + if (properties.containsKey(S3FileIOProperties.ENDPOINT)) { + builder.endpointOverride(properties.get(S3FileIOProperties.ENDPOINT)); + } + return IcebergInstructions.builder() + .s3Instructions(builder.build()) + .build(); } - - private IcebergTools() {} } diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index a1d47f5aacc..5d12d861b54 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -3,32 +3,45 @@ // package io.deephaven.iceberg; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.util.TableTools; -import io.deephaven.iceberg.util.IcebergCatalog; +import io.deephaven.extensions.s3.Credentials; +import io.deephaven.extensions.s3.S3Instructions; +import io.deephaven.iceberg.util.IcebergCatalogAdapter; import io.deephaven.iceberg.util.IcebergInstructions; import io.deephaven.iceberg.util.IcebergTools; +import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.time.DateTimeUtils; import junit.framework.TestCase; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.junit.Test; +import java.time.LocalDateTime; +import java.util.Collection; import java.util.List; public class IcebergToolsTest extends TestCase { IcebergInstructions instructions; + IcebergInstructions instructionsS3Only; @Override public void setUp() { + final S3Instructions s3Instructions = S3Instructions.builder() + .credentials(Credentials.basic("admin", "password")) + .endpointOverride("http://minio:9000") + .regionName("us-east-1") + .build(); + final ParquetInstructions parquetInstructions = ParquetInstructions.builder() + .setSpecialInstructions(s3Instructions) + .build(); + + instructionsS3Only = IcebergInstructions.builder() + .s3Instructions(s3Instructions) + .build(); instructions = IcebergInstructions.builder() - .catalogImpl(IcebergInstructions.CATALOG_IMPL.RESTCatalog) - .catalogURI("http://rest:8181") - .warehouseLocation("s3a://warehouse/wh") - .fileIOImpl(IcebergInstructions.FILEIO_IMPL.S3) - .s3AccessKeyId("admin") - .s3SecretAccessKey("password") - .s3EndpointOverride("http://minio:9000") - .s3Region("us-east-1") + .parquetInstructions(parquetInstructions) .build(); } @@ -39,28 +52,112 @@ public void testNothing() { // TODO: discuss how to perform tests since they require a full MiniIO + Iceberg setup - // @Test - // public void testListTables() { - // final IcebergCatalog catalog = IcebergTools.loadCatalog("minio-iceberg", instructions); - // - // final Namespace ns = Namespace.of("nyc"); - // final List tables = catalog.listTables(ns); - // } - // - // @Test - // public void testOpenTable() { - // final IcebergCatalog catalog = IcebergTools.loadCatalog("minio-iceberg", instructions); - // - // final Namespace ns = Namespace.of("nyc"); - // final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); - // io.deephaven.engine.table.Table table = catalog.readTable(tableId); - // - // TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); - // } - // + @Test + public void testListTables() { + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + instructions); + + final Namespace ns = Namespace.of("nyc"); + final Collection tables = adapter.listTables(ns); + } + + + @Test + public void testListTableSnapshots() { + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + instructions); + final Collection snapshots = adapter.listTableSnapshots(TableIdentifier.of("nyc", "taxis_partitioned")); + } + + @Test + public void testOpenTable() { + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + instructions); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + + @Test + public void testOpenTableDefinition() { + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofLong("year").withPartitioning(), + ColumnDefinition.ofInt("month").withPartitioning(), + ColumnDefinition.ofLong("VendorID"), + ColumnDefinition.fromGenericType("tpep_pickup_datetime", LocalDateTime.class), + ColumnDefinition.fromGenericType("tpep_dropoff_datetime", LocalDateTime.class), + ColumnDefinition.ofDouble("passenger_count")); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .s3Instructions(instructionsS3Only.s3Instructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + localInstructions); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + + @Test + public void testOpenTableSnapshot() { + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + instructions); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final List snapshots = adapter.listTableSnapshots(tableId); + + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, snapshots.get(0)); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + // @Test // public void testOpenAllTypesTable() { - // final IcebergCatalog catalog = IcebergTools.loadCatalog("minio-iceberg", instructions); + // final IcebergCatalogAdapter catalog = IcebergTools.loadCatalog("minio-iceberg", instructions); // // final Namespace ns = Namespace.of("sample"); // final TableIdentifier tableId = TableIdentifier.of(ns, "all_types"); From de402ed072b35b82801debec4dc6d6d39ae334d6 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 19 Apr 2024 15:19:45 -0700 Subject: [PATCH 11/25] PR comments, improved testing. --- .../iceberg/layout/IcebergFlatLayout.java | 42 +++- .../IcebergKeyValuePartitionedLayout.java | 91 +++++--- .../iceberg/util/IcebergCatalogAdapter.java | 46 ++-- .../iceberg/util/IcebergInstructions.java | 12 + .../IcebergToolsTest.java | 209 +++++++++++++++++- .../parquet/table/ParquetInstructions.java | 44 +++- 6 files changed, 375 insertions(+), 69 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java index 5e6036da5da..63166d7550c 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -9,6 +9,7 @@ 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.util.IcebergInstructions; import io.deephaven.parquet.table.ParquetInstructions; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; @@ -41,21 +42,40 @@ public final class IcebergFlatLayout implements TableLocationKeyFinder cache; /** - * The instructions for customizations while reading. Could be a {@link ParquetInstructions} or similar. + * The instructions for customizations while reading. */ - private final Object readInstructions; + private final IcebergInstructions instructions; + + /** + * The {@link ParquetInstructions} object that will be used to read any Parquet data files in this table. + */ + private ParquetInstructions parquetInstructions; /** * The current {@link Snapshot} to discover locations for. */ private Snapshot currentSnapshot; - private static IcebergTableLocationKey locationKey( + private IcebergTableLocationKey locationKey( final FileFormat format, - final URI fileUri, - @NotNull final Object readInstructions) { - if (format == FileFormat.PARQUET) { - return new IcebergTableParquetLocationKey(fileUri, 0, null, (ParquetInstructions) readInstructions); + final URI fileUri) { + if (format == org.apache.iceberg.FileFormat.PARQUET) { + if (parquetInstructions == null) { + // Start with user-supplied instructions (if provided). + parquetInstructions = instructions.parquetInstructions().isPresent() + ? instructions.parquetInstructions().get() + : ParquetInstructions.builder().build(); + + // Use the ParquetInstructions overrides to propagate the Iceberg instructions. + if (instructions.columnRenameMap() != null) { + parquetInstructions = parquetInstructions.withColumnRenameMap(instructions.columnRenameMap()); + } + if (instructions.s3Instructions().isPresent()) { + parquetInstructions = + parquetInstructions.withSpecialInstructions(instructions.s3Instructions().get()); + } + } + return new IcebergTableParquetLocationKey(fileUri, 0, null, parquetInstructions); } throw new UnsupportedOperationException("Unsupported file format: " + format); } @@ -64,17 +84,17 @@ private static IcebergTableLocationKey locationKey( * @param table The {@link Table} to discover locations for. * @param tableSnapshot The {@link Snapshot} from which to discover data files. * @param fileIO The file IO to use for reading manifest data files. - * @param readInstructions The instructions for customizations while reading. + * @param instructions The instructions for customizations while reading. */ public IcebergFlatLayout( @NotNull final Table table, @NotNull final Snapshot tableSnapshot, @NotNull final FileIO fileIO, - @NotNull final Object readInstructions) { + @NotNull final IcebergInstructions instructions) { this.table = table; this.currentSnapshot = tableSnapshot; this.fileIO = fileIO; - this.readInstructions = readInstructions; + this.instructions = instructions; this.cache = new HashMap<>(); } @@ -96,7 +116,7 @@ public synchronized void findKeys(@NotNull final Consumer { - final IcebergTableLocationKey key = locationKey(df.format(), fileUri, readInstructions); + final IcebergTableLocationKey key = locationKey(df.format(), fileUri); // Verify before caching. return key.verifyFileReader() ? key : null; }); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java index e4290e402ef..c31bde36c9b 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java @@ -5,21 +5,20 @@ import io.deephaven.base.FileUtils; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.TableDataException; 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.util.IcebergInstructions; import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.util.type.TypeUtils; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; import org.jetbrains.annotations.NotNull; import java.net.URI; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.function.Consumer; /** @@ -28,9 +27,13 @@ */ public final class IcebergKeyValuePartitionedLayout implements TableLocationKeyFinder { /** - * The Iceberg {@link Table} to discover locations for. + * The {@link TableDefinition} that will be used for the table. */ - private final Table table; + final TableDefinition tableDef; + /** + * The Iceberg {@link org.apache.iceberg.Table} to discover locations for. + */ + private final org.apache.iceberg.Table table; /** * The {@link FileIO} to use for passing to the catalog reading manifest data files. @@ -42,40 +45,51 @@ public final class IcebergKeyValuePartitionedLayout implements TableLocationKeyF */ private final String[] partitionColumns; + /** + * The data types of the partitioning columns. + */ + private final Class[] partitionColumnTypes; + /** * A cache of {@link IcebergTableLocationKey}s keyed by the URI of the file they represent. */ private final Map cache; /** - * The instructions for customizations while reading. Could be a {@link ParquetInstructions} or similar. + * The instructions for customizations while reading. */ private final IcebergInstructions instructions; /** - * The current {@link Snapshot} to discover locations for. + * The {@link ParquetInstructions} object that will be used to read any Parquet data files in this table. */ - private Snapshot currentSnapshot; + private ParquetInstructions parquetInstructions; - private static IcebergTableLocationKey locationKey( - final FileFormat format, + /** + * The current {@link org.apache.iceberg.Snapshot} to discover locations for. + */ + private org.apache.iceberg.Snapshot currentSnapshot; + + private IcebergTableLocationKey locationKey( + final org.apache.iceberg.FileFormat format, final URI fileUri, - final Map> partitions, - @NotNull final IcebergInstructions instructions) { + final Map> partitions) { + + if (format == org.apache.iceberg.FileFormat.PARQUET) { + if (parquetInstructions == null) { + // Start with user-supplied instructions (if provided). + parquetInstructions = instructions.parquetInstructions().isPresent() + ? instructions.parquetInstructions().get() + : ParquetInstructions.builder().build(); - if (format == FileFormat.PARQUET) { - final ParquetInstructions parquetInstructions; - if (instructions.parquetInstructions().isPresent()) { - // Accept the user supplied instructions without change - parquetInstructions = instructions.parquetInstructions().get(); - } else if (instructions.s3Instructions().isPresent()) { - // Create a default Parquet instruction object from the S3 instructions - parquetInstructions = ParquetInstructions.builder() - .setSpecialInstructions(instructions.s3Instructions().get()) - .build(); - } else { - // Create a default Parquet instruction object - parquetInstructions = ParquetInstructions.builder().build(); + // Use the ParquetInstructions overrides to propagate the Iceberg instructions. + if (instructions.columnRenameMap() != null) { + parquetInstructions = parquetInstructions.withColumnRenameMap(instructions.columnRenameMap()); + } + if (instructions.s3Instructions().isPresent()) { + parquetInstructions = + parquetInstructions.withSpecialInstructions(instructions.s3Instructions().get()); + } } return new IcebergTableParquetLocationKey(fileUri, 0, partitions, parquetInstructions); } @@ -83,24 +97,31 @@ private static IcebergTableLocationKey locationKey( } /** - * @param table The {@link Table} to discover locations for. - * @param tableSnapshot The {@link Snapshot} from which to discover data files. + * @param table The {@link org.apache.iceberg.Table} to discover locations for. + * @param tableSnapshot The {@link org.apache.iceberg.Snapshot} from which to discover data files. * @param fileIO The file IO to use for reading manifest data files. * @param partitionColumns The columns to use for partitioning. * @param instructions The instructions for customizations while reading. */ public IcebergKeyValuePartitionedLayout( - @NotNull final Table table, - @NotNull final Snapshot tableSnapshot, + @NotNull final TableDefinition tableDef, + @NotNull final org.apache.iceberg.Table table, + @NotNull final org.apache.iceberg.Snapshot tableSnapshot, @NotNull final FileIO fileIO, @NotNull final String[] partitionColumns, @NotNull final IcebergInstructions instructions) { + this.tableDef = tableDef; this.table = table; this.currentSnapshot = tableSnapshot; this.fileIO = fileIO; this.partitionColumns = partitionColumns; this.instructions = instructions; + // Compute and store the data types of the partitioning columns. + partitionColumnTypes = Arrays.stream(partitionColumns) + .map(colName -> TypeUtils.getBoxedType(tableDef.getColumn(colName).getDataType())) + .toArray(Class[]::new); + this.cache = new HashMap<>(); } @@ -124,10 +145,16 @@ public synchronized void findKeys(@NotNull final Consumer { final PartitionData partitionData = (PartitionData) df.partition(); for (int ii = 0; ii < partitionColumns.length; ++ii) { - partitions.put(partitionColumns[ii], (Comparable) partitionData.get(ii)); + final Object value = partitionData.get(ii); + if (value != null && !value.getClass().isAssignableFrom(partitionColumnTypes[ii])) { + throw new TableDataException("Partitioning column " + partitionColumns[ii] + + " has type " + value.getClass().getName() + + " but expected " + partitionColumnTypes[ii].getName()); + } + partitions.put(partitionColumns[ii], (Comparable) value); } final IcebergTableLocationKey key = - locationKey(df.format(), fileUri, partitions, instructions); + locationKey(df.format(), fileUri, partitions); // Verify before caching. return key.verifyFileReader() ? key : null; }); 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 79107b3543b..9a2f03a145c 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 @@ -33,9 +33,7 @@ import java.time.Instant; import java.time.LocalDateTime; -import java.util.ArrayList; -import java.util.List; -import java.util.Set; +import java.util.*; import java.util.stream.Collectors; public class IcebergCatalogAdapter { @@ -65,14 +63,22 @@ public class IcebergCatalogAdapter { this.instructions = instructions; } - static TableDefinition fromSchema(final Schema schema, final PartitionSpec partitionSpec) { + static TableDefinition fromSchema( + final Schema schema, + final PartitionSpec partitionSpec, + final IcebergInstructions instructions) { + final Map renameMap = instructions.columnRenameMap(); + final Set partitionNames = - partitionSpec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); + partitionSpec.fields().stream() + .map(PartitionField::name) + .map(colName -> renameMap.getOrDefault(colName, colName)) + .collect(Collectors.toSet()); final List> columns = new ArrayList<>(); for (final Types.NestedField field : schema.columns()) { - final String name = field.name(); + final String name = renameMap.getOrDefault(field.name(), field.name()); final Type type = field.type(); final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); final ColumnDefinition column; @@ -129,9 +135,7 @@ public List listTables(final Namespace namespace) { public List listTableSnapshots(@NotNull final TableIdentifier tableIdentifier) { final ArrayList snapshotIds = new ArrayList<>(); - catalog.loadTable(tableIdentifier).snapshots().forEach(snapshot -> { - snapshotIds.add(snapshot.snapshotId()); - }); + catalog.loadTable(tableIdentifier).snapshots().forEach(snapshot -> snapshotIds.add(snapshot.snapshotId())); return snapshotIds; } @@ -180,10 +184,10 @@ private Table readTableInternal( // Load the partitioning schema final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); - // Convert the Iceberg schema to a Deephaven TableDefinition - final TableDefinition schemaTableDef = fromSchema(schema, partitionSpec); - final TableDefinition tableDef = instructions.tableDefinition().isPresent() ? - instructions.tableDefinition().get() : schemaTableDef; + // Use the user-supplied defininition or the Iceberg schema for the table definition. + final TableDefinition tableDef = instructions.tableDefinition().isPresent() + ? instructions.tableDefinition().get() + : fromSchema(schema, partitionSpec, instructions); final String description; final TableLocationKeyFinder keyFinder; @@ -194,11 +198,23 @@ private Table readTableInternal( // Create the flat layout location key finder keyFinder = new IcebergFlatLayout(table, snapshot, fileIO, instructions); } else { - final String[] partitionColumns = - partitionSpec.fields().stream().map(PartitionField::name).toArray(String[]::new); + final String[] partitionColumns = partitionSpec.fields().stream() + .map(PartitionField::name) + .map(colName -> instructions.columnRenameMap().getOrDefault(colName, colName)) + .toArray(String[]::new); + + // Verify that the partitioning columns are present in the table definition. + final Map> columnDefinitionMap = tableDef.getColumnNameMap(); + final String[] missingColumns = Arrays.stream(partitionColumns) + .filter(col -> !columnDefinitionMap.containsKey(col)).toArray(String[]::new); + if (missingColumns.length > 0) { + throw new IllegalStateException("Partitioning column(s) " + Arrays.toString(missingColumns) + + " were not found in the table definition"); + } // Create the partitioning column location key finder keyFinder = new IcebergKeyValuePartitionedLayout( + tableDef, table, snapshot, fileIO, diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 15a459ae10a..6bb4c17f084 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -10,6 +10,7 @@ import org.immutables.value.Value.Check; import org.immutables.value.Value.Immutable; +import java.util.Map; import java.util.Optional; /** @@ -39,6 +40,10 @@ public static Builder builder() { */ public abstract Optional parquetInstructions(); + /** + * The {@link Map} to use for reading the Iceberg data files. + */ + public abstract Map columnRenameMap(); public interface Builder { @SuppressWarnings("unused") @@ -47,8 +52,15 @@ public interface Builder { @SuppressWarnings("unused") Builder s3Instructions(S3Instructions s3Instructions); + @SuppressWarnings("unused") Builder parquetInstructions(ParquetInstructions parquetInstructions); + @SuppressWarnings("unused") + Builder putColumnRenameMap(String key, String value); + + @SuppressWarnings("unused") + Builder putAllColumnRenameMap(Map entries); + IcebergInstructions build(); } diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index 5d12d861b54..74e9a3c507e 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -3,8 +3,10 @@ // package io.deephaven.iceberg; +import io.deephaven.base.verify.Assert; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.util.TableTools; import io.deephaven.extensions.s3.Credentials; import io.deephaven.extensions.s3.S3Instructions; @@ -102,8 +104,59 @@ public void testOpenTable() { TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); } + @Test + public void testOpenTableS3Only() { + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + instructionsS3Only); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + @Test public void testOpenTableDefinition() { + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("year").withPartitioning(), + ColumnDefinition.ofInt("month").withPartitioning(), + ColumnDefinition.ofLong("VendorID"), + ColumnDefinition.fromGenericType("tpep_pickup_datetime", LocalDateTime.class), + ColumnDefinition.fromGenericType("tpep_dropoff_datetime", LocalDateTime.class), + ColumnDefinition.ofDouble("passenger_count")); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .s3Instructions(instructionsS3Only.s3Instructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + localInstructions); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + + @Test + public void testOpenTablePartitionTypeException() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofLong("year").withPartitioning(), ColumnDefinition.ofInt("month").withPartitioning(), @@ -127,6 +180,135 @@ public void testOpenTableDefinition() { "http://minio:9000", localInstructions); + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + try { + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + Assert.statementNeverExecuted("Expected an exception for missing columns"); + } catch (final Exception e) { + Assert.eqTrue(e instanceof TableDataException, "Exception type"); + } + } + + @Test + public void testOpenTableDefinitionRename() { + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("__year").withPartitioning(), + ColumnDefinition.ofInt("__month").withPartitioning(), + ColumnDefinition.ofLong("VendorID"), + ColumnDefinition.fromGenericType("pickup_datetime", LocalDateTime.class), + ColumnDefinition.fromGenericType("dropoff_datetime", LocalDateTime.class), + ColumnDefinition.ofDouble("passenger_count")); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .s3Instructions(instructionsS3Only.s3Instructions().get()) + .putColumnRenameMap("VendorID", "vendor_id") + .putColumnRenameMap("tpep_pickup_datetime", "pickup_datetime") + .putColumnRenameMap("tpep_dropoff_datetime", "dropoff_datetime") + .putColumnRenameMap("year", "__year") + .putColumnRenameMap("month", "__month") + .build(); + + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + localInstructions); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + + @Test + public void testMissingPartitioningColumns() { + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("__year").withPartitioning(), // Incorrect name + ColumnDefinition.ofInt("__month").withPartitioning(), // Incorrect name + ColumnDefinition.ofLong("VendorID"), + ColumnDefinition.fromGenericType("pickup_datetime", LocalDateTime.class), + ColumnDefinition.fromGenericType("dropoff_datetime", LocalDateTime.class), + ColumnDefinition.ofDouble("passenger_count")); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .s3Instructions(instructionsS3Only.s3Instructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + localInstructions); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + try { + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + Assert.statementNeverExecuted("Expected an exception for missing columns"); + } catch (final IllegalStateException e) { + Assert.eqTrue(e.getMessage().startsWith("Partitioning column(s)"), "Exception message"); + Assert.eqTrue(e.getMessage().contains("were not found in the table definition"), "Exception message"); + } + } + + @Test + public void testOpenTableColumnRename() { + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .s3Instructions(instructionsS3Only.s3Instructions().get()) + .putColumnRenameMap("VendorID", "vendor_id") + .putColumnRenameMap("tpep_pickup_datetime", "pickup_datetime") + .putColumnRenameMap("tpep_dropoff_datetime", "dropoff_datetime") + .build(); + + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + localInstructions); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + + @Test + public void testOpenTableColumnRenamePartitioningColumns() { + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .s3Instructions(instructionsS3Only.s3Instructions().get()) + .putColumnRenameMap("VendorID", "vendor_id") + .putColumnRenameMap("month", "__month") + .putColumnRenameMap("year", "__year") + .build(); + + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + localInstructions); + final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); @@ -155,14 +337,21 @@ public void testOpenTableSnapshot() { TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); } - // @Test - // public void testOpenAllTypesTable() { - // final IcebergCatalogAdapter catalog = IcebergTools.loadCatalog("minio-iceberg", instructions); - // - // final Namespace ns = Namespace.of("sample"); - // final TableIdentifier tableId = TableIdentifier.of(ns, "all_types"); - // io.deephaven.engine.table.Table table = catalog.readTable(tableId); - // - // TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); - // } + @Test + public void testOpenAllTypesTable() { + final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( + "minio-iceberg", + "http://rest:8181", + "s3a://warehouse/wh", + "us-east-1", + "admin", + "password", + "http://minio:9000", + instructions); + final Namespace ns = Namespace.of("sample"); + final TableIdentifier tableId = TableIdentifier.of(ns, "all_types"); + io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java index ba2e77e3b52..afe25a50715 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java @@ -14,6 +14,7 @@ import org.jetbrains.annotations.Nullable; import java.util.Collections; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.function.Function; @@ -80,7 +81,7 @@ public static void setDefaultMaximumDictionarySize(final int maximumDictionarySi /** * @return The default for {@link #getMaximumDictionarySize()} */ - public static int getDefaltMaximumDictionarySize() { + public static int getDefaultMaximumDictionarySize() { return defaultMaximumDictionarySize; } @@ -164,6 +165,47 @@ public final String getColumnNameFromParquetColumnNameOrDefault(final String par */ public abstract boolean isRefreshing(); + public ParquetInstructions withColumnRenameMap(final Map columnRenameMap) { + // TODO: This conversion is fragile and must be updated with any change to the ParquetInstructions class. It + // would be preferred to have a more robust Immutable implementation with better copy support. + + final ParquetInstructions.Builder builder = new Builder(); + // Add all the existing column mappings. + columnRenameMap.forEach(builder::addColumnNameMapping); + + // Add all the other parameters. + builder.setCompressionCodecName(getCompressionCodecName()); + builder.setMaximumDictionaryKeys(getMaximumDictionaryKeys()); + builder.setMaximumDictionarySize(getMaximumDictionarySize()); + builder.setIsLegacyParquet(isLegacyParquet()); + builder.setTargetPageSize(getTargetPageSize()); + builder.setIsRefreshing(isRefreshing()); + builder.setSpecialInstructions(getSpecialInstructions()); + + return builder.build(); + } + + public ParquetInstructions withSpecialInstructions(final Object specialInstructions) { + // TODO: This conversion is fragile and must be updated with any change to the ParquetInstructions class. It + // would be preferred to have a more robust Immutable implementation with better copy support. + + // This version of the builder brings in all the existing column mappings from `this`. + final ParquetInstructions.Builder builder = new Builder(this); + + // Add the special instructions. + builder.setSpecialInstructions(specialInstructions); + + // Add all the other parameters. + builder.setCompressionCodecName(getCompressionCodecName()); + builder.setMaximumDictionaryKeys(getMaximumDictionaryKeys()); + builder.setMaximumDictionarySize(getMaximumDictionarySize()); + builder.setIsLegacyParquet(isLegacyParquet()); + builder.setTargetPageSize(getTargetPageSize()); + builder.setIsRefreshing(isRefreshing()); + + return builder.build(); + } + @VisibleForTesting public static boolean sameColumnNamesAndCodecMappings(final ParquetInstructions i1, final ParquetInstructions i2) { if (i1 == EMPTY) { From 9762093b40219dc5e6923f1a10cc3773ecc514f8 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 3 May 2024 14:50:10 -0700 Subject: [PATCH 12/25] WIP --- extensions/iceberg/build.gradle | 64 +++-- extensions/iceberg/gradle.properties | 3 + .../iceberg/util/IcebergCatalogAdapter.java | 25 +- .../IcebergMinIOTest.java | 35 +++ .../IcebergToolsTest.java | 121 +++++++- .../iceberg/IcebergLocalStackTest.java | 31 +++ .../TestCatalog/IcebergTestCatalog.java | 50 ++++ .../TestCatalog/IcebergTestFileIO.java | 67 +++++ .../TestCatalog/IcebergTestManifestFile.java | 129 +++++++++ .../TestCatalog/IcebergTestSnapshot.java | 134 +++++++++ .../iceberg/TestCatalog/IcebergTestTable.java | 258 ++++++++++++++++++ ...-3eae-445b-a5d5-f6b17061a5db-00001.parquet | 3 + ...-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet | 3 + ...-9822-404b-96da-8730bf21f89c-00001.parquet | 3 + ...-8358-43ed-8788-0e433efbc7b6-00001.parquet | 3 + ...-2e39-4ac3-9854-3c92a5eae25f-00004.parquet | 3 + ...-2e39-4ac3-9854-3c92a5eae25f-00005.parquet | 3 + ...-2e39-4ac3-9854-3c92a5eae25f-00002.parquet | 3 + ...-2e39-4ac3-9854-3c92a5eae25f-00006.parquet | 3 + ...-2e39-4ac3-9854-3c92a5eae25f-00003.parquet | 3 + ...-2e39-4ac3-9854-3c92a5eae25f-00008.parquet | 3 + ...-2e39-4ac3-9854-3c92a5eae25f-00001.parquet | 3 + ...-2e39-4ac3-9854-3c92a5eae25f-00007.parquet | 3 + ...-fc45-47ac-9f75-74ded525e6d6-00001.parquet | 3 + 24 files changed, 924 insertions(+), 32 deletions(-) create mode 100644 extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergMinIOTest.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/IcebergLocalStackTest.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestManifestFile.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestSnapshot.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestTable.java create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet create mode 100644 extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle index 0644b887935..cb6e5166fcb 100644 --- a/extensions/iceberg/build.gradle +++ b/extensions/iceberg/build.gradle @@ -6,7 +6,6 @@ plugins { description 'Iceberg: Support to read iceberg catalogs.' ext { - icebergVersion = '1.4.3' hadoopVersion = '3.3.6' } @@ -15,12 +14,15 @@ dependencies { api project(':engine-stringset') api project(':engine-table') - implementation project(':engine-table') implementation project(':engine-base') implementation project(':log-factory') implementation project(':Configuration') implementation depTrove3 + implementation platform('software.amazon.awssdk:bom:2.23.19') + implementation 'software.amazon.awssdk:s3' + implementation 'software.amazon.awssdk:aws-crt-client' + compileOnly depAnnotations Classpaths.inheritAutoService(project) @@ -30,38 +32,58 @@ dependencies { Classpaths.inheritAssertJ(project) Classpaths.inheritParquetHadoop(project) + + testImplementation 'org.junit.jupiter:junit-jupiter' + testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine' + testRuntimeOnly 'org.junit.platform:junit-platform-launcher' + implementation project(':extensions-parquet-base') implementation project(':extensions-parquet-table') implementation project(':extensions-s3') - implementation "org.apache.iceberg:iceberg-api:${icebergVersion}" - implementation "org.apache.iceberg:iceberg-core:${icebergVersion}" - implementation "org.apache.iceberg:iceberg-core:${icebergVersion}:tests" - implementation "org.apache.iceberg:iceberg-aws:${icebergVersion}" + implementation platform("org.apache.iceberg:iceberg-bom:1.5.0") + implementation "org.apache.iceberg:iceberg-api" + implementation "org.apache.iceberg:iceberg-core" + implementation "org.apache.iceberg:iceberg-aws" // The following lines can be uncommented when we enable support for Azure and GCP - // implementation "org.apache.iceberg:iceberg-azure:${icebergVersion}" - // implementation "org.apache.iceberg:iceberg-gcp:${icebergVersion}" - implementation "org.apache.iceberg:iceberg-bundled-guava:${icebergVersion}" + // implementation "org.apache.iceberg:iceberg-azure" + // implementation "org.apache.iceberg:iceberg-gcp" + implementation "org.apache.iceberg:iceberg-bundled-guava" + runtimeOnly "org.apache.iceberg:iceberg-aws-bundle" + // ??? BOM? implementation "org.apache.hadoop:hadoop-common:${hadoopVersion}" implementation "org.apache.hadoop:hadoop-hdfs-client:${hadoopVersion}" - runtimeOnly "org.apache.iceberg:iceberg-aws-bundle:${icebergVersion}" + // could be downstream configurable ?? testRuntimeOnly might be better // The following lines can be uncommented when we enable support for Azure and GCP - // runtimeOnly "org.apache.iceberg:iceberg-azure-bundle:${icebergVersion}" - // runtimeOnly "org.apache.iceberg:iceberg-gcp-bundle:${icebergVersion}" + // runtimeOnly "org.apache.iceberg:iceberg-azure-bundle" + // runtimeOnly "org.apache.iceberg:iceberg-gcp-bundle" - testImplementation depCommonsCompress - testImplementation depCommonsIo - testImplementation project(':engine-test-utils'), - TestTools.projectDependency(project, 'engine-base') + testImplementation "org.testcontainers:testcontainers:1.19.4" + testImplementation "org.testcontainers:junit-jupiter:1.19.4" + testImplementation "org.testcontainers:localstack:1.19.4" + testImplementation "org.testcontainers:minio:1.19.4" - Classpaths.inheritJUnitClassic(project, 'testImplementation') + testImplementation TestTools.projectDependency(project, 'extensions-s3') - testRuntimeOnly project(':log-to-slf4j'), - project(path: ':configs'), - project(path: ':test-configs') + testRuntimeOnly project(':test-configs') + testRuntimeOnly project(':log-to-slf4j') Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly') } -TestTools.addEngineOutOfBandTest(project) +test { + useJUnitPlatform { + excludeTags("testcontainers") + } +} + +tasks.register('testOutOfBand', Test) { + useJUnitPlatform { + includeTags("testcontainers") + } + systemProperty 'testcontainers.localstack.image', project.property('testcontainers.localstack.image') + systemProperty 'testcontainers.minio.image', project.property('testcontainers.minio.image') +} + + diff --git a/extensions/iceberg/gradle.properties b/extensions/iceberg/gradle.properties index c186bbfdde1..cfd384f094a 100644 --- a/extensions/iceberg/gradle.properties +++ b/extensions/iceberg/gradle.properties @@ -1 +1,4 @@ io.deephaven.project.ProjectType=JAVA_PUBLIC + +testcontainers.localstack.image=localstack/localstack:3.1.0 +testcontainers.minio.image=minio/minio:RELEASE.2024-02-04T22-36-13Z 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 9a2f03a145c..7bc8c2c16f1 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 @@ -9,6 +9,7 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.PartitionAwareSourceTable; import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; @@ -41,6 +42,8 @@ public class IcebergCatalogAdapter { private final FileIO fileIO; private final IcebergInstructions instructions; + private final WeakHashMap tableLocationProviders; + /** * Construct an IcebergCatalogAdapter given a set of configurable instructions. */ @@ -61,6 +64,7 @@ public class IcebergCatalogAdapter { this.catalog = catalog; this.fileIO = fileIO; this.instructions = instructions; + tableLocationProviders = new WeakHashMap<>(); } static TableDefinition fromSchema( @@ -232,15 +236,24 @@ private Table readTableInternal( description = "Read static iceberg table with " + keyFinder; } - return new PartitionAwareSourceTable( + final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + refreshService); + + final PartitionAwareSourceTable result = new PartitionAwareSourceTable( tableDef, description, RegionedTableComponentFactoryImpl.INSTANCE, - new PollingTableLocationProvider<>( - StandaloneTableKey.getInstance(), - keyFinder, - new IcebergTableLocationFactory(), - refreshService), + locationProvider, updateSourceRegistrar); + + if (isRefreshing) { + // Store a weak reference to the location provider. + tableLocationProviders.put(result, locationProvider); + } + + return result; } } diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergMinIOTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergMinIOTest.java new file mode 100644 index 00000000000..07ead11450b --- /dev/null +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergMinIOTest.java @@ -0,0 +1,35 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg; + + +import io.deephaven.extensions.s3.S3Instructions.Builder; +import io.deephaven.extensions.s3.testlib.SingletonContainers; +import io.deephaven.stats.util.OSUtil; +import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import software.amazon.awssdk.services.s3.S3AsyncClient; + +@Tag("testcontainers") +public class IcebergMinIOTest extends IcebergToolsTest { + + @BeforeAll + static void initContainer() { + // TODO(deephaven-core#5116): MinIO testcontainers does not work on OS X + Assumptions.assumeFalse(OSUtil.runningMacOS(), "OSUtil.runningMacOS()"); + // ensure container is started so container startup time isn't associated with a specific test + SingletonContainers.MinIO.init(); + } + + @Override + public Builder s3Instructions(Builder builder) { + return SingletonContainers.MinIO.s3Instructions(builder); + } + + @Override + public S3AsyncClient s3AsyncClient() { + return SingletonContainers.MinIO.s3AsyncClient(); + } +} diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index 74e9a3c507e..1387994d9b7 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -10,26 +10,59 @@ import io.deephaven.engine.util.TableTools; import io.deephaven.extensions.s3.Credentials; import io.deephaven.extensions.s3.S3Instructions; +import io.deephaven.extensions.s3.testlib.S3Helper; +import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; +import io.deephaven.iceberg.TestCatalog.IcebergTestFileIO; +import io.deephaven.iceberg.TestCatalog.IcebergTestSnapshot; +import io.deephaven.iceberg.TestCatalog.IcebergTestTable; import io.deephaven.iceberg.util.IcebergCatalogAdapter; import io.deephaven.iceberg.util.IcebergInstructions; import io.deephaven.iceberg.util.IcebergTools; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.time.DateTimeUtils; -import junit.framework.TestCase; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.Test; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.CreateBucketRequest; +import software.amazon.awssdk.services.s3.model.DeleteBucketRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; import java.time.LocalDateTime; +import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.UUID; -public class IcebergToolsTest extends TestCase { +public abstract class IcebergToolsTest { IcebergInstructions instructions; IcebergInstructions instructionsS3Only; - @Override - public void setUp() { + public abstract S3AsyncClient s3AsyncClient(); + + public abstract S3Instructions.Builder s3Instructions(S3Instructions.Builder builder); + + private S3AsyncClient asyncClient; + private String bucket; + + private final List keys = new ArrayList<>(); + + private IcebergTestCatalog catalog; + private IcebergTestFileIO fileIO; + + @BeforeEach + void setUp() { + bucket = UUID.randomUUID().toString(); + asyncClient = s3AsyncClient(); + asyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()); + + fileIO = IcebergTestFileIO.INSTANCE; + final S3Instructions s3Instructions = S3Instructions.builder() .credentials(Credentials.basic("admin", "password")) .endpointOverride("http://minio:9000") @@ -45,8 +78,75 @@ public void setUp() { instructions = IcebergInstructions.builder() .parquetInstructions(parquetInstructions) .build(); + + + // Create a compatible schema + // Create a snapshot + // + + S3Helper.uploadDirectory(); + + + + // Create the test catalog for the tests + catalog = IcebergTestCatalog.create(); + + // Create the schema for the tests + final Schema taxisSchema = new Schema(0, + Types.NestedField.optional(1, "VendorID", Types.LongType.get()), + Types.NestedField.optional(2, "tpep_pickup_datetime", Types.TimestampType.withoutZone()), + Types.NestedField.optional(3, "tpep_dropoff_datetime", Types.TimestampType.withoutZone()), + Types.NestedField.optional(4, "passenger_count", Types.DoubleType.get()), + Types.NestedField.optional(5, "trip_distance", Types.DoubleType.get()), + Types.NestedField.optional(6, "RatecodeID", Types.DoubleType.get()), + Types.NestedField.optional(7, "store_and_fwd_flag", Types.StringType.get()), + Types.NestedField.optional(8, "PULocationID", Types.LongType.get()), + Types.NestedField.optional(9, "DOLocationID", Types.LongType.get()), + Types.NestedField.optional(10, "payment_type", Types.LongType.get()), + Types.NestedField.optional(11, "fare_amount", Types.DoubleType.get()), + Types.NestedField.optional(12, "extra", Types.DoubleType.get()), + Types.NestedField.optional(13, "mta_tax", Types.DoubleType.get()), + Types.NestedField.optional(14, "tip_amount", Types.DoubleType.get()), + Types.NestedField.optional(15, "tolls_amount", Types.DoubleType.get()), + Types.NestedField.optional(16, "improvement_surcharge", Types.DoubleType.get()), + Types.NestedField.optional(17, "total_amount", Types.DoubleType.get()), + Types.NestedField.optional(18, "congestion_surcharge", Types.DoubleType.get()), + Types.NestedField.optional(19, "airport_fee", Types.DoubleType.get()) +// Types.NestedField.optional(20, "year", Types.IntegerType.get()), +// Types.NestedField.optional(21, "month", Types.IntegerType.get()) + ); + + final PartitionSpec taxisPartitionedSpec = PartitionSpec.builderFor(taxisSchema) +// .identity("year") +// .identity("month") + .build(); + + final IcebergTestSnapshot taxisSnapshot = IcebergTestSnapshot.create(0); + // Improve this to use the actual data files in resources + taxisSnapshot.addDataManifest(fileIO, + "s3a://warehouse/wh/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet"); + + final IcebergTestTable table = IcebergTestTable.create( + TableIdentifier.of(Namespace.of("nyc"), "taxis_single"), + taxisSchema, + taxisPartitionedSpec, + taxisSnapshot); + + catalog.addTable(TableIdentifier.of("nyc", "taxis_single"), table); } + + @AfterEach + public void tearDown() { + for (String key : keys) { + asyncClient.deleteObject(DeleteObjectRequest.builder().bucket(bucket).key(key).build()); + } + keys.clear(); + asyncClient.deleteBucket(DeleteBucketRequest.builder().bucket(bucket).build()); + asyncClient.close(); + } + + @Test public void testNothing() { // Dummy to prevent JUnit from complaining about no tests @@ -104,6 +204,17 @@ public void testOpenTable() { TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); } + @Test + public void testOpenTableTestCatalog() { + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(catalog, fileIO, instructions); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_single"); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + @Test public void testOpenTableS3Only() { final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/IcebergLocalStackTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/IcebergLocalStackTest.java new file mode 100644 index 00000000000..8426060f3bc --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/IcebergLocalStackTest.java @@ -0,0 +1,31 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg; + + +import io.deephaven.extensions.s3.S3Instructions.Builder; +import io.deephaven.extensions.s3.testlib.SingletonContainers; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import software.amazon.awssdk.services.s3.S3AsyncClient; + +@Tag("testcontainers") +public class IcebergLocalStackTest extends IcebergToolsTest { + + @BeforeAll + static void initContainer() { + // ensure container is started so container startup time isn't associated with a specific test + SingletonContainers.LocalStack.init(); + } + + @Override + public Builder s3Instructions(Builder builder) { + return SingletonContainers.LocalStack.s3Instructions(builder); + } + + @Override + public S3AsyncClient s3AsyncClient() { + return SingletonContainers.LocalStack.s3AsyncClient(); + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java new file mode 100644 index 00000000000..fa1311a129f --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java @@ -0,0 +1,50 @@ +package io.deephaven.iceberg.TestCatalog; + +import org.apache.iceberg.*; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +public class IcebergTestCatalog implements Catalog { + private Map tables; + + private IcebergTestCatalog() { + this.tables = new LinkedHashMap<>(); + } + + public static IcebergTestCatalog create() { + return new IcebergTestCatalog(); + } + + public void addTable(final TableIdentifier tableIdentifier, final IcebergTestTable table) { + tables.put(tableIdentifier, table); + } + + @Override + public List listTables(Namespace namespace) { + return List.of(); + } + + @Override + public boolean dropTable(TableIdentifier tableIdentifier, boolean b) { + final Table t = tables.remove(tableIdentifier); + return t != null; + } + + @Override + public void renameTable(TableIdentifier tableIdentifier, TableIdentifier tableIdentifier1) { + final IcebergTestTable t = tables.remove(tableIdentifier); + if (t != null) { + tables.put(tableIdentifier1, t); + } + } + + @Override + public Table loadTable(TableIdentifier tableIdentifier) { + return tables.get(tableIdentifier); + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java new file mode 100644 index 00000000000..bc62e90c1f9 --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java @@ -0,0 +1,67 @@ +package io.deephaven.iceberg.TestCatalog; + +import org.apache.iceberg.DataFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SeekableInputStream; + +import java.util.HashMap; +import java.util.Map; + +public class IcebergTestFileIO implements FileIO { + public static final IcebergTestFileIO INSTANCE = new IcebergTestFileIO(); + + private Map manifestFiles = new HashMap<>(); + + public void addManifestFile(String path, IcebergTestManifestFile manifestFile) { + manifestFiles.put(path, manifestFile); + } + + public static class IcebergTestInputFile implements InputFile { + @Override + public long getLength() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public SeekableInputStream newStream() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public String location() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public boolean exists() { + throw new UnsupportedOperationException("Not implemented"); + } + } + + @Override + public InputFile newInputFile(String s) { + if (manifestFiles.containsKey(s)) { + final ManifestFile manifestFile = manifestFiles.get(s); + final OutputFile f = null; + ManifestWriter writer = ManifestFiles.write(manifestFile.partitionSpecId(), f); + return new IcebergTestInputFile(); + } + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public OutputFile newOutputFile(String s) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void deleteFile(String s) { + throw new UnsupportedOperationException("Not implemented"); + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestManifestFile.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestManifestFile.java new file mode 100644 index 00000000000..cc1c5a6097d --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestManifestFile.java @@ -0,0 +1,129 @@ +package io.deephaven.iceberg.TestCatalog; + +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.PartitionSpec; + +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +public class IcebergTestManifestFile implements ManifestFile { + private static String path; + private final ManifestContent content; + private final List files; + private final IcebergTestSnapshot snapshot; + private final long sequenceNumber; + private final long minSequenceNumber; + + private IcebergTestManifestFile( + final ManifestContent content, + final IcebergTestSnapshot snapshot, + final long sequenceNumber, + final long minSequenceNumber) { + this.content = content; + this.snapshot = snapshot; + this.partitionSpec = partitionSpec; + this.sequenceNumber = sequenceNumber; + this.minSequenceNumber = minSequenceNumber; + + path = UUID.randomUUID() + ".manifest"; + files = new ArrayList<>(); + } + + public static IcebergTestManifestFile create( + final ManifestContent content, + final IcebergTestSnapshot snapshot, + final PartitionSpec partitionSpec) { + return new IcebergTestManifestFile(content, snapshot, partitionSpec, 0,0); + } + + public static IcebergTestManifestFile create( + final ManifestContent content, + final IcebergTestSnapshot snapshot, + final PartitionSpec partitionSpec, + final long sequenceNumber, + final long minSequenceNumber) { + return new IcebergTestManifestFile(content, snapshot, partitionSpec, sequenceNumber, minSequenceNumber); + } + + + public void addFile(String dataFilePath) { + files.add(dataFilePath); + } + + @Override + public String path() { + return path; + } + + @Override + public long length() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public int partitionSpecId() { + return snapshot.partitionSpecId; + } + + @Override + public ManifestContent content() { + return content; + } + + @Override + public long sequenceNumber() { + return sequenceNumber; + } + + @Override + public long minSequenceNumber() { + return minSequenceNumber; + } + + @Override + public Long snapshotId() { + return snapshot.snapshotId(); + } + + @Override + public Integer addedFilesCount() { + return files.size(); + } + + @Override + public Long addedRowsCount() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public Integer existingFilesCount() { + return files.size(); + } + + @Override + public Long existingRowsCount() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public Integer deletedFilesCount() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public Long deletedRowsCount() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public List partitions() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public ManifestFile copy() { + throw new UnsupportedOperationException("Not implemented"); + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestSnapshot.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestSnapshot.java new file mode 100644 index 00000000000..94b40570fe9 --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestSnapshot.java @@ -0,0 +1,134 @@ +package io.deephaven.iceberg.TestCatalog; + +import org.apache.iceberg.*; +import org.apache.iceberg.io.FileIO; +import org.testcontainers.shaded.org.apache.commons.lang3.NotImplementedException; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; + +public class IcebergTestSnapshot implements Snapshot { + private long sequenceNumber; + private long snapshotId; + private final PartitionSpec partitionSpec; + + int schemaId; + int partitionSpecId; + + private List dataManifests; + private List deleteManifests; + + private IcebergTestSnapshot( + final long sequenceNumber, + final int schemaId, + final PartitionSpec partitionSpec, + final int partitionSpecId) { + this.sequenceNumber = sequenceNumber; + this.snapshotId = ThreadLocalRandom.current().nextLong(1_000_000_000); + this.partitionSpec = partitionSpec; + this.schemaId = schemaId; + this.partitionSpecId = partitionSpecId; + + dataManifests = new ArrayList<>(); + deleteManifests = new ArrayList<>(); + } + + public static IcebergTestSnapshot create(final long sequenceNumber) { + + return new IcebergTestSnapshot(sequenceNumber, 0, 0); + } + + public static IcebergTestSnapshot create( + final long sequenceNumber, + final int schemaId, + final PartitionSpec partitionSpec, + final int partitionSpecId) { + + return new IcebergTestSnapshot(sequenceNumber, schemaId, partitionSpec, partitionSpecId); + } + + + public void addDataManifest(final IcebergTestFileIO fileIO, final String... dataFilePaths) { + final IcebergTestManifestFile manifest = + IcebergTestManifestFile.create(ManifestContent.DATA, this); + for (String dataFilePath : dataFilePaths) { + manifest.addFile(dataFilePath); + } + addDataManifest(fileIO, manifest); + } + + public void addDataManifest(final IcebergTestFileIO fileIO, final ManifestFile manifest) { + dataManifests.add(manifest); + fileIO.addManifestFile(manifest.path(), (IcebergTestManifestFile) manifest); + } + + @Override + public long sequenceNumber() { + return sequenceNumber; + } + + @Override + public long snapshotId() { + return snapshotId; + } + + @Override + public Long parentId() { + return (long)0; + } + + @Override + public Integer schemaId() { + return schemaId; + } + + @Override + public long timestampMillis() { + return 0; + } + + @Override + public List allManifests(FileIO fileIO) { + final List allManifests = new ArrayList<>(dataManifests.size() + deleteManifests.size()); + allManifests.addAll(dataManifests); + allManifests.addAll(deleteManifests); + return allManifests; + } + + @Override + public List dataManifests(FileIO fileIO) { + return dataManifests; + } + + @Override + public List deleteManifests(FileIO fileIO) { + return deleteManifests; + } + + @Override + public String operation() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public Map summary() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public Iterable addedDataFiles(FileIO fileIO) { + throw new NotImplementedException("Not implemented"); + } + + @Override + public Iterable removedDataFiles(FileIO fileIO) { + throw new NotImplementedException("Not implemented"); + } + + @Override + public String manifestListLocation() { + throw new NotImplementedException("Not implemented"); + } +} 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 new file mode 100644 index 00000000000..d86338cd004 --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestTable.java @@ -0,0 +1,258 @@ +package io.deephaven.iceberg.TestCatalog; + +import org.apache.iceberg.*; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; +import org.jetbrains.annotations.NotNull; +import org.testcontainers.shaded.org.apache.commons.lang3.NotImplementedException; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class IcebergTestTable implements Table { + private final TableIdentifier tableIdentifier; + + private final List schemas; + private final List partitionSpecs; + private final List snapshots; + + private IcebergTestTable( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final Schema initialSchema, + @NotNull final PartitionSpec initialPartitionSpec, + @NotNull final Snapshot initialSnapshot) { + this.tableIdentifier = tableIdentifier; + + schemas = new ArrayList<>(1); + schemas.add(initialSchema); + + partitionSpecs = new ArrayList<>(1); + partitionSpecs.add(initialPartitionSpec); + + snapshots = new ArrayList<>(1); + snapshots.add(initialSnapshot); + } + + public static IcebergTestTable create( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final Schema initialSchema, + @NotNull final PartitionSpec initialPartitionSpec, + @NotNull final IcebergTestSnapshot initialSnapshot) { + return new IcebergTestTable(tableIdentifier, initialSchema, initialPartitionSpec, initialSnapshot); + } + + public static IcebergTestTable create( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final Schema initialSchema, + @NotNull final IcebergTestSnapshot initialSnapshot) { + final PartitionSpec spec = PartitionSpec.builderFor(initialSchema).build(); + return new IcebergTestTable(tableIdentifier, initialSchema, spec, initialSnapshot); + } + + @Override + public void refresh() { + } + + @Override + public TableScan newScan() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public Schema schema() { + // return newest + return schemas.get(schemas.size() - 1); + } + + @Override + public Map schemas() { + final Map schemaMap = new java.util.HashMap<>(); + for (int i = 0; i < schemas.size(); i++) { + schemaMap.put(i, schemas.get(i)); + } + return schemaMap; + } + + @Override + public PartitionSpec spec() { + // return newest + return partitionSpecs.get(partitionSpecs.size() - 1); + } + + @Override + public Map specs() { + final Map specMap = new java.util.HashMap<>(); + for (int i = 0; i < partitionSpecs.size(); i++) { + specMap.put(i, partitionSpecs.get(i)); + } + return specMap; + } + + @Override + public SortOrder sortOrder() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public Map sortOrders() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public Map properties() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public String location() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public Snapshot currentSnapshot() { + return snapshots.get(snapshots.size() - 1); + } + + @Override + public Snapshot snapshot(long l) { + for (final Snapshot snapshot : snapshots) { + if (snapshot.snapshotId() == l) { + return snapshot; + } + } + return null; + } + + @Override + public Iterable snapshots() { + return snapshots; + } + + @Override + public List history() { + return List.of(); + } + + @Override + public UpdateSchema updateSchema() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public UpdatePartitionSpec updateSpec() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public UpdateProperties updateProperties() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public ReplaceSortOrder replaceSortOrder() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public UpdateLocation updateLocation() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public AppendFiles newAppend() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public RewriteFiles newRewrite() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public RewriteManifests rewriteManifests() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public OverwriteFiles newOverwrite() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public RowDelta newRowDelta() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public ReplacePartitions newReplacePartitions() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public DeleteFiles newDelete() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public ExpireSnapshots expireSnapshots() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public ManageSnapshots manageSnapshots() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public Transaction newTransaction() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public FileIO io() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public EncryptionManager encryption() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public LocationProvider locationProvider() { + throw new NotImplementedException("Not implemented"); + + } + + @Override + public List statisticsFiles() { + return List.of(); + } + + @Override + public Map refs() { + return Map.of(); + } +} diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet new file mode 100644 index 00000000000..f3efde68ddb --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:3c23a0d50d6b769f91dbabf24ab0aa93d033168a3483bb87c93c04cb3a36ab40 +size 8731886 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet new file mode 100644 index 00000000000..cb49b26e167 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:69b2ed875585804c5147c393d8920d9db0051b7aa4f5fed69c84ce1e6d681fc6 +size 16886361 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet new file mode 100644 index 00000000000..fd9f4abad97 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:3babbe04e9c88b74d1b3ac7c8d17216b81808c8430c6b703911768c9c973f23a +size 8750334 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet new file mode 100644 index 00000000000..3de9fa109f4 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:6ea6ae42f9a7607dac1390cdd9d98710fc12149dafce9be6250d6737b39f50ea +size 12830090 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet new file mode 100644 index 00000000000..73121259969 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:91eaf919c48b9b105f572b81d16df797b1c9985d57c389685a9844468b77dcb0 +size 5975 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet new file mode 100644 index 00000000000..dcde290161f --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:c5305d15f28f9d6fb522fffdbd4177e4601d390fbc10b3f0ad7abf38352f4e1b +size 7264 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet new file mode 100644 index 00000000000..51133193afb --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:4d7cbcea7ad6e082f5d0163d35a15d4f9f6cb50b81f1307ae72c389a90221fb1 +size 7277 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet new file mode 100644 index 00000000000..7010230b425 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:c248feb90c1c051a69937aebe0899507112c2c38f0d7ee4aee14a40e15f010e6 +size 49446600 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet new file mode 100644 index 00000000000..83df1c8f0a1 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:e57038ea90788ab2689d45c2b1fbd47ca5281914d325f85031f80f584507b98d +size 8646 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet new file mode 100644 index 00000000000..f31a195823e --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:182eeeaaa3f5743491c8074e395239b5311b665466f38e389f0fe2d6fd189596 +size 7734 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet new file mode 100644 index 00000000000..f496470c39d --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:109b675803353a2727d38db55696b24e2d80b02ba04fea66c671aa2e58a66202 +size 6780 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet new file mode 100644 index 00000000000..7b561414dd2 --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:666dc205cd521582d6b4a0f186aa90c0e0eec48ae5784b620b34a03ee2e51d0c +size 5975 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet new file mode 100644 index 00000000000..1b4dbbe49af --- /dev/null +++ b/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:493c715173c70d49f3a94cb8b1c35758c87604729ab2b49654f8684fbbccd489 +size 49412347 From d7c2604f3a6c576e5bb579ded513b3abb517edd4 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 14 May 2024 16:39:33 -0700 Subject: [PATCH 13/25] WIP, but test code implemented. --- .../iceberg/layout/IcebergFlatLayout.java | 28 +- .../IcebergKeyValuePartitionedLayout.java | 96 +++-- .../location/IcebergTableLocationKey.java | 4 +- .../IcebergTableParquetLocationKey.java | 2 - .../iceberg/util/IcebergCatalogAdapter.java | 40 ++- .../deephaven/iceberg/util/IcebergTools.java | 9 +- .../IcebergToolsTest.java | 331 +++++++----------- .../TestCatalog/IcebergTestCatalog.java | 59 +++- .../TestCatalog/IcebergTestFileIO.java | 80 ++--- .../TestCatalog/IcebergTestManifestFile.java | 129 ------- .../TestCatalog/IcebergTestSnapshot.java | 134 ------- .../iceberg/TestCatalog/IcebergTestTable.java | 117 +++---- ...-3eae-445b-a5d5-f6b17061a5db-00001.parquet | 0 ...-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet | 0 ...-9822-404b-96da-8730bf21f89c-00001.parquet | 0 ...-8358-43ed-8788-0e433efbc7b6-00001.parquet | 0 ...-71da-4852-849e-a6d5a98e9ae9.metadata.json | 160 +++++++++ ...-3f79-4b9e-94c8-5b46b2da591a.metadata.json | 187 ++++++++++ ...-57c8-4ea8-8b67-57bb93c1e120.metadata.json | 214 +++++++++++ ...-219d-4868-acb1-098fc5282d67.metadata.json | 241 +++++++++++++ ...418a9b-0fe5-4a50-978d-6a3c49279958-m0.avro | Bin 0 -> 8177 bytes ...aae55f-bf38-4506-b506-5832608c1dc1-m0.avro | Bin 0 -> 8158 bytes ...3f7748-dbb3-45a6-ae71-77c1b8882e8d-m0.avro | Bin 0 -> 8177 bytes ...56d517-bd9b-4e53-ac6e-119ccd017ef3-m0.avro | Bin 0 -> 8157 bytes ...-b03f7748-dbb3-45a6-ae71-77c1b8882e8d.avro | Bin 0 -> 4331 bytes ...-77aae55f-bf38-4506-b506-5832608c1dc1.avro | Bin 0 -> 4432 bytes ...-1a418a9b-0fe5-4a50-978d-6a3c49279958.avro | Bin 0 -> 4384 bytes ...-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro | Bin 0 -> 4252 bytes ...-2e39-4ac3-9854-3c92a5eae25f-00004.parquet | 0 ...-2e39-4ac3-9854-3c92a5eae25f-00005.parquet | 0 ...-2e39-4ac3-9854-3c92a5eae25f-00002.parquet | 0 ...-2e39-4ac3-9854-3c92a5eae25f-00006.parquet | 0 ...-2e39-4ac3-9854-3c92a5eae25f-00003.parquet | 0 ...-2e39-4ac3-9854-3c92a5eae25f-00008.parquet | 0 ...-2e39-4ac3-9854-3c92a5eae25f-00001.parquet | 0 ...-2e39-4ac3-9854-3c92a5eae25f-00007.parquet | 0 ...-27f3-4b56-a5dc-e1ee0e1df858.metadata.json | 180 ++++++++++ ...460130-5a60-48ae-b1e0-6aa9fa2b98b5-m0.avro | 10 + ...-69460130-5a60-48ae-b1e0-6aa9fa2b98b5.avro | Bin 0 -> 4287 bytes ...-fc45-47ac-9f75-74ded525e6d6-00001.parquet | 0 ...-c052-41d4-9e2d-63e29407b88a.metadata.json | 160 +++++++++ ...1ea117-c6b3-4853-b4f3-09aa36b77f19-m0.avro | Bin 0 -> 8189 bytes ...-651ea117-c6b3-4853-b4f3-09aa36b77f19.avro | Bin 0 -> 4260 bytes .../location/ParquetTableLocationKey.java | 3 +- 44 files changed, 1490 insertions(+), 694 deletions(-) delete mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestManifestFile.java delete mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestSnapshot.java rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet (100%) rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet (100%) rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet (100%) rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet (100%) create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00002-4caf996d-57c8-4ea8-8b67-57bb93c1e120.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00003-b7433d10-219d-4868-acb1-098fc5282d67.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/1a418a9b-0fe5-4a50-978d-6a3c49279958-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/77aae55f-bf38-4506-b506-5832608c1dc1-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/b03f7748-dbb3-45a6-ae71-77c1b8882e8d-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/ee56d517-bd9b-4e53-ac6e-119ccd017ef3-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/snap-4768271945146524109-1-b03f7748-dbb3-45a6-ae71-77c1b8882e8d.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/snap-615105126920399770-1-77aae55f-bf38-4506-b506-5832608c1dc1.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/snap-7258036030029852722-1-1a418a9b-0fe5-4a50-978d-6a3c49279958.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/snap-7480254251893511044-1-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet (100%) rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet (100%) rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet (100%) rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet (100%) rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet (100%) rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet (100%) rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet (100%) rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet (100%) create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/00000-e5584cce-27f3-4b56-a5dc-e1ee0e1df858.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/69460130-5a60-48ae-b1e0-6aa9fa2b98b5-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/snap-2327977723212680118-1-69460130-5a60-48ae-b1e0-6aa9fa2b98b5.avro rename extensions/iceberg/src/test/resources/{io.deephaven.iceberg.warehouse => warehouse}/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet (100%) create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/00000-a58846ae-c052-41d4-9e2d-63e29407b88a.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/651ea117-c6b3-4853-b4f3-09aa36b77f19-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/snap-6884618194677467995-1-651ea117-c6b3-4853-b4f3-09aa36b77f19.avro diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java index 63166d7550c..7edbe161f29 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -4,7 +4,6 @@ package io.deephaven.iceberg.layout; import io.deephaven.base.FileUtils; -import io.deephaven.base.verify.Assert; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; @@ -23,7 +22,7 @@ /** * Iceberg {@link TableLocationKeyFinder location finder} for tables without partitions that will discover data files - * from a {@link org.apache.iceberg.Snapshot} + * from a {@link Snapshot} */ public final class IcebergFlatLayout implements TableLocationKeyFinder { /** @@ -37,7 +36,7 @@ public final class IcebergFlatLayout implements TableLocationKeyFinder cache; @@ -52,9 +51,9 @@ public final class IcebergFlatLayout implements TableLocationKeyFinder locationKeyObserver) { try { // Retrieve the manifest files from the snapshot - final List manifestFiles = currentSnapshot.allManifests(fileIO); + final List manifestFiles = snapshot.allManifests(fileIO); for (final ManifestFile manifestFile : manifestFiles) { // Currently only can process manifest files with DATA content type. - Assert.eq(manifestFile.content(), "manifestFile.content()", - ManifestContent.DATA, "ManifestContent.DATA"); + if (manifestFile.content() != ManifestContent.DATA) { + throw new TableDataException( + String.format("%s:%d - only DATA manifest files are currently supported, encountered %s", + table, snapshot.snapshotId(), manifestFile.content())); + } final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO); for (DataFile df : reader) { final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); @@ -126,7 +129,8 @@ public synchronized void findKeys(@NotNull final Consumer { /** @@ -33,7 +33,7 @@ public final class IcebergKeyValuePartitionedLayout implements TableLocationKeyF /** * The Iceberg {@link org.apache.iceberg.Table} to discover locations for. */ - private final org.apache.iceberg.Table table; + private final Table table; /** * The {@link FileIO} to use for passing to the catalog reading manifest data files. @@ -41,17 +41,7 @@ public final class IcebergKeyValuePartitionedLayout implements TableLocationKeyF private final FileIO fileIO; /** - * The columns to use for partitioning. - */ - private final String[] partitionColumns; - - /** - * The data types of the partitioning columns. - */ - private final Class[] partitionColumnTypes; - - /** - * A cache of {@link IcebergTableLocationKey}s keyed by the URI of the file they represent. + * A cache of {@link IcebergTableLocationKey IcebergTableLocationKeys} keyed by the URI of the file they represent. */ private final Map cache; @@ -66,9 +56,9 @@ public final class IcebergKeyValuePartitionedLayout implements TableLocationKeyF private ParquetInstructions parquetInstructions; /** - * The current {@link org.apache.iceberg.Snapshot} to discover locations for. + * The {@link Snapshot} to discover locations for. */ - private org.apache.iceberg.Snapshot currentSnapshot; + private final Snapshot snapshot; private IcebergTableLocationKey locationKey( final org.apache.iceberg.FileFormat format, @@ -80,7 +70,7 @@ private IcebergTableLocationKey locationKey( // Start with user-supplied instructions (if provided). parquetInstructions = instructions.parquetInstructions().isPresent() ? instructions.parquetInstructions().get() - : ParquetInstructions.builder().build(); + : ParquetInstructions.EMPTY; // Use the ParquetInstructions overrides to propagate the Iceberg instructions. if (instructions.columnRenameMap() != null) { @@ -93,14 +83,14 @@ private IcebergTableLocationKey locationKey( } return new IcebergTableParquetLocationKey(fileUri, 0, partitions, parquetInstructions); } - throw new UnsupportedOperationException("Unsupported file format: " + format); + throw new UnsupportedOperationException(String.format("%s:%d - an unsupported file format %s for URI '%s'", + table, snapshot.snapshotId(), format, fileUri)); } /** * @param table The {@link org.apache.iceberg.Table} to discover locations for. * @param tableSnapshot The {@link org.apache.iceberg.Snapshot} from which to discover data files. * @param fileIO The file IO to use for reading manifest data files. - * @param partitionColumns The columns to use for partitioning. * @param instructions The instructions for customizations while reading. */ public IcebergKeyValuePartitionedLayout( @@ -108,63 +98,67 @@ public IcebergKeyValuePartitionedLayout( @NotNull final org.apache.iceberg.Table table, @NotNull final org.apache.iceberg.Snapshot tableSnapshot, @NotNull final FileIO fileIO, - @NotNull final String[] partitionColumns, @NotNull final IcebergInstructions instructions) { this.tableDef = tableDef; this.table = table; - this.currentSnapshot = tableSnapshot; + this.snapshot = tableSnapshot; this.fileIO = fileIO; - this.partitionColumns = partitionColumns; this.instructions = instructions; - // Compute and store the data types of the partitioning columns. - partitionColumnTypes = Arrays.stream(partitionColumns) - .map(colName -> TypeUtils.getBoxedType(tableDef.getColumn(colName).getDataType())) - .toArray(Class[]::new); - this.cache = new HashMap<>(); } public String toString() { - return IcebergFlatLayout.class.getSimpleName() + '[' + table.name() + ']'; + return IcebergKeyValuePartitionedLayout.class.getSimpleName() + '[' + table.name() + ']'; } @Override public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { final Map> partitions = new LinkedHashMap<>(); try { + final String[] partitionColumns = + tableDef.getPartitioningColumns().stream().map(ColumnDefinition::getName).toArray(String[]::new); + final Class[] partitionColumnTypes = Arrays.stream(partitionColumns) + .map(colName -> TypeUtils.getBoxedType(tableDef.getColumn(colName).getDataType())) + .toArray(Class[]::new); + // Retrieve the manifest files from the snapshot - final List manifestFiles = currentSnapshot.allManifests(fileIO); + final List manifestFiles = snapshot.allManifests(fileIO); for (final ManifestFile manifestFile : manifestFiles) { // Currently only can process manifest files with DATA content type. - Assert.eq(manifestFile.content(), "manifestFile.content()", - ManifestContent.DATA, "ManifestContent.DATA"); - final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO); - for (DataFile df : reader) { - final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); - final IcebergTableLocationKey locationKey = cache.computeIfAbsent(fileUri, uri -> { - final PartitionData partitionData = (PartitionData) df.partition(); - for (int ii = 0; ii < partitionColumns.length; ++ii) { - final Object value = partitionData.get(ii); - if (value != null && !value.getClass().isAssignableFrom(partitionColumnTypes[ii])) { - throw new TableDataException("Partitioning column " + partitionColumns[ii] - + " has type " + value.getClass().getName() - + " but expected " + partitionColumnTypes[ii].getName()); + if (manifestFile.content() != ManifestContent.DATA) { + throw new TableDataException( + String.format("%s:%d - only DATA manifest files are currently supported, encountered %s", + table, snapshot.snapshotId(), manifestFile.content())); + } + try (final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO)) { + for (DataFile df : reader) { + final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); + final IcebergTableLocationKey locationKey = cache.computeIfAbsent(fileUri, uri -> { + final PartitionData partitionData = (PartitionData) df.partition(); + for (int ii = 0; ii < partitionColumns.length; ++ii) { + final Object value = partitionData.get(ii); + if (value != null && !value.getClass().isAssignableFrom(partitionColumnTypes[ii])) { + throw new TableDataException("Partitioning column " + partitionColumns[ii] + + " has type " + value.getClass().getName() + + " but expected " + partitionColumnTypes[ii].getName()); + } + partitions.put(partitionColumns[ii], (Comparable) value); } - partitions.put(partitionColumns[ii], (Comparable) value); + final IcebergTableLocationKey key = + locationKey(df.format(), fileUri, partitions); + // Verify before caching. + return key.verifyFileReader() ? key : null; + }); + if (locationKey != null) { + locationKeyObserver.accept(locationKey); } - final IcebergTableLocationKey key = - locationKey(df.format(), fileUri, partitions); - // Verify before caching. - return key.verifyFileReader() ? key : null; - }); - if (locationKey != null) { - locationKeyObserver.accept(locationKey); } } } } catch (final Exception e) { - throw new TableDataException("Error finding Iceberg locations under " + currentSnapshot, e); + throw new TableDataException( + String.format("%s:%d - error finding Iceberg locations", table, snapshot.snapshotId()), e); } } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java index 42226bfa4ea..cb28907465d 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java @@ -4,7 +4,6 @@ package io.deephaven.iceberg.location; import io.deephaven.engine.table.impl.locations.TableLocationKey; -import io.deephaven.parquet.table.location.ParquetTableLocationKey; /** * {@link TableLocationKey} implementation for use with data stored in Iceberg tables. @@ -18,8 +17,7 @@ public interface IcebergTableLocationKey extends TableLocationKey { Object readInstructions(); /** - * Verify that a reader for the file can be created successfully. Delegates to - * {@link ParquetTableLocationKey#verifyFileReader()} for parquet files. + * Verify that a reader for the file can be created successfully. */ boolean verifyFileReader(); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java index 40b7d680314..e2582d77f09 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java @@ -18,7 +18,6 @@ public class IcebergTableParquetLocationKey extends ParquetTableLocationKey implements IcebergTableLocationKey { private static final String IMPLEMENTATION_NAME = IcebergTableParquetLocationKey.class.getSimpleName(); - private final ParquetInstructions readInstructions; /** * Construct a new IcebergTableParquetLocationKey for the supplied {@code fileUri} and {@code partitions}. @@ -36,7 +35,6 @@ public IcebergTableParquetLocationKey( @Nullable final Map> partitions, @NotNull final ParquetInstructions readInstructions) { super(fileUri, order, partitions, readInstructions); - this.readInstructions = readInstructions; } @Override 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 7bc8c2c16f1..ec0ba030a1f 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 @@ -42,8 +42,6 @@ public class IcebergCatalogAdapter { private final FileIO fileIO; private final IcebergInstructions instructions; - private final WeakHashMap tableLocationProviders; - /** * Construct an IcebergCatalogAdapter given a set of configurable instructions. */ @@ -64,7 +62,6 @@ public class IcebergCatalogAdapter { this.catalog = catalog; this.fileIO = fileIO; this.instructions = instructions; - tableLocationProviders = new WeakHashMap<>(); } static TableDefinition fromSchema( @@ -73,6 +70,10 @@ static TableDefinition fromSchema( final IcebergInstructions instructions) { final Map renameMap = instructions.columnRenameMap(); + final Set columnNames = instructions.tableDefinition().isPresent() + ? new HashSet<>(instructions.tableDefinition().get().getColumnNames()) + : null; + final Set partitionNames = partitionSpec.fields().stream() .map(PartitionField::name) @@ -83,6 +84,10 @@ static TableDefinition fromSchema( for (final Types.NestedField field : schema.columns()) { final String name = renameMap.getOrDefault(field.name(), field.name()); + // Skip columns that are not in the provided table definition. + if (columnNames != null && !columnNames.contains(name)) { + continue; + } final Type type = field.type(); final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); final ColumnDefinition column; @@ -138,7 +143,7 @@ public List listTables(final Namespace namespace) { } public List listTableSnapshots(@NotNull final TableIdentifier tableIdentifier) { - final ArrayList snapshotIds = new ArrayList<>(); + final List snapshotIds = new ArrayList<>(); catalog.loadTable(tableIdentifier).snapshots().forEach(snapshot -> snapshotIds.add(snapshot.snapshotId())); return snapshotIds; } @@ -188,10 +193,18 @@ private Table readTableInternal( // Load the partitioning schema final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); - // Use the user-supplied defininition or the Iceberg schema for the table definition. - final TableDefinition tableDef = instructions.tableDefinition().isPresent() - ? instructions.tableDefinition().get() - : fromSchema(schema, partitionSpec, instructions); + // Load the table-definition from the snapshot schema. + final TableDefinition icebergTableDef = fromSchema(schema, partitionSpec, instructions); + + // If the user supplied a table definition, make sure it's fully compatible. + final TableDefinition tableDef; + if (instructions.tableDefinition().isPresent()) { + final TableDefinition userTableDef = instructions.tableDefinition().get(); + tableDef = icebergTableDef.checkMutualCompatibility(userTableDef); + } else { + // Use the snapshot schema as the table definition. + tableDef = icebergTableDef; + } final String description; final TableLocationKeyFinder keyFinder; @@ -212,8 +225,9 @@ private Table readTableInternal( final String[] missingColumns = Arrays.stream(partitionColumns) .filter(col -> !columnDefinitionMap.containsKey(col)).toArray(String[]::new); if (missingColumns.length > 0) { - throw new IllegalStateException("Partitioning column(s) " + Arrays.toString(missingColumns) - + " were not found in the table definition"); + throw new IllegalStateException( + String.format("%s:%d - Partitioning column(s) %s were not found in the table definition", + table, snapshot.snapshotId(), Arrays.toString(missingColumns))); } // Create the partitioning column location key finder @@ -222,7 +236,6 @@ private Table readTableInternal( table, snapshot, fileIO, - partitionColumns, instructions); } @@ -249,11 +262,6 @@ private Table readTableInternal( locationProvider, updateSourceRegistrar); - if (isRefreshing) { - // Store a weak reference to the location provider. - tableLocationProviders.put(result, locationProvider); - } - return result; } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java index a3bb264ed26..60ad1ac92cc 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java @@ -23,6 +23,7 @@ * Tools for accessing tables in the Iceberg table format. */ public class IcebergTools { + private static final String S3_FILE_IO_CLASS = "org.apache.iceberg.aws.s3.S3FileIO"; @SuppressWarnings("unused") public static IcebergCatalogAdapter createAdapter( @@ -51,12 +52,10 @@ public static IcebergCatalogAdapter createS3Rest( final RESTCatalog catalog = new RESTCatalog(); - properties.put(CatalogProperties.CATALOG_IMPL, "org.apache.iceberg.rest.RESTCatalog"); + properties.put(CatalogProperties.CATALOG_IMPL, catalog.getClass().getName()); properties.put(CatalogProperties.URI, catalogURI); properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); - properties.put(CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.aws.s3.S3FileIO"); - // Configure the properties map from the Iceberg instructions. if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) { properties.put(S3FileIOProperties.ACCESS_KEY_ID, accessKeyId); @@ -70,9 +69,9 @@ public static IcebergCatalogAdapter createS3Rest( } // TODO: create a FileIO interface wrapping the Deephaven S3SeekableByteChannel/Provider - final FileIO fileIO = CatalogUtil.loadFileIO("org.apache.iceberg.aws.s3.S3FileIO", properties, null); + final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); - final String catalogName = name != null ? name : "IcebergTableDataService-" + catalogURI; + final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; catalog.initialize(catalogName, properties); // If the user did not supply custom read instructions, let's create some defaults. diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index 1387994d9b7..4741c27622f 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -6,38 +6,35 @@ import io.deephaven.base.verify.Assert; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.util.TableTools; -import io.deephaven.extensions.s3.Credentials; import io.deephaven.extensions.s3.S3Instructions; -import io.deephaven.extensions.s3.testlib.S3Helper; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; import io.deephaven.iceberg.TestCatalog.IcebergTestFileIO; -import io.deephaven.iceberg.TestCatalog.IcebergTestSnapshot; -import io.deephaven.iceberg.TestCatalog.IcebergTestTable; import io.deephaven.iceberg.util.IcebergCatalogAdapter; import io.deephaven.iceberg.util.IcebergInstructions; import io.deephaven.iceberg.util.IcebergTools; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.time.DateTimeUtils; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.types.Types; +import org.apache.iceberg.io.FileIO; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import software.amazon.awssdk.core.async.AsyncRequestBody; import software.amazon.awssdk.services.s3.S3AsyncClient; -import software.amazon.awssdk.services.s3.model.CreateBucketRequest; -import software.amazon.awssdk.services.s3.model.DeleteBucketRequest; -import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; +import software.amazon.awssdk.services.s3.model.*; +import java.io.File; import java.time.LocalDateTime; import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public abstract class IcebergToolsTest { IcebergInstructions instructions; @@ -52,22 +49,25 @@ public abstract class IcebergToolsTest { private final List keys = new ArrayList<>(); - private IcebergTestCatalog catalog; - private IcebergTestFileIO fileIO; + private String warehousePath; + private Catalog resourceCatalog; + private FileIO resourceFileIO; @BeforeEach - void setUp() { - bucket = UUID.randomUUID().toString(); + void setUp() throws ExecutionException, InterruptedException { + bucket = "warehouse"; asyncClient = s3AsyncClient(); - asyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()); + final CompletableFuture bucketCreated = + asyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()); - fileIO = IcebergTestFileIO.INSTANCE; + warehousePath = IcebergToolsTest.class.getResource("/warehouse").getPath(); + resourceFileIO = new IcebergTestFileIO("s3://warehouse", warehousePath); + + // Create the test catalog for the tests + resourceCatalog = IcebergTestCatalog.create(warehousePath, resourceFileIO); + + final S3Instructions s3Instructions = s3Instructions(S3Instructions.builder()).build(); - final S3Instructions s3Instructions = S3Instructions.builder() - .credentials(Credentials.basic("admin", "password")) - .endpointOverride("http://minio:9000") - .regionName("us-east-1") - .build(); final ParquetInstructions parquetInstructions = ParquetInstructions.builder() .setSpecialInstructions(s3Instructions) .build(); @@ -79,62 +79,21 @@ void setUp() { .parquetInstructions(parquetInstructions) .build(); - - // Create a compatible schema - // Create a snapshot - // - - S3Helper.uploadDirectory(); - - - - // Create the test catalog for the tests - catalog = IcebergTestCatalog.create(); - - // Create the schema for the tests - final Schema taxisSchema = new Schema(0, - Types.NestedField.optional(1, "VendorID", Types.LongType.get()), - Types.NestedField.optional(2, "tpep_pickup_datetime", Types.TimestampType.withoutZone()), - Types.NestedField.optional(3, "tpep_dropoff_datetime", Types.TimestampType.withoutZone()), - Types.NestedField.optional(4, "passenger_count", Types.DoubleType.get()), - Types.NestedField.optional(5, "trip_distance", Types.DoubleType.get()), - Types.NestedField.optional(6, "RatecodeID", Types.DoubleType.get()), - Types.NestedField.optional(7, "store_and_fwd_flag", Types.StringType.get()), - Types.NestedField.optional(8, "PULocationID", Types.LongType.get()), - Types.NestedField.optional(9, "DOLocationID", Types.LongType.get()), - Types.NestedField.optional(10, "payment_type", Types.LongType.get()), - Types.NestedField.optional(11, "fare_amount", Types.DoubleType.get()), - Types.NestedField.optional(12, "extra", Types.DoubleType.get()), - Types.NestedField.optional(13, "mta_tax", Types.DoubleType.get()), - Types.NestedField.optional(14, "tip_amount", Types.DoubleType.get()), - Types.NestedField.optional(15, "tolls_amount", Types.DoubleType.get()), - Types.NestedField.optional(16, "improvement_surcharge", Types.DoubleType.get()), - Types.NestedField.optional(17, "total_amount", Types.DoubleType.get()), - Types.NestedField.optional(18, "congestion_surcharge", Types.DoubleType.get()), - Types.NestedField.optional(19, "airport_fee", Types.DoubleType.get()) -// Types.NestedField.optional(20, "year", Types.IntegerType.get()), -// Types.NestedField.optional(21, "month", Types.IntegerType.get()) - ); - - final PartitionSpec taxisPartitionedSpec = PartitionSpec.builderFor(taxisSchema) -// .identity("year") -// .identity("month") - .build(); - - final IcebergTestSnapshot taxisSnapshot = IcebergTestSnapshot.create(0); - // Improve this to use the actual data files in resources - taxisSnapshot.addDataManifest(fileIO, - "s3a://warehouse/wh/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet"); - - final IcebergTestTable table = IcebergTestTable.create( - TableIdentifier.of(Namespace.of("nyc"), "taxis_single"), - taxisSchema, - taxisPartitionedSpec, - taxisSnapshot); - - catalog.addTable(TableIdentifier.of("nyc", "taxis_single"), table); + bucketCreated.get(); } + private void uploadParquetFiles(final File root, final String prefixToRemove) + throws ExecutionException, InterruptedException, TimeoutException { + for (final File file : root.listFiles()) { + if (file.isDirectory()) { + uploadParquetFiles(file, prefixToRemove); + } else if (file.getName().endsWith(".parquet")) { + final String key = file.getPath().substring(prefixToRemove.length() + 1); + keys.add(key); + putObject(key, AsyncRequestBody.fromFile(file)); + } + } + } @AfterEach public void tearDown() { @@ -146,7 +105,6 @@ public void tearDown() { asyncClient.close(); } - @Test public void testNothing() { // Dummy to prevent JUnit from complaining about no tests @@ -156,46 +114,41 @@ public void testNothing() { @Test public void testListTables() { - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - instructions); + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); final Namespace ns = Namespace.of("nyc"); + final Collection tables = adapter.listTables(ns); + Assert.eq(tables.size(), "tables.size()", 3, "3 tables in the namespace"); + Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "taxis")), "tables.contains(nyc/taxis)"); + Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "taxis_partitioned")), + "tables.contains(nyc/taxis_partitioned)"); + Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "taxis_single")), "tables.contains(nyc/taxis_single)"); } - @Test public void testListTableSnapshots() { - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - instructions); - final Collection snapshots = adapter.listTableSnapshots(TableIdentifier.of("nyc", "taxis_partitioned")); + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); + + final Collection snapshots = adapter.listTableSnapshots(TableIdentifier.of("nyc", "taxis")); + + Assert.eq(snapshots.size(), "snapshots.size()", 4, "4 snapshots for nyc/taxis"); + + Assert.eqTrue(snapshots.contains(7480254251893511044L), "snapshots.contains(7480254251893511044L)"); + Assert.eqTrue(snapshots.contains(4768271945146524109L), "snapshots.contains(4768271945146524109L)"); + Assert.eqTrue(snapshots.contains(7258036030029852722L), "snapshots.contains(7258036030029852722L)"); + Assert.eqTrue(snapshots.contains(615105126920399770L), "snapshots.contains(615105126920399770L)"); } @Test - public void testOpenTable() { - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - instructions); + public void testOpenTableA() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis_partitioned").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); @@ -205,8 +158,40 @@ public void testOpenTable() { } @Test - public void testOpenTableTestCatalog() { - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(catalog, fileIO, instructions); + public void testOpenTableB() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis"); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + + @Test + public void testOpenTableC() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis_single").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); + + final Namespace ns = Namespace.of("nyc"); + final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_single"); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + } + + @Test + public void testOpenTableTestCatalog() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis_single").getPath()), + warehousePath); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructions); final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_single"); @@ -216,16 +201,11 @@ public void testOpenTableTestCatalog() { } @Test - public void testOpenTableS3Only() { - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - instructionsS3Only); + public void testOpenTableS3Only() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis_partitioned").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructions); final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); @@ -235,7 +215,10 @@ public void testOpenTableS3Only() { } @Test - public void testOpenTableDefinition() { + public void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis_partitioned").getPath()), + warehousePath); + final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("year").withPartitioning(), ColumnDefinition.ofInt("month").withPartitioning(), @@ -249,15 +232,8 @@ public void testOpenTableDefinition() { .s3Instructions(instructionsS3Only.s3Instructions().get()) .build(); - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - localInstructions); + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); @@ -281,15 +257,8 @@ public void testOpenTablePartitionTypeException() { .s3Instructions(instructionsS3Only.s3Instructions().get()) .build(); - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - localInstructions); + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); @@ -297,8 +266,8 @@ public void testOpenTablePartitionTypeException() { final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); Assert.statementNeverExecuted("Expected an exception for missing columns"); - } catch (final Exception e) { - Assert.eqTrue(e instanceof TableDataException, "Exception type"); + } catch (final TableDefinition.IncompatibleTableDefinitionException e) { + Assert.eqTrue(e.getMessage().startsWith("Table definition incompatibilities"), "Exception message"); } } @@ -307,7 +276,7 @@ public void testOpenTableDefinitionRename() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("__year").withPartitioning(), ColumnDefinition.ofInt("__month").withPartitioning(), - ColumnDefinition.ofLong("VendorID"), + ColumnDefinition.ofLong("vendor_id"), ColumnDefinition.fromGenericType("pickup_datetime", LocalDateTime.class), ColumnDefinition.fromGenericType("dropoff_datetime", LocalDateTime.class), ColumnDefinition.ofDouble("passenger_count")); @@ -322,15 +291,8 @@ public void testOpenTableDefinitionRename() { .putColumnRenameMap("month", "__month") .build(); - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - localInstructions); + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); @@ -354,24 +316,16 @@ public void testMissingPartitioningColumns() { .s3Instructions(instructionsS3Only.s3Instructions().get()) .build(); - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - localInstructions); + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); try { final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); Assert.statementNeverExecuted("Expected an exception for missing columns"); - } catch (final IllegalStateException e) { - Assert.eqTrue(e.getMessage().startsWith("Partitioning column(s)"), "Exception message"); - Assert.eqTrue(e.getMessage().contains("were not found in the table definition"), "Exception message"); + } catch (final TableDefinition.IncompatibleTableDefinitionException e) { + Assert.eqTrue(e.getMessage().startsWith("Table definition incompatibilities"), "Exception message"); } } @@ -384,15 +338,8 @@ public void testOpenTableColumnRename() { .putColumnRenameMap("tpep_dropoff_datetime", "dropoff_datetime") .build(); - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - localInstructions); + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); @@ -410,15 +357,7 @@ public void testOpenTableColumnRenamePartitioningColumns() { .putColumnRenameMap("year", "__year") .build(); - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - localInstructions); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructions); final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); @@ -429,15 +368,7 @@ public void testOpenTableColumnRenamePartitioningColumns() { @Test public void testOpenTableSnapshot() { - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - instructions); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructions); final Namespace ns = Namespace.of("nyc"); final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); @@ -448,21 +379,9 @@ public void testOpenTableSnapshot() { TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); } - @Test - public void testOpenAllTypesTable() { - final IcebergCatalogAdapter adapter = IcebergTools.createS3Rest( - "minio-iceberg", - "http://rest:8181", - "s3a://warehouse/wh", - "us-east-1", - "admin", - "password", - "http://minio:9000", - instructions); - final Namespace ns = Namespace.of("sample"); - final TableIdentifier tableId = TableIdentifier.of(ns, "all_types"); - io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); - - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + private void putObject(String key, AsyncRequestBody body) + throws ExecutionException, InterruptedException, TimeoutException { + asyncClient.putObject(PutObjectRequest.builder().bucket(bucket).key(key).build(), body).get(5, + TimeUnit.SECONDS); } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java index fa1311a129f..2192a6263e8 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java @@ -1,50 +1,73 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.TestCatalog; import org.apache.iceberg.*; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.FileIO; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; +import java.io.File; +import java.util.*; public class IcebergTestCatalog implements Catalog { - private Map tables; + private final Map> namespaceTableMap; + private final Map tableMap; - private IcebergTestCatalog() { - this.tables = new LinkedHashMap<>(); - } + private IcebergTestCatalog(final String path, final FileIO fileIO) { + namespaceTableMap = new HashMap<>(); + tableMap = new HashMap<>(); + + // Assume first level is namespace. + final File root = new File(path); + for (final File namespaceFile : root.listFiles()) { + if (namespaceFile.isDirectory()) { + final Namespace namespace = Namespace.of(namespaceFile.getName()); + namespaceTableMap.putIfAbsent(namespace, new HashMap<>()); + for (final File tableFile : namespaceFile.listFiles()) { + if (tableFile.isDirectory()) { + // Second level is table name. + final TableIdentifier tableId = TableIdentifier.of(namespace, tableFile.getName()); + final Table table = IcebergTestTable.loadFromMetadata(tableFile.getAbsolutePath(), fileIO); - public static IcebergTestCatalog create() { - return new IcebergTestCatalog(); + // Add it to the maps. + namespaceTableMap.get(namespace).put(tableId, table); + tableMap.put(tableId, table); + } + } + } + } } - public void addTable(final TableIdentifier tableIdentifier, final IcebergTestTable table) { - tables.put(tableIdentifier, table); + public static IcebergTestCatalog create(final String path, final FileIO fileIO) { + return new IcebergTestCatalog(path, fileIO); } @Override public List listTables(Namespace namespace) { + if (namespaceTableMap.containsKey(namespace)) { + return new ArrayList<>(namespaceTableMap.get(namespace).keySet()); + } return List.of(); } @Override public boolean dropTable(TableIdentifier tableIdentifier, boolean b) { - final Table t = tables.remove(tableIdentifier); - return t != null; + throw new UnsupportedOperationException("Not implemented"); } @Override public void renameTable(TableIdentifier tableIdentifier, TableIdentifier tableIdentifier1) { - final IcebergTestTable t = tables.remove(tableIdentifier); - if (t != null) { - tables.put(tableIdentifier1, t); - } + throw new UnsupportedOperationException("Not implemented"); } @Override public Table loadTable(TableIdentifier tableIdentifier) { - return tables.get(tableIdentifier); + if (tableMap.containsKey(tableIdentifier)) { + return tableMap.get(tableIdentifier); + } + return null; } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java index bc62e90c1f9..03be6ca1b5e 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java @@ -1,67 +1,49 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.TestCatalog; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.ManifestWriter; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.inmemory.InMemoryFileIO; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.io.SeekableInputStream; -import java.util.HashMap; -import java.util.Map; - -public class IcebergTestFileIO implements FileIO { - public static final IcebergTestFileIO INSTANCE = new IcebergTestFileIO(); - - private Map manifestFiles = new HashMap<>(); - - public void addManifestFile(String path, IcebergTestManifestFile manifestFile) { - manifestFiles.put(path, manifestFile); - } - - public static class IcebergTestInputFile implements InputFile { - @Override - public long getLength() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public SeekableInputStream newStream() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public String location() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public boolean exists() { - throw new UnsupportedOperationException("Not implemented"); - } +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashSet; +import java.util.Set; + +public class IcebergTestFileIO extends InMemoryFileIO { + private final Set inputFiles; + private final String matchPrefix; + private final String replacePrefix; + + public IcebergTestFileIO(final String matchPrefix, final String replacePrefix) { + this.matchPrefix = matchPrefix; + this.replacePrefix = replacePrefix; + inputFiles = new HashSet<>(); } @Override public InputFile newInputFile(String s) { - if (manifestFiles.containsKey(s)) { - final ManifestFile manifestFile = manifestFiles.get(s); - final OutputFile f = null; - ManifestWriter writer = ManifestFiles.write(manifestFile.partitionSpecId(), f); - return new IcebergTestInputFile(); + if (!inputFiles.contains(s)) { + try { + final String replaced = s.replace(matchPrefix, replacePrefix); + final byte[] data = Files.readAllBytes(Path.of(replaced)); + addFile(s, data); + inputFiles.add(s); + } catch (IOException e) { + throw new RuntimeException(e); + } } - throw new UnsupportedOperationException("Not implemented"); + return super.newInputFile(s); } @Override public OutputFile newOutputFile(String s) { - throw new UnsupportedOperationException("Not implemented"); + return null; } @Override - public void deleteFile(String s) { - throw new UnsupportedOperationException("Not implemented"); - } + public void deleteFile(String s) {} } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestManifestFile.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestManifestFile.java deleted file mode 100644 index cc1c5a6097d..00000000000 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestManifestFile.java +++ /dev/null @@ -1,129 +0,0 @@ -package io.deephaven.iceberg.TestCatalog; - -import org.apache.iceberg.ManifestContent; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.PartitionSpec; - -import java.util.ArrayList; -import java.util.List; -import java.util.UUID; - -public class IcebergTestManifestFile implements ManifestFile { - private static String path; - private final ManifestContent content; - private final List files; - private final IcebergTestSnapshot snapshot; - private final long sequenceNumber; - private final long minSequenceNumber; - - private IcebergTestManifestFile( - final ManifestContent content, - final IcebergTestSnapshot snapshot, - final long sequenceNumber, - final long minSequenceNumber) { - this.content = content; - this.snapshot = snapshot; - this.partitionSpec = partitionSpec; - this.sequenceNumber = sequenceNumber; - this.minSequenceNumber = minSequenceNumber; - - path = UUID.randomUUID() + ".manifest"; - files = new ArrayList<>(); - } - - public static IcebergTestManifestFile create( - final ManifestContent content, - final IcebergTestSnapshot snapshot, - final PartitionSpec partitionSpec) { - return new IcebergTestManifestFile(content, snapshot, partitionSpec, 0,0); - } - - public static IcebergTestManifestFile create( - final ManifestContent content, - final IcebergTestSnapshot snapshot, - final PartitionSpec partitionSpec, - final long sequenceNumber, - final long minSequenceNumber) { - return new IcebergTestManifestFile(content, snapshot, partitionSpec, sequenceNumber, minSequenceNumber); - } - - - public void addFile(String dataFilePath) { - files.add(dataFilePath); - } - - @Override - public String path() { - return path; - } - - @Override - public long length() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public int partitionSpecId() { - return snapshot.partitionSpecId; - } - - @Override - public ManifestContent content() { - return content; - } - - @Override - public long sequenceNumber() { - return sequenceNumber; - } - - @Override - public long minSequenceNumber() { - return minSequenceNumber; - } - - @Override - public Long snapshotId() { - return snapshot.snapshotId(); - } - - @Override - public Integer addedFilesCount() { - return files.size(); - } - - @Override - public Long addedRowsCount() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public Integer existingFilesCount() { - return files.size(); - } - - @Override - public Long existingRowsCount() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public Integer deletedFilesCount() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public Long deletedRowsCount() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public List partitions() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public ManifestFile copy() { - throw new UnsupportedOperationException("Not implemented"); - } -} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestSnapshot.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestSnapshot.java deleted file mode 100644 index 94b40570fe9..00000000000 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestSnapshot.java +++ /dev/null @@ -1,134 +0,0 @@ -package io.deephaven.iceberg.TestCatalog; - -import org.apache.iceberg.*; -import org.apache.iceberg.io.FileIO; -import org.testcontainers.shaded.org.apache.commons.lang3.NotImplementedException; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; - -public class IcebergTestSnapshot implements Snapshot { - private long sequenceNumber; - private long snapshotId; - private final PartitionSpec partitionSpec; - - int schemaId; - int partitionSpecId; - - private List dataManifests; - private List deleteManifests; - - private IcebergTestSnapshot( - final long sequenceNumber, - final int schemaId, - final PartitionSpec partitionSpec, - final int partitionSpecId) { - this.sequenceNumber = sequenceNumber; - this.snapshotId = ThreadLocalRandom.current().nextLong(1_000_000_000); - this.partitionSpec = partitionSpec; - this.schemaId = schemaId; - this.partitionSpecId = partitionSpecId; - - dataManifests = new ArrayList<>(); - deleteManifests = new ArrayList<>(); - } - - public static IcebergTestSnapshot create(final long sequenceNumber) { - - return new IcebergTestSnapshot(sequenceNumber, 0, 0); - } - - public static IcebergTestSnapshot create( - final long sequenceNumber, - final int schemaId, - final PartitionSpec partitionSpec, - final int partitionSpecId) { - - return new IcebergTestSnapshot(sequenceNumber, schemaId, partitionSpec, partitionSpecId); - } - - - public void addDataManifest(final IcebergTestFileIO fileIO, final String... dataFilePaths) { - final IcebergTestManifestFile manifest = - IcebergTestManifestFile.create(ManifestContent.DATA, this); - for (String dataFilePath : dataFilePaths) { - manifest.addFile(dataFilePath); - } - addDataManifest(fileIO, manifest); - } - - public void addDataManifest(final IcebergTestFileIO fileIO, final ManifestFile manifest) { - dataManifests.add(manifest); - fileIO.addManifestFile(manifest.path(), (IcebergTestManifestFile) manifest); - } - - @Override - public long sequenceNumber() { - return sequenceNumber; - } - - @Override - public long snapshotId() { - return snapshotId; - } - - @Override - public Long parentId() { - return (long)0; - } - - @Override - public Integer schemaId() { - return schemaId; - } - - @Override - public long timestampMillis() { - return 0; - } - - @Override - public List allManifests(FileIO fileIO) { - final List allManifests = new ArrayList<>(dataManifests.size() + deleteManifests.size()); - allManifests.addAll(dataManifests); - allManifests.addAll(deleteManifests); - return allManifests; - } - - @Override - public List dataManifests(FileIO fileIO) { - return dataManifests; - } - - @Override - public List deleteManifests(FileIO fileIO) { - return deleteManifests; - } - - @Override - public String operation() { - throw new NotImplementedException("Not implemented"); - } - - @Override - public Map summary() { - throw new NotImplementedException("Not implemented"); - } - - @Override - public Iterable addedDataFiles(FileIO fileIO) { - throw new NotImplementedException("Not implemented"); - } - - @Override - public Iterable removedDataFiles(FileIO fileIO) { - throw new NotImplementedException("Not implemented"); - } - - @Override - public String manifestListLocation() { - throw new NotImplementedException("Not implemented"); - } -} 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 d86338cd004..d1cf5c2ee0e 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 @@ -1,76 +1,70 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// package io.deephaven.iceberg.TestCatalog; import org.apache.iceberg.*; -import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.jetbrains.annotations.NotNull; import org.testcontainers.shaded.org.apache.commons.lang3.NotImplementedException; +import java.io.File; +import java.nio.file.Path; import java.util.ArrayList; import java.util.List; import java.util.Map; public class IcebergTestTable implements Table { - private final TableIdentifier tableIdentifier; + private final TableMetadata metadata; + private final FileIO fileIO; - private final List schemas; - private final List partitionSpecs; - private final List snapshots; + private IcebergTestTable(@NotNull final String path, @NotNull final FileIO fileIO) { + final File metadataRoot = new File(path, "metadata"); + this.fileIO = fileIO; - private IcebergTestTable( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final Schema initialSchema, - @NotNull final PartitionSpec initialPartitionSpec, - @NotNull final Snapshot initialSnapshot) { - this.tableIdentifier = tableIdentifier; + final List metadataFiles = new ArrayList<>(); - schemas = new ArrayList<>(1); - schemas.add(initialSchema); - - partitionSpecs = new ArrayList<>(1); - partitionSpecs.add(initialPartitionSpec); - - snapshots = new ArrayList<>(1); - snapshots.add(initialSnapshot); - } + // Get a list of the JSON files. + for (final File file : metadataRoot.listFiles()) { + if (!file.isDirectory() && file.getName().endsWith(".json")) { + metadataFiles.add(file.getAbsolutePath()); + } + } - public static IcebergTestTable create( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final Schema initialSchema, - @NotNull final PartitionSpec initialPartitionSpec, - @NotNull final IcebergTestSnapshot initialSnapshot) { - return new IcebergTestTable(tableIdentifier, initialSchema, initialPartitionSpec, initialSnapshot); + // The last entry after sorting will be the newest / current. + metadataFiles.sort(String::compareTo); + final Path tablePath = Path.of(metadataFiles.get(metadataFiles.size() - 1)); + try { + final String tableJson = new String(java.nio.file.Files.readAllBytes(tablePath)); + metadata = TableMetadataParser.fromJson(tableJson); + } catch (Exception e) { + throw new RuntimeException("Failed to read table file: " + tablePath, e); + } } - public static IcebergTestTable create( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final Schema initialSchema, - @NotNull final IcebergTestSnapshot initialSnapshot) { - final PartitionSpec spec = PartitionSpec.builderFor(initialSchema).build(); - return new IcebergTestTable(tableIdentifier, initialSchema, spec, initialSnapshot); + public static IcebergTestTable loadFromMetadata(@NotNull final String path, @NotNull final FileIO fileIO) { + return new IcebergTestTable(path, fileIO); } @Override - public void refresh() { - } + public void refresh() {} @Override public TableScan newScan() { throw new NotImplementedException("Not implemented"); - } @Override public Schema schema() { - // return newest - return schemas.get(schemas.size() - 1); + return metadata.schema(); } @Override public Map schemas() { final Map schemaMap = new java.util.HashMap<>(); + final List schemas = metadata.schemas(); for (int i = 0; i < schemas.size(); i++) { schemaMap.put(i, schemas.get(i)); } @@ -79,12 +73,12 @@ public Map schemas() { @Override public PartitionSpec spec() { - // return newest - return partitionSpecs.get(partitionSpecs.size() - 1); + return metadata.spec(); } @Override public Map specs() { + final List partitionSpecs = metadata.specs(); final Map specMap = new java.util.HashMap<>(); for (int i = 0; i < partitionSpecs.size(); i++) { specMap.put(i, partitionSpecs.get(i)); @@ -94,32 +88,37 @@ public Map specs() { @Override public SortOrder sortOrder() { - throw new NotImplementedException("Not implemented"); - + return metadata.sortOrder(); } @Override public Map sortOrders() { - throw new NotImplementedException("Not implemented"); + final List sortOrders = metadata.sortOrders(); + final Map sortOrderMap = new java.util.HashMap<>(); + for (int i = 0; i < sortOrders.size(); i++) { + sortOrderMap.put(i, sortOrders.get(i)); + } + return sortOrderMap; } @Override public Map properties() { - throw new NotImplementedException("Not implemented"); + return metadata.properties(); } @Override public String location() { - throw new NotImplementedException("Not implemented"); + return metadata.location(); } @Override public Snapshot currentSnapshot() { - return snapshots.get(snapshots.size() - 1); + return metadata.currentSnapshot(); } @Override public Snapshot snapshot(long l) { + final List snapshots = metadata.snapshots(); for (final Snapshot snapshot : snapshots) { if (snapshot.snapshotId() == l) { return snapshot; @@ -130,129 +129,111 @@ public Snapshot snapshot(long l) { @Override public Iterable snapshots() { - return snapshots; + return metadata.snapshots(); } @Override public List history() { - return List.of(); + throw new NotImplementedException("Not implemented"); } @Override public UpdateSchema updateSchema() { throw new NotImplementedException("Not implemented"); - } @Override public UpdatePartitionSpec updateSpec() { throw new NotImplementedException("Not implemented"); - } @Override public UpdateProperties updateProperties() { throw new NotImplementedException("Not implemented"); - } @Override public ReplaceSortOrder replaceSortOrder() { throw new NotImplementedException("Not implemented"); - } @Override public UpdateLocation updateLocation() { throw new NotImplementedException("Not implemented"); - } @Override public AppendFiles newAppend() { throw new NotImplementedException("Not implemented"); - } @Override public RewriteFiles newRewrite() { throw new NotImplementedException("Not implemented"); - } @Override public RewriteManifests rewriteManifests() { throw new NotImplementedException("Not implemented"); - } @Override public OverwriteFiles newOverwrite() { throw new NotImplementedException("Not implemented"); - } @Override public RowDelta newRowDelta() { throw new NotImplementedException("Not implemented"); - } @Override public ReplacePartitions newReplacePartitions() { throw new NotImplementedException("Not implemented"); - } @Override public DeleteFiles newDelete() { throw new NotImplementedException("Not implemented"); - } @Override public ExpireSnapshots expireSnapshots() { throw new NotImplementedException("Not implemented"); - } @Override public ManageSnapshots manageSnapshots() { throw new NotImplementedException("Not implemented"); - } @Override public Transaction newTransaction() { throw new NotImplementedException("Not implemented"); - } @Override public FileIO io() { - throw new NotImplementedException("Not implemented"); - + return fileIO; } @Override public EncryptionManager encryption() { throw new NotImplementedException("Not implemented"); - } @Override public LocationProvider locationProvider() { throw new NotImplementedException("Not implemented"); - } @Override public List statisticsFiles() { - return List.of(); + return metadata.statisticsFiles(); } @Override public Map refs() { - return Map.of(); + throw new NotImplementedException("Not implemented"); } } diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json new file mode 100644 index 00000000000..0697e58f49a --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json @@ -0,0 +1,160 @@ +{ + "format-version" : 2, + "table-uuid" : "74cb0e81-37ae-4ca9-9225-18b9a885869d", + "location" : "s3://warehouse/nyc/taxis", + "last-sequence-number" : 1, + "last-updated-ms" : 1714753341825, + "last-column-id" : 19, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "VendorID", + "required" : false, + "type" : "long" + }, { + "id" : 2, + "name" : "tpep_pickup_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 3, + "name" : "tpep_dropoff_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 4, + "name" : "passenger_count", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "trip_distance", + "required" : false, + "type" : "double" + }, { + "id" : 6, + "name" : "RatecodeID", + "required" : false, + "type" : "double" + }, { + "id" : 7, + "name" : "store_and_fwd_flag", + "required" : false, + "type" : "string" + }, { + "id" : 8, + "name" : "PULocationID", + "required" : false, + "type" : "long" + }, { + "id" : 9, + "name" : "DOLocationID", + "required" : false, + "type" : "long" + }, { + "id" : 10, + "name" : "payment_type", + "required" : false, + "type" : "long" + }, { + "id" : 11, + "name" : "fare_amount", + "required" : false, + "type" : "double" + }, { + "id" : 12, + "name" : "extra", + "required" : false, + "type" : "double" + }, { + "id" : 13, + "name" : "mta_tax", + "required" : false, + "type" : "double" + }, { + "id" : 14, + "name" : "tip_amount", + "required" : false, + "type" : "double" + }, { + "id" : 15, + "name" : "tolls_amount", + "required" : false, + "type" : "double" + }, { + "id" : 16, + "name" : "improvement_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 17, + "name" : "total_amount", + "required" : false, + "type" : "double" + }, { + "id" : 18, + "name" : "congestion_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 19, + "name" : "airport_fee", + "required" : false, + "type" : "double" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-03T16:22:16.955845960Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 7480254251893511044, + "refs" : { + "main" : { + "snapshot-id" : 7480254251893511044, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 7480254251893511044, + "timestamp-ms" : 1714753341825, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "584935", + "added-files-size" : "8731886", + "changed-partition-count" : "1", + "total-records" : "584935", + "total-files-size" : "8731886", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7480254251893511044-1-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1714753341825, + "snapshot-id" : 7480254251893511044 + } ], + "metadata-log" : [ ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json new file mode 100644 index 00000000000..5f7369890f3 --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json @@ -0,0 +1,187 @@ +{ + "format-version" : 2, + "table-uuid" : "74cb0e81-37ae-4ca9-9225-18b9a885869d", + "location" : "s3://warehouse/nyc/taxis", + "last-sequence-number" : 2, + "last-updated-ms" : 1714753347072, + "last-column-id" : 19, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "VendorID", + "required" : false, + "type" : "long" + }, { + "id" : 2, + "name" : "tpep_pickup_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 3, + "name" : "tpep_dropoff_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 4, + "name" : "passenger_count", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "trip_distance", + "required" : false, + "type" : "double" + }, { + "id" : 6, + "name" : "RatecodeID", + "required" : false, + "type" : "double" + }, { + "id" : 7, + "name" : "store_and_fwd_flag", + "required" : false, + "type" : "string" + }, { + "id" : 8, + "name" : "PULocationID", + "required" : false, + "type" : "long" + }, { + "id" : 9, + "name" : "DOLocationID", + "required" : false, + "type" : "long" + }, { + "id" : 10, + "name" : "payment_type", + "required" : false, + "type" : "long" + }, { + "id" : 11, + "name" : "fare_amount", + "required" : false, + "type" : "double" + }, { + "id" : 12, + "name" : "extra", + "required" : false, + "type" : "double" + }, { + "id" : 13, + "name" : "mta_tax", + "required" : false, + "type" : "double" + }, { + "id" : 14, + "name" : "tip_amount", + "required" : false, + "type" : "double" + }, { + "id" : 15, + "name" : "tolls_amount", + "required" : false, + "type" : "double" + }, { + "id" : 16, + "name" : "improvement_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 17, + "name" : "total_amount", + "required" : false, + "type" : "double" + }, { + "id" : 18, + "name" : "congestion_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 19, + "name" : "airport_fee", + "required" : false, + "type" : "double" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-03T16:22:16.955845960Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 4768271945146524109, + "refs" : { + "main" : { + "snapshot-id" : 4768271945146524109, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 7480254251893511044, + "timestamp-ms" : 1714753341825, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "584935", + "added-files-size" : "8731886", + "changed-partition-count" : "1", + "total-records" : "584935", + "total-files-size" : "8731886", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7480254251893511044-1-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 4768271945146524109, + "parent-snapshot-id" : 7480254251893511044, + "timestamp-ms" : 1714753347072, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "1167564", + "added-files-size" : "16886361", + "changed-partition-count" : "1", + "total-records" : "1752499", + "total-files-size" : "25618247", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-4768271945146524109-1-b03f7748-dbb3-45a6-ae71-77c1b8882e8d.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1714753341825, + "snapshot-id" : 7480254251893511044 + }, { + "timestamp-ms" : 1714753347072, + "snapshot-id" : 4768271945146524109 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1714753341825, + "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00002-4caf996d-57c8-4ea8-8b67-57bb93c1e120.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00002-4caf996d-57c8-4ea8-8b67-57bb93c1e120.metadata.json new file mode 100644 index 00000000000..ba7d6d6148a --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00002-4caf996d-57c8-4ea8-8b67-57bb93c1e120.metadata.json @@ -0,0 +1,214 @@ +{ + "format-version" : 2, + "table-uuid" : "74cb0e81-37ae-4ca9-9225-18b9a885869d", + "location" : "s3://warehouse/nyc/taxis", + "last-sequence-number" : 3, + "last-updated-ms" : 1714753351429, + "last-column-id" : 19, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "VendorID", + "required" : false, + "type" : "long" + }, { + "id" : 2, + "name" : "tpep_pickup_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 3, + "name" : "tpep_dropoff_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 4, + "name" : "passenger_count", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "trip_distance", + "required" : false, + "type" : "double" + }, { + "id" : 6, + "name" : "RatecodeID", + "required" : false, + "type" : "double" + }, { + "id" : 7, + "name" : "store_and_fwd_flag", + "required" : false, + "type" : "string" + }, { + "id" : 8, + "name" : "PULocationID", + "required" : false, + "type" : "long" + }, { + "id" : 9, + "name" : "DOLocationID", + "required" : false, + "type" : "long" + }, { + "id" : 10, + "name" : "payment_type", + "required" : false, + "type" : "long" + }, { + "id" : 11, + "name" : "fare_amount", + "required" : false, + "type" : "double" + }, { + "id" : 12, + "name" : "extra", + "required" : false, + "type" : "double" + }, { + "id" : 13, + "name" : "mta_tax", + "required" : false, + "type" : "double" + }, { + "id" : 14, + "name" : "tip_amount", + "required" : false, + "type" : "double" + }, { + "id" : 15, + "name" : "tolls_amount", + "required" : false, + "type" : "double" + }, { + "id" : 16, + "name" : "improvement_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 17, + "name" : "total_amount", + "required" : false, + "type" : "double" + }, { + "id" : 18, + "name" : "congestion_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 19, + "name" : "airport_fee", + "required" : false, + "type" : "double" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-03T16:22:16.955845960Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 7258036030029852722, + "refs" : { + "main" : { + "snapshot-id" : 7258036030029852722, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 7480254251893511044, + "timestamp-ms" : 1714753341825, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "584935", + "added-files-size" : "8731886", + "changed-partition-count" : "1", + "total-records" : "584935", + "total-files-size" : "8731886", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7480254251893511044-1-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 4768271945146524109, + "parent-snapshot-id" : 7480254251893511044, + "timestamp-ms" : 1714753347072, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "1167564", + "added-files-size" : "16886361", + "changed-partition-count" : "1", + "total-records" : "1752499", + "total-files-size" : "25618247", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-4768271945146524109-1-b03f7748-dbb3-45a6-ae71-77c1b8882e8d.avro", + "schema-id" : 0 + }, { + "sequence-number" : 3, + "snapshot-id" : 7258036030029852722, + "parent-snapshot-id" : 4768271945146524109, + "timestamp-ms" : 1714753351429, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "584865", + "added-files-size" : "8750334", + "changed-partition-count" : "1", + "total-records" : "2337364", + "total-files-size" : "34368581", + "total-data-files" : "3", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7258036030029852722-1-1a418a9b-0fe5-4a50-978d-6a3c49279958.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1714753341825, + "snapshot-id" : 7480254251893511044 + }, { + "timestamp-ms" : 1714753347072, + "snapshot-id" : 4768271945146524109 + }, { + "timestamp-ms" : 1714753351429, + "snapshot-id" : 7258036030029852722 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1714753341825, + "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json" + }, { + "timestamp-ms" : 1714753347072, + "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00003-b7433d10-219d-4868-acb1-098fc5282d67.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00003-b7433d10-219d-4868-acb1-098fc5282d67.metadata.json new file mode 100644 index 00000000000..23d878d28a3 --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00003-b7433d10-219d-4868-acb1-098fc5282d67.metadata.json @@ -0,0 +1,241 @@ +{ + "format-version" : 2, + "table-uuid" : "74cb0e81-37ae-4ca9-9225-18b9a885869d", + "location" : "s3://warehouse/nyc/taxis", + "last-sequence-number" : 4, + "last-updated-ms" : 1714753355960, + "last-column-id" : 19, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "VendorID", + "required" : false, + "type" : "long" + }, { + "id" : 2, + "name" : "tpep_pickup_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 3, + "name" : "tpep_dropoff_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 4, + "name" : "passenger_count", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "trip_distance", + "required" : false, + "type" : "double" + }, { + "id" : 6, + "name" : "RatecodeID", + "required" : false, + "type" : "double" + }, { + "id" : 7, + "name" : "store_and_fwd_flag", + "required" : false, + "type" : "string" + }, { + "id" : 8, + "name" : "PULocationID", + "required" : false, + "type" : "long" + }, { + "id" : 9, + "name" : "DOLocationID", + "required" : false, + "type" : "long" + }, { + "id" : 10, + "name" : "payment_type", + "required" : false, + "type" : "long" + }, { + "id" : 11, + "name" : "fare_amount", + "required" : false, + "type" : "double" + }, { + "id" : 12, + "name" : "extra", + "required" : false, + "type" : "double" + }, { + "id" : 13, + "name" : "mta_tax", + "required" : false, + "type" : "double" + }, { + "id" : 14, + "name" : "tip_amount", + "required" : false, + "type" : "double" + }, { + "id" : 15, + "name" : "tolls_amount", + "required" : false, + "type" : "double" + }, { + "id" : 16, + "name" : "improvement_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 17, + "name" : "total_amount", + "required" : false, + "type" : "double" + }, { + "id" : 18, + "name" : "congestion_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 19, + "name" : "airport_fee", + "required" : false, + "type" : "double" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-03T16:22:16.955845960Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 615105126920399770, + "refs" : { + "main" : { + "snapshot-id" : 615105126920399770, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 7480254251893511044, + "timestamp-ms" : 1714753341825, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "584935", + "added-files-size" : "8731886", + "changed-partition-count" : "1", + "total-records" : "584935", + "total-files-size" : "8731886", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7480254251893511044-1-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 4768271945146524109, + "parent-snapshot-id" : 7480254251893511044, + "timestamp-ms" : 1714753347072, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "1167564", + "added-files-size" : "16886361", + "changed-partition-count" : "1", + "total-records" : "1752499", + "total-files-size" : "25618247", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-4768271945146524109-1-b03f7748-dbb3-45a6-ae71-77c1b8882e8d.avro", + "schema-id" : 0 + }, { + "sequence-number" : 3, + "snapshot-id" : 7258036030029852722, + "parent-snapshot-id" : 4768271945146524109, + "timestamp-ms" : 1714753351429, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "584865", + "added-files-size" : "8750334", + "changed-partition-count" : "1", + "total-records" : "2337364", + "total-files-size" : "34368581", + "total-data-files" : "3", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7258036030029852722-1-1a418a9b-0fe5-4a50-978d-6a3c49279958.avro", + "schema-id" : 0 + }, { + "sequence-number" : 4, + "snapshot-id" : 615105126920399770, + "parent-snapshot-id" : 7258036030029852722, + "timestamp-ms" : 1714753355960, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "877005", + "added-files-size" : "12830090", + "changed-partition-count" : "1", + "total-records" : "3214369", + "total-files-size" : "47198671", + "total-data-files" : "4", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-615105126920399770-1-77aae55f-bf38-4506-b506-5832608c1dc1.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1714753341825, + "snapshot-id" : 7480254251893511044 + }, { + "timestamp-ms" : 1714753347072, + "snapshot-id" : 4768271945146524109 + }, { + "timestamp-ms" : 1714753351429, + "snapshot-id" : 7258036030029852722 + }, { + "timestamp-ms" : 1714753355960, + "snapshot-id" : 615105126920399770 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1714753341825, + "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json" + }, { + "timestamp-ms" : 1714753347072, + "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json" + }, { + "timestamp-ms" : 1714753351429, + "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00002-4caf996d-57c8-4ea8-8b67-57bb93c1e120.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/1a418a9b-0fe5-4a50-978d-6a3c49279958-m0.avro b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/1a418a9b-0fe5-4a50-978d-6a3c49279958-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..cdce7ce87964bcd186acb86a2aed368bf1ead38e GIT binary patch literal 8177 zcmb_he{2)?6?d{g1A>foEF)FAetTLr0gh##^UI#n1wo^v8z~DZ4bX7;-uavyIs2}6 z=cIw5q()tA|7n=oXwn!G)7EKIHSK-x zeD^&&_F+4b0IBEi_xNmO9=+!XRKeE zr5F3zb#T$k|WG9 zzjc#!ZW^kna;hYZrBp744bhN5U~wtrXWhUF)e#lW6&2GIRf)$dlhxwLQn9X!$q`ZG z1SORuNEGucrlfdTbj)dWL>QU`E=gc*Qt)yqPe7X^pda86l$huVYu@d4=eBMrn#kc~ zjEhgeUm1^hI05mNT;b_(r0jX@yNZAfNlAKIkrUjAb9>jL<=+kg&$6si@nk|w8k|YW z;t|ME9-^d-V*)^;y!eGEHN~e44SRr=2YrY#F=1fNz)zKNmom=))|73ui*<@D>*Wie ztdkO|ri_aw@OnxUMzJH6$iW;P&7K5l}`I*2;DGmm5KvHcIzPsvU_6TIZk<#I(Y+;qmT4J8F ziYlpy_Sqy>^-;y(ptjl<4l>D}ABp>oKF)@x)vcW{j_pC-!Qpha{YOn}ZhfAgs zyr@+mE$osOmt>LCS1hu{B@!w<=7?O!w4xwuESy;mEO5gB*q!@~g=|a_;5pi9{iq`) zj*L+e5lvAP>?CUaDAcJcxqV7j*C-{eD$(Xdi*;*}d!*1(MKvZmuQtjCS z3h^-wu*)8LtiKmcNX97Y-iu5=5szv3L&2a66;p0p(Gu9@SDxfooQSuNyWw<`K~UsW0tyryr&8^-b&$oHhDn8x3^4%>l=D5mq>P0^ z?c8`M)M0yLbz9jAF(wijMM??WbfGzp+e?=dp^jrDoU$?^30NMm`o08)#)?!K zlf}ESO0c#(L{bbH3j84mnc$H8k@PXJsCZEdcI4Y+1$cF!iU3q>uo9>u`DRgEvy5{= z#RgXZmDxY~fk}d#ib0cwNa04%m!Uq~=Z&4)NOFragSv_UvLOOsPKrCPkvG>^cSX4? z*I2e908OM!;lM$w3op*Jkh_v5adOqc3U>fl;YbB+3olCoFSdmv6@o>cdXlfeD`cYP zlt(LrJ;@e!7Y|vPfXRmk%wgKI3(g8^8rw#A)?NX0Y+La}El`=+_KL7G{Y9p~o*=QU zE^AV%y1F2dY8I}fnz8Oek(Z1GKnhn@&P;zx0m;pFs3KfKG%)EBmNAL3jC0@Mb zE?$np?$*ZcEuph7!y>Dr%{8cv8uYN;unI~7nk-3&_t<>2Q133H#lto?jdd~@!B_)a zE;M%#Bb)&3g|pCP!dcMasiMW=7=1*^3$WQ7tH?@2r~~U{*l#pit!(qSsOivZ4(D*bmuNSer+fR$)A&HceX(y-~^;)~NiPRCXXHvB-l&a)^($3Dat#KqvPLWF16^jBa_JC26_n-!2rA+O3yHNlK6uYOj#I8xZ=vy~ zOwU-z+G3tTg00lEWy6T$MwqTV3(Os5dJzU974aerp9j=O)S=F_y#Yt^abl2V2(o-$!q%2aN?yR&x857S2o zuyZ%jMa*X$1FRC`*-cY5^zF%SoQvpjs-q15Zwty}Zi5QXM)m`g$?Ph{K0yWMM6^I> zP0gbm%xj{lu);wsXdMudT9?~jQ+YYYd3V{4oh`H}$#1Bs;2cAA~mcPjLr+^4iOfzj&UT89cTB{L3?gGp{PXn(l?}?uDiLt>?~NTUxs? zTitiXLjG7 z`o(vrYoBA_vY@mzS$mUpScRqc*f@oi|y z#|O|-75@;b_wf&+`YJwzW_|otG+V_-&^{mEg!WbOt!Qn-yP02Su4F!V>zQXyK7CKm zKdLVH(wow?=`HD+^yc)2bYr?YeQ$dGo+maw(=hk-jxE!Vd~mkm!!2Fs)=eM(YRzCx z-QfCLKm74M^MQ;y_3_s>o!sf$b}h4J?(k4`N5zw~y0sOtT>|Lx>9 zzE;=M^_w5m_3YUH#n0ZW9XNSjpBikM>grp()qwtbq4O0qykYGN|K7gt`+xKu4xEp^ zQvIjPTGz(w|GoIYEv@bC_Zt54p;FI{thx00^?$v9>Ti6yzV3-?^qqemiQIoC@a;`E zGw*KRiZa(~^~jH!m`wFVWbW{R+rHFf`la;qUH5HKBUigNZquGi|7-($|6<_hkKa4j c`sLc_?6)#Qw|x_rI)lG|v17XC3(HvYf3=&wKmY&$ literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/77aae55f-bf38-4506-b506-5832608c1dc1-m0.avro b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/77aae55f-bf38-4506-b506-5832608c1dc1-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..40c15577d19901412d6039b8de00d235fc48d965 GIT binary patch literal 8158 zcmb_h4{#La6{qOXSTzWyl}d(g*VuvaQtx(iBzGN@e*vNr#1Pw&2|M5IewQrSyWQ^Y zB^)85QY1Q6m|&&0OqGsST3VogETtu_6k0ltg_$r6Q>I9#Ey5tBiA_UwMEkzo+x_+~ zcgx)+Gr){{-}nCf-tWEdz2}P9+E3kfiy?O41lm^B!*f%-% zrm0~~&}6YQr3n%;u_=SV!cxf3hKUlID~it+CF!cB#^dG5s&i#&$S`oS9qWRqrji7S zLVihVDyCr9oUkjx)Mao<25Xa|pG!Fc=DGqN0XRfe!oINP+^%Xsb0VO3N-VJ{h z)b8U1#9Q`+r@@u-aO*>=h)h{c`dX0{+=zR7@ru%Khk$1pR%s}mz)4fENLe@nS;|9{ zlyO7=NR$@85TzzwXX?lYv>fO|l!*xw2_{-s!d=Qd6IfHS(H_>Riei*5fU-_bXu8^k zE#QrmE_NWjy#)G(C<9H^L`vzVQ1XdTG9cT5d_p;vP#z*#*HqmU;A2UZF`^}UAKV=H5Y}FF$oLy8&MRd+4 zk!EzLrU13oxv-i|rW6JK(;|6WfR%6@r4*BmlJ6OfmZXqZ64YP}RYyps5;3foA&vJ) zi^~cYjB$&s^N55>j|3vuHLW1XItORQ0t?(Q0e1I3Ya=VEB0NWn>>mwAjWcEkW{9RJ z6P!oX2AI%1O*LdfD@MW!Hn9<4xOw0oQe-pjd~>wg;PPJUEkHD6c2Mn^0t)f53~4;q2Jc)IQXz5gdvzRW=d0V!1fR^F=Swg0eA`at~$oh zu$W^P=keaoF8967B#gy-M{Bc1XA)KWu))Z-5`tWg=&=lz*JD9W3b8aq-1Xf~EJwjA zQnD<%Fa{ha;+^AOIK5;LRV9^x0tLsZR3~j+WRb2TQXym$C!m3Hzvq^e&QPdc=n91z z9B=GyD^nplF_BTAl)z07nq5dq(d=@(^f(b}=q2G))OJ}!$|}3>OCV^hNR_czyc}r+ zYuiIcmKaloKLjBQ9Fjk>(Fqn6E=s|MT$}6wPYhHMfQk#2163r~EQ)J}aUQ6+;5eYN z`o{n;NmNo2G+7KO+zj+(Ofzcs$Ifdcxg}UZoj?G&5CJeN#ogD)pKF}AqFk4299I^A z7E+dQ;Go@w7iL<>TS=2BIpJXO4FDD&DT8hNm?ZFH8y_hXEb`Qod<9-H7HV#Jv@_V3 zY`nL4C~7xMJ~6-?rahMvpGf|QSt&*Sz{GhX^53zodWxfYP*%K?ZUbN4f$k_b^FCy(l0_B zCI_o!5kqZj&rD=*-Q%`T#qFu+M!_A&=r#v7U35uByd^1~z1=dh1~YoqXTe2x0b>WJ zq6E-|fHf7ADTOXGyb~8S!0Y^2P_5{B)_puRijH6Tg-!f8=7uFm!)Za28N>?MkuyH| z4Q5n>XNLrXh2V~4#px(^M5Fp>m|XTlcIDUR6Q&6m&!|n5!8wCjezTnVX-^LRgBn#r z31zc9XGAMTRl$*sYVmHAvW7J(KP#18h{-qDlh zbq%mfjBhth)zG&mw{b3@$E}VE{J$+IkJ$|>JR6x0P$o006z2pLm=)1Hoi#O&axkZf zro!?EF|Tz%L~31je@*3O8Ry?+yLQ&mrX;tarh>B!&AcIYtg;(xXqDklwc86=o$!sT z+bT0%R|RS2Cf)ajbY=L@^&gGweB;G`zWU!ulb@()+qRS@7Y^i_4{tZ#dgH7;c>Qi@mj}17CcqV&Y7XjKhD8 z{QK?A>regs_=yvbg|80&1IvAUyMn_*2}z*yJbkJ@(e&O1 zG;+tr7hihly=#^KShITap}qr=n&*f5p6KiQ#%(oE{%P6tmD{TNm#kX6$gJ5kW0_Xj z(09YR4b`{LKl;^$6*cppu9>~OTqG1MaA$nLFYNA@p$ zy*Ipb)=)S$b?v~O?H5k$UwiBQw>L-5RkZB;N^tAacW=J7`5yk>12?{VO+B}%jyt`3 z{^-s7uGF1MwCp;#vHyX~a~A%RyX!wI`)|6s?}HhBr-rk}-QVkqI+~smBGA2{86AWm1F* z2Lnu066J`&gj=3u$d^!nQE-f$e^^u^s@_-!H*~QpF6kot9L2IB1{{&HsjB9)Zo1!U!z|2y}&xVO(nk$M+78TJ|O^rtLlU44@(y(EOO1r4@f*Mx{ z5}EvpsPQ&gbj_LLiZFEvT#~?AMeuVeML>ls;6Z>xP$Qx*tSPrE-P?w#>LQPo2p{c% zzcOz3aRTBkdBRiWN@;rN0ad`Jq$<8vBn7wHy?v06x$y=n%9DyX| zAxg?9CIG~8i(iORQ(R%{*ax%}=tGo=F%$D9UXjCH$~+TTle5tt)~T{=3wgdV2d@P|n#FDP5y2(eyY&>c0yA^lq z>V084hD2zEBBCssq7gvsT<{OelG*EgbF|vva$f5#Ks0Q2Q0<8V3h}WF zaLOKf=)p$RBbgnjVJWisL_DV94+VoBR4lnsRgYnhUn!Csz!hYlL4-G9M7SD2sxFX! zkcM@LJ-P4I;w1;1FfH$0>oa2?3MQf7+UGdfwQ<%kk3VM5wBdgi}`AB>~ILcHbAn&{&ZwW3hN4 z)(F;R7sRMh|#8_Wl)>Qu8Rt_j9@pkjk# zfXeD01HdFfjz^%$LZomb=*!R|yvQFruaV>yZv}N60c1l2z@!v+Un766vEGVuL$0xG zUI1E1S;B#Xb{C$RX(4YVRj@MdU~yFd7FV4I+qh9l;Kw$uI#003Q%~|0c!ey~-12B= zurJv-Z}E`T9+-UE0CSl3oPslkn#NWTo>k@n9b1t(QPWgrwlXj5EPs*ZuP;dK9FH|| zO&ed3NHyc~sb;LVP^2Vd8j!gB%9-VFHXwP~4poGYi6$mp!YC#&wsG!zQ?CemuVyu7 za1S~6P@Al_`NoCwRwF}$HjalKfFYXMv)6ecp&=xrPonGN=wrB)h&3_H`jBIe?9Ane z^R_llZwZ}!4i?!RZL&cX)S!B&VHMOEG+B}X@3Hl0p~gC*#m6=;jSVsw!B_)aPB(WD zBisNjg|pCP!dcMatD@ZI7=1*^3ozXptH?@2m;%RQ0)F#T{l))^$Sx)`5CkOvQ zjVhspvRR%oqGh8hP8{bK1>Ib|To(I*zR%NcxP z0XxU1Hc7$O8rAwxMwQ$!bG-dKHJ_$^N~=yKXO%K+=9J+)DU-kX-p<-JpQDcuVCQY3 zGnmh~2G}LWx0|MF=-ZRpIA_q~R!15B-xid|5ilZjP|bAk#?ifEe7nwm#B zn9@X3Vd;aI);b^}wJy28rt*@E^Y5}u9_n1)HSza_t(z8he5Ypm8^g-JkN2+?ANxn~ z^S9Uj;u-XD#aCwS;{=@p>XHdz7LnU{uFE1<#?>Za!aLv`v+DDFWx%2Sn$0qmlR`_UQe zrnHdp4YS?$w*`{nzw6I#<23{e@SKH&CRuC8(4GO)`0~tKDDH}qWhm$21@VVJhO7}!mo?ozh%gn zx8i{Q^5Q@3T~;7n{LAe7fBe%AYQOh-_YeA4zIp!I>OVf*zq$J2vHpYazjLuxW6>ZXF?nm-(z8Yv9jBTsFrSre?AxBmM_(3EffY={9zq^v3}bZ5Z6$+$O7 zP1HC|;@2lNPC%w;N+2-56#TPcqJ-v(Vlzbvx~i%1cxkdKTv-}23{hzjb&gk)3PvKI zUxJ#9$)an{TvvptOW=|O)+)T8OIZRcT>&cq4qg>RUs$tlSGl(hQ`JQdDFPR7hrcpv z@o@s;EqTII?Mi8Es#kerN~+>(MMiML?(OLE;%|q5XK7YxD3uTule0*fKLQ!bLxhxZ zgaJqt7rzjprnu46kq>BD(1!>U6DHzJw6Tc0gn1^grf8!*tW#y#C|&?zos`gYwN12u zHH~J| z>z?aYMIF*n7a*!>ekPD2#lb)hNUEpcyQe;;k3jaDNdtaL7goutCDu8+sN#y~oK=u! zw5lctwbi+>mR6Fo4F747yg5J%VjLx9laAo;X^obokXI7aU=gYgrz8_GQ7=In>yZ|h zWRWw5EwaKR5-L67uw2))yddigoEZu%aKi-H-TSnStf2Dn94)ecG$=JrnXMv)HASdk z4OSbVLN%IdNSIcXAj+aC8Uc!_0soLJnJMR+qtynN^;&NMq9L=DXipbVh>vA}Q}(E) z6-%gg$!w(-ucRzKVUJ1pL&2a26-#bh)f33$SC-_(;0k4*p)hYE6y|DxQgt5xLup8d z*yHejgAlsV@jDxO%?;Thlq+HLlh{07h~_LqYO>t zv+QC#-rL#bzPIUwv3T!jZLsK!qiP>ED9KiWk;@W2l)=&_k&_fImV$`8zT1iAC|HF{ zmPHrJfa6%abKDE3mkhirClgSh;5d=$q^*lA(shI@gk*{dXrSEh*(GItC{)F@g+kSi zH+Hv`t`O@*EF(`TftwyQ+mM_j+2wfYaUxXRiNh(YEfSC9Rd(N(K+ss>Dr2#D8PYJ; zwuh7?P^JohFhUkMIDaH#Jy?{#C`=z17IjppD~T2VjV1;p}x@NN5Q0=#%dHSn?RI#9~bZvp(dQBRhXNV!f@6 z(_2DkUxY<=N1JI-B{8VZX;^tR0Zo=xTGWy#*5YHEm&OJjj9{z*F6Wv%un}&6 zR>E0mGT|)f@KsS^bBsKq_yw43ja7K1!A}6|WY}*^vs>BnHc>aAAs@}qZogQL`$dSu z=-@Po7ooPbq$jdJe%)=K3fohWjeOh(`30FuCl9?8>dpCrl$So)Mb}gR=&++-5oT)1Dms2Q{LE z5XxqG)`(V!s=Ol`(c;}GWejUXenu+05aV$uK*L*PKL>cXh1=ptn%&k@oBj!O*e`Fy zLBjr(sKB`f16NU_74d;Avn;t}2p|dy^8p0q@qvNF8W|t_XCBun&#E`qcvGZj3}kIS z&mh5u>)DWD#C3xuE6*HrN0DBzK%_igun@Z8dvV(-!S#zFTji{Qph%w>*e>Vsi2>{! zpJI}NtTn3jp`0kWZRU9UcVa$C`>a--NG>R4nEWZjdQzrz^Szz5Yd%XJA;8YtMCUP| zb`7vgjBhth)R4C)yK&B=$E}Vs{J$+IkC_cBJR9i`P$tu>6z2pHm=Vz&oi#C!a4@Th zCc<(DF{gDvL}FcLe@*0N80X(*yLMKPrX;(eCW12zO}`;_tTG#GXqDm5G`AP9I^jFl zHcv=*U1M=GcgKv*@ymLqb-ew~k-xsX;JFLuMt?e{dEc^~{k4-{;U1Nj@Ou`Gow0qz z!r9@k^#AC$&)g%g-Y3p^X4h8w=_3bEtQ&l@?Z34Xe?x7&c4l|o`D+Ur%XTQspKQBM z-DBQ3OzjN3duI2GTg9@Mx6Qd!ad=|aOPk|Y=68*oaq+;qD`)X_%$YYoSPO7E8G-`=tCcWy1{ zvy@L;zlcsx-td!-iq~6b{p{*M=&cWa_wT8fet8+(DZ5fH9j&T-fA53c*Dtm7_gp;r z)ARq-{_=;D`+LW<*A3j*J@cLUkN4gCxoB;4=8kWStrMaPRz(+?GxrFSK68!^zZ`vW z!|6q%w?EXrbL{MgXKmj)?(ED|E=f3;Le{%QL$KJf$5&6%WTNP7&d~E)@iT!6U ze;D1gCdKr2iGPX;uT4nZ6|R@M@9x>#z3}GdSLc12=H8_0c?Z1qvPg$(?VF)wlMPF533*s8w>t)fn5Q(ia=D_B9KQk zh+zsSBN{SLOXf;WK)7huLzarlmS;{QhvywDNhw?ff>oeiMOF+7>d*yH(-7Uc-_)Q= zN>*T*I;l@rykmBq91;*MjHw14PS&Ldh8+E_zjWF*)w#?m|BmF7FUsY(sD;v@U}m z@sCia%v{~4 zjhu8mB3=~$twx|`ReDFsT`~;7@&t3x@R~KKAe==G^Fzn?Bt^`rjiAsbY^21%e+&3;fr{+_NVZKyopz=JSezGLt+olU z|5oo{WnpR|C6U`^5|IdJOn!lFtmK&{B-%l@f}mREnaHKcAQTQ?v^ zyCZERq(?Kk;e0x$6f#*QGn`Fl6uGdMa7>5?=aA!)xz?B77m4v;Fa!G1dvz6&kueia z{j2-gNhZVEQ|Z_X{D8k1IX3LW+1%<2!m00&F`+wTniNSW#Ey6mE`Xk&fTFZg*!*w9 ziHTKPKfn=bhF}*0;xJEk1`MF&7=U|%F2eTW5jZT%U{TMxJ7@!q6|B^QTj11T zMosn!-H^NV4Gby|F=&!w=muHp!a;Ceriqs@;M+jO)6(iCIDt2hP0r0h*f1X&SHzxU^`HF?^&gK4$q1txC2h$-HvM1GaxEYQhb@K z0+Ss&-FRIp&~TZ0GI=(LyCln+Mh$x)SP~}W!gpFAss(nCxj_ir2&c!SJ8}ve8HFTw z!Yi6j!1}&~Q)7E(24y2`f*SHmF3)L+$s_9(zKUTZ(Bpk-aRos5`3~TR0 zVorm{rTronBuKi0-yp9KkON0;is&M8nY^6NX3|+DUl__Nik!)8S^oW@M-Kc}Q;WY| z{YpHvG>|4Atvh(ZF{w?drSG=Vws{I|@mJQQKftzg`G?O-1V#Kw=>`D-}PMo#iNhO v%O?2vXy5AD(KkN-?9h%k=l^~v4jv*Lj2Jb&;;WT5R4y4Y)P7kQ6t9L-P@Vj`)Kd_^Y&)n zXXbg|d7fvV>gbv_dp_*ewOT_=pd>u7Xa{HZq!8D@=}6RciL2pMn6!T8n*7pS&@Qf0XXiqgfiTd^7E(nNQ>|nN(&@f^l(c150e2NhCy?p`1!-siC`rgZ zxzkjaB_YIMC1uP{QFtv{8V?EZ7JIO`6YNQ|pz0YYXDcy7g()MUnZAbj50+VBcg0LV z(>=okD10yn#U|UCXCo$RR-tw^+2jD+O}z(r%NU~gI;Jk#CL1Om_gHbI*JYMZP@Qdq zI}}op#n(C`G||p;dDpdc4i`S!T{7%t`M40+J3zRYv?!sWU|qI%9t)F}^#~|fmF*l6 z0$VNIOS_d*Ek)t53S}-a)OFZ{|HgMeiKUM`&(=yk#7Ddl_erEJ3s6=Kd}r;#Ufm8& zQxH_~16LzA>lPNTIDl%8t7ZxNj^ewdD1ht}bVJ3;)}SaeixkW$?LQOeVYW{O3e`NA z5{g4hF}VhNPk!BqFk4Wt8tAbOe7C5sdaQD;G!y(yo)QD+E#SNb;+hU1KDI6Dc4sC4 zX8poSrKSMvyj41|(A(Zy!{OVO#3GULm`E=x&0~8`aAjrL9f^`*MU*WiB2cgp0rPrJ!$AR1 zA7O1JgawLPPf+v6gkq43j12^o4-(LPfuR`4wju%q#jDH0%j53_K-p=@jS>`rHjfzR z)If-vvXO=2&~|x7IyD&2UGt&{^kTXF@tJi2rhK!Q6Ew&P1loJTRLSKDNlZAP2(;TC zMoY zC*599m@Fsc3n!G(NFG=xmQYmRnMppG3^q?Zp{UrqGM}&hBvKGmRZ`Y0K!a18MU4O-Xk7fvAVrYDFS`1Qqj;Ovybgox^JAQa()^?Z=$1JOt*5Do-x zSu)nwzwg?iAJ1JI`|9pDcOAUr@bov%H8!7KUH-**^U}eil|KvV<0mejS#jm+w(Acr zZJqVR&VA1>yD)g=$g!hC8*ZQ1Gdo=W8PVA52k#&K z=h)e2uOAv|yl>vni397Cv4PWN4-C#Z*}veuzL)0gsebXJ4I@A8?i${Y&L0^bx-_s1Eq|dFmi;#G*1?+KmfyeMJNn3n-<1D( z`RliTID+huL4 zq7f27u>?Z!h9U$LpG?K-gF(E-gfy)Ww!uVRs!>v7>kULh;+fkyXZF(W_Hk#={NH~& z-}lWUJ$*}>THt`8-(`wfq{6q>402X6hqxqXAW=6YuAS4MYX4TDCTEaor8BaE@Z*e( z6v^b0I|j>6AyP>`)Gcqj!NbP}~gmbheu z=ZD(MV=GAOWuq%P6MMUEK?T?e0#kk(0a*h*MGKjJx?W&`37mt5CENL=%^VWb7?~t4 zYZl2mh5rPbHiEo~Bw*>FV8~W!z9A;8sB0E--uJq|2HW8NX#zb1tSkb4Ym{6jYYE5Mv-r)SpM1h|*eK?Lh|3JJ6DpcM7iKHp>av>y@HCAg@Q*Q#;=@c_v`aQEyzc4RLOvy26ri>8iQ|i zR%pJ=%lmHVg|c1*+U_>PK31t00(%EYFBS|++E6tvJ3GG%3x@R@P&Fz$IYI)~TKY_lWIqAnq^374)w z3$^&%h>>oOg(7jjGs<`Jd?*o*gkqu4YK)^ot78BeCf2pa&>EkNJ6sKD>|8C22unt< zr}~qvWRtO2=S5rgTtDCx!w1CP)8=wt;HkcWWkNQvnj~>3L{_{G7hvy?iz1Yx5dE)$ z6CkUqet<%R2n4xUj1#B=!isriXFvdi7y+Q3poXyZcmxU-Az=Q@sXM3w8faLm4!1y2 zqXf0GPpAdCM&Cd{`Az~_$uZOdnX17-P`q#nUIl@#0?KYnxtE{_ba+HL7Y9P*l=UnW zhmOlv(#65}@>&r^pcl)XPe84!bSkimWln=sAkf*9PSqWrG>LHs6oF3L?@`?VetRm# z=P4@?+OgG*_9X)qldvbDX9c@Ue5^!j$OFcb5F%&3(_B(bkVU2j!F3}PJwV;QQ&`9- zRB|V@qwxx`jxV98ftraySqQd*KcT2tyS(q$pcSbKDr;D(ZC!RgR%`Jj6pYGM(0tSQinFMdP8EkO)VFa5NGM3w)yC_<{4^hW|Xb zZ1%%s)9*_sXRiBv>C5A1wloSW8|E&&I2S#p%7bcg$+793|8-mUFaNFK=#zYM_~R>k zfA}i?%ZV3+&?le0{rZ+YUrx>OLjRA;p1CqM@^C)UI{nZcdEVj z*PpZ_AG|R(`s(vrPoDn1>xQn~(Z?GO9)3zabT%>d&Yr95PCs#EUF+z+i+_(P#M^=YLTEXphfglw*VwR6w5Api%MWaAN%L@z|L$SW?U0a}9hVY#+;IuDL#=uD zeqlu)GY3|rfkGCz-X{FEeGB5;@GZ<--$QS{aUBg@=AhSFD5i_3%dV|8>$<+PRE$lD zx2Z;_z7qSE&oD)NgD7<0C&=|sd&o$bZ8r^UD1rMpU@qT}8TN@~0$FU_^_bXo3;smc zuc0U;Hex;+1TGuj?}`%E@;yf4_tPw>!Cp!J43XbK%poXCZ4>ENZOrfzGOpT$lETi7 zH3bopcO&9i#Pp(Gn*?&-xsX9XSu*MhtLH*~X5!uO z2hb!aTbyB`66#6EP*)UGycIL_a6fc>mMcj9VCf2873N@;oM8e*UoB90vg3XZP0Np} z%Jni&4uWb1LzFF35~bIaxa2)IWYF&BY@uIGmrsbUmLX{hd&c0`;ue~;^J3lue^AIp zF6~J&__DHG80sAmxtJPMq@gmdTDvR@Q^RHnsEn#sP6z>4i{#SY)x3yO4OA$-#5f4> z5dMa{-vj9r&&RbgsOyca#d8wLWdXZh2zQnT&f>OxpAziBgG3{51rdl>2VrlRsF}m> zDBLAVk=Q5N#xxq&pmn7eB`gk|Ka-X*?@fbZuY_rn4vr|(ntV@r-DoIVpr8hU*@n9% zbv4VXmPoVU-^`R4#c!ebE!6P?gyGm+)OlqlfYo^s_50@l$8XIKHg@jqENO7tdLR<% zjA`tMgSDbw4}*5_qY!L4GMy98qpv~Or{dvk`8KgGz|kpYOaVXZmbf}~y%)N-FN3wcK0AfWoHfR;Ii zS&*$69E9Z6#&{(Iz5=RtOVLY6f-IhxoacjxajJS2$&uyCCh7Sw*?29b81iBz^~p`^ zicIBtvEVes2@F|#k*T@EizF%TKoVrNvmDJ0$kJ05Uze&NWQR{TnU@Zxp-@jD&lKDx zEE^j&;(=mGgpe!WX^E(=i$<0PA#o#;o|Ep(DNHj8W$r{~v{8bMeTk&z&MXYdG;E1K zkyMyn>Gx~iic~^f4@Nbg%kjtRES^NdA8k3#!SQkX9p~h*b9S`# r@kbkP?S22n$?on4pa1pecYl2Oc=P0?$3Oq{L*M)M#hV{46!8B8Zp5%{ literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/00000-e5584cce-27f3-4b56-a5dc-e1ee0e1df858.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/00000-e5584cce-27f3-4b56-a5dc-e1ee0e1df858.metadata.json new file mode 100644 index 00000000000..6255133468a --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/00000-e5584cce-27f3-4b56-a5dc-e1ee0e1df858.metadata.json @@ -0,0 +1,180 @@ +{ + "format-version" : 2, + "table-uuid" : "8b920795-6a06-41b3-954b-538c9f39ea60", + "location" : "s3://warehouse/nyc/taxis_partitioned", + "last-sequence-number" : 1, + "last-updated-ms" : 1714753363900, + "last-column-id" : 21, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "VendorID", + "required" : false, + "type" : "long" + }, { + "id" : 2, + "name" : "tpep_pickup_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 3, + "name" : "tpep_dropoff_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 4, + "name" : "passenger_count", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "trip_distance", + "required" : false, + "type" : "double" + }, { + "id" : 6, + "name" : "RatecodeID", + "required" : false, + "type" : "double" + }, { + "id" : 7, + "name" : "store_and_fwd_flag", + "required" : false, + "type" : "string" + }, { + "id" : 8, + "name" : "PULocationID", + "required" : false, + "type" : "long" + }, { + "id" : 9, + "name" : "DOLocationID", + "required" : false, + "type" : "long" + }, { + "id" : 10, + "name" : "payment_type", + "required" : false, + "type" : "long" + }, { + "id" : 11, + "name" : "fare_amount", + "required" : false, + "type" : "double" + }, { + "id" : 12, + "name" : "extra", + "required" : false, + "type" : "double" + }, { + "id" : 13, + "name" : "mta_tax", + "required" : false, + "type" : "double" + }, { + "id" : 14, + "name" : "tip_amount", + "required" : false, + "type" : "double" + }, { + "id" : 15, + "name" : "tolls_amount", + "required" : false, + "type" : "double" + }, { + "id" : 16, + "name" : "improvement_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 17, + "name" : "total_amount", + "required" : false, + "type" : "double" + }, { + "id" : 18, + "name" : "congestion_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 19, + "name" : "airport_fee", + "required" : false, + "type" : "double" + }, { + "id" : 20, + "name" : "year", + "required" : false, + "type" : "int" + }, { + "id" : 21, + "name" : "month", + "required" : false, + "type" : "int" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ { + "name" : "year", + "transform" : "identity", + "source-id" : 20, + "field-id" : 1000 + }, { + "name" : "month", + "transform" : "identity", + "source-id" : 21, + "field-id" : 1001 + } ] + } ], + "last-partition-id" : 1001, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-03T16:22:36.131552469Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 2327977723212680118, + "refs" : { + "main" : { + "snapshot-id" : 2327977723212680118, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 2327977723212680118, + "timestamp-ms" : 1714753363900, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "8", + "added-records" : "3214369", + "added-files-size" : "49496251", + "changed-partition-count" : "8", + "total-records" : "3214369", + "total-files-size" : "49496251", + "total-data-files" : "8", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis_partitioned/metadata/snap-2327977723212680118-1-69460130-5a60-48ae-b1e0-6aa9fa2b98b5.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1714753363900, + "snapshot-id" : 2327977723212680118 + } ], + "metadata-log" : [ ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/69460130-5a60-48ae-b1e0-6aa9fa2b98b5-m0.avro b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/69460130-5a60-48ae-b1e0-6aa9fa2b98b5-m0.avro new file mode 100644 index 00000000000..4dfabcb5cc0 --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/69460130-5a60-48ae-b1e0-6aa9fa2b98b5-m0.avro @@ -0,0 +1,10 @@ +Obj schemaê{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"VendorID","required":false,"type":"long"},{"id":2,"name":"tpep_pickup_datetime","required":false,"type":"timestamp"},{"id":3,"name":"tpep_dropoff_datetime","required":false,"type":"timestamp"},{"id":4,"name":"passenger_count","required":false,"type":"double"},{"id":5,"name":"trip_distance","required":false,"type":"double"},{"id":6,"name":"RatecodeID","required":false,"type":"double"},{"id":7,"name":"store_and_fwd_flag","required":false,"type":"string"},{"id":8,"name":"PULocationID","required":false,"type":"long"},{"id":9,"name":"DOLocationID","required":false,"type":"long"},{"id":10,"name":"payment_type","required":false,"type":"long"},{"id":11,"name":"fare_amount","required":false,"type":"double"},{"id":12,"name":"extra","required":false,"type":"double"},{"id":13,"name":"mta_tax","required":false,"type":"double"},{"id":14,"name":"tip_amount","required":false,"type":"double"},{"id":15,"name":"tolls_amount","required":false,"type":"double"},{"id":16,"name":"improvement_surcharge","required":false,"type":"double"},{"id":17,"name":"total_amount","required":false,"type":"double"},{"id":18,"name":"congestion_surcharge","required":false,"type":"double"},{"id":19,"name":"airport_fee","required":false,"type":"double"},{"id":20,"name":"year","required":false,"type":"int"},{"id":21,"name":"month","required":false,"type":"int"}]}avro.schema¦7{"type":"record","name":"manifest_entry","fields":[{"name":"status","type":"int","field-id":0},{"name":"snapshot_id","type":["null","long"],"default":null,"field-id":1},{"name":"sequence_number","type":["null","long"],"default":null,"field-id":3},{"name":"file_sequence_number","type":["null","long"],"default":null,"field-id":4},{"name":"data_file","type":{"type":"record","name":"r2","fields":[{"name":"content","type":"int","doc":"Contents of the file: 0=data, 1=position deletes, 2=equality deletes","field-id":134},{"name":"file_path","type":"string","doc":"Location URI with FS scheme","field-id":100},{"name":"file_format","type":"string","doc":"File format name: avro, orc, or parquet","field-id":101},{"name":"partition","type":{"type":"record","name":"r102","fields":[{"name":"year","type":["null","int"],"default":null,"field-id":1000},{"name":"month","type":["null","int"],"default":null,"field-id":1001}]},"doc":"Partition data tuple, schema based on the partition spec","field-id":102},{"name":"record_count","type":"long","doc":"Number of records in the file","field-id":103},{"name":"file_size_in_bytes","type":"long","doc":"Total file size in bytes","field-id":104},{"name":"column_sizes","type":["null",{"type":"array","items":{"type":"record","name":"k117_v118","fields":[{"name":"key","type":"int","field-id":117},{"name":"value","type":"long","field-id":118}]},"logicalType":"map"}],"doc":"Map of column id to total size on disk","default":null,"field-id":108},{"name":"value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k119_v120","fields":[{"name":"key","type":"int","field-id":119},{"name":"value","type":"long","field-id":120}]},"logicalType":"map"}],"doc":"Map of column id to total count, including null and NaN","default":null,"field-id":109},{"name":"null_value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k121_v122","fields":[{"name":"key","type":"int","field-id":121},{"name":"value","type":"long","field-id":122}]},"logicalType":"map"}],"doc":"Map of column id to null value count","default":null,"field-id":110},{"name":"nan_value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k138_v139","fields":[{"name":"key","type":"int","field-id":138},{"name":"value","type":"long","field-id":139}]},"logicalType":"map"}],"doc":"Map of column id to number of NaN values in the column","default":null,"field-id":137},{"name":"lower_bounds","type":["null",{"type":"array","items":{"type":"record","name":"k126_v127","fields":[{"name":"key","type":"int","field-id":126},{"name":"value","type":"bytes","field-id":127}]},"logicalType":"map"}],"doc":"Map of column id to lower bound","default":null,"field-id":125},{"name":"upper_bounds","type":["null",{"type":"array","items":{"type":"record","name":"k129_v130","fields":[{"name":"key","type":"int","field-id":129},{"name":"value","type":"bytes","field-id":130}]},"logicalType":"map"}],"doc":"Map of column id to upper bound","default":null,"field-id":128},{"name":"key_metadata","type":["null","bytes"],"doc":"Encryption key metadata blob","default":null,"field-id":131},{"name":"split_offsets","type":["null",{"type":"array","items":"long","element-id":133}],"doc":"Splittable offsets","default":null,"field-id":132},{"name":"equality_ids","type":["null",{"type":"array","items":"int","element-id":136}],"doc":"Equality comparison field IDs","default":null,"field-id":135},{"name":"sort_order_id","type":["null","int"],"doc":"Sort order ID","default":null,"field-id":140}]},"field-id":2}]}avro.codecdeflateformat-version2"partition-spec-id0iceberg.schemaª-{"type":"struct","schema-id":0,"fields":[{"id":0,"name":"status","required":true,"type":"int"},{"id":1,"name":"snapshot_id","required":false,"type":"long"},{"id":3,"name":"sequence_number","required":false,"type":"long"},{"id":4,"name":"file_sequence_number","required":false,"type":"long"},{"id":2,"name":"data_file","required":true,"type":{"type":"struct","fields":[{"id":134,"name":"content","required":true,"type":"int","doc":"Contents of the file: 0=data, 1=position deletes, 2=equality deletes"},{"id":100,"name":"file_path","required":true,"type":"string","doc":"Location URI with FS scheme"},{"id":101,"name":"file_format","required":true,"type":"string","doc":"File format name: avro, orc, or parquet"},{"id":102,"name":"partition","required":true,"type":{"type":"struct","fields":[{"id":1000,"name":"year","required":false,"type":"int"},{"id":1001,"name":"month","required":false,"type":"int"}]},"doc":"Partition data tuple, schema based on the partition spec"},{"id":103,"name":"record_count","required":true,"type":"long","doc":"Number of records in the file"},{"id":104,"name":"file_size_in_bytes","required":true,"type":"long","doc":"Total file size in bytes"},{"id":108,"name":"column_sizes","required":false,"type":{"type":"map","key-id":117,"key":"int","value-id":118,"value":"long","value-required":true},"doc":"Map of column id to total size on disk"},{"id":109,"name":"value_counts","required":false,"type":{"type":"map","key-id":119,"key":"int","value-id":120,"value":"long","value-required":true},"doc":"Map of column id to total count, including null and NaN"},{"id":110,"name":"null_value_counts","required":false,"type":{"type":"map","key-id":121,"key":"int","value-id":122,"value":"long","value-required":true},"doc":"Map of column id to null value count"},{"id":137,"name":"nan_value_counts","required":false,"type":{"type":"map","key-id":138,"key":"int","value-id":139,"value":"long","value-required":true},"doc":"Map of column id to number of NaN values in the column"},{"id":125,"name":"lower_bounds","required":false,"type":{"type":"map","key-id":126,"key":"int","value-id":127,"value":"binary","value-required":true},"doc":"Map of column id to lower bound"},{"id":128,"name":"upper_bounds","required":false,"type":{"type":"map","key-id":129,"key":"int","value-id":130,"value":"binary","value-required":true},"doc":"Map of column id to upper bound"},{"id":131,"name":"key_metadata","required":false,"type":"binary","doc":"Encryption key metadata blob"},{"id":132,"name":"split_offsets","required":false,"type":{"type":"list","element-id":133,"element":"long","element-required":true},"doc":"Splittable offsets"},{"id":135,"name":"equality_ids","required":false,"type":{"type":"list","element-id":136,"element":"int","element-required":true},"doc":"Equality comparison field IDs"},{"id":140,"name":"sort_order_id","required":false,"type":"int","doc":"Sort order ID"}]}}]}partition-specœ[{"name":"year","transform":"identity","source-id":20,"field-id":1000},{"name":"month","transform":"identity","source-id":21,"field-id":1001}]contentdataàÙ‘•J Pq(Ü´ªˆ”ݘolgÇŸçñår<1ÞÍzdÇÃÕ½y7×(gÇnFSTr{6ƒµêª®:¶™ÎU+ÝÚ®qG;íE(«è °h@1 ¶Nt‘FUJU-·îFäUÉZ­éÄ´ Ai€¼XÇ +íÄ=wöÅŽ}Ûš.Hã$ÿüÓù|çïÝçù>ßÇÍ”§Çß9;~ÒüöåV¦ÓßÌï(lÞ¶³¯Þº{cº˜ßµ¥ïŽíùÅ-Å-Û¶îJß•/æÓ» ù«²™L.}÷¶­ÅÍ«ºÒ¶uæVtv÷d²›rÙ®Îl!×Ó¹<¿1×Ù³¢{ygncO6ß]Ȳݛ:ÙÁËuç´÷ì,#·Ü°fõ­_\‹ž\‚*¥É™¼)˜Ø ›‘u¢5‰3%S6©©˜qS5¦f&צÖç`Ä! £0Š EA1$!Q¤ 8RQi(éÈà0ƒA +ˆ$€’ ’¼Ãš?fçð6N4¦¿kìýmð¢}[ìúŸ8 zŸÍöâj@ØßA_Å窻£~GDXíbþ—$¿“ýŽúâwq¿Sý.áwšß%…Sí¤÷Jÿ*B‚Sœ·z˜g/ælW æìh¾¾‰æÑ%Hš.ï üç‡içV¿þF7öâþ~pê7ÂË]ƒÿô‡2>ò‹ÏDÞ=~•xâðÓ(:ùÈ{ˆ ¼¹2v|ô`Hzïè7äf>IÇý:¤<óvw|ú[íê¾CÏq‰éo_«Ùû–&‹©Á"{Žì:+<++˜•0+VDV¢¬Vb¬¸?Of…²¢°gEe%ÁŠÆJ’•+s#襑°38œq^-Ç‘³_s^f,AŸB`ÿy~µÍ¡Ì~V8ñJ›O!BŸBèS)œÝc×(ŒÚ²ø³Ÿ²í­‘*…ýëí*…ý_²ãbÎÝN¨â&¶½³Ç®QÈÙ5 +ßI +q) ×Æï«°;¶ÿk÷iWÅÊßoþþé9ªÅW*•Êk}䆪 +{½A·‰â'`ME­#b[£Š=OUý_0dï×mÂFUÅänCŸ?Ÿ4¾Š^ÕS»ï)ÃWaø*zT´K ˜²ÙêXZÈPêjJ•%ˆÃïnqnr rƒ„BâA.ÁÈÃP¼?z?9c»¤”KV rÆKPýL” v&`j²g„9ÌccÆ,â(&8†%,cŠÇ*N` 'q +/ÒTn¿îlߌÓgþÆ:oŒ£ÊÞs«žì‡wÆÇûP É +50Àd¯ìw­MvŒ5îã-À`&;å‚ZL=~¾}ÂUAüoÖÕTˆŽ +ïôИ§bÊW1ã«@óT, PQ¾ñ­ûH9d´VÁnš²·óƒU,ÂT±à“k5UĬýßàå6¾Ü&œ„ø"uEѱ©ÀØ“!Ɇò1HÅ)ÏÂø0T+m‰cP®cÜâ,Þ,l…­ˆ%ZQ‹X1K²d‹ZŠ·T+aiVÒJY‹Ä¸M]gtÜóËà«m AâÌÁí«ð™SÁŒsÁŒû>9âÓ1ò¾Œ;îuWÛ­ƒ„g~ÍŒÛÅRu"227ºNq'ŒÈµŸ÷'†«âh¶ðP£ŠÉÕŒ/3:q~s¦Æxÿu îܳƒÛYÿ¾ÞñÑ2¾âJâÐ皟pâÐÇ0ÜÛµ_»óç<ãàøƒ¿cà[.öf`-ãs<háØó½îËFs,ntlïéË °ìPÙÿµfÉÇ_ÅGHbØ+Z¤ÏìòüºâBÂÄ3¨HŽ è†±!$ »™d)Ãn&9J Cí‰:¿Ö9×ëa=¢‹zT'zL—tY§º¢ÇuUOèšžÔSúb-_'î½à…U£ð½«ï;Ýè×®-%ÿì×í~þ@6ZÍæ^4ŸË$~=´|í¿ÔÿÀ†×\—¦Ï±®¦BªFîÊW²ó3I›¹/~S€_ßÄ"Éˌք»ÁªDøÔ",_3Wä×ÝMˆ;~Mì{<Ä ¾„„cïGác.â/Ãè$Gaì (= eÒ1¨ ÁøT_‚‰!¨ Õ!N8Â`&"’(!$F$"J'*I$IŠ,â—`hãûÑ%¬ûÿmÌ!ži¸‡é5~$áìË$ËZ¬*âA&Þó§À®ŠþG7ܲµN‹$3,ŸëšIþé«‚"I‰sFó@•Œ…˜øÄ"˜x¦ç +L<ÛDø3Ž‰Ó׫„?Žøa$Ø.áC.á“0z²uåa(†ò¤£5Âg\ÂYG8å(OŠi˜F¨H£”Е¨L)Uhœª4A5š¤)ºX ËuKöØåÛÜy¹„§þ§ÿÞ 2q{ÝßÓw{ã4¾Wº·N‹)îju<;?¦´"<8¦Ì6˜¸«W%w—úšÂÏ6›øàÙ‘•J Pq(Ü´ªˆ \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/snap-2327977723212680118-1-69460130-5a60-48ae-b1e0-6aa9fa2b98b5.avro b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/snap-2327977723212680118-1-69460130-5a60-48ae-b1e0-6aa9fa2b98b5.avro new file mode 100644 index 0000000000000000000000000000000000000000..d75fbc1402fc5b1dfb101760bb08d74f0bb18442 GIT binary patch literal 4287 zcmbW4&ub(_6vtx%iyq8Hi3itMw31V@bWb*!$-u&j>q20`#fdt*%hJ|#SEoDa=^pz> z#&L-U#jBVh?B>UkL_v_SvKMcA5cD9TH(@V=NAcjrzd&F0kE-hF?#WC}!}L_Y&#U*U zzVAD4w>F-cp21t5d&$>K(#A)#_oSfLAyQrPh~|2_G%q>0O~1FXV;RH`RKv0fJT@$1 z>%LUKb1%1|g98&*B!T?Evz&GKuDcqtli}2eoN67WhTDZhg3`q)7CIsBbO5z1prTY%-@zT+ZSTTrDzXkBdQ<71cr!?*OaCnL$|@O5=*XORI2Z*t7tpQN_+NA)smzU)np86WX=} zDwJGe?0L8cKf~Q`gY>cIgIeh=$+h%|XC#u!0=66Vt?dTV^JvVC`Z zUV_`!29Zc+Ol_NOtP!@`7_@_Kg<#F(O`A{t=l*tGpRNLX|A$Z zsjgJ3RrpjY%Qab1Y8Qdi#=*iC@qCzTla<+2j2Ge=FsWR$Gy)@IJ(>DXUu7p5hIOdX zu^0P+XftwV*eA0&?+cQt@4=X`JeVeR2u0WtAHxOE^Al5)JcZ5w5>7^}O8o#wR2YI? zEWrtsK*V8Q>7?2DW{s zo=l!?aF?)bVAQY&f+b->F8oi6MRkQ8WNr{*H^S)|=}w)(Sw^AAo$!j*3b2tc;nd8Y znL#-VTi{PP6=qlR|21nxN}-kmqngg;=+EjXo`l2rxXRj(v0>#tB;z!AT-qSs7J^Am^_3tN-Cja=cY1`jCR*#yy2RrKR zho{Hy?>|!e&7E5p)i0ZC{agM0;|B*{A0OWC@AaGgZ(YM`86TMT$>xD+oNTss_D&D= zy^WL2`+Mg8eREg0cJ$N3&65Ys_jZ5KreB+ydg1rO@bRTjp7j2G_}RqNPhUO%%5PVd fnG16l=Dzs#pR02}zl@&ytMaKiJ&S&s$YuCH(NMLq literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/io.deephaven.iceberg.warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet rename to extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/00000-a58846ae-c052-41d4-9e2d-63e29407b88a.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/00000-a58846ae-c052-41d4-9e2d-63e29407b88a.metadata.json new file mode 100644 index 00000000000..9f3bf63f6bf --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/00000-a58846ae-c052-41d4-9e2d-63e29407b88a.metadata.json @@ -0,0 +1,160 @@ +{ + "format-version" : 2, + "table-uuid" : "0cfe47f4-5403-4ea0-ad4b-b4c5c0711cd6", + "location" : "s3://warehouse/nyc/taxis_single", + "last-sequence-number" : 1, + "last-updated-ms" : 1714753336329, + "last-column-id" : 19, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "VendorID", + "required" : false, + "type" : "long" + }, { + "id" : 2, + "name" : "tpep_pickup_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 3, + "name" : "tpep_dropoff_datetime", + "required" : false, + "type" : "timestamp" + }, { + "id" : 4, + "name" : "passenger_count", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "trip_distance", + "required" : false, + "type" : "double" + }, { + "id" : 6, + "name" : "RatecodeID", + "required" : false, + "type" : "double" + }, { + "id" : 7, + "name" : "store_and_fwd_flag", + "required" : false, + "type" : "string" + }, { + "id" : 8, + "name" : "PULocationID", + "required" : false, + "type" : "long" + }, { + "id" : 9, + "name" : "DOLocationID", + "required" : false, + "type" : "long" + }, { + "id" : 10, + "name" : "payment_type", + "required" : false, + "type" : "long" + }, { + "id" : 11, + "name" : "fare_amount", + "required" : false, + "type" : "double" + }, { + "id" : 12, + "name" : "extra", + "required" : false, + "type" : "double" + }, { + "id" : 13, + "name" : "mta_tax", + "required" : false, + "type" : "double" + }, { + "id" : 14, + "name" : "tip_amount", + "required" : false, + "type" : "double" + }, { + "id" : 15, + "name" : "tolls_amount", + "required" : false, + "type" : "double" + }, { + "id" : 16, + "name" : "improvement_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 17, + "name" : "total_amount", + "required" : false, + "type" : "double" + }, { + "id" : 18, + "name" : "congestion_surcharge", + "required" : false, + "type" : "double" + }, { + "id" : 19, + "name" : "airport_fee", + "required" : false, + "type" : "double" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-03T16:22:09.674882012Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 6884618194677467995, + "refs" : { + "main" : { + "snapshot-id" : 6884618194677467995, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 6884618194677467995, + "timestamp-ms" : 1714753336329, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1714753288452", + "added-data-files" : "1", + "added-records" : "3214369", + "added-files-size" : "49412347", + "changed-partition-count" : "1", + "total-records" : "3214369", + "total-files-size" : "49412347", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/nyc/taxis_single/metadata/snap-6884618194677467995-1-651ea117-c6b3-4853-b4f3-09aa36b77f19.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1714753336329, + "snapshot-id" : 6884618194677467995 + } ], + "metadata-log" : [ ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/651ea117-c6b3-4853-b4f3-09aa36b77f19-m0.avro b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/651ea117-c6b3-4853-b4f3-09aa36b77f19-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..6d16a630d8b3f161c58fdc7843793b2d6884bcaf GIT binary patch literal 8189 zcmb_heQXrh5x38-aZ9kVf($9N*=$T}wP$*}3;s&2m|!A|fY1=r6yj{&?Y*-X@7->9 z_Y8-zYD21+s7W!Aa6!;wH6%7lt(LZFtbQQ@tre0|DWU3z+NNz9RMVh}KT@@h)Xv+z z-M4qXTkg)504aO#&Bt$kGjC?{NO)iA^1BSF7st@C6$7l9)G*u18m69*Og6~czck1Z zwlx%FqcT<^2HU!KfF)nJAR9+9a(*|CM^wGD18(T}%L!S>@N*O?1`awR6*b<&4hHR; zynEBsuqJA<)R)l22r{uLgTUNU$j^p}Vwx*TNEa2+RZWdXi<7n9m8BuW!0{fei;|j% z6C`r^6;Tsm1-s^Ka7CE93@*uFZCvtmDMLV`E8tOpLsBEy7uJm1P3~>ORCO$(ctniu zhrbHy@o@s;EqlV#>`K|$wM~_fDXVc`E7F47;@<9fr10Au@GQkD4JBhZZi*Hub4MUe zIY&tuMFfCYVe#WAHSqycM?RosK+jPo#!Mud=s*E?Df3KVO~FQcSf?tAQMdrgIyt84 zYCpDsHxjzki}aoX=*v+CnyQJE!cC#%lb~ckwgdUZVl1INM6#}_x+z9+KAv5@R7pj2 z&c=~u^s1%^wbi+>myIVB1^&|_d3TVF;3!HcCfiEBr!-oUIIkqA!5FHJ7*E8)ST91F z;E@)U6)YMH7P;Od5-L3siCovToFMBwoS6?SaKi-H-TSPKY($mdIof3ZXfSG&F?%sX zG_^9JHljAjaBZ4u$b?o*1S{CYMv&p#z(1tOX43iQXtlu=yw+QQXvplP+EWD-;$s=$ zls%^F(N1Q+Z1yq_?_w-I5szv3xlqW1iX}Iy>M`W;D?{?b;0j}(VF+(p8N$^dqv{g* zhtZG@u_yPvTD;_d6QzGzT1iA zD0qyNEQ>CT0mq4W=eQS6FBv3NNyMN)!Eq|pNn00Lr0a-O2-(CjXrSEhnI)x<}5uqby?3N>fiWCwV0plSiA_)sxWwPcz_aZNGK0~H@y z090207z8FsN+JSH7DEa*1AQ5@1MTp~&TAyOC0apUL;!h?0GO8I?rY@FHQrlM?#MNs zFA6{lDN8tT(C)%>GmZ0B(m0AQI#@z8fF-mP!8T!D68N!AXekmb^3;=j1zs^0YHoS7 zGuW4Gg12}m>VBAf!hktUdrrYwKuzNt3D25}fR1m>ov2wVGv8Dcc9y@4<*zSD{05IT z2~AsEkVrKXim7J2w@_pxV-}Ev;>wxjZ$2P-*$!1ijA0XzE@2*%Slc-Fz1bX>^rU7r zW^j*j?lECS4g1DL@Kz&3gEmft9e{z&{MqZgkkAm4(I?gQ3G^}CMZ}s2W_`#pM|SRV zBzRjJr?-U8z5t8tjyBz(MrzPzr(u=U7&KY30q?Q(Xfd4~M2nAYUK$%@FoLlLxSVb7 zAV#Tax1G+btt)Fr!y}7F=`} zFn@5$O8{L6SW`ifQs^?nJ8@A1yw1-D)ry{H-6v3^==hag*y4|4ZdgJzoE9{xK`e(I z8RL`RU`91~c1SQ-i0(+1pN@P-G^&q=$z?xeS9WbaVOoUojM_vQoH3YXH_NG?_T=C{ zs8J=9P&UgmMznlXcH z{qnXtNZ7wp6(rYS;3{aeqCU`NmL-=C0aQVLK7gPcKJbuOi{pd;%;P%cS@mWcZwmB` zhpf%z86?<3J)1X-xNZn^<(XyfD9{T5h?K(%0YbNMFYY)cxPI|;tDG?q6zCHV+vOZS z@qnG?QjjoS-Y zopAZy?y6MRRY{t;lAj(w5?*`w+bj3IXKp`#aL0pzpH)iX-@p2*^uL$Be&&yJ_uR8; z)dyVd>b?KCIriz=bJ2IlYG0nc(e!@z}!;i;SPq#e(le_+U@P|g((rcstxp3%( z4{k60!L~h@>fScyKKH`R(*wK11GNp&69;~Df9LF_)lZ(>^p)MAt8H_U^ZzQi0U|76zlCC`22cNYWfZ`;e4jJy-5`8;#Dx}$Qa^1kKw1=!2&PnVAz3)Jw; zLscF1LzQcnj|bSd+skJvDrzelE1s9;lCP~gFjRN%@};3lZX`6qZ3%7RWt9ZNlv5|^|Xpt&a;zvpeT~+SH9pZg2u|K=8&tX2!9dAA86l&iH-ayf^cG z-@Mh?J9YXD?)%<_K(|O2PtM&@=&(nWrs5Ob^9^NPadDS@?_$?BNkBEzb_hH+ZQ>Y# z(%iX|ThYVRf)z=ifcmz(3*QY-hd9?f9aGzL(R=S+MFX2!=*DPBkFC4)F5 zm$_P(5CQoxB(6?0H|%zZFY?Zmq?N2f@hUX1h?{|;0S*v048nFEH8rl1T9sH`8|t$V z@0vG&LxR%9DHb{*?{o@PlTcAATHxYdV0kptknF~i6|5`N!ZhB)3`*r3u^o##N) zy|AoZue0JH=%znJ=`yKMl`SbQ`N$3^9CvN8&{vY>Gos6Fh!2IaVDQV)5t@~GzV5y^ z$kifK_P7{qSy3%i`3~?}%ni!RP#Rb6U0Q{?VbcPXMwL6qg@CC=d}((SH*_2YR4BQ` z*!S@eeulf>1?l6@r?t|rRvPIM&qySb1#G(k+*uYljoa}&hhP^T#2R_s4?(<4gxz7R zW){1naF-kh@jOunN*%(!7!7CLGl+f~w(ej`&ZOE1q*rT=##T*j zsM}k$dL2I7+toGTbTQrN6F-2twz_#P72}O~2CS-UwoYJV>?TwH=a=|NhG8A*Z0yB; zAlghNGwhSuoc9IE)c0Xb*gi~?CWIpVh%ey+*!hVmYM#R9e+efeR;7MGAT~LIUo6Q9 zltAQRUhE7wK&^9t@B|BlUB)8_Se3)lo(p$S0?ibxwS-#`)EcK2`-DZv3;G5Qtz6{L zBFC@@vbBJN5WKA^UcrDbfy$>P?=ctlR@}6l|75#h<2ru^kkTHycSYaak1k5 zWTtgVr7~wR=QQ{ZRMC50sinj7B1!H*5Jb1r8Z8Y-%TpA;B2LN5JJi$!&tA7o(=VmBh_8R<@)!aSo;hGs^erP++lW(<2`~Jgw+RaDL z9>0Hdtc}|DK3LNp9X*(Q{_OGi=IC(L9zFHUy~Br=Wj!A|Ul_;x9lUouHb&<0{_Uef z3m&->)B1Y6|K)@Bjr;$7Gr4>BQ~TFnKKpn4TEG9tuF<*O`2MR;&Yu5qC6~tkhmNrt literal 0 HcmV?d00001 diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java index a33cd48f609..98c78848e35 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java @@ -35,7 +35,8 @@ public class ParquetTableLocationKey extends URITableLocationKey { private ParquetFileReader fileReader; private ParquetMetadata metadata; private int[] rowGroupIndices; - private final ParquetInstructions readInstructions; + + protected final ParquetInstructions readInstructions; /** * Construct a new ParquetTableLocationKey for the supplied {@code file} and {@code partitions}. From a507402c5c54ca3f93b6222cce99be410ed2c8ab Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 20 May 2024 16:20:42 -0700 Subject: [PATCH 14/25] Tests simplified and passing. --- .../iceberg/layout/IcebergBaseLayout.java | 141 ++++++++++ .../iceberg/layout/IcebergFlatLayout.java | 106 +------- .../IcebergKeyValuePartitionedLayout.java | 139 ++-------- .../location/IcebergTableLocationKey.java | 5 - .../IcebergTableParquetLocationKey.java | 4 +- .../iceberg/util/IcebergCatalogAdapter.java | 9 +- .../IcebergToolsTest.java | 232 +++++++++-------- ...-3eae-445b-a5d5-f6b17061a5db-00001.parquet | 3 - ...-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet | 3 - ...-9822-404b-96da-8730bf21f89c-00001.parquet | 3 - ...-8358-43ed-8788-0e433efbc7b6-00001.parquet | 3 - ...-71da-4852-849e-a6d5a98e9ae9.metadata.json | 160 ------------ ...-3f79-4b9e-94c8-5b46b2da591a.metadata.json | 187 -------------- ...-57c8-4ea8-8b67-57bb93c1e120.metadata.json | 214 ---------------- ...-219d-4868-acb1-098fc5282d67.metadata.json | 241 ------------------ ...418a9b-0fe5-4a50-978d-6a3c49279958-m0.avro | Bin 8177 -> 0 bytes ...aae55f-bf38-4506-b506-5832608c1dc1-m0.avro | Bin 8158 -> 0 bytes ...3f7748-dbb3-45a6-ae71-77c1b8882e8d-m0.avro | Bin 8177 -> 0 bytes ...56d517-bd9b-4e53-ac6e-119ccd017ef3-m0.avro | Bin 8157 -> 0 bytes ...-b03f7748-dbb3-45a6-ae71-77c1b8882e8d.avro | Bin 4331 -> 0 bytes ...-77aae55f-bf38-4506-b506-5832608c1dc1.avro | Bin 4432 -> 0 bytes ...-1a418a9b-0fe5-4a50-978d-6a3c49279958.avro | Bin 4384 -> 0 bytes ...-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro | Bin 4252 -> 0 bytes ...-2e39-4ac3-9854-3c92a5eae25f-00004.parquet | 3 - ...-2e39-4ac3-9854-3c92a5eae25f-00005.parquet | 3 - ...-2e39-4ac3-9854-3c92a5eae25f-00002.parquet | 3 - ...-2e39-4ac3-9854-3c92a5eae25f-00006.parquet | 3 - ...-2e39-4ac3-9854-3c92a5eae25f-00003.parquet | 3 - ...-2e39-4ac3-9854-3c92a5eae25f-00008.parquet | 3 - ...-2e39-4ac3-9854-3c92a5eae25f-00001.parquet | 3 - ...-2e39-4ac3-9854-3c92a5eae25f-00007.parquet | 3 - ...-27f3-4b56-a5dc-e1ee0e1df858.metadata.json | 180 ------------- ...460130-5a60-48ae-b1e0-6aa9fa2b98b5-m0.avro | 10 - ...-69460130-5a60-48ae-b1e0-6aa9fa2b98b5.avro | Bin 4287 -> 0 bytes ...-fc45-47ac-9f75-74ded525e6d6-00001.parquet | 3 - ...-c052-41d4-9e2d-63e29407b88a.metadata.json | 160 ------------ ...1ea117-c6b3-4853-b4f3-09aa36b77f19-m0.avro | Bin 8189 -> 0 bytes ...-651ea117-c6b3-4853-b4f3-09aa36b77f19.avro | Bin 4260 -> 0 bytes ...e7d-4830-aedb-14dab6fabdd6-0-00001.parquet | 3 + ...f10-4e08-89ae-f1b1b578ce63-0-00001.parquet | 3 + ...256-4a04-9248-d7c69bec4881-0-00001.parquet | 3 + ...3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet | 3 + ...-a414-468b-b7b2-78558b2e8c1f.metadata.json | 91 +++++++ ...-9c8a-4fa9-911d-e356b2b04061.metadata.json | 118 +++++++++ ...-a010-4afe-8461-f5261787aae9.metadata.json | 145 +++++++++++ ...-48c8-4f6d-bd6b-9b452de74dc3.metadata.json | 172 +++++++++++++ ...f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro | Bin 0 -> 6976 bytes ...be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro | Bin 0 -> 6978 bytes ...bfad1c-123c-452c-814d-298a1483a99f-m0.avro | Bin 0 -> 6978 bytes ...1cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro | Bin 0 -> 6978 bytes ...-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro | Bin 0 -> 4399 bytes ...-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro | Bin 0 -> 4224 bytes ...-77bfad1c-123c-452c-814d-298a1483a99f.avro | Bin 0 -> 4353 bytes ...-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro | Bin 0 -> 4302 bytes ...66f-4474-a2a5-ad5e580e8f54-0-00001.parquet | 3 + ...66f-4474-a2a5-ad5e580e8f54-0-00001.parquet | 3 + ...-14fb-4a05-b8af-2c9f5986dfe6.metadata.json | 111 ++++++++ ...305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro | Bin 0 -> 7627 bytes ...-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro | Bin 0 -> 4251 bytes ...7ce-4968-b83d-30b13e1e7151-0-00001.parquet | 3 + ...-ae4e-42dd-ab04-884f6fd52372.metadata.json | 91 +++++++ ...95af41-65d8-41f0-8674-13624d6129c6-m0.avro | Bin 0 -> 6982 bytes ...-3395af41-65d8-41f0-8674-13624d6129c6.avro | Bin 0 -> 4227 bytes .../parquet/table/ParquetInstructions.java | 56 +--- 64 files changed, 1054 insertions(+), 1578 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00002-4caf996d-57c8-4ea8-8b67-57bb93c1e120.metadata.json delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00003-b7433d10-219d-4868-acb1-098fc5282d67.metadata.json delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/1a418a9b-0fe5-4a50-978d-6a3c49279958-m0.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/77aae55f-bf38-4506-b506-5832608c1dc1-m0.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/b03f7748-dbb3-45a6-ae71-77c1b8882e8d-m0.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/ee56d517-bd9b-4e53-ac6e-119ccd017ef3-m0.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/snap-4768271945146524109-1-b03f7748-dbb3-45a6-ae71-77c1b8882e8d.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/snap-615105126920399770-1-77aae55f-bf38-4506-b506-5832608c1dc1.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/snap-7258036030029852722-1-1a418a9b-0fe5-4a50-978d-6a3c49279958.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/snap-7480254251893511044-1-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/00000-e5584cce-27f3-4b56-a5dc-e1ee0e1df858.metadata.json delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/69460130-5a60-48ae-b1e0-6aa9fa2b98b5-m0.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/snap-2327977723212680118-1-69460130-5a60-48ae-b1e0-6aa9fa2b98b5.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/00000-a58846ae-c052-41d4-9e2d-63e29407b88a.metadata.json delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/651ea117-c6b3-4853-b4f3-09aa36b77f19-m0.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/snap-6884618194677467995-1-651ea117-c6b3-4853-b4f3-09aa36b77f19.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro 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 new file mode 100644 index 00000000000..e97e8938873 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -0,0 +1,141 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.layout; + +import io.deephaven.base.FileUtils; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.locations.TableDataException; +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.util.IcebergInstructions; +import io.deephaven.parquet.table.ParquetInstructions; +import org.apache.iceberg.*; +import org.apache.iceberg.io.FileIO; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.net.URI; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; + +public abstract class IcebergBaseLayout implements TableLocationKeyFinder { + /** + * The {@link TableDefinition} that will be used for the table. + */ + final TableDefinition tableDef; + + /** + * The Iceberg {@link Table} to discover locations for. + */ + final Table table; + + /** + * The {@link Snapshot} to discover locations for. + */ + final Snapshot snapshot; + + /** + * The {@link FileIO} to use for passing to the catalog reading manifest data files. + */ + final FileIO fileIO; + + /** + * The instructions for customizations while reading. + */ + final IcebergInstructions instructions; + + /** + * A cache of {@link IcebergTableLocationKey IcebergTableLocationKeys} keyed by the URI of the file they represent. + */ + final Map cache; + + /** + * The {@link ParquetInstructions} object that will be used to read any Parquet data files in this table. + */ + ParquetInstructions parquetInstructions; + + protected IcebergTableLocationKey locationKey( + final org.apache.iceberg.FileFormat format, + final URI fileUri, + @Nullable final Map> partitions) { + + if (format == org.apache.iceberg.FileFormat.PARQUET) { + if (parquetInstructions == null) { + // Start with user-supplied instructions (if provided). + final ParquetInstructions.Builder builder = instructions.parquetInstructions().isPresent() + ? new ParquetInstructions.Builder(instructions.parquetInstructions().get()) + : new ParquetInstructions.Builder(); + + if (instructions.columnRenameMap() != null) { + for (Map.Entry entry : instructions.columnRenameMap().entrySet()) { + builder.addColumnNameMapping(entry.getKey(), entry.getValue()); + } + } + if (instructions.s3Instructions().isPresent()) { + builder.setSpecialInstructions(instructions.s3Instructions().get()); + } + parquetInstructions = builder.build(); + } + return new IcebergTableParquetLocationKey(fileUri, 0, partitions, parquetInstructions); + } + throw new UnsupportedOperationException(String.format("%s:%d - an unsupported file format %s for URI '%s'", + table, snapshot.snapshotId(), format, fileUri)); + } + + /** + * @param tableDef The {@link TableDefinition} that will be used for the table. + * @param table The {@link Table} to discover locations for. + * @param tableSnapshot The {@link Snapshot} from which to discover data files. + * @param fileIO The file IO to use for reading manifest data files. + * @param instructions The instructions for customizations while reading. + */ + public IcebergBaseLayout( + @NotNull final TableDefinition tableDef, + @NotNull final Table table, + @NotNull final Snapshot tableSnapshot, + @NotNull final FileIO fileIO, + @NotNull final IcebergInstructions instructions) { + this.tableDef = tableDef; + this.table = table; + this.snapshot = tableSnapshot; + this.fileIO = fileIO; + this.instructions = instructions; + + this.cache = new HashMap<>(); + } + + abstract IcebergTableLocationKey keyFromDataFile(DataFile df, URI fileUri); + + @Override + public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { + try { + // Retrieve the manifest files from the snapshot + final List manifestFiles = snapshot.allManifests(fileIO); + for (final ManifestFile manifestFile : manifestFiles) { + // Currently only can process manifest files with DATA content type. + if (manifestFile.content() != ManifestContent.DATA) { + throw new TableDataException( + String.format("%s:%d - only DATA manifest files are currently supported, encountered %s", + table, snapshot.snapshotId(), manifestFile.content())); + } + try (final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO)) { + for (DataFile df : reader) { + final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); + final IcebergTableLocationKey locationKey = + cache.computeIfAbsent(fileUri, uri -> keyFromDataFile(df, fileUri)); + if (locationKey != null) { + locationKeyObserver.accept(locationKey); + } + } + } + } + } catch (final Exception e) { + throw new TableDataException( + String.format("%s:%d - error finding Iceberg locations", table, snapshot.snapshotId()), e); + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java index 7edbe161f29..ac4c19283f9 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -3,134 +3,44 @@ // package io.deephaven.iceberg.layout; -import io.deephaven.base.FileUtils; -import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.TableDefinition; 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.util.IcebergInstructions; -import io.deephaven.parquet.table.ParquetInstructions; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; import org.jetbrains.annotations.NotNull; import java.net.URI; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.function.Consumer; /** * Iceberg {@link TableLocationKeyFinder location finder} for tables without partitions that will discover data files * from a {@link Snapshot} */ -public final class IcebergFlatLayout implements TableLocationKeyFinder { - /** - * The Iceberg {@link Table} to discover locations for. - */ - private final Table table; - - /** - * The {@link FileIO} to use for passing to the catalog reading manifest data files. - */ - private final FileIO fileIO; - - /** - * A cache of {@link IcebergTableLocationKey IcebergTableLocationKeys} keyed by the URI of the file they represent. - */ - private final Map cache; - - /** - * The instructions for customizations while reading. - */ - private final IcebergInstructions instructions; - - /** - * The {@link ParquetInstructions} object that will be used to read any Parquet data files in this table. - */ - private ParquetInstructions parquetInstructions; - - /** - * The {@link Snapshot} to discover locations for. - */ - private final Snapshot snapshot; - - private IcebergTableLocationKey locationKey( - final FileFormat format, - final URI fileUri) { - if (format == org.apache.iceberg.FileFormat.PARQUET) { - if (parquetInstructions == null) { - // Start with user-supplied instructions (if provided). - parquetInstructions = instructions.parquetInstructions().isPresent() - ? instructions.parquetInstructions().get() - : ParquetInstructions.EMPTY; - - // Use the ParquetInstructions overrides to propagate the Iceberg instructions. - if (instructions.columnRenameMap() != null) { - parquetInstructions = parquetInstructions.withColumnRenameMap(instructions.columnRenameMap()); - } - if (instructions.s3Instructions().isPresent()) { - parquetInstructions = - parquetInstructions.withSpecialInstructions(instructions.s3Instructions().get()); - } - } - return new IcebergTableParquetLocationKey(fileUri, 0, null, parquetInstructions); - } - throw new UnsupportedOperationException(String.format("%s:%d - an unsupported file format %s for URI '%s'", - table, snapshot.snapshotId(), format, fileUri)); - } - +public final class IcebergFlatLayout extends IcebergBaseLayout { /** + * @param tableDef The {@link TableDefinition} that will be used for the table. * @param table The {@link Table} to discover locations for. * @param tableSnapshot The {@link Snapshot} from which to discover data files. * @param fileIO The file IO to use for reading manifest data files. * @param instructions The instructions for customizations while reading. */ public IcebergFlatLayout( + @NotNull final TableDefinition tableDef, @NotNull final Table table, @NotNull final Snapshot tableSnapshot, @NotNull final FileIO fileIO, @NotNull final IcebergInstructions instructions) { - this.table = table; - this.snapshot = tableSnapshot; - this.fileIO = fileIO; - this.instructions = instructions; - - this.cache = new HashMap<>(); + super(tableDef, table, tableSnapshot, fileIO, instructions); } + @Override public String toString() { return IcebergFlatLayout.class.getSimpleName() + '[' + table.name() + ']'; } @Override - public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { - try { - // Retrieve the manifest files from the snapshot - final List manifestFiles = snapshot.allManifests(fileIO); - for (final ManifestFile manifestFile : manifestFiles) { - // Currently only can process manifest files with DATA content type. - if (manifestFile.content() != ManifestContent.DATA) { - throw new TableDataException( - String.format("%s:%d - only DATA manifest files are currently supported, encountered %s", - table, snapshot.snapshotId(), manifestFile.content())); - } - final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO); - for (DataFile df : reader) { - final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); - final IcebergTableLocationKey locationKey = cache.computeIfAbsent(fileUri, uri -> { - final IcebergTableLocationKey key = locationKey(df.format(), fileUri); - // Verify before caching. - return key.verifyFileReader() ? key : null; - }); - if (locationKey != null) { - locationKeyObserver.accept(locationKey); - } - } - } - } catch (final Exception e) { - throw new TableDataException( - String.format("%s:%d - error finding Iceberg locations", table, snapshot.snapshotId()), e); - } + IcebergTableLocationKey keyFromDataFile(DataFile df, URI fileUri) { + return locationKey(df.format(), fileUri, null); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java index 79a075ee089..2483fbe5837 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java @@ -3,15 +3,12 @@ // package io.deephaven.iceberg.layout; -import io.deephaven.base.FileUtils; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.TableDataException; 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.util.IcebergInstructions; -import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.util.type.TypeUtils; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; @@ -19,77 +16,19 @@ import java.net.URI; import java.util.*; -import java.util.function.Consumer; /** * Iceberg {@link TableLocationKeyFinder location finder} for tables with partitions that will discover data files from * a {@link Snapshot} */ -public final class IcebergKeyValuePartitionedLayout implements TableLocationKeyFinder { - /** - * The {@link TableDefinition} that will be used for the table. - */ - final TableDefinition tableDef; - /** - * The Iceberg {@link org.apache.iceberg.Table} to discover locations for. - */ - private final Table table; - - /** - * The {@link FileIO} to use for passing to the catalog reading manifest data files. - */ - private final FileIO fileIO; - - /** - * A cache of {@link IcebergTableLocationKey IcebergTableLocationKeys} keyed by the URI of the file they represent. - */ - private final Map cache; - - /** - * The instructions for customizations while reading. - */ - private final IcebergInstructions instructions; +public final class IcebergKeyValuePartitionedLayout extends IcebergBaseLayout { + private final String[] partitionColumns; + private final Class[] partitionColumnTypes; /** - * The {@link ParquetInstructions} object that will be used to read any Parquet data files in this table. - */ - private ParquetInstructions parquetInstructions; - - /** - * The {@link Snapshot} to discover locations for. - */ - private final Snapshot snapshot; - - private IcebergTableLocationKey locationKey( - final org.apache.iceberg.FileFormat format, - final URI fileUri, - final Map> partitions) { - - if (format == org.apache.iceberg.FileFormat.PARQUET) { - if (parquetInstructions == null) { - // Start with user-supplied instructions (if provided). - parquetInstructions = instructions.parquetInstructions().isPresent() - ? instructions.parquetInstructions().get() - : ParquetInstructions.EMPTY; - - // Use the ParquetInstructions overrides to propagate the Iceberg instructions. - if (instructions.columnRenameMap() != null) { - parquetInstructions = parquetInstructions.withColumnRenameMap(instructions.columnRenameMap()); - } - if (instructions.s3Instructions().isPresent()) { - parquetInstructions = - parquetInstructions.withSpecialInstructions(instructions.s3Instructions().get()); - } - } - return new IcebergTableParquetLocationKey(fileUri, 0, partitions, parquetInstructions); - } - throw new UnsupportedOperationException(String.format("%s:%d - an unsupported file format %s for URI '%s'", - table, snapshot.snapshotId(), format, fileUri)); - } - - /** - * @param table The {@link org.apache.iceberg.Table} to discover locations for. - * @param tableSnapshot The {@link org.apache.iceberg.Snapshot} from which to discover data files. + * @param tableDef The {@link TableDefinition} that will be used for the table. + * @param table The {@link Table} to discover locations for. + * @param tableSnapshot The {@link Snapshot} from which to discover data files. * @param fileIO The file IO to use for reading manifest data files. * @param instructions The instructions for customizations while reading. */ @@ -99,66 +38,34 @@ public IcebergKeyValuePartitionedLayout( @NotNull final org.apache.iceberg.Snapshot tableSnapshot, @NotNull final FileIO fileIO, @NotNull final IcebergInstructions instructions) { - this.tableDef = tableDef; - this.table = table; - this.snapshot = tableSnapshot; - this.fileIO = fileIO; - this.instructions = instructions; + super(tableDef, table, tableSnapshot, fileIO, instructions); - this.cache = new HashMap<>(); + partitionColumns = + tableDef.getPartitioningColumns().stream().map(ColumnDefinition::getName).toArray(String[]::new); + partitionColumnTypes = Arrays.stream(partitionColumns) + .map(colName -> TypeUtils.getBoxedType(tableDef.getColumn(colName).getDataType())) + .toArray(Class[]::new); } + @Override public String toString() { return IcebergKeyValuePartitionedLayout.class.getSimpleName() + '[' + table.name() + ']'; } @Override - public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { + IcebergTableLocationKey keyFromDataFile(DataFile df, URI fileUri) { final Map> partitions = new LinkedHashMap<>(); - try { - final String[] partitionColumns = - tableDef.getPartitioningColumns().stream().map(ColumnDefinition::getName).toArray(String[]::new); - final Class[] partitionColumnTypes = Arrays.stream(partitionColumns) - .map(colName -> TypeUtils.getBoxedType(tableDef.getColumn(colName).getDataType())) - .toArray(Class[]::new); - // Retrieve the manifest files from the snapshot - final List manifestFiles = snapshot.allManifests(fileIO); - for (final ManifestFile manifestFile : manifestFiles) { - // Currently only can process manifest files with DATA content type. - if (manifestFile.content() != ManifestContent.DATA) { - throw new TableDataException( - String.format("%s:%d - only DATA manifest files are currently supported, encountered %s", - table, snapshot.snapshotId(), manifestFile.content())); - } - try (final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO)) { - for (DataFile df : reader) { - final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); - final IcebergTableLocationKey locationKey = cache.computeIfAbsent(fileUri, uri -> { - final PartitionData partitionData = (PartitionData) df.partition(); - for (int ii = 0; ii < partitionColumns.length; ++ii) { - final Object value = partitionData.get(ii); - if (value != null && !value.getClass().isAssignableFrom(partitionColumnTypes[ii])) { - throw new TableDataException("Partitioning column " + partitionColumns[ii] - + " has type " + value.getClass().getName() - + " but expected " + partitionColumnTypes[ii].getName()); - } - partitions.put(partitionColumns[ii], (Comparable) value); - } - final IcebergTableLocationKey key = - locationKey(df.format(), fileUri, partitions); - // Verify before caching. - return key.verifyFileReader() ? key : null; - }); - if (locationKey != null) { - locationKeyObserver.accept(locationKey); - } - } - } + final PartitionData partitionData = (PartitionData) df.partition(); + for (int ii = 0; ii < partitionColumns.length; ++ii) { + final Object value = partitionData.get(ii); + if (value != null && !value.getClass().isAssignableFrom(partitionColumnTypes[ii])) { + throw new TableDataException("Partitioning column " + partitionColumns[ii] + + " has type " + value.getClass().getName() + + " but expected " + partitionColumnTypes[ii].getName()); } - } catch (final Exception e) { - throw new TableDataException( - String.format("%s:%d - error finding Iceberg locations", table, snapshot.snapshotId()), e); + partitions.put(partitionColumns[ii], (Comparable) value); } + return locationKey(df.format(), fileUri, partitions); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java index cb28907465d..d6d5d4fb514 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java @@ -15,9 +15,4 @@ public interface IcebergTableLocationKey extends TableLocationKey { * @return the read instructions */ Object readInstructions(); - - /** - * Verify that a reader for the file can be created successfully. - */ - boolean verifyFileReader(); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java index e2582d77f09..6240a1113c2 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java @@ -16,9 +16,10 @@ * {@link TableLocationKey} implementation for use with data stored in Iceberg tables in the parquet format. */ public class IcebergTableParquetLocationKey extends ParquetTableLocationKey implements IcebergTableLocationKey { - private static final String IMPLEMENTATION_NAME = IcebergTableParquetLocationKey.class.getSimpleName(); + private final ParquetInstructions readInstructions; + /** * Construct a new IcebergTableParquetLocationKey for the supplied {@code fileUri} and {@code partitions}. * @@ -35,6 +36,7 @@ public IcebergTableParquetLocationKey( @Nullable final Map> partitions, @NotNull final ParquetInstructions readInstructions) { super(fileUri, order, partitions, readInstructions); + this.readInstructions = readInstructions; } @Override 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 ec0ba030a1f..3a5882d0a91 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 @@ -213,7 +213,7 @@ private Table readTableInternal( if (partitionSpec.isUnpartitioned()) { // Create the flat layout location key finder - keyFinder = new IcebergFlatLayout(table, snapshot, fileIO, instructions); + keyFinder = new IcebergFlatLayout(tableDef, table, snapshot, fileIO, instructions); } else { final String[] partitionColumns = partitionSpec.fields().stream() .map(PartitionField::name) @@ -231,12 +231,7 @@ private Table readTableInternal( } // Create the partitioning column location key finder - keyFinder = new IcebergKeyValuePartitionedLayout( - tableDef, - table, - snapshot, - fileIO, - instructions); + keyFinder = new IcebergKeyValuePartitionedLayout(tableDef, table, snapshot, fileIO, instructions); } if (isRefreshing) { diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index 4741c27622f..f683fbf3a50 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -27,7 +27,7 @@ import software.amazon.awssdk.services.s3.model.*; import java.io.File; -import java.time.LocalDateTime; +import java.time.Instant; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -57,8 +57,7 @@ public abstract class IcebergToolsTest { void setUp() throws ExecutionException, InterruptedException { bucket = "warehouse"; asyncClient = s3AsyncClient(); - final CompletableFuture bucketCreated = - asyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()); + asyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()).get(); warehousePath = IcebergToolsTest.class.getResource("/warehouse").getPath(); resourceFileIO = new IcebergTestFileIO("s3://warehouse", warehousePath); @@ -78,8 +77,6 @@ void setUp() throws ExecutionException, InterruptedException { instructions = IcebergInstructions.builder() .parquetInstructions(parquetInstructions) .build(); - - bucketCreated.get(); } private void uploadParquetFiles(final File root, final String prefixToRemove) @@ -89,42 +86,43 @@ private void uploadParquetFiles(final File root, final String prefixToRemove) uploadParquetFiles(file, prefixToRemove); } else if (file.getName().endsWith(".parquet")) { final String key = file.getPath().substring(prefixToRemove.length() + 1); + keys.add(key); - putObject(key, AsyncRequestBody.fromFile(file)); + final CompletableFuture future = asyncClient.putObject( + PutObjectRequest.builder().bucket(bucket).key(key).build(), + AsyncRequestBody.fromFile(file)); + + final PutObjectResponse response = future.get(10, TimeUnit.SECONDS); + if (!response.sdkHttpResponse().isSuccessful()) { + Assert.statementNeverExecuted("Failed to upload file: " + file.getPath()); + } } } } @AfterEach - public void tearDown() { + public void tearDown() throws ExecutionException, InterruptedException { for (String key : keys) { - asyncClient.deleteObject(DeleteObjectRequest.builder().bucket(bucket).key(key).build()); + asyncClient.deleteObject(DeleteObjectRequest.builder().bucket(bucket).key(key).build()).get(); } keys.clear(); - asyncClient.deleteBucket(DeleteBucketRequest.builder().bucket(bucket).build()); + asyncClient.deleteBucket(DeleteBucketRequest.builder().bucket(bucket).build()).get(); asyncClient.close(); } - @Test - public void testNothing() { - // Dummy to prevent JUnit from complaining about no tests - } - - // TODO: discuss how to perform tests since they require a full MiniIO + Iceberg setup - @Test public void testListTables() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); - final Namespace ns = Namespace.of("nyc"); + final Namespace ns = Namespace.of("sales"); final Collection tables = adapter.listTables(ns); Assert.eq(tables.size(), "tables.size()", 3, "3 tables in the namespace"); - Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "taxis")), "tables.contains(nyc/taxis)"); - Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "taxis_partitioned")), - "tables.contains(nyc/taxis_partitioned)"); - Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "taxis_single")), "tables.contains(nyc/taxis_single)"); + Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_multi")), "tables.contains(sales_multi)"); + Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_partitioned")), + "tables.contains(sales_partitioned)"); + Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_single")), "tables.contains(sales_single)"); } @Test @@ -132,100 +130,91 @@ public void testListTableSnapshots() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); - final Collection snapshots = adapter.listTableSnapshots(TableIdentifier.of("nyc", "taxis")); + final Collection snapshots = adapter.listTableSnapshots(TableIdentifier.of("sales", "sales_multi")); - Assert.eq(snapshots.size(), "snapshots.size()", 4, "4 snapshots for nyc/taxis"); + Assert.eq(snapshots.size(), "snapshots.size()", 4, "4 snapshots for sales/sales_multi"); - Assert.eqTrue(snapshots.contains(7480254251893511044L), "snapshots.contains(7480254251893511044L)"); - Assert.eqTrue(snapshots.contains(4768271945146524109L), "snapshots.contains(4768271945146524109L)"); - Assert.eqTrue(snapshots.contains(7258036030029852722L), "snapshots.contains(7258036030029852722L)"); - Assert.eqTrue(snapshots.contains(615105126920399770L), "snapshots.contains(615105126920399770L)"); + Assert.eqTrue(snapshots.contains(2001582482032951248L), "snapshots.contains(2001582482032951248)"); + Assert.eqTrue(snapshots.contains(8325605756612719366L), "snapshots.contains(8325605756612719366L)"); + Assert.eqTrue(snapshots.contains(3247344357341484163L), "snapshots.contains(3247344357341484163L)"); + Assert.eqTrue(snapshots.contains(1792185872197984875L), "snapshots.contains(1792185872197984875L)"); } @Test public void testOpenTableA() throws ExecutionException, InterruptedException, TimeoutException { - uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis_partitioned").getPath()), + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), warehousePath); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @Test public void testOpenTableB() throws ExecutionException, InterruptedException, TimeoutException { - uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis").getPath()), + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_multi").getPath()), warehousePath); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @Test public void testOpenTableC() throws ExecutionException, InterruptedException, TimeoutException { - uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis_single").getPath()), + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_single").getPath()), warehousePath); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_single"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); - - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); - } - - @Test - public void testOpenTableTestCatalog() throws ExecutionException, InterruptedException, TimeoutException { - uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis_single").getPath()), - warehousePath); - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructions); - - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_single"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_single"); final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @Test public void testOpenTableS3Only() throws ExecutionException, InterruptedException, TimeoutException { - uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis_partitioned").getPath()), + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), warehousePath); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructions); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @Test public void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { - uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/nyc/taxis_partitioned").getPath()), + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), warehousePath); final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("year").withPartitioning(), ColumnDefinition.ofInt("month").withPartitioning(), - ColumnDefinition.ofLong("VendorID"), - ColumnDefinition.fromGenericType("tpep_pickup_datetime", LocalDateTime.class), - ColumnDefinition.fromGenericType("tpep_dropoff_datetime", LocalDateTime.class), - ColumnDefinition.ofDouble("passenger_count")); + ColumnDefinition.ofString("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofInt("Units_Sold"), + ColumnDefinition.ofDouble("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) @@ -235,11 +224,12 @@ public void testOpenTableDefinition() throws ExecutionException, InterruptedExce final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @Test @@ -247,10 +237,11 @@ public void testOpenTablePartitionTypeException() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofLong("year").withPartitioning(), ColumnDefinition.ofInt("month").withPartitioning(), - ColumnDefinition.ofLong("VendorID"), - ColumnDefinition.fromGenericType("tpep_pickup_datetime", LocalDateTime.class), - ColumnDefinition.fromGenericType("tpep_dropoff_datetime", LocalDateTime.class), - ColumnDefinition.ofDouble("passenger_count")); + ColumnDefinition.ofLong("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofDouble("Units_Sold"), + ColumnDefinition.ofLong("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) @@ -260,8 +251,8 @@ public void testOpenTablePartitionTypeException() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); try { final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); @@ -272,21 +263,27 @@ public void testOpenTablePartitionTypeException() { } @Test - public void testOpenTableDefinitionRename() { + public void testOpenTableDefinitionRename() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("__year").withPartitioning(), ColumnDefinition.ofInt("__month").withPartitioning(), - ColumnDefinition.ofLong("vendor_id"), - ColumnDefinition.fromGenericType("pickup_datetime", LocalDateTime.class), - ColumnDefinition.fromGenericType("dropoff_datetime", LocalDateTime.class), - ColumnDefinition.ofDouble("passenger_count")); + ColumnDefinition.ofString("RegionName"), + ColumnDefinition.ofString("ItemType"), + ColumnDefinition.ofInt("UnitsSold"), + ColumnDefinition.ofDouble("UnitPrice"), + ColumnDefinition.fromGenericType("OrderDate", Instant.class)); final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .s3Instructions(instructionsS3Only.s3Instructions().get()) - .putColumnRenameMap("VendorID", "vendor_id") - .putColumnRenameMap("tpep_pickup_datetime", "pickup_datetime") - .putColumnRenameMap("tpep_dropoff_datetime", "dropoff_datetime") + .putColumnRenameMap("Region", "RegionName") + .putColumnRenameMap("Item_Type", "ItemType") + .putColumnRenameMap("Units_Sold", "UnitsSold") + .putColumnRenameMap("Unit_Price", "UnitPrice") + .putColumnRenameMap("Order_Date", "OrderDate") .putColumnRenameMap("year", "__year") .putColumnRenameMap("month", "__month") .build(); @@ -294,11 +291,12 @@ public void testOpenTableDefinitionRename() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @Test @@ -306,10 +304,11 @@ public void testMissingPartitioningColumns() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("__year").withPartitioning(), // Incorrect name ColumnDefinition.ofInt("__month").withPartitioning(), // Incorrect name - ColumnDefinition.ofLong("VendorID"), - ColumnDefinition.fromGenericType("pickup_datetime", LocalDateTime.class), - ColumnDefinition.fromGenericType("dropoff_datetime", LocalDateTime.class), - ColumnDefinition.ofDouble("passenger_count")); + ColumnDefinition.ofLong("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofDouble("Units_Sold"), + ColumnDefinition.ofLong("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) @@ -319,8 +318,8 @@ public void testMissingPartitioningColumns() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); try { final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); Assert.statementNeverExecuted("Expected an exception for missing columns"); @@ -330,26 +329,33 @@ public void testMissingPartitioningColumns() { } @Test - public void testOpenTableColumnRename() { + public void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + final IcebergInstructions localInstructions = IcebergInstructions.builder() .s3Instructions(instructionsS3Only.s3Instructions().get()) - .putColumnRenameMap("VendorID", "vendor_id") - .putColumnRenameMap("tpep_pickup_datetime", "pickup_datetime") - .putColumnRenameMap("tpep_dropoff_datetime", "dropoff_datetime") + .putColumnRenameMap("RegionName", "Region") + .putColumnRenameMap("ItemType", "Item_Type") .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @Test - public void testOpenTableColumnRenamePartitioningColumns() { + public void testOpenTableColumnRenamePartitioningColumns() + throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + final IcebergInstructions localInstructions = IcebergInstructions.builder() .s3Instructions(instructionsS3Only.s3Instructions().get()) .putColumnRenameMap("VendorID", "vendor_id") @@ -357,31 +363,39 @@ public void testOpenTableColumnRenamePartitioningColumns() { .putColumnRenameMap("year", "__year") .build(); - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructions); + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @Test - public void testOpenTableSnapshot() { + public void testOpenTableSnapshot() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_multi").getPath()), + warehousePath); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructions); - final Namespace ns = Namespace.of("nyc"); - final TableIdentifier tableId = TableIdentifier.of(ns, "taxis_partitioned"); + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); final List snapshots = adapter.listTableSnapshots(tableId); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, snapshots.get(0)); + // Verify we retrieved all the rows. + final io.deephaven.engine.table.Table table0 = adapter.snapshotTable(tableId, snapshots.get(0)); + Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); - TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); - } + final io.deephaven.engine.table.Table table1 = adapter.snapshotTable(tableId, snapshots.get(1)); + Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); - private void putObject(String key, AsyncRequestBody body) - throws ExecutionException, InterruptedException, TimeoutException { - asyncClient.putObject(PutObjectRequest.builder().bucket(bucket).key(key).build(), body).get(5, - TimeUnit.SECONDS); + final io.deephaven.engine.table.Table table2 = adapter.snapshotTable(tableId, snapshots.get(2)); + Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); + + final io.deephaven.engine.table.Table table3 = adapter.snapshotTable(tableId, snapshots.get(3)); + Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); } } diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet deleted file mode 100644 index f3efde68ddb..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-10-461b1549-3eae-445b-a5d5-f6b17061a5db-00001.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:3c23a0d50d6b769f91dbabf24ab0aa93d033168a3483bb87c93c04cb3a36ab40 -size 8731886 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet deleted file mode 100644 index cb49b26e167..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-16-1b8a6089-b6c8-4f59-ba21-68d76d4fbb5f-00001.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:69b2ed875585804c5147c393d8920d9db0051b7aa4f5fed69c84ce1e6d681fc6 -size 16886361 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet deleted file mode 100644 index fd9f4abad97..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-22-78d55c70-9822-404b-96da-8730bf21f89c-00001.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:3babbe04e9c88b74d1b3ac7c8d17216b81808c8430c6b703911768c9c973f23a -size 8750334 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet deleted file mode 100644 index 3de9fa109f4..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/data/00002-28-ee6f6bd7-8358-43ed-8788-0e433efbc7b6-00001.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:6ea6ae42f9a7607dac1390cdd9d98710fc12149dafce9be6250d6737b39f50ea -size 12830090 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json deleted file mode 100644 index 0697e58f49a..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json +++ /dev/null @@ -1,160 +0,0 @@ -{ - "format-version" : 2, - "table-uuid" : "74cb0e81-37ae-4ca9-9225-18b9a885869d", - "location" : "s3://warehouse/nyc/taxis", - "last-sequence-number" : 1, - "last-updated-ms" : 1714753341825, - "last-column-id" : 19, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "VendorID", - "required" : false, - "type" : "long" - }, { - "id" : 2, - "name" : "tpep_pickup_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 3, - "name" : "tpep_dropoff_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 4, - "name" : "passenger_count", - "required" : false, - "type" : "double" - }, { - "id" : 5, - "name" : "trip_distance", - "required" : false, - "type" : "double" - }, { - "id" : 6, - "name" : "RatecodeID", - "required" : false, - "type" : "double" - }, { - "id" : 7, - "name" : "store_and_fwd_flag", - "required" : false, - "type" : "string" - }, { - "id" : 8, - "name" : "PULocationID", - "required" : false, - "type" : "long" - }, { - "id" : 9, - "name" : "DOLocationID", - "required" : false, - "type" : "long" - }, { - "id" : 10, - "name" : "payment_type", - "required" : false, - "type" : "long" - }, { - "id" : 11, - "name" : "fare_amount", - "required" : false, - "type" : "double" - }, { - "id" : 12, - "name" : "extra", - "required" : false, - "type" : "double" - }, { - "id" : 13, - "name" : "mta_tax", - "required" : false, - "type" : "double" - }, { - "id" : 14, - "name" : "tip_amount", - "required" : false, - "type" : "double" - }, { - "id" : 15, - "name" : "tolls_amount", - "required" : false, - "type" : "double" - }, { - "id" : 16, - "name" : "improvement_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 17, - "name" : "total_amount", - "required" : false, - "type" : "double" - }, { - "id" : 18, - "name" : "congestion_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 19, - "name" : "airport_fee", - "required" : false, - "type" : "double" - } ] - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ ] - } ], - "last-partition-id" : 999, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "root", - "created-at" : "2024-05-03T16:22:16.955845960Z", - "write.format.default" : "parquet", - "write.parquet.compression-codec" : "zstd" - }, - "current-snapshot-id" : 7480254251893511044, - "refs" : { - "main" : { - "snapshot-id" : 7480254251893511044, - "type" : "branch" - } - }, - "snapshots" : [ { - "sequence-number" : 1, - "snapshot-id" : 7480254251893511044, - "timestamp-ms" : 1714753341825, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "584935", - "added-files-size" : "8731886", - "changed-partition-count" : "1", - "total-records" : "584935", - "total-files-size" : "8731886", - "total-data-files" : "1", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7480254251893511044-1-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro", - "schema-id" : 0 - } ], - "statistics" : [ ], - "snapshot-log" : [ { - "timestamp-ms" : 1714753341825, - "snapshot-id" : 7480254251893511044 - } ], - "metadata-log" : [ ] -} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json deleted file mode 100644 index 5f7369890f3..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json +++ /dev/null @@ -1,187 +0,0 @@ -{ - "format-version" : 2, - "table-uuid" : "74cb0e81-37ae-4ca9-9225-18b9a885869d", - "location" : "s3://warehouse/nyc/taxis", - "last-sequence-number" : 2, - "last-updated-ms" : 1714753347072, - "last-column-id" : 19, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "VendorID", - "required" : false, - "type" : "long" - }, { - "id" : 2, - "name" : "tpep_pickup_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 3, - "name" : "tpep_dropoff_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 4, - "name" : "passenger_count", - "required" : false, - "type" : "double" - }, { - "id" : 5, - "name" : "trip_distance", - "required" : false, - "type" : "double" - }, { - "id" : 6, - "name" : "RatecodeID", - "required" : false, - "type" : "double" - }, { - "id" : 7, - "name" : "store_and_fwd_flag", - "required" : false, - "type" : "string" - }, { - "id" : 8, - "name" : "PULocationID", - "required" : false, - "type" : "long" - }, { - "id" : 9, - "name" : "DOLocationID", - "required" : false, - "type" : "long" - }, { - "id" : 10, - "name" : "payment_type", - "required" : false, - "type" : "long" - }, { - "id" : 11, - "name" : "fare_amount", - "required" : false, - "type" : "double" - }, { - "id" : 12, - "name" : "extra", - "required" : false, - "type" : "double" - }, { - "id" : 13, - "name" : "mta_tax", - "required" : false, - "type" : "double" - }, { - "id" : 14, - "name" : "tip_amount", - "required" : false, - "type" : "double" - }, { - "id" : 15, - "name" : "tolls_amount", - "required" : false, - "type" : "double" - }, { - "id" : 16, - "name" : "improvement_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 17, - "name" : "total_amount", - "required" : false, - "type" : "double" - }, { - "id" : 18, - "name" : "congestion_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 19, - "name" : "airport_fee", - "required" : false, - "type" : "double" - } ] - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ ] - } ], - "last-partition-id" : 999, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "root", - "created-at" : "2024-05-03T16:22:16.955845960Z", - "write.format.default" : "parquet", - "write.parquet.compression-codec" : "zstd" - }, - "current-snapshot-id" : 4768271945146524109, - "refs" : { - "main" : { - "snapshot-id" : 4768271945146524109, - "type" : "branch" - } - }, - "snapshots" : [ { - "sequence-number" : 1, - "snapshot-id" : 7480254251893511044, - "timestamp-ms" : 1714753341825, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "584935", - "added-files-size" : "8731886", - "changed-partition-count" : "1", - "total-records" : "584935", - "total-files-size" : "8731886", - "total-data-files" : "1", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7480254251893511044-1-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro", - "schema-id" : 0 - }, { - "sequence-number" : 2, - "snapshot-id" : 4768271945146524109, - "parent-snapshot-id" : 7480254251893511044, - "timestamp-ms" : 1714753347072, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "1167564", - "added-files-size" : "16886361", - "changed-partition-count" : "1", - "total-records" : "1752499", - "total-files-size" : "25618247", - "total-data-files" : "2", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-4768271945146524109-1-b03f7748-dbb3-45a6-ae71-77c1b8882e8d.avro", - "schema-id" : 0 - } ], - "statistics" : [ ], - "snapshot-log" : [ { - "timestamp-ms" : 1714753341825, - "snapshot-id" : 7480254251893511044 - }, { - "timestamp-ms" : 1714753347072, - "snapshot-id" : 4768271945146524109 - } ], - "metadata-log" : [ { - "timestamp-ms" : 1714753341825, - "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json" - } ] -} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00002-4caf996d-57c8-4ea8-8b67-57bb93c1e120.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00002-4caf996d-57c8-4ea8-8b67-57bb93c1e120.metadata.json deleted file mode 100644 index ba7d6d6148a..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00002-4caf996d-57c8-4ea8-8b67-57bb93c1e120.metadata.json +++ /dev/null @@ -1,214 +0,0 @@ -{ - "format-version" : 2, - "table-uuid" : "74cb0e81-37ae-4ca9-9225-18b9a885869d", - "location" : "s3://warehouse/nyc/taxis", - "last-sequence-number" : 3, - "last-updated-ms" : 1714753351429, - "last-column-id" : 19, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "VendorID", - "required" : false, - "type" : "long" - }, { - "id" : 2, - "name" : "tpep_pickup_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 3, - "name" : "tpep_dropoff_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 4, - "name" : "passenger_count", - "required" : false, - "type" : "double" - }, { - "id" : 5, - "name" : "trip_distance", - "required" : false, - "type" : "double" - }, { - "id" : 6, - "name" : "RatecodeID", - "required" : false, - "type" : "double" - }, { - "id" : 7, - "name" : "store_and_fwd_flag", - "required" : false, - "type" : "string" - }, { - "id" : 8, - "name" : "PULocationID", - "required" : false, - "type" : "long" - }, { - "id" : 9, - "name" : "DOLocationID", - "required" : false, - "type" : "long" - }, { - "id" : 10, - "name" : "payment_type", - "required" : false, - "type" : "long" - }, { - "id" : 11, - "name" : "fare_amount", - "required" : false, - "type" : "double" - }, { - "id" : 12, - "name" : "extra", - "required" : false, - "type" : "double" - }, { - "id" : 13, - "name" : "mta_tax", - "required" : false, - "type" : "double" - }, { - "id" : 14, - "name" : "tip_amount", - "required" : false, - "type" : "double" - }, { - "id" : 15, - "name" : "tolls_amount", - "required" : false, - "type" : "double" - }, { - "id" : 16, - "name" : "improvement_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 17, - "name" : "total_amount", - "required" : false, - "type" : "double" - }, { - "id" : 18, - "name" : "congestion_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 19, - "name" : "airport_fee", - "required" : false, - "type" : "double" - } ] - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ ] - } ], - "last-partition-id" : 999, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "root", - "created-at" : "2024-05-03T16:22:16.955845960Z", - "write.format.default" : "parquet", - "write.parquet.compression-codec" : "zstd" - }, - "current-snapshot-id" : 7258036030029852722, - "refs" : { - "main" : { - "snapshot-id" : 7258036030029852722, - "type" : "branch" - } - }, - "snapshots" : [ { - "sequence-number" : 1, - "snapshot-id" : 7480254251893511044, - "timestamp-ms" : 1714753341825, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "584935", - "added-files-size" : "8731886", - "changed-partition-count" : "1", - "total-records" : "584935", - "total-files-size" : "8731886", - "total-data-files" : "1", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7480254251893511044-1-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro", - "schema-id" : 0 - }, { - "sequence-number" : 2, - "snapshot-id" : 4768271945146524109, - "parent-snapshot-id" : 7480254251893511044, - "timestamp-ms" : 1714753347072, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "1167564", - "added-files-size" : "16886361", - "changed-partition-count" : "1", - "total-records" : "1752499", - "total-files-size" : "25618247", - "total-data-files" : "2", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-4768271945146524109-1-b03f7748-dbb3-45a6-ae71-77c1b8882e8d.avro", - "schema-id" : 0 - }, { - "sequence-number" : 3, - "snapshot-id" : 7258036030029852722, - "parent-snapshot-id" : 4768271945146524109, - "timestamp-ms" : 1714753351429, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "584865", - "added-files-size" : "8750334", - "changed-partition-count" : "1", - "total-records" : "2337364", - "total-files-size" : "34368581", - "total-data-files" : "3", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7258036030029852722-1-1a418a9b-0fe5-4a50-978d-6a3c49279958.avro", - "schema-id" : 0 - } ], - "statistics" : [ ], - "snapshot-log" : [ { - "timestamp-ms" : 1714753341825, - "snapshot-id" : 7480254251893511044 - }, { - "timestamp-ms" : 1714753347072, - "snapshot-id" : 4768271945146524109 - }, { - "timestamp-ms" : 1714753351429, - "snapshot-id" : 7258036030029852722 - } ], - "metadata-log" : [ { - "timestamp-ms" : 1714753341825, - "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json" - }, { - "timestamp-ms" : 1714753347072, - "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json" - } ] -} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00003-b7433d10-219d-4868-acb1-098fc5282d67.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00003-b7433d10-219d-4868-acb1-098fc5282d67.metadata.json deleted file mode 100644 index 23d878d28a3..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/00003-b7433d10-219d-4868-acb1-098fc5282d67.metadata.json +++ /dev/null @@ -1,241 +0,0 @@ -{ - "format-version" : 2, - "table-uuid" : "74cb0e81-37ae-4ca9-9225-18b9a885869d", - "location" : "s3://warehouse/nyc/taxis", - "last-sequence-number" : 4, - "last-updated-ms" : 1714753355960, - "last-column-id" : 19, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "VendorID", - "required" : false, - "type" : "long" - }, { - "id" : 2, - "name" : "tpep_pickup_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 3, - "name" : "tpep_dropoff_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 4, - "name" : "passenger_count", - "required" : false, - "type" : "double" - }, { - "id" : 5, - "name" : "trip_distance", - "required" : false, - "type" : "double" - }, { - "id" : 6, - "name" : "RatecodeID", - "required" : false, - "type" : "double" - }, { - "id" : 7, - "name" : "store_and_fwd_flag", - "required" : false, - "type" : "string" - }, { - "id" : 8, - "name" : "PULocationID", - "required" : false, - "type" : "long" - }, { - "id" : 9, - "name" : "DOLocationID", - "required" : false, - "type" : "long" - }, { - "id" : 10, - "name" : "payment_type", - "required" : false, - "type" : "long" - }, { - "id" : 11, - "name" : "fare_amount", - "required" : false, - "type" : "double" - }, { - "id" : 12, - "name" : "extra", - "required" : false, - "type" : "double" - }, { - "id" : 13, - "name" : "mta_tax", - "required" : false, - "type" : "double" - }, { - "id" : 14, - "name" : "tip_amount", - "required" : false, - "type" : "double" - }, { - "id" : 15, - "name" : "tolls_amount", - "required" : false, - "type" : "double" - }, { - "id" : 16, - "name" : "improvement_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 17, - "name" : "total_amount", - "required" : false, - "type" : "double" - }, { - "id" : 18, - "name" : "congestion_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 19, - "name" : "airport_fee", - "required" : false, - "type" : "double" - } ] - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ ] - } ], - "last-partition-id" : 999, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "root", - "created-at" : "2024-05-03T16:22:16.955845960Z", - "write.format.default" : "parquet", - "write.parquet.compression-codec" : "zstd" - }, - "current-snapshot-id" : 615105126920399770, - "refs" : { - "main" : { - "snapshot-id" : 615105126920399770, - "type" : "branch" - } - }, - "snapshots" : [ { - "sequence-number" : 1, - "snapshot-id" : 7480254251893511044, - "timestamp-ms" : 1714753341825, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "584935", - "added-files-size" : "8731886", - "changed-partition-count" : "1", - "total-records" : "584935", - "total-files-size" : "8731886", - "total-data-files" : "1", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7480254251893511044-1-ee56d517-bd9b-4e53-ac6e-119ccd017ef3.avro", - "schema-id" : 0 - }, { - "sequence-number" : 2, - "snapshot-id" : 4768271945146524109, - "parent-snapshot-id" : 7480254251893511044, - "timestamp-ms" : 1714753347072, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "1167564", - "added-files-size" : "16886361", - "changed-partition-count" : "1", - "total-records" : "1752499", - "total-files-size" : "25618247", - "total-data-files" : "2", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-4768271945146524109-1-b03f7748-dbb3-45a6-ae71-77c1b8882e8d.avro", - "schema-id" : 0 - }, { - "sequence-number" : 3, - "snapshot-id" : 7258036030029852722, - "parent-snapshot-id" : 4768271945146524109, - "timestamp-ms" : 1714753351429, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "584865", - "added-files-size" : "8750334", - "changed-partition-count" : "1", - "total-records" : "2337364", - "total-files-size" : "34368581", - "total-data-files" : "3", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-7258036030029852722-1-1a418a9b-0fe5-4a50-978d-6a3c49279958.avro", - "schema-id" : 0 - }, { - "sequence-number" : 4, - "snapshot-id" : 615105126920399770, - "parent-snapshot-id" : 7258036030029852722, - "timestamp-ms" : 1714753355960, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "877005", - "added-files-size" : "12830090", - "changed-partition-count" : "1", - "total-records" : "3214369", - "total-files-size" : "47198671", - "total-data-files" : "4", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis/metadata/snap-615105126920399770-1-77aae55f-bf38-4506-b506-5832608c1dc1.avro", - "schema-id" : 0 - } ], - "statistics" : [ ], - "snapshot-log" : [ { - "timestamp-ms" : 1714753341825, - "snapshot-id" : 7480254251893511044 - }, { - "timestamp-ms" : 1714753347072, - "snapshot-id" : 4768271945146524109 - }, { - "timestamp-ms" : 1714753351429, - "snapshot-id" : 7258036030029852722 - }, { - "timestamp-ms" : 1714753355960, - "snapshot-id" : 615105126920399770 - } ], - "metadata-log" : [ { - "timestamp-ms" : 1714753341825, - "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00000-4d3ab82d-71da-4852-849e-a6d5a98e9ae9.metadata.json" - }, { - "timestamp-ms" : 1714753347072, - "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00001-e8705414-3f79-4b9e-94c8-5b46b2da591a.metadata.json" - }, { - "timestamp-ms" : 1714753351429, - "metadata-file" : "s3://warehouse/nyc/taxis/metadata/00002-4caf996d-57c8-4ea8-8b67-57bb93c1e120.metadata.json" - } ] -} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/1a418a9b-0fe5-4a50-978d-6a3c49279958-m0.avro b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/1a418a9b-0fe5-4a50-978d-6a3c49279958-m0.avro deleted file mode 100644 index cdce7ce87964bcd186acb86a2aed368bf1ead38e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8177 zcmb_he{2)?6?d{g1A>foEF)FAetTLr0gh##^UI#n1wo^v8z~DZ4bX7;-uavyIs2}6 z=cIw5q()tA|7n=oXwn!G)7EKIHSK-x zeD^&&_F+4b0IBEi_xNmO9=+!XRKeE zr5F3zb#T$k|WG9 zzjc#!ZW^kna;hYZrBp744bhN5U~wtrXWhUF)e#lW6&2GIRf)$dlhxwLQn9X!$q`ZG z1SORuNEGucrlfdTbj)dWL>QU`E=gc*Qt)yqPe7X^pda86l$huVYu@d4=eBMrn#kc~ zjEhgeUm1^hI05mNT;b_(r0jX@yNZAfNlAKIkrUjAb9>jL<=+kg&$6si@nk|w8k|YW z;t|ME9-^d-V*)^;y!eGEHN~e44SRr=2YrY#F=1fNz)zKNmom=))|73ui*<@D>*Wie ztdkO|ri_aw@OnxUMzJH6$iW;P&7K5l}`I*2;DGmm5KvHcIzPsvU_6TIZk<#I(Y+;qmT4J8F ziYlpy_Sqy>^-;y(ptjl<4l>D}ABp>oKF)@x)vcW{j_pC-!Qpha{YOn}ZhfAgs zyr@+mE$osOmt>LCS1hu{B@!w<=7?O!w4xwuESy;mEO5gB*q!@~g=|a_;5pi9{iq`) zj*L+e5lvAP>?CUaDAcJcxqV7j*C-{eD$(Xdi*;*}d!*1(MKvZmuQtjCS z3h^-wu*)8LtiKmcNX97Y-iu5=5szv3L&2a66;p0p(Gu9@SDxfooQSuNyWw<`K~UsW0tyryr&8^-b&$oHhDn8x3^4%>l=D5mq>P0^ z?c8`M)M0yLbz9jAF(wijMM??WbfGzp+e?=dp^jrDoU$?^30NMm`o08)#)?!K zlf}ESO0c#(L{bbH3j84mnc$H8k@PXJsCZEdcI4Y+1$cF!iU3q>uo9>u`DRgEvy5{= z#RgXZmDxY~fk}d#ib0cwNa04%m!Uq~=Z&4)NOFragSv_UvLOOsPKrCPkvG>^cSX4? z*I2e908OM!;lM$w3op*Jkh_v5adOqc3U>fl;YbB+3olCoFSdmv6@o>cdXlfeD`cYP zlt(LrJ;@e!7Y|vPfXRmk%wgKI3(g8^8rw#A)?NX0Y+La}El`=+_KL7G{Y9p~o*=QU zE^AV%y1F2dY8I}fnz8Oek(Z1GKnhn@&P;zx0m;pFs3KfKG%)EBmNAL3jC0@Mb zE?$np?$*ZcEuph7!y>Dr%{8cv8uYN;unI~7nk-3&_t<>2Q133H#lto?jdd~@!B_)a zE;M%#Bb)&3g|pCP!dcMasiMW=7=1*^3$WQ7tH?@2r~~U{*l#pit!(qSsOivZ4(D*bmuNSer+fR$)A&HceX(y-~^;)~NiPRCXXHvB-l&a)^($3Dat#KqvPLWF16^jBa_JC26_n-!2rA+O3yHNlK6uYOj#I8xZ=vy~ zOwU-z+G3tTg00lEWy6T$MwqTV3(Os5dJzU974aerp9j=O)S=F_y#Yt^abl2V2(o-$!q%2aN?yR&x857S2o zuyZ%jMa*X$1FRC`*-cY5^zF%SoQvpjs-q15Zwty}Zi5QXM)m`g$?Ph{K0yWMM6^I> zP0gbm%xj{lu);wsXdMudT9?~jQ+YYYd3V{4oh`H}$#1Bs;2cAA~mcPjLr+^4iOfzj&UT89cTB{L3?gGp{PXn(l?}?uDiLt>?~NTUxs? zTitiXLjG7 z`o(vrYoBA_vY@mzS$mUpScRqc*f@oi|y z#|O|-75@;b_wf&+`YJwzW_|otG+V_-&^{mEg!WbOt!Qn-yP02Su4F!V>zQXyK7CKm zKdLVH(wow?=`HD+^yc)2bYr?YeQ$dGo+maw(=hk-jxE!Vd~mkm!!2Fs)=eM(YRzCx z-QfCLKm74M^MQ;y_3_s>o!sf$b}h4J?(k4`N5zw~y0sOtT>|Lx>9 zzE;=M^_w5m_3YUH#n0ZW9XNSjpBikM>grp()qwtbq4O0qykYGN|K7gt`+xKu4xEp^ zQvIjPTGz(w|GoIYEv@bC_Zt54p;FI{thx00^?$v9>Ti6yzV3-?^qqemiQIoC@a;`E zGw*KRiZa(~^~jH!m`wFVWbW{R+rHFf`la;qUH5HKBUigNZquGi|7-($|6<_hkKa4j c`sLc_?6)#Qw|x_rI)lG|v17XC3(HvYf3=&wKmY&$ diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/77aae55f-bf38-4506-b506-5832608c1dc1-m0.avro b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis/metadata/77aae55f-bf38-4506-b506-5832608c1dc1-m0.avro deleted file mode 100644 index 40c15577d19901412d6039b8de00d235fc48d965..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8158 zcmb_h4{#La6{qOXSTzWyl}d(g*VuvaQtx(iBzGN@e*vNr#1Pw&2|M5IewQrSyWQ^Y zB^)85QY1Q6m|&&0OqGsST3VogETtu_6k0ltg_$r6Q>I9#Ey5tBiA_UwMEkzo+x_+~ zcgx)+Gr){{-}nCf-tWEdz2}P9+E3kfiy?O41lm^B!*f%-% zrm0~~&}6YQr3n%;u_=SV!cxf3hKUlID~it+CF!cB#^dG5s&i#&$S`oS9qWRqrji7S zLVihVDyCr9oUkjx)Mao<25Xa|pG!Fc=DGqN0XRfe!oINP+^%Xsb0VO3N-VJ{h z)b8U1#9Q`+r@@u-aO*>=h)h{c`dX0{+=zR7@ru%Khk$1pR%s}mz)4fENLe@nS;|9{ zlyO7=NR$@85TzzwXX?lYv>fO|l!*xw2_{-s!d=Qd6IfHS(H_>Riei*5fU-_bXu8^k zE#QrmE_NWjy#)G(C<9H^L`vzVQ1XdTG9cT5d_p;vP#z*#*HqmU;A2UZF`^}UAKV=H5Y}FF$oLy8&MRd+4 zk!EzLrU13oxv-i|rW6JK(;|6WfR%6@r4*BmlJ6OfmZXqZ64YP}RYyps5;3foA&vJ) zi^~cYjB$&s^N55>j|3vuHLW1XItORQ0t?(Q0e1I3Ya=VEB0NWn>>mwAjWcEkW{9RJ z6P!oX2AI%1O*LdfD@MW!Hn9<4xOw0oQe-pjd~>wg;PPJUEkHD6c2Mn^0t)f53~4;q2Jc)IQXz5gdvzRW=d0V!1fR^F=Swg0eA`at~$oh zu$W^P=keaoF8967B#gy-M{Bc1XA)KWu))Z-5`tWg=&=lz*JD9W3b8aq-1Xf~EJwjA zQnD<%Fa{ha;+^AOIK5;LRV9^x0tLsZR3~j+WRb2TQXym$C!m3Hzvq^e&QPdc=n91z z9B=GyD^nplF_BTAl)z07nq5dq(d=@(^f(b}=q2G))OJ}!$|}3>OCV^hNR_czyc}r+ zYuiIcmKaloKLjBQ9Fjk>(Fqn6E=s|MT$}6wPYhHMfQk#2163r~EQ)J}aUQ6+;5eYN z`o{n;NmNo2G+7KO+zj+(Ofzcs$Ifdcxg}UZoj?G&5CJeN#ogD)pKF}AqFk4299I^A z7E+dQ;Go@w7iL<>TS=2BIpJXO4FDD&DT8hNm?ZFH8y_hXEb`Qod<9-H7HV#Jv@_V3 zY`nL4C~7xMJ~6-?rahMvpGf|QSt&*Sz{GhX^53zodWxfYP*%K?ZUbN4f$k_b^FCy(l0_B zCI_o!5kqZj&rD=*-Q%`T#qFu+M!_A&=r#v7U35uByd^1~z1=dh1~YoqXTe2x0b>WJ zq6E-|fHf7ADTOXGyb~8S!0Y^2P_5{B)_puRijH6Tg-!f8=7uFm!)Za28N>?MkuyH| z4Q5n>XNLrXh2V~4#px(^M5Fp>m|XTlcIDUR6Q&6m&!|n5!8wCjezTnVX-^LRgBn#r z31zc9XGAMTRl$*sYVmHAvW7J(KP#18h{-qDlh zbq%mfjBhth)zG&mw{b3@$E}VE{J$+IkJ$|>JR6x0P$o006z2pLm=)1Hoi#O&axkZf zro!?EF|Tz%L~31je@*3O8Ry?+yLQ&mrX;tarh>B!&AcIYtg;(xXqDklwc86=o$!sT z+bT0%R|RS2Cf)ajbY=L@^&gGweB;G`zWU!ulb@()+qRS@7Y^i_4{tZ#dgH7;c>Qi@mj}17CcqV&Y7XjKhD8 z{QK?A>regs_=yvbg|80&1IvAUyMn_*2}z*yJbkJ@(e&O1 zG;+tr7hihly=#^KShITap}qr=n&*f5p6KiQ#%(oE{%P6tmD{TNm#kX6$gJ5kW0_Xj z(09YR4b`{LKl;^$6*cppu9>~OTqG1MaA$nLFYNA@p$ zy*Ipb)=)S$b?v~O?H5k$UwiBQw>L-5RkZB;N^tAacW=J7`5yk>12?{VO+B}%jyt`3 z{^-s7uGF1MwCp;#vHyX~a~A%RyX!wI`)|6s?}HhBr-rk}-QVkqI+~smBGA2{86AWm1F* z2Lnu066J`&gj=3u$d^!nQE-f$e^^u^s@_-!H*~QpF6kot9L2IB1{{&HsjB9)Zo1!U!z|2y}&xVO(nk$M+78TJ|O^rtLlU44@(y(EOO1r4@f*Mx{ z5}EvpsPQ&gbj_LLiZFEvT#~?AMeuVeML>ls;6Z>xP$Qx*tSPrE-P?w#>LQPo2p{c% zzcOz3aRTBkdBRiWN@;rN0ad`Jq$<8vBn7wHy?v06x$y=n%9DyX| zAxg?9CIG~8i(iORQ(R%{*ax%}=tGo=F%$D9UXjCH$~+TTle5tt)~T{=3wgdV2d@P|n#FDP5y2(eyY&>c0yA^lq z>V084hD2zEBBCssq7gvsT<{OelG*EgbF|vva$f5#Ks0Q2Q0<8V3h}WF zaLOKf=)p$RBbgnjVJWisL_DV94+VoBR4lnsRgYnhUn!Csz!hYlL4-G9M7SD2sxFX! zkcM@LJ-P4I;w1;1FfH$0>oa2?3MQf7+UGdfwQ<%kk3VM5wBdgi}`AB>~ILcHbAn&{&ZwW3hN4 z)(F;R7sRMh|#8_Wl)>Qu8Rt_j9@pkjk# zfXeD01HdFfjz^%$LZomb=*!R|yvQFruaV>yZv}N60c1l2z@!v+Un766vEGVuL$0xG zUI1E1S;B#Xb{C$RX(4YVRj@MdU~yFd7FV4I+qh9l;Kw$uI#003Q%~|0c!ey~-12B= zurJv-Z}E`T9+-UE0CSl3oPslkn#NWTo>k@n9b1t(QPWgrwlXj5EPs*ZuP;dK9FH|| zO&ed3NHyc~sb;LVP^2Vd8j!gB%9-VFHXwP~4poGYi6$mp!YC#&wsG!zQ?CemuVyu7 za1S~6P@Al_`NoCwRwF}$HjalKfFYXMv)6ecp&=xrPonGN=wrB)h&3_H`jBIe?9Ane z^R_llZwZ}!4i?!RZL&cX)S!B&VHMOEG+B}X@3Hl0p~gC*#m6=;jSVsw!B_)aPB(WD zBisNjg|pCP!dcMatD@ZI7=1*^3ozXptH?@2m;%RQ0)F#T{l))^$Sx)`5CkOvQ zjVhspvRR%oqGh8hP8{bK1>Ib|To(I*zR%NcxP z0XxU1Hc7$O8rAwxMwQ$!bG-dKHJ_$^N~=yKXO%K+=9J+)DU-kX-p<-JpQDcuVCQY3 zGnmh~2G}LWx0|MF=-ZRpIA_q~R!15B-xid|5ilZjP|bAk#?ifEe7nwm#B zn9@X3Vd;aI);b^}wJy28rt*@E^Y5}u9_n1)HSza_t(z8he5Ypm8^g-JkN2+?ANxn~ z^S9Uj;u-XD#aCwS;{=@p>XHdz7LnU{uFE1<#?>Za!aLv`v+DDFWx%2Sn$0qmlR`_UQe zrnHdp4YS?$w*`{nzw6I#<23{e@SKH&CRuC8(4GO)`0~tKDDH}qWhm$21@VVJhO7}!mo?ozh%gn zx8i{Q^5Q@3T~;7n{LAe7fBe%AYQOh-_YeA4zIp!I>OVf*zq$J2vHpYazjLuxW6>ZXF?nm-(z8Yv9jBTsFrSre?AxBmM_(3EffY={9zq^v3}bZ5Z6$+$O7 zP1HC|;@2lNPC%w;N+2-56#TPcqJ-v(Vlzbvx~i%1cxkdKTv-}23{hzjb&gk)3PvKI zUxJ#9$)an{TvvptOW=|O)+)T8OIZRcT>&cq4qg>RUs$tlSGl(hQ`JQdDFPR7hrcpv z@o@s;EqTII?Mi8Es#kerN~+>(MMiML?(OLE;%|q5XK7YxD3uTule0*fKLQ!bLxhxZ zgaJqt7rzjprnu46kq>BD(1!>U6DHzJw6Tc0gn1^grf8!*tW#y#C|&?zos`gYwN12u zHH~J| z>z?aYMIF*n7a*!>ekPD2#lb)hNUEpcyQe;;k3jaDNdtaL7goutCDu8+sN#y~oK=u! zw5lctwbi+>mR6Fo4F747yg5J%VjLx9laAo;X^obokXI7aU=gYgrz8_GQ7=In>yZ|h zWRWw5EwaKR5-L67uw2))yddigoEZu%aKi-H-TSnStf2Dn94)ecG$=JrnXMv)HASdk z4OSbVLN%IdNSIcXAj+aC8Uc!_0soLJnJMR+qtynN^;&NMq9L=DXipbVh>vA}Q}(E) z6-%gg$!w(-ucRzKVUJ1pL&2a26-#bh)f33$SC-_(;0k4*p)hYE6y|DxQgt5xLup8d z*yHejgAlsV@jDxO%?;Thlq+HLlh{07h~_LqYO>t zv+QC#-rL#bzPIUwv3T!jZLsK!qiP>ED9KiWk;@W2l)=&_k&_fImV$`8zT1iAC|HF{ zmPHrJfa6%abKDE3mkhirClgSh;5d=$q^*lA(shI@gk*{dXrSEh*(GItC{)F@g+kSi zH+Hv`t`O@*EF(`TftwyQ+mM_j+2wfYaUxXRiNh(YEfSC9Rd(N(K+ss>Dr2#D8PYJ; zwuh7?P^JohFhUkMIDaH#Jy?{#C`=z17IjppD~T2VjV1;p}x@NN5Q0=#%dHSn?RI#9~bZvp(dQBRhXNV!f@6 z(_2DkUxY<=N1JI-B{8VZX;^tR0Zo=xTGWy#*5YHEm&OJjj9{z*F6Wv%un}&6 zR>E0mGT|)f@KsS^bBsKq_yw43ja7K1!A}6|WY}*^vs>BnHc>aAAs@}qZogQL`$dSu z=-@Po7ooPbq$jdJe%)=K3fohWjeOh(`30FuCl9?8>dpCrl$So)Mb}gR=&++-5oT)1Dms2Q{LE z5XxqG)`(V!s=Ol`(c;}GWejUXenu+05aV$uK*L*PKL>cXh1=ptn%&k@oBj!O*e`Fy zLBjr(sKB`f16NU_74d;Avn;t}2p|dy^8p0q@qvNF8W|t_XCBun&#E`qcvGZj3}kIS z&mh5u>)DWD#C3xuE6*HrN0DBzK%_igun@Z8dvV(-!S#zFTji{Qph%w>*e>Vsi2>{! zpJI}NtTn3jp`0kWZRU9UcVa$C`>a--NG>R4nEWZjdQzrz^Szz5Yd%XJA;8YtMCUP| zb`7vgjBhth)R4C)yK&B=$E}Vs{J$+IkC_cBJR9i`P$tu>6z2pHm=Vz&oi#C!a4@Th zCc<(DF{gDvL}FcLe@*0N80X(*yLMKPrX;(eCW12zO}`;_tTG#GXqDm5G`AP9I^jFl zHcv=*U1M=GcgKv*@ymLqb-ew~k-xsX;JFLuMt?e{dEc^~{k4-{;U1Nj@Ou`Gow0qz z!r9@k^#AC$&)g%g-Y3p^X4h8w=_3bEtQ&l@?Z34Xe?x7&c4l|o`D+Ur%XTQspKQBM z-DBQ3OzjN3duI2GTg9@Mx6Qd!ad=|aOPk|Y=68*oaq+;qD`)X_%$YYoSPO7E8G-`=tCcWy1{ zvy@L;zlcsx-td!-iq~6b{p{*M=&cWa_wT8fet8+(DZ5fH9j&T-fA53c*Dtm7_gp;r z)ARq-{_=;D`+LW<*A3j*J@cLUkN4gCxoB;4=8kWStrMaPRz(+?GxrFSK68!^zZ`vW z!|6q%w?EXrbL{MgXKmj)?(ED|E=f3;Le{%QL$KJf$5&6%WTNP7&d~E)@iT!6U ze;D1gCdKr2iGPX;uT4nZ6|R@M@9x>#z3}GdSLc12=H8_0c?Z1qvPg$(?VF)wlMPF533*s8w>t)fn5Q(ia=D_B9KQk zh+zsSBN{SLOXf;WK)7huLzarlmS;{QhvywDNhw?ff>oeiMOF+7>d*yH(-7Uc-_)Q= zN>*T*I;l@rykmBq91;*MjHw14PS&Ldh8+E_zjWF*)w#?m|BmF7FUsY(sD;v@U}m z@sCia%v{~4 zjhu8mB3=~$twx|`ReDFsT`~;7@&t3x@R~KKAe==G^Fzn?Bt^`rjiAsbY^21%e+&3;fr{+_NVZKyopz=JSezGLt+olU z|5oo{WnpR|C6U`^5|IdJOn!lFtmK&{B-%l@f}mREnaHKcAQTQ?v^ zyCZERq(?Kk;e0x$6f#*QGn`Fl6uGdMa7>5?=aA!)xz?B77m4v;Fa!G1dvz6&kueia z{j2-gNhZVEQ|Z_X{D8k1IX3LW+1%<2!m00&F`+wTniNSW#Ey6mE`Xk&fTFZg*!*w9 ziHTKPKfn=bhF}*0;xJEk1`MF&7=U|%F2eTW5jZT%U{TMxJ7@!q6|B^QTj11T zMosn!-H^NV4Gby|F=&!w=muHp!a;Ceriqs@;M+jO)6(iCIDt2hP0r0h*f1X&SHzxU^`HF?^&gK4$q1txC2h$-HvM1GaxEYQhb@K z0+Ss&-FRIp&~TZ0GI=(LyCln+Mh$x)SP~}W!gpFAss(nCxj_ir2&c!SJ8}ve8HFTw z!Yi6j!1}&~Q)7E(24y2`f*SHmF3)L+$s_9(zKUTZ(Bpk-aRos5`3~TR0 zVorm{rTronBuKi0-yp9KkON0;is&M8nY^6NX3|+DUl__Nik!)8S^oW@M-Kc}Q;WY| z{YpHvG>|4Atvh(ZF{w?drSG=Vws{I|@mJQQKftzg`G?O-1V#Kw=>`D-}PMo#iNhO v%O?2vXy5AD(KkN-?9h%k=l^~v4jv*Lj2Jb&;;WT5R4y4Y)P7kQ6t9L-P@Vj`)Kd_^Y&)n zXXbg|d7fvV>gbv_dp_*ewOT_=pd>u7Xa{HZq!8D@=}6RciL2pMn6!T8n*7pS&@Qf0XXiqgfiTd^7E(nNQ>|nN(&@f^l(c150e2NhCy?p`1!-siC`rgZ zxzkjaB_YIMC1uP{QFtv{8V?EZ7JIO`6YNQ|pz0YYXDcy7g()MUnZAbj50+VBcg0LV z(>=okD10yn#U|UCXCo$RR-tw^+2jD+O}z(r%NU~gI;Jk#CL1Om_gHbI*JYMZP@Qdq zI}}op#n(C`G||p;dDpdc4i`S!T{7%t`M40+J3zRYv?!sWU|qI%9t)F}^#~|fmF*l6 z0$VNIOS_d*Ek)t53S}-a)OFZ{|HgMeiKUM`&(=yk#7Ddl_erEJ3s6=Kd}r;#Ufm8& zQxH_~16LzA>lPNTIDl%8t7ZxNj^ewdD1ht}bVJ3;)}SaeixkW$?LQOeVYW{O3e`NA z5{g4hF}VhNPk!BqFk4Wt8tAbOe7C5sdaQD;G!y(yo)QD+E#SNb;+hU1KDI6Dc4sC4 zX8poSrKSMvyj41|(A(Zy!{OVO#3GULm`E=x&0~8`aAjrL9f^`*MU*WiB2cgp0rPrJ!$AR1 zA7O1JgawLPPf+v6gkq43j12^o4-(LPfuR`4wju%q#jDH0%j53_K-p=@jS>`rHjfzR z)If-vvXO=2&~|x7IyD&2UGt&{^kTXF@tJi2rhK!Q6Ew&P1loJTRLSKDNlZAP2(;TC zMoY zC*599m@Fsc3n!G(NFG=xmQYmRnMppG3^q?Zp{UrqGM}&hBvKGmRZ`Y0K!a18MU4O-Xk7fvAVrYDFS`1Qqj;Ovybgox^JAQa()^?Z=$1JOt*5Do-x zSu)nwzwg?iAJ1JI`|9pDcOAUr@bov%H8!7KUH-**^U}eil|KvV<0mejS#jm+w(Acr zZJqVR&VA1>yD)g=$g!hC8*ZQ1Gdo=W8PVA52k#&K z=h)e2uOAv|yl>vni397Cv4PWN4-C#Z*}veuzL)0gsebXJ4I@A8?i${Y&L0^bx-_s1Eq|dFmi;#G*1?+KmfyeMJNn3n-<1D( z`RliTID+huL4 zq7f27u>?Z!h9U$LpG?K-gF(E-gfy)Ww!uVRs!>v7>kULh;+fkyXZF(W_Hk#={NH~& z-}lWUJ$*}>THt`8-(`wfq{6q>402X6hqxqXAW=6YuAS4MYX4TDCTEaor8BaE@Z*e( z6v^b0I|j>6AyP>`)Gcqj!NbP}~gmbheu z=ZD(MV=GAOWuq%P6MMUEK?T?e0#kk(0a*h*MGKjJx?W&`37mt5CENL=%^VWb7?~t4 zYZl2mh5rPbHiEo~Bw*>FV8~W!z9A;8sB0E--uJq|2HW8NX#zb1tSkb4Ym{6jYYE5Mv-r)SpM1h|*eK?Lh|3JJ6DpcM7iKHp>av>y@HCAg@Q*Q#;=@c_v`aQEyzc4RLOvy26ri>8iQ|i zR%pJ=%lmHVg|c1*+U_>PK31t00(%EYFBS|++E6tvJ3GG%3x@R@P&Fz$IYI)~TKY_lWIqAnq^374)w z3$^&%h>>oOg(7jjGs<`Jd?*o*gkqu4YK)^ot78BeCf2pa&>EkNJ6sKD>|8C22unt< zr}~qvWRtO2=S5rgTtDCx!w1CP)8=wt;HkcWWkNQvnj~>3L{_{G7hvy?iz1Yx5dE)$ z6CkUqet<%R2n4xUj1#B=!isriXFvdi7y+Q3poXyZcmxU-Az=Q@sXM3w8faLm4!1y2 zqXf0GPpAdCM&Cd{`Az~_$uZOdnX17-P`q#nUIl@#0?KYnxtE{_ba+HL7Y9P*l=UnW zhmOlv(#65}@>&r^pcl)XPe84!bSkimWln=sAkf*9PSqWrG>LHs6oF3L?@`?VetRm# z=P4@?+OgG*_9X)qldvbDX9c@Ue5^!j$OFcb5F%&3(_B(bkVU2j!F3}PJwV;QQ&`9- zRB|V@qwxx`jxV98ftraySqQd*KcT2tyS(q$pcSbKDr;D(ZC!RgR%`Jj6pYGM(0tSQinFMdP8EkO)VFa5NGM3w)yC_<{4^hW|Xb zZ1%%s)9*_sXRiBv>C5A1wloSW8|E&&I2S#p%7bcg$+793|8-mUFaNFK=#zYM_~R>k zfA}i?%ZV3+&?le0{rZ+YUrx>OLjRA;p1CqM@^C)UI{nZcdEVj z*PpZ_AG|R(`s(vrPoDn1>xQn~(Z?GO9)3zabT%>d&Yr95PCs#EUF+z+i+_(P#M^=YLTEXphfglw*VwR6w5Api%MWaAN%L@z|L$SW?U0a}9hVY#+;IuDL#=uD zeqlu)GY3|rfkGCz-X{FEeGB5;@GZ<--$QS{aUBg@=AhSFD5i_3%dV|8>$<+PRE$lD zx2Z;_z7qSE&oD)NgD7<0C&=|sd&o$bZ8r^UD1rMpU@qT}8TN@~0$FU_^_bXo3;smc zuc0U;Hex;+1TGuj?}`%E@;yf4_tPw>!Cp!J43XbK%poXCZ4>ENZOrfzGOpT$lETi7 zH3bopcO&9i#Pp(Gn*?&-xsX9XSu*MhtLH*~X5!uO z2hb!aTbyB`66#6EP*)UGycIL_a6fc>mMcj9VCf2873N@;oM8e*UoB90vg3XZP0Np} z%Jni&4uWb1LzFF35~bIaxa2)IWYF&BY@uIGmrsbUmLX{hd&c0`;ue~;^J3lue^AIp zF6~J&__DHG80sAmxtJPMq@gmdTDvR@Q^RHnsEn#sP6z>4i{#SY)x3yO4OA$-#5f4> z5dMa{-vj9r&&RbgsOyca#d8wLWdXZh2zQnT&f>OxpAziBgG3{51rdl>2VrlRsF}m> zDBLAVk=Q5N#xxq&pmn7eB`gk|Ka-X*?@fbZuY_rn4vr|(ntV@r-DoIVpr8hU*@n9% zbv4VXmPoVU-^`R4#c!ebE!6P?gyGm+)OlqlfYo^s_50@l$8XIKHg@jqENO7tdLR<% zjA`tMgSDbw4}*5_qY!L4GMy98qpv~Or{dvk`8KgGz|kpYOaVXZmbf}~y%)N-FN3wcK0AfWoHfR;Ii zS&*$69E9Z6#&{(Iz5=RtOVLY6f-IhxoacjxajJS2$&uyCCh7Sw*?29b81iBz^~p`^ zicIBtvEVes2@F|#k*T@EizF%TKoVrNvmDJ0$kJ05Uze&NWQR{TnU@Zxp-@jD&lKDx zEE^j&;(=mGgpe!WX^E(=i$<0PA#o#;o|Ep(DNHj8W$r{~v{8bMeTk&z&MXYdG;E1K zkyMyn>Gx~iic~^f4@Nbg%kjtRES^NdA8k3#!SQkX9p~h*b9S`# r@kbkP?S22n$?on4pa1pecYl2Oc=P0?$3Oq{L*M)M#hV{46!8B8Zp5%{ diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet deleted file mode 100644 index 73121259969..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2003/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00004.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:91eaf919c48b9b105f572b81d16df797b1c9985d57c389685a9844468b77dcb0 -size 5975 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet deleted file mode 100644 index dcde290161f..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2008/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00005.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:c5305d15f28f9d6fb522fffdbd4177e4601d390fbc10b3f0ad7abf38352f4e1b -size 7264 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet deleted file mode 100644 index 51133193afb..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2009/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00002.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:4d7cbcea7ad6e082f5d0163d35a15d4f9f6cb50b81f1307ae72c389a90221fb1 -size 7277 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet deleted file mode 100644 index 7010230b425..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month-12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00006.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:c248feb90c1c051a69937aebe0899507112c2c38f0d7ee4aee14a40e15f010e6 -size 49446600 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet deleted file mode 100644 index 83df1c8f0a1..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2021/month=11/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00003.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:e57038ea90788ab2689d45c2b1fbd47ca5281914d325f85031f80f584507b98d -size 8646 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet deleted file mode 100644 index f31a195823e..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=1/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00008.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:182eeeaaa3f5743491c8074e395239b5311b665466f38e389f0fe2d6fd189596 -size 7734 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet deleted file mode 100644 index f496470c39d..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2022/month=2/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00001.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:109b675803353a2727d38db55696b24e2d80b02ba04fea66c671aa2e58a66202 -size 6780 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet deleted file mode 100644 index 7b561414dd2..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/data/year=2028/month=12/00000-38-5902f321-2e39-4ac3-9854-3c92a5eae25f-00007.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:666dc205cd521582d6b4a0f186aa90c0e0eec48ae5784b620b34a03ee2e51d0c -size 5975 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/00000-e5584cce-27f3-4b56-a5dc-e1ee0e1df858.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/00000-e5584cce-27f3-4b56-a5dc-e1ee0e1df858.metadata.json deleted file mode 100644 index 6255133468a..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/00000-e5584cce-27f3-4b56-a5dc-e1ee0e1df858.metadata.json +++ /dev/null @@ -1,180 +0,0 @@ -{ - "format-version" : 2, - "table-uuid" : "8b920795-6a06-41b3-954b-538c9f39ea60", - "location" : "s3://warehouse/nyc/taxis_partitioned", - "last-sequence-number" : 1, - "last-updated-ms" : 1714753363900, - "last-column-id" : 21, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "VendorID", - "required" : false, - "type" : "long" - }, { - "id" : 2, - "name" : "tpep_pickup_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 3, - "name" : "tpep_dropoff_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 4, - "name" : "passenger_count", - "required" : false, - "type" : "double" - }, { - "id" : 5, - "name" : "trip_distance", - "required" : false, - "type" : "double" - }, { - "id" : 6, - "name" : "RatecodeID", - "required" : false, - "type" : "double" - }, { - "id" : 7, - "name" : "store_and_fwd_flag", - "required" : false, - "type" : "string" - }, { - "id" : 8, - "name" : "PULocationID", - "required" : false, - "type" : "long" - }, { - "id" : 9, - "name" : "DOLocationID", - "required" : false, - "type" : "long" - }, { - "id" : 10, - "name" : "payment_type", - "required" : false, - "type" : "long" - }, { - "id" : 11, - "name" : "fare_amount", - "required" : false, - "type" : "double" - }, { - "id" : 12, - "name" : "extra", - "required" : false, - "type" : "double" - }, { - "id" : 13, - "name" : "mta_tax", - "required" : false, - "type" : "double" - }, { - "id" : 14, - "name" : "tip_amount", - "required" : false, - "type" : "double" - }, { - "id" : 15, - "name" : "tolls_amount", - "required" : false, - "type" : "double" - }, { - "id" : 16, - "name" : "improvement_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 17, - "name" : "total_amount", - "required" : false, - "type" : "double" - }, { - "id" : 18, - "name" : "congestion_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 19, - "name" : "airport_fee", - "required" : false, - "type" : "double" - }, { - "id" : 20, - "name" : "year", - "required" : false, - "type" : "int" - }, { - "id" : 21, - "name" : "month", - "required" : false, - "type" : "int" - } ] - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ { - "name" : "year", - "transform" : "identity", - "source-id" : 20, - "field-id" : 1000 - }, { - "name" : "month", - "transform" : "identity", - "source-id" : 21, - "field-id" : 1001 - } ] - } ], - "last-partition-id" : 1001, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "root", - "created-at" : "2024-05-03T16:22:36.131552469Z", - "write.format.default" : "parquet", - "write.parquet.compression-codec" : "zstd" - }, - "current-snapshot-id" : 2327977723212680118, - "refs" : { - "main" : { - "snapshot-id" : 2327977723212680118, - "type" : "branch" - } - }, - "snapshots" : [ { - "sequence-number" : 1, - "snapshot-id" : 2327977723212680118, - "timestamp-ms" : 1714753363900, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "8", - "added-records" : "3214369", - "added-files-size" : "49496251", - "changed-partition-count" : "8", - "total-records" : "3214369", - "total-files-size" : "49496251", - "total-data-files" : "8", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis_partitioned/metadata/snap-2327977723212680118-1-69460130-5a60-48ae-b1e0-6aa9fa2b98b5.avro", - "schema-id" : 0 - } ], - "statistics" : [ ], - "snapshot-log" : [ { - "timestamp-ms" : 1714753363900, - "snapshot-id" : 2327977723212680118 - } ], - "metadata-log" : [ ] -} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/69460130-5a60-48ae-b1e0-6aa9fa2b98b5-m0.avro b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/69460130-5a60-48ae-b1e0-6aa9fa2b98b5-m0.avro deleted file mode 100644 index 4dfabcb5cc0..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/69460130-5a60-48ae-b1e0-6aa9fa2b98b5-m0.avro +++ /dev/null @@ -1,10 +0,0 @@ -Obj schemaê{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"VendorID","required":false,"type":"long"},{"id":2,"name":"tpep_pickup_datetime","required":false,"type":"timestamp"},{"id":3,"name":"tpep_dropoff_datetime","required":false,"type":"timestamp"},{"id":4,"name":"passenger_count","required":false,"type":"double"},{"id":5,"name":"trip_distance","required":false,"type":"double"},{"id":6,"name":"RatecodeID","required":false,"type":"double"},{"id":7,"name":"store_and_fwd_flag","required":false,"type":"string"},{"id":8,"name":"PULocationID","required":false,"type":"long"},{"id":9,"name":"DOLocationID","required":false,"type":"long"},{"id":10,"name":"payment_type","required":false,"type":"long"},{"id":11,"name":"fare_amount","required":false,"type":"double"},{"id":12,"name":"extra","required":false,"type":"double"},{"id":13,"name":"mta_tax","required":false,"type":"double"},{"id":14,"name":"tip_amount","required":false,"type":"double"},{"id":15,"name":"tolls_amount","required":false,"type":"double"},{"id":16,"name":"improvement_surcharge","required":false,"type":"double"},{"id":17,"name":"total_amount","required":false,"type":"double"},{"id":18,"name":"congestion_surcharge","required":false,"type":"double"},{"id":19,"name":"airport_fee","required":false,"type":"double"},{"id":20,"name":"year","required":false,"type":"int"},{"id":21,"name":"month","required":false,"type":"int"}]}avro.schema¦7{"type":"record","name":"manifest_entry","fields":[{"name":"status","type":"int","field-id":0},{"name":"snapshot_id","type":["null","long"],"default":null,"field-id":1},{"name":"sequence_number","type":["null","long"],"default":null,"field-id":3},{"name":"file_sequence_number","type":["null","long"],"default":null,"field-id":4},{"name":"data_file","type":{"type":"record","name":"r2","fields":[{"name":"content","type":"int","doc":"Contents of the file: 0=data, 1=position deletes, 2=equality deletes","field-id":134},{"name":"file_path","type":"string","doc":"Location URI with FS scheme","field-id":100},{"name":"file_format","type":"string","doc":"File format name: avro, orc, or parquet","field-id":101},{"name":"partition","type":{"type":"record","name":"r102","fields":[{"name":"year","type":["null","int"],"default":null,"field-id":1000},{"name":"month","type":["null","int"],"default":null,"field-id":1001}]},"doc":"Partition data tuple, schema based on the partition spec","field-id":102},{"name":"record_count","type":"long","doc":"Number of records in the file","field-id":103},{"name":"file_size_in_bytes","type":"long","doc":"Total file size in bytes","field-id":104},{"name":"column_sizes","type":["null",{"type":"array","items":{"type":"record","name":"k117_v118","fields":[{"name":"key","type":"int","field-id":117},{"name":"value","type":"long","field-id":118}]},"logicalType":"map"}],"doc":"Map of column id to total size on disk","default":null,"field-id":108},{"name":"value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k119_v120","fields":[{"name":"key","type":"int","field-id":119},{"name":"value","type":"long","field-id":120}]},"logicalType":"map"}],"doc":"Map of column id to total count, including null and NaN","default":null,"field-id":109},{"name":"null_value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k121_v122","fields":[{"name":"key","type":"int","field-id":121},{"name":"value","type":"long","field-id":122}]},"logicalType":"map"}],"doc":"Map of column id to null value count","default":null,"field-id":110},{"name":"nan_value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k138_v139","fields":[{"name":"key","type":"int","field-id":138},{"name":"value","type":"long","field-id":139}]},"logicalType":"map"}],"doc":"Map of column id to number of NaN values in the column","default":null,"field-id":137},{"name":"lower_bounds","type":["null",{"type":"array","items":{"type":"record","name":"k126_v127","fields":[{"name":"key","type":"int","field-id":126},{"name":"value","type":"bytes","field-id":127}]},"logicalType":"map"}],"doc":"Map of column id to lower bound","default":null,"field-id":125},{"name":"upper_bounds","type":["null",{"type":"array","items":{"type":"record","name":"k129_v130","fields":[{"name":"key","type":"int","field-id":129},{"name":"value","type":"bytes","field-id":130}]},"logicalType":"map"}],"doc":"Map of column id to upper bound","default":null,"field-id":128},{"name":"key_metadata","type":["null","bytes"],"doc":"Encryption key metadata blob","default":null,"field-id":131},{"name":"split_offsets","type":["null",{"type":"array","items":"long","element-id":133}],"doc":"Splittable offsets","default":null,"field-id":132},{"name":"equality_ids","type":["null",{"type":"array","items":"int","element-id":136}],"doc":"Equality comparison field IDs","default":null,"field-id":135},{"name":"sort_order_id","type":["null","int"],"doc":"Sort order ID","default":null,"field-id":140}]},"field-id":2}]}avro.codecdeflateformat-version2"partition-spec-id0iceberg.schemaª-{"type":"struct","schema-id":0,"fields":[{"id":0,"name":"status","required":true,"type":"int"},{"id":1,"name":"snapshot_id","required":false,"type":"long"},{"id":3,"name":"sequence_number","required":false,"type":"long"},{"id":4,"name":"file_sequence_number","required":false,"type":"long"},{"id":2,"name":"data_file","required":true,"type":{"type":"struct","fields":[{"id":134,"name":"content","required":true,"type":"int","doc":"Contents of the file: 0=data, 1=position deletes, 2=equality deletes"},{"id":100,"name":"file_path","required":true,"type":"string","doc":"Location URI with FS scheme"},{"id":101,"name":"file_format","required":true,"type":"string","doc":"File format name: avro, orc, or parquet"},{"id":102,"name":"partition","required":true,"type":{"type":"struct","fields":[{"id":1000,"name":"year","required":false,"type":"int"},{"id":1001,"name":"month","required":false,"type":"int"}]},"doc":"Partition data tuple, schema based on the partition spec"},{"id":103,"name":"record_count","required":true,"type":"long","doc":"Number of records in the file"},{"id":104,"name":"file_size_in_bytes","required":true,"type":"long","doc":"Total file size in bytes"},{"id":108,"name":"column_sizes","required":false,"type":{"type":"map","key-id":117,"key":"int","value-id":118,"value":"long","value-required":true},"doc":"Map of column id to total size on disk"},{"id":109,"name":"value_counts","required":false,"type":{"type":"map","key-id":119,"key":"int","value-id":120,"value":"long","value-required":true},"doc":"Map of column id to total count, including null and NaN"},{"id":110,"name":"null_value_counts","required":false,"type":{"type":"map","key-id":121,"key":"int","value-id":122,"value":"long","value-required":true},"doc":"Map of column id to null value count"},{"id":137,"name":"nan_value_counts","required":false,"type":{"type":"map","key-id":138,"key":"int","value-id":139,"value":"long","value-required":true},"doc":"Map of column id to number of NaN values in the column"},{"id":125,"name":"lower_bounds","required":false,"type":{"type":"map","key-id":126,"key":"int","value-id":127,"value":"binary","value-required":true},"doc":"Map of column id to lower bound"},{"id":128,"name":"upper_bounds","required":false,"type":{"type":"map","key-id":129,"key":"int","value-id":130,"value":"binary","value-required":true},"doc":"Map of column id to upper bound"},{"id":131,"name":"key_metadata","required":false,"type":"binary","doc":"Encryption key metadata blob"},{"id":132,"name":"split_offsets","required":false,"type":{"type":"list","element-id":133,"element":"long","element-required":true},"doc":"Splittable offsets"},{"id":135,"name":"equality_ids","required":false,"type":{"type":"list","element-id":136,"element":"int","element-required":true},"doc":"Equality comparison field IDs"},{"id":140,"name":"sort_order_id","required":false,"type":"int","doc":"Sort order ID"}]}}]}partition-specœ[{"name":"year","transform":"identity","source-id":20,"field-id":1000},{"name":"month","transform":"identity","source-id":21,"field-id":1001}]contentdataàÙ‘•J Pq(Ü´ªˆ”ݘolgÇŸçñår<1ÞÍzdÇÃÕ½y7×(gÇnFSTr{6ƒµêª®:¶™ÎU+ÝÚ®qG;íE(«è °h@1 ¶Nt‘FUJU-·îFäUÉZ­éÄ´ Ai€¼XÇ -íÄ=wöÅŽ}Ûš.Hã$ÿüÓù|çïÝçù>ßÇÍ”§Çß9;~ÒüöåV¦ÓßÌï(lÞ¶³¯Þº{cº˜ßµ¥ïŽíùÅ-Å-Û¶îJß•/æÓ» ù«²™L.}÷¶­ÅÍ«ºÒ¶uæVtv÷d²›rÙ®Îl!×Ó¹<¿1×Ù³¢{ygncO6ß]Ȳݛ:ÙÁËuç´÷ì,#·Ü°fõ­_\‹ž\‚*¥É™¼)˜Ø ›‘u¢5‰3%S6©©˜qS5¦f&צÖç`Ä! £0Š EA1$!Q¤ 8RQi(éÈà0ƒA -ˆ$€’ ’¼Ãš?fçð6N4¦¿kìýmð¢}[ìúŸ8 zŸÍöâj@ØßA_Å窻£~GDXíbþ—$¿“ýŽúâwq¿Sý.áwšß%…Sí¤÷Jÿ*B‚Sœ·z˜g/ælW æìh¾¾‰æÑ%Hš.ï üç‡içV¿þF7öâþ~pê7ÂË]ƒÿô‡2>ò‹ÏDÞ=~•xâðÓ(:ùÈ{ˆ ¼¹2v|ô`Hzïè7äf>IÇý:¤<óvw|ú[íê¾CÏq‰éo_«Ùû–&‹©Á"{Žì:+<++˜•0+VDV¢¬Vb¬¸?Of…²¢°gEe%ÁŠÆJ’•+s#襑°38œq^-Ç‘³_s^f,AŸB`ÿy~µÍ¡Ì~V8ñJ›O!BŸBèS)œÝc×(ŒÚ²ø³Ÿ²í­‘*…ýëí*…ý_²ãbÎÝN¨â&¶½³Ç®QÈÙ5 -ßI -q) ×Æï«°;¶ÿk÷iWÅÊßoþþé9ªÅW*•Êk}䆪 -{½A·‰â'`ME­#b[£Š=OUý_0dï×mÂFUÅänCŸ?Ÿ4¾Š^ÕS»ï)ÃWaø*zT´K ˜²ÙêXZÈPêjJ•%ˆÃïnqnr rƒ„BâA.ÁÈÃP¼?z?9c»¤”KV rÆKPýL” v&`j²g„9ÌccÆ,â(&8†%,cŠÇ*N` 'q -/ÒTn¿îlߌÓgþÆ:oŒ£ÊÞs«žì‡wÆÇûP É -50Àd¯ìw­MvŒ5îã-À`&;å‚ZL=~¾}ÂUAüoÖÕTˆŽ -ïôИ§bÊW1ã«@óT, PQ¾ñ­ûH9d´VÁnš²·óƒU,ÂT±à“k5UĬýßàå6¾Ü&œ„ø"uEѱ©ÀØ“!Ɇò1HÅ)ÏÂø0T+m‰cP®cÜâ,Þ,l…­ˆ%ZQ‹X1K²d‹ZŠ·T+aiVÒJY‹Ä¸M]gtÜóËà«m AâÌÁí«ð™SÁŒsÁŒû>9âÓ1ò¾Œ;îuWÛ­ƒ„g~ÍŒÛÅRu"227ºNq'ŒÈµŸ÷'†«âh¶ðP£ŠÉÕŒ/3:q~s¦Æxÿu îܳƒÛYÿ¾ÞñÑ2¾âJâÐ皟pâÐÇ0ÜÛµ_»óç<ãàøƒ¿cà[.öf`-ãs<háØó½îËFs,ntlïéË °ìPÙÿµfÉÇ_ÅGHbØ+Z¤ÏìòüºâBÂÄ3¨HŽ è†±!$ »™d)Ãn&9J Cí‰:¿Ö9×ëa=¢‹zT'zL—tY§º¢ÇuUOèšžÔSúb-_'î½à…U£ð½«ï;Ýè×®-%ÿì×í~þ@6ZÍæ^4ŸË$~=´|í¿ÔÿÀ†×\—¦Ï±®¦BªFîÊW²ó3I›¹/~S€_ßÄ"Éˌք»ÁªDøÔ",_3Wä×ÝMˆ;~Mì{<Ä ¾„„cïGác.â/Ãè$Gaì (= eÒ1¨ ÁøT_‚‰!¨ Õ!N8Â`&"’(!$F$"J'*I$IŠ,â—`hãûÑ%¬ûÿmÌ!ži¸‡é5~$áìË$ËZ¬*âA&Þó§À®ŠþG7ܲµN‹$3,ŸëšIþé«‚"I‰sFó@•Œ…˜øÄ"˜x¦ç -L<ÛDø3Ž‰Ó׫„?Žøa$Ø.áC.á“0z²uåa(†ò¤£5Âg\ÂYG8å(OŠi˜F¨H£”Е¨L)Uhœª4A5š¤)ºX ËuKöØåÛÜy¹„§þ§ÿÞ 2q{ÝßÓw{ã4¾Wº·N‹)îju<;?¦´"<8¦Ì6˜¸«W%w—úšÂÏ6›øàÙ‘•J Pq(Ü´ªˆ \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/snap-2327977723212680118-1-69460130-5a60-48ae-b1e0-6aa9fa2b98b5.avro b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_partitioned/metadata/snap-2327977723212680118-1-69460130-5a60-48ae-b1e0-6aa9fa2b98b5.avro deleted file mode 100644 index d75fbc1402fc5b1dfb101760bb08d74f0bb18442..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4287 zcmbW4&ub(_6vtx%iyq8Hi3itMw31V@bWb*!$-u&j>q20`#fdt*%hJ|#SEoDa=^pz> z#&L-U#jBVh?B>UkL_v_SvKMcA5cD9TH(@V=NAcjrzd&F0kE-hF?#WC}!}L_Y&#U*U zzVAD4w>F-cp21t5d&$>K(#A)#_oSfLAyQrPh~|2_G%q>0O~1FXV;RH`RKv0fJT@$1 z>%LUKb1%1|g98&*B!T?Evz&GKuDcqtli}2eoN67WhTDZhg3`q)7CIsBbO5z1prTY%-@zT+ZSTTrDzXkBdQ<71cr!?*OaCnL$|@O5=*XORI2Z*t7tpQN_+NA)smzU)np86WX=} zDwJGe?0L8cKf~Q`gY>cIgIeh=$+h%|XC#u!0=66Vt?dTV^JvVC`Z zUV_`!29Zc+Ol_NOtP!@`7_@_Kg<#F(O`A{t=l*tGpRNLX|A$Z zsjgJ3RrpjY%Qab1Y8Qdi#=*iC@qCzTla<+2j2Ge=FsWR$Gy)@IJ(>DXUu7p5hIOdX zu^0P+XftwV*eA0&?+cQt@4=X`JeVeR2u0WtAHxOE^Al5)JcZ5w5>7^}O8o#wR2YI? zEWrtsK*V8Q>7?2DW{s zo=l!?aF?)bVAQY&f+b->F8oi6MRkQ8WNr{*H^S)|=}w)(Sw^AAo$!j*3b2tc;nd8Y znL#-VTi{PP6=qlR|21nxN}-kmqngg;=+EjXo`l2rxXRj(v0>#tB;z!AT-qSs7J^Am^_3tN-Cja=cY1`jCR*#yy2RrKR zho{Hy?>|!e&7E5p)i0ZC{agM0;|B*{A0OWC@AaGgZ(YM`86TMT$>xD+oNTss_D&D= zy^WL2`+Mg8eREg0cJ$N3&65Ys_jZ5KreB+ydg1rO@bRTjp7j2G_}RqNPhUO%%5PVd fnG16l=Dzs#pR02}zl@&ytMaKiJ&S&s$YuCH(NMLq diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet deleted file mode 100644 index 1b4dbbe49af..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/data/00002-4-825545b9-fc45-47ac-9f75-74ded525e6d6-00001.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:493c715173c70d49f3a94cb8b1c35758c87604729ab2b49654f8684fbbccd489 -size 49412347 diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/00000-a58846ae-c052-41d4-9e2d-63e29407b88a.metadata.json b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/00000-a58846ae-c052-41d4-9e2d-63e29407b88a.metadata.json deleted file mode 100644 index 9f3bf63f6bf..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/00000-a58846ae-c052-41d4-9e2d-63e29407b88a.metadata.json +++ /dev/null @@ -1,160 +0,0 @@ -{ - "format-version" : 2, - "table-uuid" : "0cfe47f4-5403-4ea0-ad4b-b4c5c0711cd6", - "location" : "s3://warehouse/nyc/taxis_single", - "last-sequence-number" : 1, - "last-updated-ms" : 1714753336329, - "last-column-id" : 19, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "VendorID", - "required" : false, - "type" : "long" - }, { - "id" : 2, - "name" : "tpep_pickup_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 3, - "name" : "tpep_dropoff_datetime", - "required" : false, - "type" : "timestamp" - }, { - "id" : 4, - "name" : "passenger_count", - "required" : false, - "type" : "double" - }, { - "id" : 5, - "name" : "trip_distance", - "required" : false, - "type" : "double" - }, { - "id" : 6, - "name" : "RatecodeID", - "required" : false, - "type" : "double" - }, { - "id" : 7, - "name" : "store_and_fwd_flag", - "required" : false, - "type" : "string" - }, { - "id" : 8, - "name" : "PULocationID", - "required" : false, - "type" : "long" - }, { - "id" : 9, - "name" : "DOLocationID", - "required" : false, - "type" : "long" - }, { - "id" : 10, - "name" : "payment_type", - "required" : false, - "type" : "long" - }, { - "id" : 11, - "name" : "fare_amount", - "required" : false, - "type" : "double" - }, { - "id" : 12, - "name" : "extra", - "required" : false, - "type" : "double" - }, { - "id" : 13, - "name" : "mta_tax", - "required" : false, - "type" : "double" - }, { - "id" : 14, - "name" : "tip_amount", - "required" : false, - "type" : "double" - }, { - "id" : 15, - "name" : "tolls_amount", - "required" : false, - "type" : "double" - }, { - "id" : 16, - "name" : "improvement_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 17, - "name" : "total_amount", - "required" : false, - "type" : "double" - }, { - "id" : 18, - "name" : "congestion_surcharge", - "required" : false, - "type" : "double" - }, { - "id" : 19, - "name" : "airport_fee", - "required" : false, - "type" : "double" - } ] - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ ] - } ], - "last-partition-id" : 999, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "root", - "created-at" : "2024-05-03T16:22:09.674882012Z", - "write.format.default" : "parquet", - "write.parquet.compression-codec" : "zstd" - }, - "current-snapshot-id" : 6884618194677467995, - "refs" : { - "main" : { - "snapshot-id" : 6884618194677467995, - "type" : "branch" - } - }, - "snapshots" : [ { - "sequence-number" : 1, - "snapshot-id" : 6884618194677467995, - "timestamp-ms" : 1714753336329, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1714753288452", - "added-data-files" : "1", - "added-records" : "3214369", - "added-files-size" : "49412347", - "changed-partition-count" : "1", - "total-records" : "3214369", - "total-files-size" : "49412347", - "total-data-files" : "1", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "s3://warehouse/nyc/taxis_single/metadata/snap-6884618194677467995-1-651ea117-c6b3-4853-b4f3-09aa36b77f19.avro", - "schema-id" : 0 - } ], - "statistics" : [ ], - "snapshot-log" : [ { - "timestamp-ms" : 1714753336329, - "snapshot-id" : 6884618194677467995 - } ], - "metadata-log" : [ ] -} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/651ea117-c6b3-4853-b4f3-09aa36b77f19-m0.avro b/extensions/iceberg/src/test/resources/warehouse/nyc/taxis_single/metadata/651ea117-c6b3-4853-b4f3-09aa36b77f19-m0.avro deleted file mode 100644 index 6d16a630d8b3f161c58fdc7843793b2d6884bcaf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8189 zcmb_heQXrh5x38-aZ9kVf($9N*=$T}wP$*}3;s&2m|!A|fY1=r6yj{&?Y*-X@7->9 z_Y8-zYD21+s7W!Aa6!;wH6%7lt(LZFtbQQ@tre0|DWU3z+NNz9RMVh}KT@@h)Xv+z z-M4qXTkg)504aO#&Bt$kGjC?{NO)iA^1BSF7st@C6$7l9)G*u18m69*Og6~czck1Z zwlx%FqcT<^2HU!KfF)nJAR9+9a(*|CM^wGD18(T}%L!S>@N*O?1`awR6*b<&4hHR; zynEBsuqJA<)R)l22r{uLgTUNU$j^p}Vwx*TNEa2+RZWdXi<7n9m8BuW!0{fei;|j% z6C`r^6;Tsm1-s^Ka7CE93@*uFZCvtmDMLV`E8tOpLsBEy7uJm1P3~>ORCO$(ctniu zhrbHy@o@s;EqlV#>`K|$wM~_fDXVc`E7F47;@<9fr10Au@GQkD4JBhZZi*Hub4MUe zIY&tuMFfCYVe#WAHSqycM?RosK+jPo#!Mud=s*E?Df3KVO~FQcSf?tAQMdrgIyt84 zYCpDsHxjzki}aoX=*v+CnyQJE!cC#%lb~ckwgdUZVl1INM6#}_x+z9+KAv5@R7pj2 z&c=~u^s1%^wbi+>myIVB1^&|_d3TVF;3!HcCfiEBr!-oUIIkqA!5FHJ7*E8)ST91F z;E@)U6)YMH7P;Od5-L3siCovToFMBwoS6?SaKi-H-TSPKY($mdIof3ZXfSG&F?%sX zG_^9JHljAjaBZ4u$b?o*1S{CYMv&p#z(1tOX43iQXtlu=yw+QQXvplP+EWD-;$s=$ zls%^F(N1Q+Z1yq_?_w-I5szv3xlqW1iX}Iy>M`W;D?{?b;0j}(VF+(p8N$^dqv{g* zhtZG@u_yPvTD;_d6QzGzT1iA zD0qyNEQ>CT0mq4W=eQS6FBv3NNyMN)!Eq|pNn00Lr0a-O2-(CjXrSEhnI)x<}5uqby?3N>fiWCwV0plSiA_)sxWwPcz_aZNGK0~H@y z090207z8FsN+JSH7DEa*1AQ5@1MTp~&TAyOC0apUL;!h?0GO8I?rY@FHQrlM?#MNs zFA6{lDN8tT(C)%>GmZ0B(m0AQI#@z8fF-mP!8T!D68N!AXekmb^3;=j1zs^0YHoS7 zGuW4Gg12}m>VBAf!hktUdrrYwKuzNt3D25}fR1m>ov2wVGv8Dcc9y@4<*zSD{05IT z2~AsEkVrKXim7J2w@_pxV-}Ev;>wxjZ$2P-*$!1ijA0XzE@2*%Slc-Fz1bX>^rU7r zW^j*j?lECS4g1DL@Kz&3gEmft9e{z&{MqZgkkAm4(I?gQ3G^}CMZ}s2W_`#pM|SRV zBzRjJr?-U8z5t8tjyBz(MrzPzr(u=U7&KY30q?Q(Xfd4~M2nAYUK$%@FoLlLxSVb7 zAV#Tax1G+btt)Fr!y}7F=`} zFn@5$O8{L6SW`ifQs^?nJ8@A1yw1-D)ry{H-6v3^==hag*y4|4ZdgJzoE9{xK`e(I z8RL`RU`91~c1SQ-i0(+1pN@P-G^&q=$z?xeS9WbaVOoUojM_vQoH3YXH_NG?_T=C{ zs8J=9P&UgmMznlXcH z{qnXtNZ7wp6(rYS;3{aeqCU`NmL-=C0aQVLK7gPcKJbuOi{pd;%;P%cS@mWcZwmB` zhpf%z86?<3J)1X-xNZn^<(XyfD9{T5h?K(%0YbNMFYY)cxPI|;tDG?q6zCHV+vOZS z@qnG?QjjoS-Y zopAZy?y6MRRY{t;lAj(w5?*`w+bj3IXKp`#aL0pzpH)iX-@p2*^uL$Be&&yJ_uR8; z)dyVd>b?KCIriz=bJ2IlYG0nc(e!@z}!;i;SPq#e(le_+U@P|g((rcstxp3%( z4{k60!L~h@>fScyKKH`R(*wK11GNp&69;~Df9LF_)lZ(>^p)MAt8H_U^ZzQi0U|76zlCC`22cNYWfZ`;e4jJy-5`8;#Dx}$Qa^1kKw1=!2&PnVAz3)Jw; zLscF1LzQcnj|bSd+skJvDrzelE1s9;lCP~gFjRN%@};3lZX`6qZ3%7RWt9ZNlv5|^|Xpt&a;zvpeT~+SH9pZg2u|K=8&tX2!9dAA86l&iH-ayf^cG z-@Mh?J9YXD?)%<_K(|O2PtM&@=&(nWrs5Ob^9^NPadDS@?_$?BNkBEzb_hH+ZQ>Y# z(%iX|ThYVRf)z=ifcmz(3*QY-hd9?f9aGzL(R=S+MFX2!=*DPBkFC4)F5 zm$_P(5CQoxB(6?0H|%zZFY?Zmq?N2f@hUX1h?{|;0S*v048nFEH8rl1T9sH`8|t$V z@0vG&LxR%9DHb{*?{o@PlTcAATHxYdV0kptknF~i6|5`N!ZhB)3`*r3u^o##N) zy|AoZue0JH=%znJ=`yKMl`SbQ`N$3^9CvN8&{vY>Gos6Fh!2IaVDQV)5t@~GzV5y^ z$kifK_P7{qSy3%i`3~?}%ni!RP#Rb6U0Q{?VbcPXMwL6qg@CC=d}((SH*_2YR4BQ` z*!S@eeulf>1?l6@r?t|rRvPIM&qySb1#G(k+*uYljoa}&hhP^T#2R_s4?(<4gxz7R zW){1naF-kh@jOunN*%(!7!7CLGl+f~w(ej`&ZOE1q*rT=##T*j zsM}k$dL2I7+toGTbTQrN6F-2twz_#P72}O~2CS-UwoYJV>?TwH=a=|NhG8A*Z0yB; zAlghNGwhSuoc9IE)c0Xb*gi~?CWIpVh%ey+*!hVmYM#R9e+efeR;7MGAT~LIUo6Q9 zltAQRUhE7wK&^9t@B|BlUB)8_Se3)lo(p$S0?ibxwS-#`)EcK2`-DZv3;G5Qtz6{L zBFC@@vbBJN5WKA^UcrDbfy$>P?=ctlR@}6l|75#h<2ru^kkTHycSYaak1k5 zWTtgVr7~wR=QQ{ZRMC50sinj7B1!H*5Jb1r8Z8Y-%TpA;B2LN5JJi$!&tA7o(=VmBh_8R<@)!aSo;hGs^erP++lW(<2`~Jgw+RaDL z9>0Hdtc}|DK3LNp9X*(Q{_OGi=IC(L9zFHUy~Br=Wj!A|Ul_;x9lUouHb&<0{_Uef z3m&->)B1Y6|K)@Bjr;$7Gr4>BQ~TFnKKpn4TEG9tuF<*O`2MR;&Yu5qC6~tkhmNrt diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet new file mode 100644 index 00000000000..04d259decea --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:9749ae2922aa9d21b7e779142d6c2476d0444c2c24f7e93397e6750147180724 +size 176970 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet new file mode 100644 index 00000000000..206e1f84a6c --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:d5007c70d0e8be33ef012c48d7f067b47812b22747c145cfa4bab4bef944fb0f +size 331675 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet new file mode 100644 index 00000000000..6d552bf0e9a --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:c02909448106787f93e8a2550e9db360aedd9ab25c925e8a536b18096aa8ed91 +size 176482 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet new file mode 100644 index 00000000000..540e5bc3392 --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:c872a3defa86826496fbc76fa4235c858d45edb5441e196c2c812f4c8a1166e7 +size 256855 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json new file mode 100644 index 00000000000..53c1afcba7b --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json @@ -0,0 +1,91 @@ +{ + "format-version" : 2, + "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "location" : "s3://warehouse/sales/sales_multi", + "last-sequence-number" : 1, + "last-updated-ms" : 1716234545155, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-20T19:49:04.669537174Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 2001582482032951248, + "refs" : { + "main" : { + "snapshot-id" : 2001582482032951248, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545155, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18266", + "added-files-size" : "176970", + "changed-partition-count" : "1", + "total-records" : "18266", + "total-files-size" : "176970", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1716234545155, + "snapshot-id" : 2001582482032951248 + } ], + "metadata-log" : [ ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json new file mode 100644 index 00000000000..5aa1996ce92 --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json @@ -0,0 +1,118 @@ +{ + "format-version" : 2, + "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "location" : "s3://warehouse/sales/sales_multi", + "last-sequence-number" : 2, + "last-updated-ms" : 1716234545529, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-20T19:49:04.669537174Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 8325605756612719366, + "refs" : { + "main" : { + "snapshot-id" : 8325605756612719366, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545155, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18266", + "added-files-size" : "176970", + "changed-partition-count" : "1", + "total-records" : "18266", + "total-files-size" : "176970", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 8325605756612719366, + "parent-snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545529, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "36107", + "added-files-size" : "331675", + "changed-partition-count" : "1", + "total-records" : "54373", + "total-files-size" : "508645", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1716234545155, + "snapshot-id" : 2001582482032951248 + }, { + "timestamp-ms" : 1716234545529, + "snapshot-id" : 8325605756612719366 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1716234545155, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json new file mode 100644 index 00000000000..1be00194c4e --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json @@ -0,0 +1,145 @@ +{ + "format-version" : 2, + "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "location" : "s3://warehouse/sales/sales_multi", + "last-sequence-number" : 3, + "last-updated-ms" : 1716234545865, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-20T19:49:04.669537174Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 3247344357341484163, + "refs" : { + "main" : { + "snapshot-id" : 3247344357341484163, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545155, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18266", + "added-files-size" : "176970", + "changed-partition-count" : "1", + "total-records" : "18266", + "total-files-size" : "176970", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 8325605756612719366, + "parent-snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545529, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "36107", + "added-files-size" : "331675", + "changed-partition-count" : "1", + "total-records" : "54373", + "total-files-size" : "508645", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro", + "schema-id" : 0 + }, { + "sequence-number" : 3, + "snapshot-id" : 3247344357341484163, + "parent-snapshot-id" : 8325605756612719366, + "timestamp-ms" : 1716234545865, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18230", + "added-files-size" : "176482", + "changed-partition-count" : "1", + "total-records" : "72603", + "total-files-size" : "685127", + "total-data-files" : "3", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1716234545155, + "snapshot-id" : 2001582482032951248 + }, { + "timestamp-ms" : 1716234545529, + "snapshot-id" : 8325605756612719366 + }, { + "timestamp-ms" : 1716234545865, + "snapshot-id" : 3247344357341484163 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1716234545155, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json" + }, { + "timestamp-ms" : 1716234545529, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json new file mode 100644 index 00000000000..16cca0f3f8d --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json @@ -0,0 +1,172 @@ +{ + "format-version" : 2, + "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "location" : "s3://warehouse/sales/sales_multi", + "last-sequence-number" : 4, + "last-updated-ms" : 1716234546189, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-20T19:49:04.669537174Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 1792185872197984875, + "refs" : { + "main" : { + "snapshot-id" : 1792185872197984875, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545155, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18266", + "added-files-size" : "176970", + "changed-partition-count" : "1", + "total-records" : "18266", + "total-files-size" : "176970", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 8325605756612719366, + "parent-snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545529, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "36107", + "added-files-size" : "331675", + "changed-partition-count" : "1", + "total-records" : "54373", + "total-files-size" : "508645", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro", + "schema-id" : 0 + }, { + "sequence-number" : 3, + "snapshot-id" : 3247344357341484163, + "parent-snapshot-id" : 8325605756612719366, + "timestamp-ms" : 1716234545865, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18230", + "added-files-size" : "176482", + "changed-partition-count" : "1", + "total-records" : "72603", + "total-files-size" : "685127", + "total-data-files" : "3", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro", + "schema-id" : 0 + }, { + "sequence-number" : 4, + "snapshot-id" : 1792185872197984875, + "parent-snapshot-id" : 3247344357341484163, + "timestamp-ms" : 1716234546189, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "27397", + "added-files-size" : "256855", + "changed-partition-count" : "1", + "total-records" : "100000", + "total-files-size" : "941982", + "total-data-files" : "4", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1716234545155, + "snapshot-id" : 2001582482032951248 + }, { + "timestamp-ms" : 1716234545529, + "snapshot-id" : 8325605756612719366 + }, { + "timestamp-ms" : 1716234545865, + "snapshot-id" : 3247344357341484163 + }, { + "timestamp-ms" : 1716234546189, + "snapshot-id" : 1792185872197984875 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1716234545155, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json" + }, { + "timestamp-ms" : 1716234545529, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json" + }, { + "timestamp-ms" : 1716234545865, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..d871fbb9fd098d297903c3e1dcd7f46abe821c6e GIT binary patch literal 6976 zcmb_gYlsz99N)|}iIBzTu5NW0bV0nEcV^aoT#&?6ShaO=6_%D|I(P2e-D&5}TxaI4 zuCkOG#uiv8$|_tockK@-d(sX?94g; z$M66A&5q;?4>dgI>BB|_U#UMJ`D0l_N=TmX=5$|*Nx_>|GbJVBF)3{tR?3qS0|zAX zM2<-|&XDsvjUm&q;k;|?$(gPJFVooajF=$Mw1=eeSa4HeH@kf!qwOMEtG=wV%e!sU z_q0C8N=4i@ZJ+6FE4Zrlx~3kbIOXJ$mceeXW4Cv>DZ|w^VL$4&Z)Ob7$C<3ZUm71A zUy4Uv=cxeDW9t~yuAw_Fu|E%G2HR#D?rDbYyJPTN06l*Mg!(xTUge`jJj|a9B1K>h z&)PWa4LiOD0Shk-NOsP$;9tvum<-0Gl##|c%a;=5xhP2%B|&HnTQ@X2mq{9K4boIW zTH3S>&6}~vHbG~N3aswfKETfIO98SeM~57LCV1%~Cyo4J0})LL z6kku&#*n-|>v*OQRe(~4W%z~{L&|#a4_l@`7CsBL(lE)YXgviai~V7)9jcHN4aA2U zu-(xyRj_+!H`;6Z!)Qw%qWMHTrrrc{<8c8MDmU%887%l!BzX(Cf`T)Mh7zr`|K`D`PLm3(y}N4;}0cg&=|{@^e|shy@;El*FEqLI5wp zo~c7#*3gT*R|M~a>e6&4Cx(^geWw5=zk49-IT3(=Nh=N1v$Mig+J>C8$;P1sR?5%o6`BxRjoO)i_A zU64pOQ)}sFir6TMim?nxYHjU|`dbZ1VzxsU(K3dQ$(B%(Nm5`On>U+n-5tx)y$SB2 z@E%H9PBJnss@RRZEUa;wlTLevUw!t56A~7}4D6Nptxx5{u#brKF^~+Xu~2s9cBG1{ zO}JaaW~Bo{@dIhOJN4kzb(0A;k*m7$?6#pE>s>-jWh8q|9CcG2>SYc{AM`!pofa zmJ+9`1aSIGSwW3bIC&Qt7dJq3z7$j%J>eUI%8lamtGuw;r(+?qa-4^gLX#(A73?UY zPf^5-Yv5l?s0K#Ks?)KhN{Yty@i4LMhw3V?Eh0>_K+m{MJcEmfS$VU<{u%J!XrjiI z@PrDoyojPzqpBih<61gd{TGonF2A6Z8N`GR#Uz+T!8njR=s)m*A!yBX)aGA-j$ra8 zLL`E>Tm`8$AY1`JRZ=VNLl^Ub%H<@0E2z!~5JXn{GCn9!ShM3pH0NPX$-I)w(VJ;{ zra;wJ@(c=Wrk<6M5$1*pX_22VWq8Lly-*>MDtMtn=w|N416cy|OW_|;iwMCqeNx&h z`lJAMp-)NE#`H5PebCC4Jg{=a{hgc7%f4t;=aQ=`8KrW`s6xrqZoar#GxJqGgdiWp zCAxz75;GuZF_GIeSHovd@#0)T59^MW>5=b@g$pW-jr;>>G5Jp^;R!CVAfjb9Yi=IT z!J;La3o9SQverQ&a_b8BYc8+AxacjL+1bWhS@A;61s52apCLi43KwfwmEqUSb^*Ex z&m9dC1t@9uB6R{#6aSyhS0n{llD z!|#4ryCb&so6py__I$s#=j-m)BX9h1`Q5Hnr|z$wST%8f9(rr!?U6H+lV>dZ*gGR< z)AN5mzr6qUbNwynrp`6BG_{;MJ+Sip>83l&Zy(yVVrt3CrAtnpJNbKk=e+tOSB`AI zdF^86&3TKKU4J@#^6_7-HGjK%+CK0;zSQzn-QtJ0UaXT>G4 zsj+c#va@l6)O_^QQxma`Jrmt09<}e?TWvL?7qjz+Kl|&+`&Tx8ad6w!t9QS>d(-Lv z<_wjK8mFH#r2w(!rz&DUSO*uUx8&g*^K79M};iPWuz-bc*(UvBKXG3Ull|81YT Pclnu)w^U!KuC-_=G?LA@|%FYk@nh|ln@=M*)M7kK|oo1Wt{NrBJn$jG^TW^gZ$ zpNIl)ss((0D;m}<6V5wmS5bEmyv(bHiv&iXZVvO~f`3yAZ}xboplrih>%J_9m$#d` z=PCo1k&C&ln_j55z3i&8#nH7m#W|~(HBfjv72e+L?K+h2mRpqhFf?kULhoH2OLfL=NRLcO93uS(Iv9+uAek-{(s zXHC_1M=Vc)fCU!@d9!F3@ULM(OojwLhw^IC@c0aV&PozlNf26OYDh7Qg)DL!kR~(I z^16W(cgiB$8Id{FQxz;XG_5Acjs#~afz>S21K7j+ypL?o(jdpz`Y&D1%5&Zb;;^O+ zm+Z!B1y1a?Em!xT3b-6HkcV7>le)n_)zH1M;8~!Rgh`ZH>j@yX>WxtCP=&Z?AU?!^ zUQ1Jng6&&-xZS!p!fhDfNIqeYi8sF7WRgLJ$jw_$L1p}kNZtUhaQ+z%=S_ygxhim$ zqv1cCtvV3<@+1?JY;wQ}k9Z$iUmJTdS%&`5c;MicKnRC1<-DS8Ai?)g<+7@aavZ>m zu@BX8u8p*a_Y&j1UtNl371^+oybrYYlXS*W^$)w8?kmB_MMPI-u)YT=x~XKxAmX9# zeqjX)wppHPkaXc(IF7{!$JwB<$)H(Av0y6rIF%ZdZHTPuI4bTCI`mNJ%Hi{9OW7%k zoyw>vrUP&M7GCNQI}w&qqm{r-2F+2`DAMW*ykwjZ)6nQ}Um4al)gWy>!xvN=`d@%X zpK4>QeGfT3$9Wd~FhT?zTtB+I6D+FTl#=PFO@4r<2dXZBN=h~YRaew3ifbw37^tM= z6rdvcEI=k{MllCHg2RoQQ_URLulC1d#~O)mDI}=V2%scl0Lw}|oJR4vCb1pmfw?A0 z4FO1yBElgD{jN|u(?qtDnyNYNV999!OYUkQ+vJKOh|4y)t3j~vsK@mRdBqXb!uIG_ zaBQ;4Z1XUz-LNxd0drU(gN8GOo+fqRJnL)#I;o>}p;qb4QfEWh5q~-2Z!Absim|3> z+tUjY?q+f$-ArN|MWh(3fFw88&WOMDfW&4ybP=V1JQZ&V6`92Q#)b1{y{S24Hrbot z9v9r>vWAt7jf>27BiDvCPOnMQr~1e*(BXh$1$2y(U!c#F`(kf#iI-Dmw~Uz4tL40z> zI%@hP0d|2;S>060XH@c_jVgIy<*@rZHJ_G!WL2k<>na(kcFD+0$uw?0yIF_k%XA1q zKCnx44fFZX0Kdh=Zqrl^ojuXTxrUywI~ux+zcZFEs4zB451_@AKBWXFsKBy_R@tnn zc{B$jOEeW$J&0AUgG8j(mG9S7UYT+6TXtw?J8fmrg_;U3Gqf~A{8*JQ*03tWuQl8S z$R<2{aA;v^>za#~xu)0uxO?=^{$mrreDuRRw-R^I%=cz%*$dY{AH99$oBRJRSdvH_ zOtvf?ym{-$ik9Q~-;T7L{C)9?8y~i;y>$1diFXh6-hb<>L+i$8HD69$O`T858{0RZ zyVfW5_fMpHq)pfQ+Rye*q&CXEslJZn*<`AHB6WGr>5glwHVRJ*eQm!!V$Yv9zqsJ$ zjxWD3KYs49#aAEAAAWxH+{%a3Y4gmkLRz@kv}o3*Pn*Q0&2E#pxcRmHOI}R;m`Eo+ zOC*jSO($0Ktp~q8c0gFuf1u~>dFH))D~;ASSHEJ<9l7|=6aQUU^XR3m=>$*Ues+*&uOlQbGEln z80{Zjf`=VzwGZgn+7N0-Q!IzrUw|@)O+5qmWX<%P5qQplUN{0my}S#r3eh4S7S8#R zA}|MMO>Db^mM25Nf(yF@GjAC1uVFz<`eTBsWpLi`gcNzsOG@&RAhepPXtJ5lr8TDp zX%Z(bqZ^v+PFrLXCsM^8mWkZZw2B}*BAl5FtYVoSz#iTgd}LKifgIoHzjTq6LEfN- zh^7=utRre;D7ntIT-}2zK&obFp614oxDNcohVG37&jPJ9Oi78io&sWHZ;)w+DkMb% z@u3EES_-BLwr^`kdv$LRZRtfcpNPlQ8((fB!J$IsW-KR%IloFIZvj`3e+H4fNg5Wy6AdE3xpzK0m4v8$;F z;3e3H>X2(|N{RO(=e=KDvSQ`=u+qE_w06;SCQoJ_U`-uNxN&>^xKkx`+Qz)cR#VQl1Cbp>8>P9$5Q(UHC~pexv*Z9OIAunqk$ zKw~Sm3D&-cNLP_(!H*zB!6EgdyIHWPa#KpQmfGY8cxIq#1E|DAEl{Te`S;#$s{ zylu}cNTi!dwRAI)ZxkiPSOz4iwsuDStp+4M+o6ldInBdlOPG{Nf^S?nZ#J8XGh)-d z3GSib9!eWlIx;R2-;G=w);QV9WL(XwK6`@+2@7Ek_R7N6C$VAJOT>B@NCwncAiHuq zlK9mo*ezkRpMpjHMq6x9Gc#y&u&^pt4wfw4g?UVe7HZ!_v_#m(r?G1}fEFc5<#-1% zA`DP3oP{M52o>mvRMF(;7#pHw0xY7mO0iT;sR!!};AD;dDjOfx92XYyI#CGs#W>j) zAr5tkMxcJs+Xf1Z?AEGqeX5T90v!%0RzSxX`33sSxG(XRlz1^^c8iD^yIRbf>D~}r zX2f?gajHrHqra3D)F_3Kcb;)E19;~rgG!^veM69#QH*|-7dG>BEI>AyVBw_D6o^;_ zJ4)zN6ft8O*w+%OA*5v0>DW>wMPvF{n8NIb>ME}-B1|(t&zMatgG-26d9#B4>GR)c zqQ;c4gz~ezgrZfWsv>1$T3W;UFClA8eo-lh5ED8S6JQqk;~?2W|A7w-eru+qw(tsc z_>(skAmP7dDoCvX;qn2hl3Fnz+Cm?wTt)(zg6ez#L1eWrkC`ge6q~V+A#js-FZt_Y*=|So}GW^-z#;e<|yf*ufH7n`{X~*t=I&6 zQX`Hve|7uLn(kP~_czzXyM9^IHPIe_=kSBeAGa+(_HgB)<%b^5L#MMBvguP}$B$%A zesV%pht6fk2KSuF>T3GK6X{b%=Gc+rMjGSu+0zTQ<<7r;ct_uk<2$b{{k>sP!=KOm zm8t*v{g2Lc?(ST0^o=E-eZFkx^&1b~{UCem#TVwCZs-}Cd+^r-M`s_re4yd^iScXW zH^#4ye>*;Y{rc5ePyb=Ze&|#eIvt%C`t0w1?8f)@%z60s#`V&<|9-l&xTAh|-I#Fc zzWEKFd-1jTtGC{MvTNeySjR-K-MQ*kdg0@*Z0(M>ch>DY{NBd(%g!!7+k5ZT1w9v) U-@bVE{@vM^Uiws-H3^~r0UDGY{{R30 literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..462894acbdef4351207c59d84392e303f1c0d9eb GIT binary patch literal 6978 zcmb_gTWl0n7={W+kf2dOG<`TsXcgJQ&J2{cNh68Sh{4iO8iCSG&(5B0r#m~d%*;Xx zX$WeJJZPE*YlWmKL@q`j026O-Dj`VV1)?S1*_m^8yE|;RO*TzB zbIyPH{_8he69cm!S?KEhB!fSjdqDCAvqWlS2sErdX(;C3RKlB^J(AJ3Q>_(WR>R9( zmf^Wtr)?%9ZX1>t>RnrKRqJpJJxX!X&LvC|-d-2p-s&WYqiw`q)NRkm5ZA+*thZkp z>KR&!_d51!AJCC?A=D1hZHL;QhcbgLBL(*~VtLLWJQqOE9|56W&V^U`Xi*RI=ln=f zn1izx&bs}!r$NAi3*C~HGfnu{v>_%vF)2w>IA?lN3wt$Di|Gy2wr;ub&{Q zsRhMbsoEHlTeG%ncu)l>NlfAqH-?l}@DH1YHyAt%w6ZYCs%Sj}B#XU%t{tk777fIQ z8L-9HF;mdBZ8O?uc>SoY6R~`v9y4!zx$(Gw3X_|%oeUQIDw5m=t|0#mqIuJTXs*VP z?dbFm%3=p%UzlWaTucr);W6(+>&s&=#|zN!844Wi2!tStDe`hzlf--vF-l;UBoV+% zu@BWDH%s&)?-jv&zq&Nt&WT}Vc^_zPXX#9%>K}HI;VYrY6^Wjl!Nz`~8J3n9gouZ} z`-K%K*lv5+Wa)xjI8Ma}$HkzD$)MY2E@Nr*IF}lfZHO#(985ce0X-DDa`?QsrKDwf zy|!1Dn*wkA7M|}AX+mX`X(ez|KyxoPbG*6&F9j#$CTMiDuk;x@Hd$M5kuo?7{Vzb{ zMVzHr`yL`Ai98#A6d?uI}OKk$M)aEL(O`TK(QQ4+8R|ys!^|W3guMk5mY>$2g zM<$ypHV@O@2Rl;&FozW~XgJg8X-Xr_v-MR#r!oq zW@i>8+RfBzx|t$2ilSmH0g_r>J7fM<0+N{R&_%Qi@i5&ICS{W38yC)-O_uHqX4&2Z z_fT*TB}_XJ85dRTMs60?IL%I_T;f%py}^Wpg)jqqWq#{Z`7rFHVm%Bb18OXgUA`Tu z;%XD@may4R!6JX7Ei|Z+8?-4{SamxCOP1lnJZ3`+ZQe+=MA#;#v1>bk79&Zecn38i z3{WSWg(VXR73hdmvDVKqK1AsRSj=dZZYPPp2&^-KlhyjGY{OpSxUi7dC{nmDHqd<$ z;!qQ>1?mUAtuN2WF0TsLr^?7L(BXh$1$2y)U!c#N`%-Udi5F64w}6=OtA)Io?G3?Y zPJAa5r-}q{`pZ~Bl~Oo)7a12fKy-dGs4RNIHw2X%#pzdRVKYz10%YYl4<~~rPsGaD zQAD4jh#A+wzm_l!AtkF!M_YvyjqBrK3bP-otF*R=FwFox<2LaOE+S^7%?kRb&wrzd z8dt&-%Fps5idKoLvXqT$X$tGVh^%q>1*IH9OzBWef?4E`19?6B2R<xypl`N zn<;vxK-HG>3<_+zo=qYnp&KfsMSi}N;2l%+LWM*s2Ymt0ZFDCJ8=6-uUh^To|NG+*UI z2=YN(qRW^sg$DR7CUTqRYWVCaUYyJ53A>|dxb!`^JKBb?-dythy%ja%bM(ZnKM&sf@#_2Ey?IQH z$7^w{@uO?suGt!U;p(L|4eejAX}_|$;kDu0pPp>4d*|-T!*z%6&O;OF57GnU1IZ&} z=JE9S!1#%D0-F=Z#*U7>o9-VO8B30ijvX0EoJvnTv@LVyrQsdhcO2dI#p5@YEnfEf zljq6m>qkC3v1Rv`J7YUX&z<|e`|`!xhu*ri;jw2Q=veslS2Y*R4OeDg*u7!lqp4op ztKaCok?8GB-0YooINP+eum1e}=DPZ>XFmDqm(yqW#ApBa`SOV;t{?pM)`GeRS0Czl z+B|8Vx@u2pm1uK#V`Q>hh8>waF@bZQAX_xh_RPn{lJQ9rx&(#7Qyi>@uW_TKF? YbI)|G{P&+P-9HZPI{uA3YZ5~L11%RLd;kCd literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro new file mode 100644 index 0000000000000000000000000000000000000000..c6e3e85459ba2cbb6885352e7aaef3756e017fef GIT binary patch literal 4399 zcmbVPZERCj7*;?|1|0C zxzF>S=ly!RE!ul?`3l&lDm9vrKuNf-YJk)GQ;2KeR3s>BjPr0ZOq$=5P!{7z(<5_UlMYXi43!m1M3Z4kODMW4A?}N1PMBHI6HvGB zFb|8@m&IaMiWU(RBVVRkO#KLd$&i<*ur-D;3|Q8#8z&I8PfZ zFzyDcX6B1GTWD2DXT|E$*_OaOmP-MiEqbCrK~SPVUR*X=jsOK=9Rc#9viTyNZ;l7M zOS?JQkR%R|olMt4RfYZdXWZ;bJnP8wxx-NnctcK2yJn&}2cRfxxJgZ52REuH5`r>* zU=PJk)xgs#4xrp`kFSK;l(=~$36MBJAC!!_L#LjpKuTsoyYIv~oI4AVLOBO#48@@# z>0E;ulY3Bvm=Gk~{%RzOJBkicM^(s|aYY3XU)$eYQn)$k%)P+R3p-)ej4FYvDt z1%!7*PiD=Jts#|+x3eLbd(Jih)@8h|?Vef2*+w9<=2g5GL=~@(1`LHriI=eU&HAxf z_-q!k@bmDvhE-kzKq+bo1!*j1_2(gBw|8zUKnVCCA)qFp2-s3g0VN9%GAHNM4&)(q z2^KEF9#Gag!kXI{6vJGkQ6QvVA0f?U5sIM<7hxGFpMMUY+brbaV+$qQJWz0&H?lNK z#wSaZ^(T~qCdt{KB{MktG$#R1JIPLnYdz=D$5nh;)`A4j(_|BUN`@!;Fm`}4(DXX; zluY2rlMCNV#{ii#rqQ6f#G#~-aY^Q}giRzqHfOhx2aE?HlbiV`W>3+2Qn}O-*!F|6 zyJodx4GKAfTy}&i8p=Uy84=3rx-)Skg=lja63U8~RpuY6+k50C6=l4e%|&ee9WBO$ zP%^ro+4Y|Im5i_&29VFLt~*GduIk`4by&Yu0-O-`ef5+}P za%r&f_`vQb-kdnye&CwacA)*t%6(nqEl*wn{MlCY`p_$1F6?UCF?{g-vR|Ki$G2zr z@}r}ZQ|aE{ggXPbEL-~ck=Df{>$Y$1PBj^S+;jKl7uqjfUGwm<6Mg?&|NCdn_^5L7 zlisW9W%+@NhtD)VSk^IBb@1yYBZn3p4qtEjxct7+hc;a}`)AZ>UR`aJO?~jpU3;?x F`VSgC?eqWu literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro new file mode 100644 index 0000000000000000000000000000000000000000..28d439215a04848e617058919ca10091fdd5fbd9 GIT binary patch literal 4224 zcmbVPO^X~w7*6&e3bPR=vP1~6Y$OMnk?z@@?Cd}eVie=3^oEHNc7fV-9j)8e^`k}^+atk& zitdF9_KlEX5AgsIbw41;4bc9W5t@ga0UHYN5J$}A@mO?7Yzvr0;BLS~)F%9hu3tkj zB@SXC8bvOfjJKu0+F`&*@_v>D73{V2M~cEeVgrKm)OL}6)xiumka5)^9w|j`Y$_C# zydRUmCRPyp`y`Ta7gAzor%-wdjRqtrkfIT$2s;kpH&2S1X31zPT)qqWS%_~Z96^(y z{BTYSl~8XwL3Kr;NzaukySg zC}cE7`7tF&`i25Y?zogerzZn46ZQ4%Y@UEhRJC4W^7(p5r?jU9vFB;9cCx9(QH00vGYq>A zyG}fxT#iv&@8msQ*ok}(U^k#Jq#QWM-48>L-~b+^OYue&!?x-p9E{WTHQ+-D!{d2K z#EA~E7f&voEu{k`Spr?Wla_Pdo=1v#eP} z0iP}ckB?dNkVF)OY^8N6-@eUs>8}_UU7LVL9Aq;8-3y|VL7iip>s)F8lFR4?biyk3bjP2;`#+P(-x&60*fnZrMZA* zkbD~xd_}WR!KXG#**r*aSvMh?OXCX>RsD&ikVVS%b7_WLPp1+Xa+1=56y&)gKZWKi zSql-oA&V{YvoyTOk75TTgDh{Jo}~%$^i0Fo;x`+vpWKy4seq(8}YU5F`77~}&3p_^}^bCUmufGHh$9;)n z8=mH_?aa%6yjfrW{o^lcKm7V3diMKZ<(IR)A8&fz*WXxQ-TeH^&#YVbA6WNKPj1y- zd_=eDTdgP5+^(DT?fR4RKi5voz1@fL>OIqByZ0XMvXjQ@`M7BwAMfq$G*3RQ9W%Q6 ncm1bN{_I81UpPHo`{%cR@BH=P+eeRgYPH_um1izLRSMug$_SL1 literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro new file mode 100644 index 0000000000000000000000000000000000000000..8774b62b1e731a45959b78d041f4d8503027c302 GIT binary patch literal 4353 zcmbVPTWB0r7;X|!u@YLW@xc&J25qfwWM}VLY-lm5Bm^zl1S%!2Co^ZWqq8&Hnb~aD zG*~Rb3zllYdJ94yN<@lQr6Eug>Vpp=RKa@5gH+Q{DaA;Oih_z~Zs(lY-PwyT3p;z} z`~LIazHc9!c&eqn15Rn`gSwDKIe4UNnlnmy#3eZm392S?0ZxHA^Lq{|QU>WpIwQ#l zKh8)<7IiLpV7lT&9vWGE!U56^O;RTDcTp8E&af&#LsAtmHZll`l92^_Q@~;;s6CR% zC8NRMOrR25M#?0ME~^aosA@nNm;nM^dIkYW0TU$y>2A0VFwX$zp=L;CJZU|TgfwO* zflG=(qE6u-!C*fq=tu;H3W}O!l*b!lUVi*MW_QPipadVRa84m ze3aqJNjN_Z@eQj*+$6v~?9$>&us1EEiZN7HC}~}Rc|EHdo`QHU=A1CAVq~FV-=PLe zFkHc6c8Z=B)Pi57_LzAAxR6>2xW||r1tSb3Iwa`^?sRKFYN8G}Cu;z+$+ufUtdHAo z^-QgJD-*4$#fn%xdA3tv9;=oCo^5)Pfx4h11HQOyvD^gIg>@6)i^|rEOuo4u>?u9K zDFs>P@Y->v7HS$S;h*ua=kTs0&&!vi77Zrcp7!iSa}PjC(eaR)z%K5Ds>%o|_<_9? zN3{apRv7@5lD)pN=1}6{k!3*Q1XEBhlrNoLrUNON1?{~P=W>qMBZW#8P7!5bK{mLg z8IyNWB$yCnJpNjG0#6iOq;6I@dlwV_eeUW7)?2`O3uIIcKzwa$Hq8Y26SD$Q$mjnD zvGrE-#QeLZ2+vx1U>5Rh;gM6aAaP?x*%y-K9U#;krr=4_RapWS@=p8BKST$ zo@M2004PO>P>{i5q~C{xRqyInfDrI;LO@MG1F+4Q0!kJoWNyu=9q=LbBrMg0J)o>H z!dl%JG{W4VQ6QwjFd?mG5gMUPHDDPiU!;uBYZiR?*hZ-|4-}l%jfiH`_(Vime?lo} zkzD<3n!(l6ssucpB)cFUdG^VVr}-+Z1qq&~#U}Y_8lL2bu>+KWme);B(*$mMs^JIe z8X$YdG#WH787S*yU6Oq)V-tywm1zrkz<3a{xtV`r_7?3Wol6~oZ9gcxXIHz{pq?|R zW=Cj76IEy}BSKj{cP5Uc9&HsvLRqm`IsZ_--ouwvQm{6gLTvpVZN!97GOABrztJ?A zzwz)`3yN#j3wsVb=o$tCzkUyHIMbJ~wjmJ?MPtEeJQ|Dfp*Y_gj>R^u3~d>G&-mc# z>}`iPFB~0Sy6Jq|YhNx8we8uq`=`%8TF)t&pD)On_m9u*-S|6j>*KAf#}+%fXZP=a zVXXh!8y9}tIdJC29V6doU)}N5z*hO|`9Ch7o<7z$cVm9xsu+5DPChjM)}^lRK0Y{o z|Fw(S`iWC#SJvKbz3*bn@QW?;p{=iUK5#j5``Era+xp&q^vc;S|NOg_`+ez?OX=gG z?!FV_p%42u1FIlQ4PzW(cS`?(+1 zpMUA0(eCZS$y;_lcm3NVJ3iYzD7<`cY4p!$`7gRw7SAnwv$i_*vxr3`9C*j B+3)}W literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro b/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro new file mode 100644 index 0000000000000000000000000000000000000000..b5659a5c9b9c1352e4e14c33433464d2c3786a47 GIT binary patch literal 4302 zcmbVPTWB0r7>=Qh8Zc0!q1e(M2GO=AWoELsECk!6sRQT9xoh4M};)L7MI8(zs;7D*axCmQhA7E|v`wk;i2N znYt_G7gt+O)F7^q6Csd`9m858-*sCfI7e&^V#Bt;@uwC*-M|%axBz^1hB{&BQa&xq zjqz4&6In|lx@im8$7~EuKm!D>@e%@t1xhOzxlyPtmO*tw@+z;D&`+ z)CKY*SU3bc7wG`opzavBIbKzO)ocqR|NSrvBG_#3qYLaZz!e0d)aHRatwRjQfikTl z6SYLHWCaQrz33rJLq*H0mXO2aj-{j!o&v!sP_G~>MhfcC1yI)!y}4i1AWKSG;HnxV zKRxjsvFoHsKy)~wC6y3w+C-HRsQgfht_5pu#m2FM#4i?}pjE*Yh=V(HV38+VSZJrX zMa}k-Ds_1sHbB)k?sL#SS}e-3LN2B2*SvJ5gbw6nK6Ly}Qp9(`_G$kFEJ`a26Z{08*s@AA0GvW) z6l|~r=}#aL)w{hFU<68*5pWaG18hI0fRm*eSyXdw2NFnQ2`ltr4>)Uvv9>n`y)gG^ z6d0*I$w=E-gkC5MJy-_LmuljRn}q~Eu~Ayh0|)1IW1`tNJ`+*VpKuCZq)0#eW{C8( zErG%(DJV!xo)hvDYrYn1!Gc$Ku~~llhG+Q^>;Pxr<&DzQH$jx1cK9-118mPwqrvl1 zhNjEbCELd)F_Gk0leVx2f(K!nTlgmyY|$y!x!e&1_Jgy>c6DS8x;cY(c7$g%*M`h1OCPk(fF`}aYBp#*Ob%Cl7y|+B-b)@}1v)2O~f4vCgjJ_0=b~eK>dbqhG)E z;GYBg?ZzvGq0y@sp84_FlNT?Xe`DkJt<*a=PJN{=zWv$orE8O3Lu-4E-*YK@W$v|i)tTq&^-re%ZG6zkz5eR#t@~#dHm|K|k7u5q|D*K6fy&+k pkL^DA)0rKk`>r?c?D*mH`I|St|NG+RulvruH~sC=e}}W6qOc?7^ZTF%WkqW9eZ|W zGc%j8X-p##85tpJ+LYX0(mteZB-9A0rWKV)BsUncRM3K;>Wh0*8lY6|Ls9&I_|KV{ zbIy);*Xy;VC{kw6`7hsp{Z0l_V{5-~r>kZ49GY%^O!SI*T}+Cu=M*$gjEVj)aU(4z z6EQJk=w{j#laD?o;xAH6v``M8Kd6rywgu-M{cyo>boeoYOjnO_0u5_aoQU~1WqPyE z({t(&)>`*vgv$m%~ZiE*e6|I74!aq$L z!ha|xru7Uen4Xx#-+4(AFA1_pw=`X~3b~YyVe-}2FHPa3WeihS-4%;$=R~HFhg2+= znpPELM}{+tfz=?A06V=e`pBkj4a#Yc|D!9|8Ntiy0@joi61%b5m>})W+pgh3!3k;I z)IHse3G#0651EEn48Mh1Nth&sx1Io!MoNaD85Km;SkhJ(L|ZMVKFXWiZpR(B82)t=#GG+D|m&xsmFY*k&r^J zo)#d`I18xbf}7X1NE~D?4t|}hnqA-%j8tH#wVxCg&M*J4D;T~K9K(p{#kFJ{(^bP# zQ$@%<8U??^Lj^;&hfI>Vf(ys7`0zNNK73kgwpqwoDn8Dnh9ys9j2s8yUS+`GfKf!h zN1NNYBz3AsB&jR(#-Ce3uNv2}j3D`=O$}~xXdXdkfz^5FCFg|HMZ#&?qlSjeA%BAA zP+pwC$wcaDKgwgQeGdgAEqFHk!3Ys>a3ve=I9OD@Jtn%MHu(Ww9jLkiDml>zRNYat z7_LFaaZt&L6+lJuIR=@enT0g;6akMA0aV=oE7vmxdZ1}X9Lj59n}l9LT8pc8^VtGD-eHc zf+X+ctSRL4s|ynDW=bR7Oy(O!q!=rJq%_vfh`;rK#AiEn5jCfK2yY3CGD-A}qw{93 zr8&hs+2G)w5Z)6~rk$!87lrReZXOm&)y`yG-K#%)!wCtiX%05rVEiK|*v6-^Yde4z zVTP3$P`F)^g zWRdQRal9`=9GVg>z;~gyjRxHCQkNu0e%mzxxF(rY}Q2=^(uO3cQg$b zzp$1s#4t9356~uqr@_nAkD>JV4R!{A0 zXRR!{z%#*Rh6Xdlk5&01534!+X`#CZ*}Ts*!T*;9YjgPU>>$g<%MhMOX<$k4a1Q}- z?SiA}WG;cfY~_cAf`5|%=l}iH&mNIKk@o-g>g=Zu2uJ<<<;%%FXQ%dtA|Jya#v@r8=?7r*I^>3Yd{PZhbpLz1(U;Xoo@7}sP_un6H z-m;}g4f(PtTs7`{uPhKbX6*?tx=J`p@oj(&g#qMWOpY D0AU9c literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro b/extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro new file mode 100644 index 0000000000000000000000000000000000000000..17af22ffd9f53d228dab8461c2eef00489774fe8 GIT binary patch literal 4251 zcmbVPPly{;9Nz9p=w=UDDHTe64$@OnW|B>^8|CBsr zu`KkaRD0-d1(D)S1TP|Zkb3fLP8Ltz6a>XnyL$Hb=FfX?GMW4_w@luA-|zkYeP7;a zzk6!+3_kGOi@wn%J$(249fc12L}@A>Fk2@BzOQSdl*P7* zvoE9Du7thkQfwm@Ao8sb2(ld19#Y~b;l{w41l-3SwOBkYzfX)7F!R7Ihw`X(_z_*b zf&!nIh`MOtS#&(!iUe!84kh9HaTa8-SEC<&U zXcCkhPH3SL@=eF6>Jn8LO3Qa}-|xCKRgm<>;uEYYw2NtUhbb&-HN(O>#cvsIkXNb8 zEH4NO=?zhGOv;hEE1mzjX#g&VdBw!|Nl7L)Pxn6wo*?Nek^rqqjwynV0iKiBO9v;Ha zFzh|pb^Q7Ga`bBIM$*%%oyhh8wj3XZlmREWZP&F4cHlv@6!*LUwp9mVXBe%oE*nZ1 z9@|DdPIQ3nV0`IpN*yT3Lg@6Jl!WuvEK=-D;511G2R2ojET;6L*pNc7Vf?*T8zxFD z(j==&w2L|ao5|`$;aezt3w2x%VYqfUn=C>3#IisH{r>+T4&PdwSUbFRSW;lxdaxQ| zQQ0`;7t@1Y55vY_S0UKwigZrBkJdH4vVF_Z(qGojSq1@(*pFrY`xkj9gE|KW)49k1gqKlM z>K==6)(FHh?}08@9>_-%pa^gA1!SM~PsE~SS-9Zm@T8Vit^p7f6;8niOOpN^5?Q?` zTLDg>H8_DV0dv4EVhRLVjguuc7j__rG?g%Y0ec`=>zsA6F_?#WPNTp{)hZ{QWD(|} z)aS4af^U6{FKrfb_~b^(ng;c&NLVSFy4vOf_NqDYB;F3gbV=|loeOj1;klsxC; zC)IozYr%upM6r2(7KZ2fk?eqA5amtMvoJxDo=NzsSOa{|Fry*z(!sXR*CpS_Hke2_ zHl{87f#gB>=9d16MO$=}cdl>*k^K5|gIUgCk{uBl-Izcd8WF*ox-)kqvuGz6 z62S^$75_s`dyiaF%K>d>LLB}boyUX-GNDgtzp*e`zVS#|3xUh(1)d`cdWyk-*I$N) zW4=VP4bL&xwsiWh#~(hpeDd9Uzw9>mF4P|`A1(j-<8|Bq@|)Hd*B^fVpmpQX*R4mN z9p6}a>G#vckFQm>D$kzn>Lf4cYdt52({s_i`f;qUH~r~m%B_QKm|&#sPsJ9Y0{c&y%j#n``l S^v=b`M}I6`tEx+x*!}|((4WKr literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet b/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet new file mode 100644 index 00000000000..6394c55b6b2 --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:bd1139eaf58ad06a1aaaf2d9d8505225d4dd7ac9e7a40f8f4b324ec364e76de5 +size 729342 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json b/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json new file mode 100644 index 00000000000..0e56acb0f0d --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json @@ -0,0 +1,91 @@ +{ + "format-version" : 2, + "table-uuid" : "f1d2074d-58b0-4087-8edb-8d85f1472553", + "location" : "s3://warehouse/sales/sales_single", + "last-sequence-number" : 1, + "last-updated-ms" : 1716234544074, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-05-20T19:49:02.681248048Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 5481576066981634597, + "refs" : { + "main" : { + "snapshot-id" : 5481576066981634597, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 5481576066981634597, + "timestamp-ms" : 1716234544074, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "100000", + "added-files-size" : "729342", + "changed-partition-count" : "1", + "total-records" : "100000", + "total-files-size" : "729342", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1716234544074, + "snapshot-id" : 5481576066981634597 + } ], + "metadata-log" : [ ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro b/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..3a70db60f6c201e993ad9e8019a795ae71a756d5 GIT binary patch literal 6982 zcmb_gZEVzJ9G4vijj$LHCOYx49}I?@UfaFgE-qVuEkhB;2D%JpX?wk`Tdyr`Z)A`e zV(=3&3iyE--6t_fm8kgZ*&nc)Lm*o66O?sBgq>@}t*Nm*oWp*Fr z@Do1CnQ|VV@7G3j%Y^fewy&T&8obQOhN~qRfx0=uO(p%CLU?nFr{$%cSZmdn#qjbj zQ}Vp3W}kHw_C#7+nucDNSkFZ?zX4rHP@5#ws(M= z8lGAsPdL_#KA^{1LZ}^0wH$1J5z4%5>N&V4X{P5)!gB`n;t>$)6QNz3USdu ze24*CEmbB8cJ*&T`*m*=^$Z}APuOGPjW0KqVo)J+bC#2r8NVWud%zXspFucpG6?5t z5?PLl{~%j-Aoit6#;4fifD<0^KD53v_I#=Y{o$#=!M;EU!k8kjU>jP}_fSTP>}pvA z@M7#kb;z|fHR8R%c<)!2q*?_wtR(LPt=mXCzZDU%TK z(09ME0tGuQPc}%pAQz5f@xgI6Xlyd5mQl!?50BH1SujMa?tMzl{1ZJJE5U2C&5T8nR8CQv`9@CbriI79RDuULmg#K`m^L zeg(%So5(f~!`cr!lLDB-3K=w<1@ttb73W!74bTa#l?$~@XBOIO!jAZhh`+HQ2`!8@ z1>0U+kZ?B>Yw2bJ+bAN%SOz4qwsuDRtp+4E+o6j{dCimYmM|xiIN!K%-gKL)Gij5( z3GSib9#RZTiH(cMb|cq@HBPc}Ial+l&)#4{!a|sby|TFViF6naV6mPIBm-(JkX^YQ ziR@|jW;`7n|_D z2yv)OH3Id6-ZoNXWLGwX>r-{)7wB+6u>v|q$uH1n%6+l7xWr2-vs*&U=+#o*O!kK0 zG9|uqiBnYqDE%d@phhW_yt9mp8o)X~7gQ2G<{N@YjiU6cys*WmV*#>!iiVRwQzT*) z?1<2(IATUM(61##LrBT0)6r8UMWgy?n8NIb>ME}-CQOS!&!|l_gCoSOyjem2^!aZ* zQKL#|Lit%9p=i~psz}+WmULME5wb?*my~h{F{VRF4rY-*4)`|m4}4(oTQeE8#aE!i zpS+m>3I8osfoly2mk&^t)QbAh9{ND!QW8KFRObT-!mE859|S0@#qlAY^Mp?Ew35rw zn|XRBK-E_A3<_+ap3Na6p&KHkMRC5A;T`kzLWD%B;DrdGTeuewWC@{P0{w^@Aq4aE zNocL;lK|KSJ}J5>lh3H+K@(N-z{+9wcWOQ@`^c(JC0A84LgkVXnUbm9e0H-A%@^qq zf_z|?=nCd@p#gr2iQT5D8ajKTi*p4%VRtli7k_6gT~J|c6dyo~DSk=`PEdg*5iPS> zQ}bvJMwVzQtb7p5S_g?ptt;KHsk{>7;F(kB;@0&DUgjR!F!j1N*)TJ{`n!`~zw+icnYvGxs!HbOKIiX0)*gKR+TQHm zz3HC@4ddh)X?pO3506SiAAK%;bmI8X^5_0MBHq2>_TL*=`RdAhE4sR7duLOZ(?8DK zn(3XLdAWVp><>4tZfv`JG2K0T3thgRzL~oD`}LW=*{+69e(O7f&Ye2jeE#rD7x?w5 znU>oZlGB~nj&9wv_5PhN&t6Smy7JY?^wOVC|N6zAwtrTvJ#?{aat+EQlzMchPI*6- zcr5XBVp*agkyy2AS)%vwrE_WVX!kEK@SpX6-_+Up&bx`*4fRW|o%nB)^v3#j`_U%{ z*A2b(?}`nB=QrB=G1FFhlpV_S+v?DcHAi{}#}15bQC40*ZZy{Yg-)Z7pL%cT(9qqw SXFEFYuRFDR?3u(Il>P^)2O69J literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro b/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro new file mode 100644 index 0000000000000000000000000000000000000000..a7720ef0bd70d2af15a81796bd8d1e8b200849bf GIT binary patch literal 4227 zcmbVPO>Y}T7;bXnkVe9w5~QjOE5yl0yK(#xOFbY8Qd9>}oT92xWsP^o@mBj`c4lKt z6p3>o2UHFLMZIx>12@DkfCI^eUOBb=0-zEXB;MJNnOU#b9~`}&eV%vTkLP{%gWiLc z^=tSj@ZX~5fVlXx8^;&0%H1srIj@gyZpC{J-bgR>-KGjz~jD69E)nLgX7$Z zA!Y+OkpNK^*xn)hwtN$C+P;aI?R)6X-VPesY=GYDpom?dR%}aa)pY%&oX2*Ecc`K} zzJk5)GwdK9AfooC1lb@PW&fDejl*`L1}7RNWW=ehRevfX%UCyA~zZe1tlMc z#4}0P3*8LkbW8s}byQUu&RqAb?7X*b2 z#wb0e2SWSN13f6qwTT#!w0Y5e^ z1BTl4A_H?lMFt9S)ncUym9XkO?z8puvEwKxdy82*M~cR}mK z^GR_GYI-y6>5L|F9l*9nVMsY}ire#jhhPsL#G<$#grHV^guQVrUjsgrFg%WfM4add zJK?10Y%3io$s*|NowStm{yb9bP2sdiABPUpT0Ew#P&Abg92ozg+k=UcLYih(i?vwr zf0#-yirzxeTd3~`2*b6b+2jc-CRP_jI2`^D;^?jAiH(!{CuI$$%>`*lmP+$PT&xqi zE(XQmS0UIO$aK!UkKO>-m!oZL`W7+Q;pz@%BV{fVTIMN0SQ(hTW7ol0QHNs0@SS6gwapWO>u{EKQK6XBxgPWkBc*HySc8ee6&nmqL#nFp+R$cM`T8uQ)nY2B3Uzc7LH^d?G!^ISz%cv zU#MB{QAlcgu$%cJjy^{hF(Hyn?x(EZSemTZcx0@F#O3t@&k+Yb!(hPcZ$QIwU!vHC z7r1M?@~qVOe*dF8ul)WZ`RB*o2Y;+ST{-yWuH$_BL-+CBufF-Rd++?a?)llHduy*h z`@H(sZ*=cD?c6SPO1DeTPfshmFFk3mg_rG4neDDUS!0!pi(Pj0^zw`L-o@H!`Ozh# pSN|^k`tZ-=;MI+@v-iXE^M6M_Ki;Wse7&);e(`kWp>ZV_!+(Rtm;C?$ literal 0 HcmV?d00001 diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java index 6c4e57fb34b..300e2095d1d 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java @@ -18,7 +18,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Map; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -148,7 +147,7 @@ public enum ParquetFileLayout { *
  • A single parquet {@value ParquetUtils#COMMON_METADATA_FILE_NAME} file * */ - METADATA_PARTITIONED; + METADATA_PARTITIONED } private static final boolean DEFAULT_GENERATE_METADATA_FILES = false; @@ -246,61 +245,16 @@ public abstract ParquetInstructions withTableDefinitionAndLayout(final TableDefi */ public abstract String baseNameForPartitionedParquetData(); - public ParquetInstructions withColumnRenameMap(final Map columnRenameMap) { - // TODO: This conversion is fragile and must be updated with any change to the ParquetInstructions class. It - // would be preferred to have a more robust Immutable implementation with better copy support. - - final ParquetInstructions.Builder builder = new Builder(); - // Add all the existing column mappings. - columnRenameMap.forEach(builder::addColumnNameMapping); - - // Add all the other parameters. - builder.setCompressionCodecName(getCompressionCodecName()); - builder.setMaximumDictionaryKeys(getMaximumDictionaryKeys()); - builder.setMaximumDictionarySize(getMaximumDictionarySize()); - builder.setIsLegacyParquet(isLegacyParquet()); - builder.setTargetPageSize(getTargetPageSize()); - builder.setIsRefreshing(isRefreshing()); - builder.setSpecialInstructions(getSpecialInstructions()); - builder.setGenerateMetadataFiles(generateMetadataFiles()); - builder.setBaseNameForPartitionedParquetData(baseNameForPartitionedParquetData()); - - return builder.build(); - } - - public ParquetInstructions withSpecialInstructions(final Object specialInstructions) { - // TODO: This conversion is fragile and must be updated with any change to the ParquetInstructions class. It - // would be preferred to have a more robust Immutable implementation with better copy support. - - // This version of the builder brings in all the existing column mappings from `this`. - final ParquetInstructions.Builder builder = new Builder(this); - - // Add the special instructions. - builder.setSpecialInstructions(specialInstructions); - - // Add all the other parameters. - builder.setCompressionCodecName(getCompressionCodecName()); - builder.setMaximumDictionaryKeys(getMaximumDictionaryKeys()); - builder.setMaximumDictionarySize(getMaximumDictionarySize()); - builder.setIsLegacyParquet(isLegacyParquet()); - builder.setTargetPageSize(getTargetPageSize()); - builder.setIsRefreshing(isRefreshing()); - builder.setGenerateMetadataFiles(generateMetadataFiles()); - builder.setBaseNameForPartitionedParquetData(baseNameForPartitionedParquetData()); - - return builder.build(); - } - @VisibleForTesting public static boolean sameColumnNamesAndCodecMappings(final ParquetInstructions i1, final ParquetInstructions i2) { if (i1 == EMPTY) { if (i2 == EMPTY) { return true; } - return ((ReadOnly) i2).columnNameToInstructions.size() == 0; + return ((ReadOnly) i2).columnNameToInstructions.isEmpty(); } if (i2 == EMPTY) { - return ((ReadOnly) i1).columnNameToInstructions.size() == 0; + return ((ReadOnly) i1).columnNameToInstructions.isEmpty(); } return ReadOnly.sameCodecMappings((ReadOnly) i1, (ReadOnly) i2); } @@ -753,7 +707,7 @@ public Builder(final ParquetInstructions parquetInstructions) { } private void newColumnNameToInstructionsMap() { - columnNameToInstructions = new KeyedObjectHashMap<>(new KeyedObjectKey.Basic() { + columnNameToInstructions = new KeyedObjectHashMap<>(new KeyedObjectKey.Basic<>() { @Override public String getKey(@NotNull final ColumnInstructions value) { return value.getColumnName(); @@ -763,7 +717,7 @@ public String getKey(@NotNull final ColumnInstructions value) { private void newParquetColumnNameToInstructionsMap() { parquetColumnNameToInstructions = - new KeyedObjectHashMap<>(new KeyedObjectKey.Basic() { + new KeyedObjectHashMap<>(new KeyedObjectKey.Basic<>() { @Override public String getKey(@NotNull final ColumnInstructions value) { return value.getParquetColumnName(); From 039010286be94ef2d5ef69f6a62d2b7e2c142e9a Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 22 May 2024 09:47:26 -0700 Subject: [PATCH 15/25] Gradle cleanup. --- extensions/iceberg/build.gradle | 5 ----- 1 file changed, 5 deletions(-) diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle index cb6e5166fcb..5b764989e2c 100644 --- a/extensions/iceberg/build.gradle +++ b/extensions/iceberg/build.gradle @@ -11,20 +11,16 @@ ext { dependencies { api project(':engine-api') - api project(':engine-stringset') api project(':engine-table') implementation project(':engine-base') implementation project(':log-factory') implementation project(':Configuration') - implementation depTrove3 implementation platform('software.amazon.awssdk:bom:2.23.19') implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:aws-crt-client' - compileOnly depAnnotations - Classpaths.inheritAutoService(project) Classpaths.inheritImmutables(project) @@ -51,7 +47,6 @@ dependencies { implementation "org.apache.iceberg:iceberg-bundled-guava" runtimeOnly "org.apache.iceberg:iceberg-aws-bundle" - // ??? BOM? implementation "org.apache.hadoop:hadoop-common:${hadoopVersion}" implementation "org.apache.hadoop:hadoop-hdfs-client:${hadoopVersion}" From 4539bee48f76f148ff3f0cec2e22f0eb5a3ed7eb Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 28 May 2024 08:50:14 -0700 Subject: [PATCH 16/25] Simplified Iceberg instructions. --- .../iceberg/layout/IcebergBaseLayout.java | 9 ++- .../iceberg/util/IcebergCatalogAdapter.java | 65 ++++++++-------- .../iceberg/util/IcebergInstructions.java | 20 +---- .../deephaven/iceberg/util/IcebergTools.java | 35 +-------- .../IcebergToolsTest.java | 77 ++++++++----------- 5 files changed, 76 insertions(+), 130 deletions(-) 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 e97e8938873..93df4a30106 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 @@ -66,15 +66,16 @@ protected IcebergTableLocationKey locationKey( if (format == org.apache.iceberg.FileFormat.PARQUET) { if (parquetInstructions == null) { // Start with user-supplied instructions (if provided). - final ParquetInstructions.Builder builder = instructions.parquetInstructions().isPresent() - ? new ParquetInstructions.Builder(instructions.parquetInstructions().get()) - : new ParquetInstructions.Builder(); + final ParquetInstructions.Builder builder = new ParquetInstructions.Builder(); - if (instructions.columnRenameMap() != null) { + // Add any column rename mappings. + if (!instructions.columnRenameMap().isEmpty()) { for (Map.Entry entry : instructions.columnRenameMap().entrySet()) { builder.addColumnNameMapping(entry.getKey(), entry.getValue()); } } + + // Add the S3 instructions. if (instructions.s3Instructions().isPresent()) { builder.setSpecialInstructions(instructions.s3Instructions().get()); } 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 3a5882d0a91..a8a4434c541 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 @@ -31,6 +31,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.time.Instant; import java.time.LocalDateTime; @@ -40,50 +41,37 @@ public class IcebergCatalogAdapter { private final Catalog catalog; private final FileIO fileIO; - private final IcebergInstructions instructions; /** - * Construct an IcebergCatalogAdapter given a set of configurable instructions. + * Construct an IcebergCatalogAdapter from a catalog and file IO. */ - @SuppressWarnings("unused") IcebergCatalogAdapter( @NotNull final Catalog catalog, @NotNull final FileIO fileIO) { - this(catalog, fileIO, IcebergInstructions.builder().build()); - } - - /** - * Construct an IcebergCatalogAdapter given a set of configurable instructions. - */ - IcebergCatalogAdapter( - @NotNull final Catalog catalog, - @NotNull final FileIO fileIO, - @NotNull final IcebergInstructions instructions) { this.catalog = catalog; this.fileIO = fileIO; - this.instructions = instructions; } static TableDefinition fromSchema( - final Schema schema, - final PartitionSpec partitionSpec, - final IcebergInstructions instructions) { - final Map renameMap = instructions.columnRenameMap(); + @NotNull final Schema schema, + @NotNull final PartitionSpec partitionSpec, + @Nullable final TableDefinition tableDefinition, + @NotNull final Map columnRenameMap) { - final Set columnNames = instructions.tableDefinition().isPresent() - ? new HashSet<>(instructions.tableDefinition().get().getColumnNames()) + final Set columnNames = tableDefinition != null + ? new HashSet<>(tableDefinition.getColumnNames()) : null; final Set partitionNames = partitionSpec.fields().stream() .map(PartitionField::name) - .map(colName -> renameMap.getOrDefault(colName, colName)) + .map(colName -> columnRenameMap.getOrDefault(colName, colName)) .collect(Collectors.toSet()); final List> columns = new ArrayList<>(); for (final Types.NestedField field : schema.columns()) { - final String name = renameMap.getOrDefault(field.name(), field.name()); + final String name = columnRenameMap.getOrDefault(field.name(), field.name()); // Skip columns that are not in the provided table definition. if (columnNames != null && !columnNames.contains(name)) { continue; @@ -102,7 +90,7 @@ static TableDefinition fromSchema( return TableDefinition.of(columns); } - static io.deephaven.qst.type.Type convertPrimitiveType(final Type icebergType) { + static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type icebergType) { final Type.TypeID typeId = icebergType.typeId(); switch (typeId) { case BOOLEAN: @@ -137,7 +125,7 @@ static io.deephaven.qst.type.Type convertPrimitiveType(final Type icebergType } @SuppressWarnings("unused") - public List listTables(final Namespace namespace) { + public List listTables(@NotNull final Namespace namespace) { // TODO: have this return a Deephaven Table of table identifiers return catalog.listTables(namespace); } @@ -152,11 +140,14 @@ public List listTableSnapshots(@NotNull final TableIdentifier tableIdentif * Read the latest static snapshot of a table from the Iceberg catalog. * * @param tableIdentifier The table identifier to load + * @param instructions The instructions for customizations while reading * @return The loaded table */ @SuppressWarnings("unused") - public Table snapshotTable(@NotNull final TableIdentifier tableIdentifier) { - return readTableInternal(tableIdentifier, -1, false); + public Table snapshotTable( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final IcebergInstructions instructions) { + return readTableInternal(tableIdentifier, -1, false, instructions); } /** @@ -164,19 +155,22 @@ public Table snapshotTable(@NotNull final TableIdentifier tableIdentifier) { * * @param tableIdentifier The table identifier to load * @param snapshotId The snapshot ID to load + * @param instructions The instructions for customizations while reading * @return The loaded table */ @SuppressWarnings("unused") public Table snapshotTable( @NotNull final TableIdentifier tableIdentifier, - final long snapshotId) { - return readTableInternal(tableIdentifier, snapshotId, false); + final long snapshotId, + @NotNull final IcebergInstructions instructions) { + return readTableInternal(tableIdentifier, snapshotId, false, instructions); } private Table readTableInternal( @NotNull final TableIdentifier tableIdentifier, final long snapshotId, - final boolean isRefreshing) { + final boolean isRefreshing, + @NotNull final IcebergInstructions instructions) { // Load the table from the catalog final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); @@ -193,13 +187,17 @@ private Table readTableInternal( // Load the partitioning schema final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); - // Load the table-definition from the snapshot schema. - final TableDefinition icebergTableDef = fromSchema(schema, partitionSpec, instructions); + // Get the user supplied table definition and column rename map (if any). + final TableDefinition userTableDef = instructions.tableDefinition().orElse(null); + + // Get the table definition from the schema (potentially limited by the user supplied table definition and + // applying column renames). + final TableDefinition icebergTableDef = + fromSchema(schema, partitionSpec, userTableDef, instructions.columnRenameMap()); // If the user supplied a table definition, make sure it's fully compatible. final TableDefinition tableDef; - if (instructions.tableDefinition().isPresent()) { - final TableDefinition userTableDef = instructions.tableDefinition().get(); + if (userTableDef != null) { tableDef = icebergTableDef.checkMutualCompatibility(userTableDef); } else { // Use the snapshot schema as the table definition. @@ -215,6 +213,7 @@ private Table readTableInternal( // Create the flat layout location key finder keyFinder = new IcebergFlatLayout(tableDef, table, snapshot, fileIO, instructions); } else { + // Get the partitioning columns (applying column renames). final String[] partitionColumns = partitionSpec.fields().stream() .map(PartitionField::name) .map(colName -> instructions.columnRenameMap().getOrDefault(colName, colName)) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 6bb4c17f084..ee1bbd9c138 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -6,8 +6,6 @@ import io.deephaven.annotations.BuildableStyle; import io.deephaven.engine.table.TableDefinition; import io.deephaven.extensions.s3.S3Instructions; -import io.deephaven.parquet.table.ParquetInstructions; -import org.immutables.value.Value.Check; import org.immutables.value.Value.Immutable; import java.util.Map; @@ -30,16 +28,10 @@ public static Builder builder() { public abstract Optional tableDefinition(); /** - * The {@link S3Instructions} to use for reading the Iceberg data files. This is mutually exclusive with - * {@link #parquetInstructions()} which can be used to provide custom instructions for reading the data files. + * The {@link S3Instructions} to use for reading the Iceberg data files. */ public abstract Optional s3Instructions(); - /** - * The {@link ParquetInstructions} to use for reading the Iceberg data files. - */ - public abstract Optional parquetInstructions(); - /** * The {@link Map} to use for reading the Iceberg data files. */ @@ -52,9 +44,6 @@ public interface Builder { @SuppressWarnings("unused") Builder s3Instructions(S3Instructions s3Instructions); - @SuppressWarnings("unused") - Builder parquetInstructions(ParquetInstructions parquetInstructions); - @SuppressWarnings("unused") Builder putColumnRenameMap(String key, String value); @@ -63,11 +52,4 @@ public interface Builder { IcebergInstructions build(); } - - @Check - final void checkInstructions() { - if (s3Instructions().isPresent() && parquetInstructions().isPresent()) { - throw new IllegalArgumentException("Only one of s3Instructions or parquetInstructions may be provided"); - } - } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java index 60ad1ac92cc..079ef5b17b9 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java @@ -3,8 +3,6 @@ // package io.deephaven.iceberg.util; -import io.deephaven.extensions.s3.Credentials; -import io.deephaven.extensions.s3.S3Instructions; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.aws.AwsClientProperties; @@ -28,9 +26,8 @@ public class IcebergTools { @SuppressWarnings("unused") public static IcebergCatalogAdapter createAdapter( final Catalog catalog, - final FileIO fileIO, - final IcebergInstructions instructions) { - return new IcebergCatalogAdapter(catalog, fileIO, instructions); + final FileIO fileIO) { + return new IcebergCatalogAdapter(catalog, fileIO); } private IcebergTools() {} @@ -43,9 +40,7 @@ public static IcebergCatalogAdapter createS3Rest( @Nullable final String region, @Nullable final String accessKeyId, @Nullable final String secretAccessKey, - @Nullable final String endpointOverride, - @Nullable final IcebergInstructions specialInstructions) { - + @Nullable final String endpointOverride) { // Set up the properties map for the Iceberg catalog final Map properties = new HashMap<>(); @@ -74,29 +69,7 @@ public static IcebergCatalogAdapter createS3Rest( final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; catalog.initialize(catalogName, properties); - // If the user did not supply custom read instructions, let's create some defaults. - final IcebergInstructions instructions = specialInstructions != null - ? specialInstructions - : buildInstructions(properties); - - return new IcebergCatalogAdapter(catalog, fileIO, instructions); + return new IcebergCatalogAdapter(catalog, fileIO); } - private static IcebergInstructions buildInstructions(final Map properties) { - final S3Instructions.Builder builder = S3Instructions.builder(); - if (properties.containsKey(S3FileIOProperties.ACCESS_KEY_ID) - && properties.containsKey(S3FileIOProperties.SECRET_ACCESS_KEY)) { - builder.credentials(Credentials.basic(properties.get(S3FileIOProperties.ACCESS_KEY_ID), - properties.get(S3FileIOProperties.SECRET_ACCESS_KEY))); - } - if (properties.containsKey(AwsClientProperties.CLIENT_REGION)) { - builder.regionName(properties.get(AwsClientProperties.CLIENT_REGION)); - } - if (properties.containsKey(S3FileIOProperties.ENDPOINT)) { - builder.endpointOverride(properties.get(S3FileIOProperties.ENDPOINT)); - } - return IcebergInstructions.builder() - .s3Instructions(builder.build()) - .build(); - } } diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index f683fbf3a50..0c3f3a0bfa8 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -13,7 +13,6 @@ import io.deephaven.iceberg.util.IcebergCatalogAdapter; import io.deephaven.iceberg.util.IcebergInstructions; import io.deephaven.iceberg.util.IcebergTools; -import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.time.DateTimeUtils; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -38,7 +37,6 @@ public abstract class IcebergToolsTest { IcebergInstructions instructions; - IcebergInstructions instructionsS3Only; public abstract S3AsyncClient s3AsyncClient(); @@ -67,15 +65,8 @@ void setUp() throws ExecutionException, InterruptedException { final S3Instructions s3Instructions = s3Instructions(S3Instructions.builder()).build(); - final ParquetInstructions parquetInstructions = ParquetInstructions.builder() - .setSpecialInstructions(s3Instructions) - .build(); - - instructionsS3Only = IcebergInstructions.builder() - .s3Instructions(s3Instructions) - .build(); instructions = IcebergInstructions.builder() - .parquetInstructions(parquetInstructions) + .s3Instructions(s3Instructions) .build(); } @@ -113,7 +104,7 @@ public void tearDown() throws ExecutionException, InterruptedException { @Test public void testListTables() { final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); @@ -128,7 +119,7 @@ public void testListTables() { @Test public void testListTableSnapshots() { final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Collection snapshots = adapter.listTableSnapshots(TableIdentifier.of("sales", "sales_multi")); @@ -146,11 +137,11 @@ public void testOpenTableA() throws ExecutionException, InterruptedException, Ti warehousePath); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -162,11 +153,11 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti warehousePath); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, instructions); Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @@ -177,11 +168,11 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti warehousePath); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructionsS3Only); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_single"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -192,11 +183,11 @@ public void testOpenTableS3Only() throws ExecutionException, InterruptedExceptio uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), warehousePath); - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructions); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -218,15 +209,15 @@ public void testOpenTableDefinition() throws ExecutionException, InterruptedExce final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructionsS3Only.s3Instructions().get()) + .s3Instructions(instructions.s3Instructions().get()) .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -245,16 +236,16 @@ public void testOpenTablePartitionTypeException() { final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructionsS3Only.s3Instructions().get()) + .s3Instructions(instructions.s3Instructions().get()) .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); try { - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); Assert.statementNeverExecuted("Expected an exception for missing columns"); } catch (final TableDefinition.IncompatibleTableDefinitionException e) { @@ -278,7 +269,7 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructionsS3Only.s3Instructions().get()) + .s3Instructions(instructions.s3Instructions().get()) .putColumnRenameMap("Region", "RegionName") .putColumnRenameMap("Item_Type", "ItemType") .putColumnRenameMap("Units_Sold", "UnitsSold") @@ -289,11 +280,11 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -312,16 +303,16 @@ public void testMissingPartitioningColumns() { final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructionsS3Only.s3Instructions().get()) + .s3Instructions(instructions.s3Instructions().get()) .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); try { - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); Assert.statementNeverExecuted("Expected an exception for missing columns"); } catch (final TableDefinition.IncompatibleTableDefinitionException e) { Assert.eqTrue(e.getMessage().startsWith("Table definition incompatibilities"), "Exception message"); @@ -334,17 +325,17 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx warehousePath); final IcebergInstructions localInstructions = IcebergInstructions.builder() - .s3Instructions(instructionsS3Only.s3Instructions().get()) + .s3Instructions(instructions.s3Instructions().get()) .putColumnRenameMap("RegionName", "Region") .putColumnRenameMap("ItemType", "Item_Type") .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -357,18 +348,18 @@ public void testOpenTableColumnRenamePartitioningColumns() warehousePath); final IcebergInstructions localInstructions = IcebergInstructions.builder() - .s3Instructions(instructionsS3Only.s3Instructions().get()) + .s3Instructions(instructions.s3Instructions().get()) .putColumnRenameMap("VendorID", "vendor_id") .putColumnRenameMap("month", "__month") .putColumnRenameMap("year", "__year") .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO, localInstructions); + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId); + final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -379,23 +370,23 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_multi").getPath()), warehousePath); - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO, instructions); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); final List snapshots = adapter.listTableSnapshots(tableId); // Verify we retrieved all the rows. - final io.deephaven.engine.table.Table table0 = adapter.snapshotTable(tableId, snapshots.get(0)); + final io.deephaven.engine.table.Table table0 = adapter.snapshotTable(tableId, snapshots.get(0), instructions); Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); - final io.deephaven.engine.table.Table table1 = adapter.snapshotTable(tableId, snapshots.get(1)); + final io.deephaven.engine.table.Table table1 = adapter.snapshotTable(tableId, snapshots.get(1), instructions); Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); - final io.deephaven.engine.table.Table table2 = adapter.snapshotTable(tableId, snapshots.get(2)); + final io.deephaven.engine.table.Table table2 = adapter.snapshotTable(tableId, snapshots.get(2), instructions); Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); - final io.deephaven.engine.table.Table table3 = adapter.snapshotTable(tableId, snapshots.get(3)); + final io.deephaven.engine.table.Table table3 = adapter.snapshotTable(tableId, snapshots.get(3), instructions); Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); } } From c5d6be18e89d892efe3b134467ebaf8169906c0c Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 30 May 2024 16:32:05 -0700 Subject: [PATCH 17/25] Addressed many PR comments. --- .../iceberg/layout/IcebergBaseLayout.java | 11 +- .../IcebergKeyValuePartitionedLayout.java | 2 + .../location/IcebergTableLocationFactory.java | 6 +- .../location/IcebergTableParquetLocation.java | 29 --- .../IcebergTableParquetLocationKey.java | 2 +- .../iceberg/util/IcebergCatalogAdapter.java | 183 ++++++++++++++---- .../iceberg/util/IcebergInstructions.java | 3 +- .../IcebergToolsTest.java | 95 ++++++--- .../TestCatalog/IcebergTestCatalog.java | 35 +++- ...48746c-d0a8-4697-8a8b-cdde507ffd02.parquet | 3 + ...-2420-4247-88af-3583a772d037.metadata.json | 95 +++++++++ ...-bce4-47f1-b2b8-7a4d90c1c524.metadata.json | 125 ++++++++++++ ...21fde5-e926-4c4e-a30a-010a244ca3ce-m0.avro | Bin 0 -> 7496 bytes ...-e421fde5-e926-4c4e-a30a-010a244ca3ce.avro | Bin 0 -> 4227 bytes 14 files changed, 489 insertions(+), 100 deletions(-) delete mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocation.java create mode 100644 extensions/iceberg/src/test/resources/warehouse/sample/all_types/b648746c-d0a8-4697-8a8b-cdde507ffd02.parquet create mode 100644 extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-3edeb19d-2420-4247-88af-3583a772d037.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-4dbde963-bce4-47f1-b2b8-7a4d90c1c524.metadata.json create mode 100644 extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/e421fde5-e926-4c4e-a30a-010a244ca3ce-m0.avro create mode 100644 extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/snap-5615734881085848694-1-e421fde5-e926-4c4e-a30a-010a244ca3ce.avro 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 93df4a30106..84c5b40619c 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 @@ -54,7 +54,8 @@ public abstract class IcebergBaseLayout implements TableLocationKeyFinder cache; /** - * The {@link ParquetInstructions} object that will be used to read any Parquet data files in this table. + * The {@link ParquetInstructions} object that will be used to read any Parquet data files in this table. Only + * accessed while synchronized on {@code this}. */ ParquetInstructions parquetInstructions; @@ -68,6 +69,9 @@ protected IcebergTableLocationKey locationKey( // Start with user-supplied instructions (if provided). final ParquetInstructions.Builder builder = new ParquetInstructions.Builder(); + // Add the table definition. + builder.setTableDefinition(tableDef); + // Add any column rename mappings. if (!instructions.columnRenameMap().isEmpty()) { for (Map.Entry entry : instructions.columnRenameMap().entrySet()) { @@ -76,9 +80,8 @@ protected IcebergTableLocationKey locationKey( } // Add the S3 instructions. - if (instructions.s3Instructions().isPresent()) { - builder.setSpecialInstructions(instructions.s3Instructions().get()); - } + instructions.s3Instructions().ifPresent(builder::setSpecialInstructions); + parquetInstructions = builder.build(); } return new IcebergTableParquetLocationKey(fileUri, 0, partitions, parquetInstructions); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java index 2483fbe5837..04ff6e0bb4b 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java @@ -40,6 +40,8 @@ public IcebergKeyValuePartitionedLayout( @NotNull final IcebergInstructions instructions) { super(tableDef, table, tableSnapshot, fileIO, instructions); + // + partitionColumns = tableDef.getPartitioningColumns().stream().map(ColumnDefinition::getName).toArray(String[]::new); partitionColumnTypes = Arrays.stream(partitionColumns) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java index 773df6371de..dc91d1c45fd 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java @@ -7,6 +7,9 @@ import io.deephaven.engine.table.impl.locations.TableLocation; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.location.ParquetTableLocation; +import io.deephaven.parquet.table.location.ParquetTableLocationKey; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -22,7 +25,8 @@ public TableLocation makeLocation(@NotNull final TableKey tableKey, @NotNull final IcebergTableLocationKey locationKey, @Nullable final TableDataRefreshService refreshService) { if (locationKey instanceof IcebergTableParquetLocationKey) { - return new IcebergTableParquetLocation(tableKey, locationKey); + return new ParquetTableLocation(tableKey, (ParquetTableLocationKey) locationKey, + (ParquetInstructions) locationKey.readInstructions()); } throw new UnsupportedOperationException("Unsupported location key type: " + locationKey.getClass()); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocation.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocation.java deleted file mode 100644 index 9575c3bd42c..00000000000 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocation.java +++ /dev/null @@ -1,29 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.iceberg.location; - -import io.deephaven.engine.table.impl.locations.TableKey; -import io.deephaven.engine.table.impl.locations.TableLocation; -import io.deephaven.parquet.table.ParquetInstructions; -import io.deephaven.parquet.table.location.ParquetTableLocation; -import io.deephaven.parquet.table.location.ParquetTableLocationKey; -import org.jetbrains.annotations.NotNull; - -/** - * {@link TableLocation} implementation for use with data stored in Iceberg tables in the parquet format. - */ -public class IcebergTableParquetLocation extends ParquetTableLocation { - private static final String IMPLEMENTATION_NAME = IcebergTableParquetLocation.class.getSimpleName(); - - public IcebergTableParquetLocation(@NotNull final TableKey tableKey, - @NotNull final IcebergTableLocationKey tableLocationKey) { - super(tableKey, (ParquetTableLocationKey) tableLocationKey, - (ParquetInstructions) tableLocationKey.readInstructions()); - } - - @Override - public String getImplementationName() { - return IMPLEMENTATION_NAME; - } -} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java index 6240a1113c2..e356d0ecb92 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java @@ -45,7 +45,7 @@ public String getImplementationName() { } @Override - public Object readInstructions() { + public ParquetInstructions readInstructions() { return readInstructions; } } 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 a8a4434c541..bde8aca8e45 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 @@ -3,17 +3,17 @@ // package io.deephaven.iceberg.util; -import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.PartitionAwareSourceTable; +import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import io.deephaven.iceberg.layout.IcebergFlatLayout; @@ -26,6 +26,7 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.types.Type; @@ -52,14 +53,14 @@ public class IcebergCatalogAdapter { this.fileIO = fileIO; } - static TableDefinition fromSchema( + private static TableDefinition fromSchema( @NotNull final Schema schema, @NotNull final PartitionSpec partitionSpec, @Nullable final TableDefinition tableDefinition, @NotNull final Map columnRenameMap) { final Set columnNames = tableDefinition != null - ? new HashSet<>(tableDefinition.getColumnNames()) + ? tableDefinition.getColumnNameSet() : null; final Set partitionNames = @@ -119,21 +120,141 @@ static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type ic case FIXED: // Fall through case BINARY: return io.deephaven.qst.type.Type.find(byte[].class); + case UUID: // Fall through + case STRUCT: // Fall through + case LIST: // Fall through + case MAP: // Fall through default: throw new TableDataException("Unsupported iceberg column type " + typeId.name()); } } - @SuppressWarnings("unused") + public List listNamespaces() { + return listNamespaces(Namespace.empty()); + } + + public List listNamespaces(@NotNull Namespace namespace) { + if (catalog instanceof org.apache.iceberg.catalog.SupportsNamespaces) { + final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; + return nsCatalog.listNamespaces(namespace); + } + throw new UnsupportedOperationException(String.format( + "%s does not implement org.apache.iceberg.catalog.SupportsNamespaces", catalog.getClass().getName())); + } + + public Table listNamespacesAsTable() { + return listNamespacesAsTable(Namespace.empty()); + } + + public Table listNamespacesAsTable(@NotNull Namespace namespace) { + final List namespaces = listNamespaces(); + final long size = namespaces.size(); + + // Create and return a table containing the namespaces as strings + final Map> columnSourceMap = new LinkedHashMap<>(); + + // Create the column source(s) + final WritableColumnSource namespaceColumn = + ArrayBackedColumnSource.getMemoryColumnSource(size, String.class, null); + columnSourceMap.put("namespace", namespaceColumn); + + final WritableColumnSource objectColumn = + ArrayBackedColumnSource.getMemoryColumnSource(size, Object.class, Namespace.class); + columnSourceMap.put("namespace_object", objectColumn); + + // Populate the column source(s) + for (int i = 0; i < size; i++) { + final Namespace ns = namespaces.get(i); + namespaceColumn.set(i, ns.toString()); + objectColumn.set(i, ns); + } + + // Create and return the table + return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); + } + public List listTables(@NotNull final Namespace namespace) { - // TODO: have this return a Deephaven Table of table identifiers return catalog.listTables(namespace); } - public List listTableSnapshots(@NotNull final TableIdentifier tableIdentifier) { - final List snapshotIds = new ArrayList<>(); - catalog.loadTable(tableIdentifier).snapshots().forEach(snapshot -> snapshotIds.add(snapshot.snapshotId())); - return snapshotIds; + public Table listTablesAsTable(@NotNull final Namespace namespace) { + final List tableIdentifiers = listTables(namespace); + final long size = tableIdentifiers.size(); + + // Create and return a table containing the namespaces as strings + final Map> columnSourceMap = new LinkedHashMap<>(); + + // Create the column source(s) + final WritableColumnSource namespaceColumn = + ArrayBackedColumnSource.getMemoryColumnSource(size, String.class, null); + columnSourceMap.put("namespace", namespaceColumn); + + final WritableColumnSource tableColumn = + ArrayBackedColumnSource.getMemoryColumnSource(size, String.class, null); + columnSourceMap.put("table_name", tableColumn); + + final WritableColumnSource objectColumn = + ArrayBackedColumnSource.getMemoryColumnSource(size, Object.class, TableIdentifier.class); + columnSourceMap.put("table_identifier_object", objectColumn); + + // Populate the column source(s) + for (int i = 0; i < size; i++) { + final TableIdentifier tableIdentifier = tableIdentifiers.get(i); + namespaceColumn.set(i, tableIdentifier.namespace().toString()); + tableColumn.set(i, tableIdentifier.name()); + objectColumn.set(i, tableIdentifier); + } + + // Create and return the table + return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); + } + + public List listSnapshots(@NotNull final TableIdentifier tableIdentifier) { + final List snapshots = new ArrayList<>(); + catalog.loadTable(tableIdentifier).snapshots().forEach(snapshots::add); + return snapshots; + } + + public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier) { + final List snapshots = listSnapshots(tableIdentifier); + final long size = snapshots.size(); + + // Create and return a table containing the namespaces as strings + final Map> columnSourceMap = new LinkedHashMap<>(); + + // Create the column source(s) + final WritableColumnSource idColumn = + ArrayBackedColumnSource.getMemoryColumnSource(size, Long.class, null); + columnSourceMap.put("id", idColumn); + + final WritableColumnSource timestampMsColumn = + ArrayBackedColumnSource.getMemoryColumnSource(size, Long.class, null); + columnSourceMap.put("timestamp_ms", timestampMsColumn); + + final WritableColumnSource operationColumn = + ArrayBackedColumnSource.getMemoryColumnSource(size, String.class, null); + columnSourceMap.put("operation", operationColumn); + + final WritableColumnSource summaryColumn = + ArrayBackedColumnSource.getMemoryColumnSource(size, Object.class, Map.class); + columnSourceMap.put("summary", summaryColumn); + + final WritableColumnSource objectColumn = + ArrayBackedColumnSource.getMemoryColumnSource(size, Object.class, Snapshot.class); + columnSourceMap.put("snapshot_object", objectColumn); + + // Populate the column source(s) + for (int i = 0; i < size; i++) { + final Snapshot snapshot = snapshots.get(i); + idColumn.set(i, snapshot.snapshotId()); + timestampMsColumn.set(i, snapshot.timestampMillis()); + operationColumn.set(i, snapshot.operation()); + summaryColumn.set(i, snapshot.summary()); + objectColumn.set(i, snapshot); + } + + // Create and return the table + return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); } /** @@ -144,50 +265,44 @@ public List listTableSnapshots(@NotNull final TableIdentifier tableIdentif * @return The loaded table */ @SuppressWarnings("unused") - public Table snapshotTable( + public Table readTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final IcebergInstructions instructions) { - return readTableInternal(tableIdentifier, -1, false, instructions); + return readTableInternal(tableIdentifier, null, instructions); } /** * Read a static snapshot of a table from the Iceberg catalog. * * @param tableIdentifier The table identifier to load - * @param snapshotId The snapshot ID to load + * @param tableSnapshot The {@link Snapshot snapshot} to load * @param instructions The instructions for customizations while reading * @return The loaded table */ @SuppressWarnings("unused") - public Table snapshotTable( + public Table readTable( @NotNull final TableIdentifier tableIdentifier, - final long snapshotId, + @NotNull final Snapshot tableSnapshot, @NotNull final IcebergInstructions instructions) { - return readTableInternal(tableIdentifier, snapshotId, false, instructions); + return readTableInternal(tableIdentifier, tableSnapshot, instructions); } private Table readTableInternal( @NotNull final TableIdentifier tableIdentifier, - final long snapshotId, - final boolean isRefreshing, + @Nullable final Snapshot tableSnapshot, @NotNull final IcebergInstructions instructions) { // Load the table from the catalog final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); // Do we want the latest or a specific snapshot? - final Snapshot snapshot; - if (snapshotId < 0) { - snapshot = table.currentSnapshot(); - } else { - snapshot = table.snapshot(snapshotId); - } + final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); final Schema schema = table.schemas().get(snapshot.schemaId()); // Load the partitioning schema final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); - // Get the user supplied table definition and column rename map (if any). + // Get the user supplied table definition. final TableDefinition userTableDef = instructions.tableDefinition().orElse(null); // Get the table definition from the schema (potentially limited by the user supplied table definition and @@ -198,7 +313,7 @@ private Table readTableInternal( // If the user supplied a table definition, make sure it's fully compatible. final TableDefinition tableDef; if (userTableDef != null) { - tableDef = icebergTableDef.checkMutualCompatibility(userTableDef); + tableDef = icebergTableDef.checkCompatibility(userTableDef); } else { // Use the snapshot schema as the table definition. tableDef = icebergTableDef; @@ -233,15 +348,9 @@ private Table readTableInternal( keyFinder = new IcebergKeyValuePartitionedLayout(tableDef, table, snapshot, fileIO, instructions); } - if (isRefreshing) { - refreshService = TableDataRefreshService.getSharedRefreshService(); - updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); - description = "Read refreshing iceberg table with " + keyFinder; - } else { - refreshService = null; - updateSourceRegistrar = null; - description = "Read static iceberg table with " + keyFinder; - } + refreshService = null; + updateSourceRegistrar = null; + description = "Read static iceberg table with " + keyFinder; final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>( StandaloneTableKey.getInstance(), diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index ee1bbd9c138..4513e9bd48b 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -33,7 +33,8 @@ public static Builder builder() { public abstract Optional s3Instructions(); /** - * The {@link Map} to use for reading the Iceberg data files. + * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading the Iceberg + * data files. */ public abstract Map columnRenameMap(); diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index 0c3f3a0bfa8..02b135a3274 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -3,8 +3,10 @@ // package io.deephaven.iceberg; +import gnu.trove.list.array.TLongArrayList; import io.deephaven.base.verify.Assert; 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.extensions.s3.S3Instructions; @@ -14,6 +16,7 @@ import io.deephaven.iceberg.util.IcebergInstructions; import io.deephaven.iceberg.util.IcebergTools; import io.deephaven.time.DateTimeUtils; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -34,6 +37,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; public abstract class IcebergToolsTest { IcebergInstructions instructions; @@ -101,10 +105,25 @@ public void tearDown() throws ExecutionException, InterruptedException { asyncClient.close(); } + @Test + public void testListNamespaces() { + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Collection namespaces = adapter.listNamespaces(); + final Collection namespaceNames = + namespaces.stream().map(Namespace::toString).collect(Collectors.toList()); + + Assert.eq(namespaceNames.size(), "namespaceNames.size()", 2, "2 namespace in the catalog"); + Assert.eqTrue(namespaceNames.contains("sales"), "namespaceNames.contains(sales)"); + Assert.eqTrue(namespaceNames.contains("sample"), "namespaceNames.contains(sample)"); + + final Table table = adapter.listNamespacesAsTable(); + Assert.eq(table.size(), "table.size()", 2, "2 namespace in the catalog"); + } + @Test public void testListTables() { - final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); @@ -114,21 +133,29 @@ public void testListTables() { Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_partitioned")), "tables.contains(sales_partitioned)"); Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_single")), "tables.contains(sales_single)"); + + final Table table = adapter.listTablesAsTable(ns); + Assert.eq(table.size(), "table.size()", 3, "3 tables in the namespace"); } @Test - public void testListTableSnapshots() { - final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + public void testListSnapshots() { + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final TLongArrayList snapshotIds = new TLongArrayList(); + final TableIdentifier tableIdentifier =TableIdentifier.of("sales", "sales_multi"); + adapter.listSnapshots(tableIdentifier) + .forEach(snapshot -> snapshotIds.add(snapshot.snapshotId())); - final Collection snapshots = adapter.listTableSnapshots(TableIdentifier.of("sales", "sales_multi")); + Assert.eq(snapshotIds.size(), "snapshots.size()", 4, "4 snapshots for sales/sales_multi"); - Assert.eq(snapshots.size(), "snapshots.size()", 4, "4 snapshots for sales/sales_multi"); + Assert.eqTrue(snapshotIds.contains(2001582482032951248L), "snapshots.contains(2001582482032951248)"); + Assert.eqTrue(snapshotIds.contains(8325605756612719366L), "snapshots.contains(8325605756612719366L)"); + Assert.eqTrue(snapshotIds.contains(3247344357341484163L), "snapshots.contains(3247344357341484163L)"); + Assert.eqTrue(snapshotIds.contains(1792185872197984875L), "snapshots.contains(1792185872197984875L)"); - Assert.eqTrue(snapshots.contains(2001582482032951248L), "snapshots.contains(2001582482032951248)"); - Assert.eqTrue(snapshots.contains(8325605756612719366L), "snapshots.contains(8325605756612719366L)"); - Assert.eqTrue(snapshots.contains(3247344357341484163L), "snapshots.contains(3247344357341484163L)"); - Assert.eqTrue(snapshots.contains(1792185872197984875L), "snapshots.contains(1792185872197984875L)"); + final Table table = adapter.listSnapshotsAsTable(tableIdentifier); + Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); } @Test @@ -141,7 +168,7 @@ public void testOpenTableA() throws ExecutionException, InterruptedException, Ti final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, instructions); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -157,7 +184,7 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, instructions); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @@ -172,7 +199,7 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_single"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, instructions); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -187,7 +214,7 @@ public void testOpenTableS3Only() throws ExecutionException, InterruptedExceptio final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, instructions); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -217,7 +244,7 @@ public void testOpenTableDefinition() throws ExecutionException, InterruptedExce final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -245,7 +272,7 @@ public void testOpenTablePartitionTypeException() { final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); try { - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); Assert.statementNeverExecuted("Expected an exception for missing columns"); } catch (final TableDefinition.IncompatibleTableDefinitionException e) { @@ -284,7 +311,7 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -312,7 +339,7 @@ public void testMissingPartitioningColumns() { final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); try { - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); Assert.statementNeverExecuted("Expected an exception for missing columns"); } catch (final TableDefinition.IncompatibleTableDefinitionException e) { Assert.eqTrue(e.getMessage().startsWith("Table definition incompatibilities"), "Exception message"); @@ -335,7 +362,7 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -359,7 +386,7 @@ public void testOpenTableColumnRenamePartitioningColumns() final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.snapshotTable(tableId, localInstructions); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -374,19 +401,35 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - final List snapshots = adapter.listTableSnapshots(tableId); + final List snapshots = adapter.listSnapshots(tableId); // Verify we retrieved all the rows. - final io.deephaven.engine.table.Table table0 = adapter.snapshotTable(tableId, snapshots.get(0), instructions); + final io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0), instructions); Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); - final io.deephaven.engine.table.Table table1 = adapter.snapshotTable(tableId, snapshots.get(1), instructions); + final io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1), instructions); Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); - final io.deephaven.engine.table.Table table2 = adapter.snapshotTable(tableId, snapshots.get(2), instructions); + final io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2), instructions); Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); - final io.deephaven.engine.table.Table table3 = adapter.snapshotTable(tableId, snapshots.get(3), instructions); + final io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3), instructions); Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); } + + @Test + public void testOpenAllTypesTable() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sample/all_types").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sample"); + final TableIdentifier tableId = TableIdentifier.of(ns, "all_types"); + final List snapshots = adapter.listSnapshots(tableId); + + // Verify we retrieved all the rows. + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + Assert.eq(table.size(), "table.size()", 10, "10 rows in the table"); + } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java index 2192a6263e8..e62fbd282e0 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java @@ -6,13 +6,16 @@ import org.apache.iceberg.*; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.io.FileIO; import java.io.File; import java.util.*; -public class IcebergTestCatalog implements Catalog { +public class IcebergTestCatalog implements Catalog, SupportsNamespaces { private final Map> namespaceTableMap; private final Map tableMap; @@ -70,4 +73,34 @@ public Table loadTable(TableIdentifier tableIdentifier) { } return null; } + + @Override + public void createNamespace(Namespace namespace, Map map) { + + } + + @Override + public List listNamespaces(Namespace namespace) throws NoSuchNamespaceException { + return new ArrayList<>(namespaceTableMap.keySet()); + } + + @Override + public Map loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException { + return Map.of(); + } + + @Override + public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException { + return false; + } + + @Override + public boolean setProperties(Namespace namespace, Map map) throws NoSuchNamespaceException { + return false; + } + + @Override + public boolean removeProperties(Namespace namespace, Set set) throws NoSuchNamespaceException { + return false; + } } diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/b648746c-d0a8-4697-8a8b-cdde507ffd02.parquet b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/b648746c-d0a8-4697-8a8b-cdde507ffd02.parquet new file mode 100644 index 00000000000..d726ec831f3 --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/b648746c-d0a8-4697-8a8b-cdde507ffd02.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:6c41d07c03f719700f7ef7bf4f38ec4f9d4f7ac31dc5524ae6f315fdf26d4d9e +size 3904 diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-3edeb19d-2420-4247-88af-3583a772d037.metadata.json b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-3edeb19d-2420-4247-88af-3583a772d037.metadata.json new file mode 100644 index 00000000000..7c96ddd34ef --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-3edeb19d-2420-4247-88af-3583a772d037.metadata.json @@ -0,0 +1,95 @@ +{ + "format-version" : 2, + "table-uuid" : "0eb53bb4-0f8b-403a-8386-7427eb2cd719", + "location" : "s3://warehouse/sample/all_types", + "last-sequence-number" : 0, + "last-updated-ms" : 1717109442450, + "last-column-id" : 12, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "booleanField", + "required" : false, + "type" : "boolean" + }, { + "id" : 2, + "name" : "integerField", + "required" : false, + "type" : "int" + }, { + "id" : 3, + "name" : "longField", + "required" : false, + "type" : "long" + }, { + "id" : 4, + "name" : "floatField", + "required" : false, + "type" : "float" + }, { + "id" : 5, + "name" : "doubleField", + "required" : false, + "type" : "double" + }, { + "id" : 6, + "name" : "stringField", + "required" : false, + "type" : "string" + }, { + "id" : 7, + "name" : "dateField", + "required" : false, + "type" : "date" + }, { + "id" : 8, + "name" : "timeField", + "required" : false, + "type" : "time" + }, { + "id" : 9, + "name" : "timestampField", + "required" : false, + "type" : "timestamp" + }, { + "id" : 10, + "name" : "decimalField", + "required" : false, + "type" : "decimal(9, 4)" + }, { + "id" : 11, + "name" : "fixedField", + "required" : false, + "type" : "fixed[10]" + }, { + "id" : 12, + "name" : "binaryField", + "required" : false, + "type" : "binary" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : -1, + "refs" : { }, + "snapshots" : [ ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ ], + "metadata-log" : [ ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-4dbde963-bce4-47f1-b2b8-7a4d90c1c524.metadata.json b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-4dbde963-bce4-47f1-b2b8-7a4d90c1c524.metadata.json new file mode 100644 index 00000000000..6ec810befc0 --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-4dbde963-bce4-47f1-b2b8-7a4d90c1c524.metadata.json @@ -0,0 +1,125 @@ +{ + "format-version" : 2, + "table-uuid" : "0eb53bb4-0f8b-403a-8386-7427eb2cd719", + "location" : "s3://warehouse/sample/all_types", + "last-sequence-number" : 1, + "last-updated-ms" : 1717109460647, + "last-column-id" : 12, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "booleanField", + "required" : false, + "type" : "boolean" + }, { + "id" : 2, + "name" : "integerField", + "required" : false, + "type" : "int" + }, { + "id" : 3, + "name" : "longField", + "required" : false, + "type" : "long" + }, { + "id" : 4, + "name" : "floatField", + "required" : false, + "type" : "float" + }, { + "id" : 5, + "name" : "doubleField", + "required" : false, + "type" : "double" + }, { + "id" : 6, + "name" : "stringField", + "required" : false, + "type" : "string" + }, { + "id" : 7, + "name" : "dateField", + "required" : false, + "type" : "date" + }, { + "id" : 8, + "name" : "timeField", + "required" : false, + "type" : "time" + }, { + "id" : 9, + "name" : "timestampField", + "required" : false, + "type" : "timestamp" + }, { + "id" : 10, + "name" : "decimalField", + "required" : false, + "type" : "decimal(9, 4)" + }, { + "id" : 11, + "name" : "fixedField", + "required" : false, + "type" : "fixed[10]" + }, { + "id" : 12, + "name" : "binaryField", + "required" : false, + "type" : "binary" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 5615734881085848694, + "refs" : { + "main" : { + "snapshot-id" : 5615734881085848694, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 5615734881085848694, + "timestamp-ms" : 1717109460647, + "summary" : { + "operation" : "append", + "added-data-files" : "1", + "added-records" : "10", + "added-files-size" : "3904", + "changed-partition-count" : "1", + "total-records" : "10", + "total-files-size" : "3904", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sample/all_types/metadata/snap-5615734881085848694-1-e421fde5-e926-4c4e-a30a-010a244ca3ce.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1717109460647, + "snapshot-id" : 5615734881085848694 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1717109442450, + "metadata-file" : "s3://warehouse/sample/all_types/metadata/00000-3edeb19d-2420-4247-88af-3583a772d037.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/e421fde5-e926-4c4e-a30a-010a244ca3ce-m0.avro b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/e421fde5-e926-4c4e-a30a-010a244ca3ce-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..0d7f444c9dff1b2784b8aa2e2aef76aaeda9763e GIT binary patch literal 7496 zcmb_gZHyJw8O9YGAX~F2@fR+qQ-1*7l{+*0c^izIE(lbS2y3di-kdx4+}&~K&fJ}u zyRcZ=n6$~NKTIu2f6!F=gNlICh8U|61;2>I1UDK#0)!CbM=Vuat)V|^e9t*E=bX8F zzxM7X8xrQepU-*T_q^}9EBD}49qoZNMvC~&_J_1^szkJm7KDD;3bmBRf9bOGS|**+ z3N~@_ftK0%kVe1gDb2-2DxdQ_hhTRj{T;M?GEugD0wM+M1SA!kjl{IQDQ>_J25dJZ zqr`7y0K|l{DU==09c`pZWrS+CP%Stf4jbt*A)(wOl=EIW=a5F)TuP|-3UvV4ZX7t5 z66$?IJ&(i2zClK)_6ya}E;dx9GD3AAp&Ep^SZb6u)n|gjhz<;j_>w1Iy<2!$JkcWx#wV8N zQ|d^SgWM(vT5mL~;Vc0w&fzpIvX)e@8@aV?T{%f%e=Td-y-)(W;mvcbGfb}J@<);LS; ziqL9;>>D668(7POX2QTsAe;9r=u|)FKL*Gvpm2;J>M4WL>#5rm($|-~zz%KCMS0?o zkOV1YtcU!tV~11mx7aH?Cf!t{XMmJ&I3}W}y%fqE{?7xr!?Q4RFnsrBw95|1(8leE z@rh>4vf*~qX$2K#x8V6jOi3cUv}$=Hq=L8%qP)o<%GDI|e2e~rO4x_9Pm)YetKA;FWjM^M68PJ4JCVCU0KU3 ztH;WCAA8-(=uA(Q3kS&NPAGB}re`zQ-b1pso6SwZiHqEMVZ{!1cp-KeT~Gkx)O;+i z9-2x9%X7*_H%rB()wpa0ve@@A?GScIig34z@0BfOT-W=ule*p?XTvYzs6&hsY9m2- zB`m#9L30v2Wm#Quma23=G&Mr{i_JNQUY#3kr|F&cN`62+Qli0nKHJ@}&tF>q-8 z*ugkNRJ$pq`zvGO2fQ#)4FFU|x)rDfD#MbvMvPOSGSc&ait#xGOtPGE9x55p#*MI> zM_cjM`q-&KQmrgI)CC04&?$h45*N2oeXbd5M`_44!)OUWh7>an9OPZ0Hq&&qle*Yl zaInmN0LvU`foz3a(2wQ*9oOw~P4M9AFNMK-_TV z(bJ4x%Co)}pfh@F7iyKxZ1lB+9m^N7eCvW_^eEAkOQnSciFPxym2PIJjiRC$tAJ#- z*3MYI&48q`9lA)iNJ31fgjr0|cyQvr+2C6KREf<^pod~TlykgX-E%S3ZWNSYjLUk3 zLO{ai*&E-GFc21Du8gKWQ(lJKso4<2E(kRi+pgV?Om(!0XG@su=MWL!6_Nq<%78Y+ z1FPi~VaT!rxR2STg*FdSFLh#5X&iVypylU*%hmP{8iY8Y?H~(7Cdh)1x-Pmo$K)kS zZ-5nSt+G7W_^yTM9N1e|@=>;HlK24(r7ckpaS-+aHB3EE_ zDFGnKCH3epNT7aq=$UIz~G$dyh)X<5U;18ga~FF_RwT(I;23=6p1} zqDg=9m<0Qwx~hAtb4&}cp2?UbgDV!Z>S4wGlb?9~MNK-9gknI`wL)FdnsHTgWs_dy zn^?uNChaFmSwKwJp_B%95x)*}c$cGNaVBCnyK1AKzz4s1GcgkUSLuS*8Z2BKpe9!< znS*?sV|L{dARRQ90~o52g8_xLFgeuUc|=mmTVb{JW{#N|P_?x(g94jxX0w(Nk%lRs zYpd)XbIig7BGt&kgrl2(77e!qk(VJK!7CPmIp$=*b9jxM48ShVDSCg6K6FVZ4J${T z-(~o+>?=lfX}PJAF>05Lsgz9X;j5Fi2;Y>K5MZYc(KW)?L;$?S)SafK8+rFs4$d|F zi0-Ht~l1rOb>678(kKA`|N5|1a`w!eRdZPE~ zt{u;8e)RpfW(M!yr)3`5KR$D6aKiuQu8#XRbewu)P+z(>v-{kxuRi|&?tQnO_|pRy zE`E9Is_Pf^-*oDoX;Lg*Jo3zQZD*HV*>e2Sy)VuDvi;nh>;87;g`$?I72z8I literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/snap-5615734881085848694-1-e421fde5-e926-4c4e-a30a-010a244ca3ce.avro b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/snap-5615734881085848694-1-e421fde5-e926-4c4e-a30a-010a244ca3ce.avro new file mode 100644 index 0000000000000000000000000000000000000000..3f76b00c6dd8c9b3a1cd3b8f70a7a8676b419bb5 GIT binary patch literal 4227 zcmbVPO>f*p7;YnkIQ0VRsg#k05QhZW$!@Y)sj4bUQKd?tZb2bYWoGT!*vYKD*yAiq z6^R@`>Mhws5Or4s4xlPRLfkkZfmCtgA8_J=dI9cS`i?(l#_RQda9r>BdER+Hp7-%P z&7F&vF5|txep|f}2Qvpwq(C7HTyGnG+r9-jcYF&o*Z0tScW_vpJ9r4fGBi7A;|Smv&Tr7g_{B!3UCJp%;oWzVTV{IFpI!lkBO)o@F%)` z6Gb7h5%W5hhAt+029qHF>%6GO7w!+lBnh#CON< zLX)8Ea7GK2P;WXywW?6Xp_rkEJE7yVTtV^|OHc5sFbA{b4ii}Pl>!Uzl+d*NsH{>q zcwP_`GU%b~n35yCqCk?5-H<`2#{)7E^{w>e1YkA!Nh{dXac{;wGb!F;q65EMh&7kz zBnAA~xC|I-(~AsD1r-@6#Z`-yC15ISmVi=JwO(TKxjZCOx}|v$r5eaiI<+_ma1Z{5 zVQ+)hiRYu@7*zFI*3&snV+S(uu=~Aif zi;Fd)b{m6Y@T(APIWnCS@1w5)>`U=BwtSmdm*DCYGjcf-LtuKXx@uI{Rw}icq1URl zN^Nz$as_zWm@Vy*AcRG>uzWRB-=#$K7mO>eMPNs4r)&PFS41bnc8)B*=Mn=DUq;Vu z_jDN-jX=8Q1K1000PCX;P(-x&9BZHVPr_mpS;WFG;mL_rsR57_%Ys4-ma_UwNYw2; z-U#ATKZwzK(p3x`>Qhh~`jU{I%u5T?P{^gwV+tk` zj*aXV@j&q)Lb;VMu|!4JMdwOKkk}8&p3~~g8ccHr$OMT@Q9MU&Qg}=qx5glF9v)_ZxGQl^c(owUD^HUf?;BpeGm%c>Psq zIPOan+wcN+Z5M7GefGz{&zrxz{rO){A07YvH^sWz%9v<9y?)6_U6k>P=jYFf* literal 0 HcmV?d00001 From 23e4a1820ba2fcb48d19da389c61fa45d9ef03ae Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 30 May 2024 17:37:29 -0700 Subject: [PATCH 18/25] Attempted to handle partitioning columns correctly. --- .../IcebergKeyValuePartitionedLayout.java | 59 +++++++++++++----- .../iceberg/util/IcebergCatalogAdapter.java | 19 +----- .../IcebergToolsTest.java | 62 ++++++++++++++++++- 3 files changed, 108 insertions(+), 32 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java index 04ff6e0bb4b..4889a455f28 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java @@ -22,8 +22,21 @@ * a {@link Snapshot} */ public final class IcebergKeyValuePartitionedLayout extends IcebergBaseLayout { - private final String[] partitionColumns; - private final Class[] partitionColumnTypes; + private final String[] inputPartitionColumnNames; + + private class ColumnData { + final String name; + final Class type; + final int index; + + public ColumnData(String name, Class type, int index) { + this.name = name; + this.type = type; + this.index = index; + } + } + + private final List outputPartitionColumns; /** * @param tableDef The {@link TableDefinition} that will be used for the table. @@ -37,16 +50,33 @@ public IcebergKeyValuePartitionedLayout( @NotNull final org.apache.iceberg.Table table, @NotNull final org.apache.iceberg.Snapshot tableSnapshot, @NotNull final FileIO fileIO, + @NotNull final PartitionSpec partitionSpec, @NotNull final IcebergInstructions instructions) { super(tableDef, table, tableSnapshot, fileIO, instructions); - // + // Get the list of (potentially renamed) columns on which the Iceberg table is partitioned. This will be the + // order of the values in DataFile.partition() collection. + inputPartitionColumnNames = partitionSpec.fields().stream() + .map(PartitionField::name) + .map(col -> instructions.columnRenameMap().getOrDefault(col, col)) + .toArray(String[]::new); - partitionColumns = - tableDef.getPartitioningColumns().stream().map(ColumnDefinition::getName).toArray(String[]::new); - partitionColumnTypes = Arrays.stream(partitionColumns) - .map(colName -> TypeUtils.getBoxedType(tableDef.getColumn(colName).getDataType())) - .toArray(Class[]::new); + outputPartitionColumns = new ArrayList<>(); + + // Get the list of columns in the table definition that are included in the partition columns. + final List outputCols = tableDef.getColumnNames(); + outputCols.retainAll(List.of(inputPartitionColumnNames)); + + for (String col : outputCols) { + // Is this so inefficient that it's worth it to use a map? + for (int i = 0; i < inputPartitionColumnNames.length; i++) { + if (inputPartitionColumnNames[i].equals(col)) { + outputPartitionColumns + .add(new ColumnData(col, TypeUtils.getBoxedType(tableDef.getColumn(col).getDataType()), i)); + break; + } + } + } } @Override @@ -59,14 +89,15 @@ IcebergTableLocationKey keyFromDataFile(DataFile df, URI fileUri) { final Map> partitions = new LinkedHashMap<>(); final PartitionData partitionData = (PartitionData) df.partition(); - for (int ii = 0; ii < partitionColumns.length; ++ii) { - final Object value = partitionData.get(ii); - if (value != null && !value.getClass().isAssignableFrom(partitionColumnTypes[ii])) { - throw new TableDataException("Partitioning column " + partitionColumns[ii] + for (ColumnData colData : outputPartitionColumns) { + final String colName = colData.name; + final Object value = partitionData.get(colData.index); + if (value != null && !value.getClass().isAssignableFrom(colData.type)) { + throw new TableDataException("Partitioning column " + colName + " has type " + value.getClass().getName() - + " but expected " + partitionColumnTypes[ii].getName()); + + " but expected " + colData.type.getName()); } - partitions.put(partitionColumns[ii], (Comparable) value); + partitions.put(colName, (Comparable) value); } return locationKey(df.format(), fileUri, partitions); } 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 bde8aca8e45..1e88da91f3d 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 @@ -328,24 +328,9 @@ private Table readTableInternal( // Create the flat layout location key finder keyFinder = new IcebergFlatLayout(tableDef, table, snapshot, fileIO, instructions); } else { - // Get the partitioning columns (applying column renames). - final String[] partitionColumns = partitionSpec.fields().stream() - .map(PartitionField::name) - .map(colName -> instructions.columnRenameMap().getOrDefault(colName, colName)) - .toArray(String[]::new); - - // Verify that the partitioning columns are present in the table definition. - final Map> columnDefinitionMap = tableDef.getColumnNameMap(); - final String[] missingColumns = Arrays.stream(partitionColumns) - .filter(col -> !columnDefinitionMap.containsKey(col)).toArray(String[]::new); - if (missingColumns.length > 0) { - throw new IllegalStateException( - String.format("%s:%d - Partitioning column(s) %s were not found in the table definition", - table, snapshot.snapshotId(), Arrays.toString(missingColumns))); - } - // Create the partitioning column location key finder - keyFinder = new IcebergKeyValuePartitionedLayout(tableDef, table, snapshot, fileIO, instructions); + keyFinder = new IcebergKeyValuePartitionedLayout(tableDef, table, snapshot, fileIO, partitionSpec, + instructions); } refreshService = null; diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java index 02b135a3274..f5534be7fea 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java @@ -143,7 +143,7 @@ public void testListSnapshots() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final TLongArrayList snapshotIds = new TLongArrayList(); - final TableIdentifier tableIdentifier =TableIdentifier.of("sales", "sales_multi"); + final TableIdentifier tableIdentifier = TableIdentifier.of("sales", "sales_multi"); adapter.listSnapshots(tableIdentifier) .forEach(snapshot -> snapshotIds.add(snapshot.snapshotId())); @@ -317,6 +317,66 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } + @Test + public void testSkippedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("year").withPartitioning(), + // Omitting month partitioning column + ColumnDefinition.ofString("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofInt("Units_Sold"), + ColumnDefinition.ofDouble("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .s3Instructions(instructions.s3Instructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testReorderedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("month").withPartitioning(), + ColumnDefinition.ofInt("year").withPartitioning(), + ColumnDefinition.ofString("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofInt("Units_Sold"), + ColumnDefinition.ofDouble("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .s3Instructions(instructions.s3Instructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + @Test public void testMissingPartitioningColumns() { final TableDefinition tableDef = TableDefinition.of( From fa2e79debb3252be8a720bd47abde8551b011d39 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 31 May 2024 16:34:41 -0700 Subject: [PATCH 19/25] Getting close to final. --- .../iceberg/layout/IcebergBaseLayout.java | 4 +- .../IcebergKeyValuePartitionedLayout.java | 58 ++--- .../iceberg/util/IcebergCatalogAdapter.java | 208 +++++++++++++----- .../iceberg/util/IcebergInstructions.java | 6 +- .../{ => util}/IcebergLocalStackTest.java | 2 +- .../iceberg/util}/IcebergMinIOTest.java | 2 +- .../iceberg/util}/IcebergToolsTest.java | 117 ++++++++-- 7 files changed, 292 insertions(+), 105 deletions(-) rename extensions/iceberg/src/test/java/io/deephaven/iceberg/{ => util}/IcebergLocalStackTest.java (96%) rename extensions/iceberg/src/test/java/{io.deephaven.iceberg => io/deephaven/iceberg/util}/IcebergMinIOTest.java (96%) rename extensions/iceberg/src/test/java/{io.deephaven.iceberg => io/deephaven/iceberg/util}/IcebergToolsTest.java (81%) 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 84c5b40619c..141b17e4a3c 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 @@ -73,8 +73,8 @@ protected IcebergTableLocationKey locationKey( builder.setTableDefinition(tableDef); // Add any column rename mappings. - if (!instructions.columnRenameMap().isEmpty()) { - for (Map.Entry entry : instructions.columnRenameMap().entrySet()) { + if (!instructions.columnRename().isEmpty()) { + for (Map.Entry entry : instructions.columnRename().entrySet()) { builder.addColumnNameMapping(entry.getKey(), entry.getValue()); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java index 4889a455f28..8795d9800c7 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java @@ -10,20 +10,20 @@ import io.deephaven.iceberg.location.IcebergTableLocationKey; import io.deephaven.iceberg.util.IcebergInstructions; import io.deephaven.util.type.TypeUtils; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; import org.jetbrains.annotations.NotNull; import java.net.URI; import java.util.*; +import java.util.stream.Collectors; /** * Iceberg {@link TableLocationKeyFinder location finder} for tables with partitions that will discover data files from * a {@link Snapshot} */ public final class IcebergKeyValuePartitionedLayout extends IcebergBaseLayout { - private final String[] inputPartitionColumnNames; - private class ColumnData { final String name; final Class type; @@ -36,13 +36,14 @@ public ColumnData(String name, Class type, int index) { } } - private final List outputPartitionColumns; + private final List outputPartitioningColumns; /** * @param tableDef The {@link TableDefinition} that will be used for the table. * @param table The {@link Table} to discover locations for. * @param tableSnapshot The {@link Snapshot} from which to discover data files. * @param fileIO The file IO to use for reading manifest data files. + * @param partitionSpec The Iceberg {@link PartitionSpec partition spec} for the table. * @param instructions The instructions for customizations while reading. */ public IcebergKeyValuePartitionedLayout( @@ -54,29 +55,28 @@ public IcebergKeyValuePartitionedLayout( @NotNull final IcebergInstructions instructions) { super(tableDef, table, tableSnapshot, fileIO, instructions); - // Get the list of (potentially renamed) columns on which the Iceberg table is partitioned. This will be the - // order of the values in DataFile.partition() collection. - inputPartitionColumnNames = partitionSpec.fields().stream() + // We can assume due to upstream validation that there are no duplicate names (after renaming) that are included + // in the output definition, so we can ignore duplicates. + final MutableInt icebergIndex = new MutableInt(0); + final Map availablePartitioningColumns = partitionSpec.fields().stream() .map(PartitionField::name) - .map(col -> instructions.columnRenameMap().getOrDefault(col, col)) - .toArray(String[]::new); - - outputPartitionColumns = new ArrayList<>(); - - // Get the list of columns in the table definition that are included in the partition columns. - final List outputCols = tableDef.getColumnNames(); - outputCols.retainAll(List.of(inputPartitionColumnNames)); + .map(name -> instructions.columnRename().getOrDefault(name, name)) + .collect(Collectors.toMap( + name -> name, + name -> icebergIndex.getAndIncrement(), + (v1, v2) -> v1, + LinkedHashMap::new)); - for (String col : outputCols) { - // Is this so inefficient that it's worth it to use a map? - for (int i = 0; i < inputPartitionColumnNames.length; i++) { - if (inputPartitionColumnNames[i].equals(col)) { - outputPartitionColumns - .add(new ColumnData(col, TypeUtils.getBoxedType(tableDef.getColumn(col).getDataType()), i)); - break; - } - } - } + outputPartitioningColumns = tableDef.getColumnStream() + .map((final ColumnDefinition columnDef) -> { + final Integer index = availablePartitioningColumns.get(columnDef.getName()); + if (index == null) { + return null; + } + return new ColumnData(columnDef.getName(), TypeUtils.getBoxedType(columnDef.getDataType()), index); + }) + .filter(Objects::nonNull) + .collect(Collectors.toList()); } @Override @@ -89,15 +89,15 @@ IcebergTableLocationKey keyFromDataFile(DataFile df, URI fileUri) { final Map> partitions = new LinkedHashMap<>(); final PartitionData partitionData = (PartitionData) df.partition(); - for (ColumnData colData : outputPartitionColumns) { + for (final ColumnData colData : outputPartitioningColumns) { final String colName = colData.name; - final Object value = partitionData.get(colData.index); - if (value != null && !value.getClass().isAssignableFrom(colData.type)) { + final Object colValue = partitionData.get(colData.index); + if (colValue != null && !colData.type.isAssignableFrom(colValue.getClass())) { throw new TableDataException("Partitioning column " + colName - + " has type " + value.getClass().getName() + + " has type " + colValue.getClass().getName() + " but expected " + colData.type.getName()); } - partitions.put(colName, (Comparable) value); + partitions.put(colName, (Comparable) colValue); } return locationKey(df.format(), fileUri, partitions); } 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 1e88da91f3d..42df421dc02 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 @@ -13,7 +13,7 @@ import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; -import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import io.deephaven.iceberg.layout.IcebergFlatLayout; @@ -53,11 +53,21 @@ public class IcebergCatalogAdapter { this.fileIO = fileIO; } + /** + * Create a single {@link TableDefinition} from a given Schema, PartitionSpec, and TableDefinition. Takes into + * account {@link Map column rename instructions} + * + * @param schema The schema of the table. + * @param partitionSpec The partition specification of the table. + * @param tableDefinition The table definition. + * @param columnRename The map for renaming columns. + * @return The generated TableDefinition. + */ private static TableDefinition fromSchema( @NotNull final Schema schema, @NotNull final PartitionSpec partitionSpec, @Nullable final TableDefinition tableDefinition, - @NotNull final Map columnRenameMap) { + @NotNull final Map columnRename) { final Set columnNames = tableDefinition != null ? tableDefinition.getColumnNameSet() @@ -66,13 +76,13 @@ private static TableDefinition fromSchema( final Set partitionNames = partitionSpec.fields().stream() .map(PartitionField::name) - .map(colName -> columnRenameMap.getOrDefault(colName, colName)) + .map(colName -> columnRename.getOrDefault(colName, colName)) .collect(Collectors.toSet()); final List> columns = new ArrayList<>(); for (final Types.NestedField field : schema.columns()) { - final String name = columnRenameMap.getOrDefault(field.name(), field.name()); + final String name = columnRename.getOrDefault(field.name(), field.name()); // Skip columns that are not in the provided table definition. if (columnNames != null && !columnNames.contains(name)) { continue; @@ -91,6 +101,12 @@ private static TableDefinition fromSchema( return TableDefinition.of(columns); } + /** + * Convert an Iceberg data type to a Deephaven type. + * + * @param icebergType The Iceberg data type to be converted. + * @return The converted Deephaven type. + */ static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type icebergType) { final Type.TypeID typeId = icebergType.typeId(); switch (typeId) { @@ -129,12 +145,26 @@ static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type ic } } + /** + * List all {@link Namespace namespaces} in the catalog. This method is only supported if the catalog implements + * {@link SupportsNamespaces} for namespace discovery. See {@link SupportsNamespaces#listNamespaces(Namespace)}. + * + * @return A list of all namespaces. + */ public List listNamespaces() { return listNamespaces(Namespace.empty()); } - public List listNamespaces(@NotNull Namespace namespace) { - if (catalog instanceof org.apache.iceberg.catalog.SupportsNamespaces) { + /** + * List all {@link Namespace namespaces} in a given namespace. This method is only supported if the catalog + * implements {@link SupportsNamespaces} for namespace discovery. See + * {@link SupportsNamespaces#listNamespaces(Namespace)}. + * + * @param namespace The namespace to list namespaces in. + * @return A list of all namespaces in the given namespace. + */ + public List listNamespaces(@NotNull final Namespace namespace) { + if (catalog instanceof SupportsNamespaces) { final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; return nsCatalog.listNamespaces(namespace); } @@ -142,41 +172,65 @@ public List listNamespaces(@NotNull Namespace namespace) { "%s does not implement org.apache.iceberg.catalog.SupportsNamespaces", catalog.getClass().getName())); } + /** + * List all {@link Namespace namespaces} in the catalog as a Deephaven {@link Table table}. The resulting table will + * be static and contain the same information as {@link #listNamespaces()}. + * + * @return A {@link Table table} of all namespaces. + */ public Table listNamespacesAsTable() { return listNamespacesAsTable(Namespace.empty()); } - public Table listNamespacesAsTable(@NotNull Namespace namespace) { - final List namespaces = listNamespaces(); + /** + * List all {@link Namespace namespaces} in a given namespace as a Deephaven {@link Table table}. The resulting + * table will be static and contain the same information as {@link #listNamespaces(Namespace)}. + * + * @return A {@link Table table} of all namespaces. + */ + public Table listNamespacesAsTable(@NotNull final Namespace namespace) { + final List namespaces = listNamespaces(namespace); final long size = namespaces.size(); // Create and return a table containing the namespaces as strings final Map> columnSourceMap = new LinkedHashMap<>(); // Create the column source(s) - final WritableColumnSource namespaceColumn = - ArrayBackedColumnSource.getMemoryColumnSource(size, String.class, null); - columnSourceMap.put("namespace", namespaceColumn); + final String[] namespaceArr = new String[(int) size]; + columnSourceMap.put("namespace", InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceArr)); - final WritableColumnSource objectColumn = - ArrayBackedColumnSource.getMemoryColumnSource(size, Object.class, Namespace.class); - columnSourceMap.put("namespace_object", objectColumn); + final Namespace[] namespaceObjectArr = new Namespace[(int) size]; + columnSourceMap.put("namespace_object", + InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceObjectArr, Namespace.class, null)); - // Populate the column source(s) + // Populate the column source arrays for (int i = 0; i < size; i++) { final Namespace ns = namespaces.get(i); - namespaceColumn.set(i, ns.toString()); - objectColumn.set(i, ns); + namespaceArr[i] = ns.toString(); + namespaceObjectArr[i] = ns; } // Create and return the table return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); } + /** + * List all Iceberg {@link TableIdentifier tables} in a given namespace. + * + * @param namespace The namespace to list tables in. + * @return A list of all tables in the given namespace. + */ public List listTables(@NotNull final Namespace namespace) { return catalog.listTables(namespace); } + /** + * List all Iceberg {@link TableIdentifier tables} in a given namespace as a Deephaven {@link Table table}. The + * resulting table will be static and contain the same information as {@link #listTables(Namespace)}. + * + * @param namespace The namespace from which to gather the tables + * @return A list of all tables in the given namespace. + */ public Table listTablesAsTable(@NotNull final Namespace namespace) { final List tableIdentifiers = listTables(namespace); final long size = tableIdentifiers.size(); @@ -185,36 +239,47 @@ public Table listTablesAsTable(@NotNull final Namespace namespace) { final Map> columnSourceMap = new LinkedHashMap<>(); // Create the column source(s) - final WritableColumnSource namespaceColumn = - ArrayBackedColumnSource.getMemoryColumnSource(size, String.class, null); - columnSourceMap.put("namespace", namespaceColumn); + final String[] namespaceArr = new String[(int) size]; + columnSourceMap.put("namespace", InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceArr)); - final WritableColumnSource tableColumn = - ArrayBackedColumnSource.getMemoryColumnSource(size, String.class, null); - columnSourceMap.put("table_name", tableColumn); + final String[] tableNameArr = new String[(int) size]; + columnSourceMap.put("table_name", InMemoryColumnSource.getImmutableMemoryColumnSource(tableNameArr)); - final WritableColumnSource objectColumn = - ArrayBackedColumnSource.getMemoryColumnSource(size, Object.class, TableIdentifier.class); - columnSourceMap.put("table_identifier_object", objectColumn); + final TableIdentifier[] tableIdentifierArr = new TableIdentifier[(int) size]; + columnSourceMap.put("table_identifier_object", + InMemoryColumnSource.getImmutableMemoryColumnSource(tableIdentifierArr, Namespace.class, null)); - // Populate the column source(s) + // Populate the column source arrays for (int i = 0; i < size; i++) { final TableIdentifier tableIdentifier = tableIdentifiers.get(i); - namespaceColumn.set(i, tableIdentifier.namespace().toString()); - tableColumn.set(i, tableIdentifier.name()); - objectColumn.set(i, tableIdentifier); + namespaceArr[i] = tableIdentifier.namespace().toString(); + tableNameArr[i] = tableIdentifier.name(); + tableIdentifierArr[i] = tableIdentifier; } // Create and return the table return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); } + /** + * List all {@link Snapshot snapshots} of a given Iceberg table. + * + * @param tableIdentifier The identifier of the table from which to gather snapshots. + * @return A list of all snapshots of the given table. + */ public List listSnapshots(@NotNull final TableIdentifier tableIdentifier) { final List snapshots = new ArrayList<>(); catalog.loadTable(tableIdentifier).snapshots().forEach(snapshots::add); return snapshots; } + /** + * List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting + * table will be static and contain the same information as {@link #listSnapshots(TableIdentifier)}. + * + * @param tableIdentifier The identifier of the table from which to gather snapshots. + * @return A list of all tables in the given namespace. + */ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier) { final List snapshots = listSnapshots(tableIdentifier); final long size = snapshots.size(); @@ -223,34 +288,31 @@ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier final Map> columnSourceMap = new LinkedHashMap<>(); // Create the column source(s) - final WritableColumnSource idColumn = - ArrayBackedColumnSource.getMemoryColumnSource(size, Long.class, null); - columnSourceMap.put("id", idColumn); + final long[] idArr = new long[(int) size]; + columnSourceMap.put("id", InMemoryColumnSource.getImmutableMemoryColumnSource(idArr)); - final WritableColumnSource timestampMsColumn = - ArrayBackedColumnSource.getMemoryColumnSource(size, Long.class, null); - columnSourceMap.put("timestamp_ms", timestampMsColumn); + final long[] timestampArr = new long[(int) size]; + columnSourceMap.put("timestamp_ms", InMemoryColumnSource.getImmutableMemoryColumnSource(timestampArr)); - final WritableColumnSource operationColumn = - ArrayBackedColumnSource.getMemoryColumnSource(size, String.class, null); - columnSourceMap.put("operation", operationColumn); + final String[] operatorArr = new String[(int) size]; + columnSourceMap.put("operation", InMemoryColumnSource.getImmutableMemoryColumnSource(operatorArr)); - final WritableColumnSource summaryColumn = - ArrayBackedColumnSource.getMemoryColumnSource(size, Object.class, Map.class); - columnSourceMap.put("summary", summaryColumn); + final Map[] summaryArr = new Map[(int) size]; + columnSourceMap.put("summary", + InMemoryColumnSource.getImmutableMemoryColumnSource(summaryArr, Map.class, null)); - final WritableColumnSource objectColumn = - ArrayBackedColumnSource.getMemoryColumnSource(size, Object.class, Snapshot.class); - columnSourceMap.put("snapshot_object", objectColumn); + final Snapshot[] snapshotArr = new Snapshot[(int) size]; + columnSourceMap.put("snapshot_object", + InMemoryColumnSource.getImmutableMemoryColumnSource(snapshotArr, Snapshot.class, null)); // Populate the column source(s) for (int i = 0; i < size; i++) { final Snapshot snapshot = snapshots.get(i); - idColumn.set(i, snapshot.snapshotId()); - timestampMsColumn.set(i, snapshot.timestampMillis()); - operationColumn.set(i, snapshot.operation()); - summaryColumn.set(i, snapshot.summary()); - objectColumn.set(i, snapshot); + idArr[i] = snapshot.snapshotId(); + timestampArr[i] = snapshot.timestampMillis(); + operatorArr[i] = snapshot.operation(); + summaryArr[i] = snapshot.summary(); + snapshotArr[i] = snapshot; } // Create and return the table @@ -258,7 +320,7 @@ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier } /** - * Read the latest static snapshot of a table from the Iceberg catalog. + * Read the latest static snapshot of an Iceberg table from the Iceberg catalog. * * @param tableIdentifier The table identifier to load * @param instructions The instructions for customizations while reading @@ -272,7 +334,30 @@ public Table readTable( } /** - * Read a static snapshot of a table from the Iceberg catalog. + * Read a static snapshot of an Iceberg table from the Iceberg catalog. + * + * @param tableIdentifier The table identifier to load + * @param tableSnapshotId The snapshot id to load + * @param instructions The instructions for customizations while reading + * @return The loaded table + */ + @SuppressWarnings("unused") + public Table readTable( + @NotNull final TableIdentifier tableIdentifier, + final long tableSnapshotId, + @NotNull final IcebergInstructions instructions) { + + // Find the snapshot with the given snapshot id + final Snapshot tableSnapshot = listSnapshots(tableIdentifier).stream() + .filter(snapshot -> snapshot.snapshotId() == tableSnapshotId) + .findFirst() + .orElseThrow(() -> new IllegalArgumentException("Snapshot with id " + tableSnapshotId + " not found")); + + return readTableInternal(tableIdentifier, tableSnapshot, instructions); + } + + /** + * Read a static snapshot of an Iceberg table from the Iceberg catalog. * * @param tableIdentifier The table identifier to load * @param tableSnapshot The {@link Snapshot snapshot} to load @@ -308,12 +393,29 @@ private Table readTableInternal( // Get the table definition from the schema (potentially limited by the user supplied table definition and // applying column renames). final TableDefinition icebergTableDef = - fromSchema(schema, partitionSpec, userTableDef, instructions.columnRenameMap()); + fromSchema(schema, partitionSpec, userTableDef, instructions.columnRename()); // If the user supplied a table definition, make sure it's fully compatible. final TableDefinition tableDef; if (userTableDef != null) { tableDef = icebergTableDef.checkCompatibility(userTableDef); + + // Ensure that the user has not marked non-partitioned columns as partitioned. + final Set userPartitionColumns = userTableDef.getPartitioningColumns().stream() + .map(ColumnDefinition::getName) + .collect(Collectors.toSet()); + final Set partitionColumns = tableDef.getPartitioningColumns().stream() + .map(ColumnDefinition::getName) + .collect(Collectors.toSet()); + + // The working partitioning column set must be a super-set of the user-supplied set. + if (!partitionColumns.containsAll(userPartitionColumns)) { + final Set invalidColumns = new HashSet<>(userPartitionColumns); + invalidColumns.removeAll(partitionColumns); + + throw new TableDataException("The following columns are not partitioned in the Iceberg table: " + + invalidColumns); + } } else { // Use the snapshot schema as the table definition. tableDef = icebergTableDef; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 4513e9bd48b..5438459cc3f 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -36,7 +36,7 @@ public static Builder builder() { * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading the Iceberg * data files. */ - public abstract Map columnRenameMap(); + public abstract Map columnRename(); public interface Builder { @SuppressWarnings("unused") @@ -46,10 +46,10 @@ public interface Builder { Builder s3Instructions(S3Instructions s3Instructions); @SuppressWarnings("unused") - Builder putColumnRenameMap(String key, String value); + Builder putColumnRename(String key, String value); @SuppressWarnings("unused") - Builder putAllColumnRenameMap(Map entries); + Builder putAllColumnRename(Map entries); IcebergInstructions build(); } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/IcebergLocalStackTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java similarity index 96% rename from extensions/iceberg/src/test/java/io/deephaven/iceberg/IcebergLocalStackTest.java rename to extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java index 8426060f3bc..578e358985e 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/IcebergLocalStackTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java @@ -1,7 +1,7 @@ // // Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending // -package io.deephaven.iceberg; +package io.deephaven.iceberg.util; import io.deephaven.extensions.s3.S3Instructions.Builder; diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergMinIOTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java similarity index 96% rename from extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergMinIOTest.java rename to extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java index 07ead11450b..804d2d01746 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergMinIOTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java @@ -1,7 +1,7 @@ // // Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending // -package io.deephaven.iceberg; +package io.deephaven.iceberg.util; import io.deephaven.extensions.s3.S3Instructions.Builder; diff --git a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java similarity index 81% rename from extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java rename to extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index f5534be7fea..1850675afd1 100644 --- a/extensions/iceberg/src/test/java/io.deephaven.iceberg/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -1,20 +1,18 @@ // // Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending // -package io.deephaven.iceberg; +package io.deephaven.iceberg.util; import gnu.trove.list.array.TLongArrayList; import io.deephaven.base.verify.Assert; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.util.TableTools; import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; import io.deephaven.iceberg.TestCatalog.IcebergTestFileIO; -import io.deephaven.iceberg.util.IcebergCatalogAdapter; -import io.deephaven.iceberg.util.IcebergInstructions; -import io.deephaven.iceberg.util.IcebergTools; import io.deephaven.time.DateTimeUtils; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; @@ -297,13 +295,13 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .s3Instructions(instructions.s3Instructions().get()) - .putColumnRenameMap("Region", "RegionName") - .putColumnRenameMap("Item_Type", "ItemType") - .putColumnRenameMap("Units_Sold", "UnitsSold") - .putColumnRenameMap("Unit_Price", "UnitPrice") - .putColumnRenameMap("Order_Date", "OrderDate") - .putColumnRenameMap("year", "__year") - .putColumnRenameMap("month", "__month") + .putColumnRename("Region", "RegionName") + .putColumnRename("Item_Type", "ItemType") + .putColumnRename("Units_Sold", "UnitsSold") + .putColumnRename("Unit_Price", "UnitPrice") + .putColumnRename("Order_Date", "OrderDate") + .putColumnRename("year", "__year") + .putColumnRename("month", "__month") .build(); final IcebergCatalogAdapter adapter = @@ -377,6 +375,64 @@ public void testReorderedPartitioningColumn() throws ExecutionException, Interru Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } + @Test + public void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofString("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofInt("Units_Sold"), + ColumnDefinition.ofDouble("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .s3Instructions(instructions.s3Instructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testIncorrectPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("month").withPartitioning(), + ColumnDefinition.ofInt("year").withPartitioning(), + ColumnDefinition.ofString("Region").withPartitioning(), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofInt("Units_Sold"), + ColumnDefinition.ofDouble("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .s3Instructions(instructions.s3Instructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + + try { + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + Assert.statementNeverExecuted("Expected an exception for missing columns"); + } catch (final TableDataException e) { + Assert.eqTrue(e.getMessage().startsWith("The following columns are not partitioned"), "Exception message"); + } + } + @Test public void testMissingPartitioningColumns() { final TableDefinition tableDef = TableDefinition.of( @@ -413,8 +469,8 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx final IcebergInstructions localInstructions = IcebergInstructions.builder() .s3Instructions(instructions.s3Instructions().get()) - .putColumnRenameMap("RegionName", "Region") - .putColumnRenameMap("ItemType", "Item_Type") + .putColumnRename("RegionName", "Region") + .putColumnRename("ItemType", "Item_Type") .build(); final IcebergCatalogAdapter adapter = @@ -436,9 +492,9 @@ public void testOpenTableColumnRenamePartitioningColumns() final IcebergInstructions localInstructions = IcebergInstructions.builder() .s3Instructions(instructions.s3Instructions().get()) - .putColumnRenameMap("VendorID", "vendor_id") - .putColumnRenameMap("month", "__month") - .putColumnRenameMap("year", "__year") + .putColumnRename("VendorID", "vendor_id") + .putColumnRename("month", "__month") + .putColumnRename("year", "__year") .build(); final IcebergCatalogAdapter adapter = @@ -463,6 +519,35 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); final List snapshots = adapter.listSnapshots(tableId); + // Verify we retrieved all the rows. + final io.deephaven.engine.table.Table table0 = + adapter.readTable(tableId, snapshots.get(0).snapshotId(), instructions); + Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); + + final io.deephaven.engine.table.Table table1 = + adapter.readTable(tableId, snapshots.get(1).snapshotId(), instructions); + Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); + + final io.deephaven.engine.table.Table table2 = + adapter.readTable(tableId, snapshots.get(2).snapshotId(), instructions); + Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); + + final io.deephaven.engine.table.Table table3 = + adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions); + Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_multi").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); + final List snapshots = adapter.listSnapshots(tableId); + // Verify we retrieved all the rows. final io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0), instructions); Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); From d6065e488d707aaaf1dc5dbe88c9e9ced119b8e5 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 31 May 2024 16:56:59 -0700 Subject: [PATCH 20/25] Another rev from comments. --- .../iceberg/layout/IcebergBaseLayout.java | 4 ++-- .../IcebergKeyValuePartitionedLayout.java | 2 +- .../iceberg/util/IcebergCatalogAdapter.java | 21 +++++++++------- .../iceberg/util/IcebergInstructions.java | 6 ++--- .../iceberg/util/IcebergToolsTest.java | 24 +++++++++---------- 5 files changed, 31 insertions(+), 26 deletions(-) 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 141b17e4a3c..27daf87fd34 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 @@ -73,8 +73,8 @@ protected IcebergTableLocationKey locationKey( builder.setTableDefinition(tableDef); // Add any column rename mappings. - if (!instructions.columnRename().isEmpty()) { - for (Map.Entry entry : instructions.columnRename().entrySet()) { + if (!instructions.columnRenames().isEmpty()) { + for (Map.Entry entry : instructions.columnRenames().entrySet()) { builder.addColumnNameMapping(entry.getKey(), entry.getValue()); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java index 8795d9800c7..47ec05dfd74 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java @@ -60,7 +60,7 @@ public IcebergKeyValuePartitionedLayout( final MutableInt icebergIndex = new MutableInt(0); final Map availablePartitioningColumns = partitionSpec.fields().stream() .map(PartitionField::name) - .map(name -> instructions.columnRename().getOrDefault(name, name)) + .map(name -> instructions.columnRenames().getOrDefault(name, name)) .collect(Collectors.toMap( name -> name, name -> icebergIndex.getAndIncrement(), 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 42df421dc02..3dc2f059779 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 @@ -197,7 +197,8 @@ public Table listNamespacesAsTable(@NotNull final Namespace namespace) { // Create the column source(s) final String[] namespaceArr = new String[(int) size]; - columnSourceMap.put("namespace", InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceArr)); + columnSourceMap.put("namespace", + InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceArr, String.class, null)); final Namespace[] namespaceObjectArr = new Namespace[(int) size]; columnSourceMap.put("namespace_object", @@ -240,14 +241,16 @@ public Table listTablesAsTable(@NotNull final Namespace namespace) { // Create the column source(s) final String[] namespaceArr = new String[(int) size]; - columnSourceMap.put("namespace", InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceArr)); + columnSourceMap.put("namespace", + InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceArr, String.class, null)); final String[] tableNameArr = new String[(int) size]; - columnSourceMap.put("table_name", InMemoryColumnSource.getImmutableMemoryColumnSource(tableNameArr)); + columnSourceMap.put("table_name", + InMemoryColumnSource.getImmutableMemoryColumnSource(tableNameArr, String.class, null)); final TableIdentifier[] tableIdentifierArr = new TableIdentifier[(int) size]; columnSourceMap.put("table_identifier_object", - InMemoryColumnSource.getImmutableMemoryColumnSource(tableIdentifierArr, Namespace.class, null)); + InMemoryColumnSource.getImmutableMemoryColumnSource(tableIdentifierArr, TableIdentifier.class, null)); // Populate the column source arrays for (int i = 0; i < size; i++) { @@ -289,13 +292,15 @@ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier // Create the column source(s) final long[] idArr = new long[(int) size]; - columnSourceMap.put("id", InMemoryColumnSource.getImmutableMemoryColumnSource(idArr)); + columnSourceMap.put("id", InMemoryColumnSource.getImmutableMemoryColumnSource(idArr, long.class, null)); final long[] timestampArr = new long[(int) size]; - columnSourceMap.put("timestamp_ms", InMemoryColumnSource.getImmutableMemoryColumnSource(timestampArr)); + columnSourceMap.put("timestamp_ms", + InMemoryColumnSource.getImmutableMemoryColumnSource(timestampArr, long.class, null)); final String[] operatorArr = new String[(int) size]; - columnSourceMap.put("operation", InMemoryColumnSource.getImmutableMemoryColumnSource(operatorArr)); + columnSourceMap.put("operation", + InMemoryColumnSource.getImmutableMemoryColumnSource(operatorArr, String.class, null)); final Map[] summaryArr = new Map[(int) size]; columnSourceMap.put("summary", @@ -393,7 +398,7 @@ private Table readTableInternal( // Get the table definition from the schema (potentially limited by the user supplied table definition and // applying column renames). final TableDefinition icebergTableDef = - fromSchema(schema, partitionSpec, userTableDef, instructions.columnRename()); + fromSchema(schema, partitionSpec, userTableDef, instructions.columnRenames()); // If the user supplied a table definition, make sure it's fully compatible. final TableDefinition tableDef; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 5438459cc3f..7616241e4d1 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -36,7 +36,7 @@ public static Builder builder() { * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading the Iceberg * data files. */ - public abstract Map columnRename(); + public abstract Map columnRenames(); public interface Builder { @SuppressWarnings("unused") @@ -46,10 +46,10 @@ public interface Builder { Builder s3Instructions(S3Instructions s3Instructions); @SuppressWarnings("unused") - Builder putColumnRename(String key, String value); + Builder putColumnRenames(String key, String value); @SuppressWarnings("unused") - Builder putAllColumnRename(Map entries); + Builder putAllColumnRenames(Map entries); IcebergInstructions build(); } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 1850675afd1..22721651d91 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -295,13 +295,13 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .s3Instructions(instructions.s3Instructions().get()) - .putColumnRename("Region", "RegionName") - .putColumnRename("Item_Type", "ItemType") - .putColumnRename("Units_Sold", "UnitsSold") - .putColumnRename("Unit_Price", "UnitPrice") - .putColumnRename("Order_Date", "OrderDate") - .putColumnRename("year", "__year") - .putColumnRename("month", "__month") + .putColumnRenames("Region", "RegionName") + .putColumnRenames("Item_Type", "ItemType") + .putColumnRenames("Units_Sold", "UnitsSold") + .putColumnRenames("Unit_Price", "UnitPrice") + .putColumnRenames("Order_Date", "OrderDate") + .putColumnRenames("year", "__year") + .putColumnRenames("month", "__month") .build(); final IcebergCatalogAdapter adapter = @@ -469,8 +469,8 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx final IcebergInstructions localInstructions = IcebergInstructions.builder() .s3Instructions(instructions.s3Instructions().get()) - .putColumnRename("RegionName", "Region") - .putColumnRename("ItemType", "Item_Type") + .putColumnRenames("RegionName", "Region") + .putColumnRenames("ItemType", "Item_Type") .build(); final IcebergCatalogAdapter adapter = @@ -492,9 +492,9 @@ public void testOpenTableColumnRenamePartitioningColumns() final IcebergInstructions localInstructions = IcebergInstructions.builder() .s3Instructions(instructions.s3Instructions().get()) - .putColumnRename("VendorID", "vendor_id") - .putColumnRename("month", "__month") - .putColumnRename("year", "__year") + .putColumnRenames("VendorID", "vendor_id") + .putColumnRenames("month", "__month") + .putColumnRenames("year", "__year") .build(); final IcebergCatalogAdapter adapter = From ea5ca0e6e26b04769fa5bc2a6f6525cdc193f635 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 3 Jun 2024 13:39:17 -0700 Subject: [PATCH 21/25] WIP, some updates. --- .../iceberg/util/IcebergCatalogAdapter.java | 5 ++- .../iceberg/util/IcebergToolsTest.java | 11 ++++++ ...528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet | 3 ++ ...48746c-d0a8-4697-8a8b-cdde507ffd02.parquet | 3 -- ...4659-434a-b422-227b8b15dfb9.metadata.json} | 11 ++++-- ...5cf6-4266-9f0c-5ae723cd77ba.metadata.json} | 33 ++++++++++-------- ...8d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro | Bin 0 -> 7582 bytes ...21fde5-e926-4c4e-a30a-010a244ca3ce-m0.avro | Bin 7496 -> 0 bytes ...388d67e5-5760-40c1-a460-c8c4ee5b8629.avro} | Bin 4227 -> 4226 bytes 9 files changed, 45 insertions(+), 21 deletions(-) create mode 100644 extensions/iceberg/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet delete mode 100644 extensions/iceberg/src/test/resources/warehouse/sample/all_types/b648746c-d0a8-4697-8a8b-cdde507ffd02.parquet rename extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/{00000-3edeb19d-2420-4247-88af-3583a772d037.metadata.json => 00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json} (89%) rename extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/{00001-4dbde963-bce4-47f1-b2b8-7a4d90c1c524.metadata.json => 00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json} (78%) create mode 100644 extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro delete mode 100644 extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/e421fde5-e926-4c4e-a30a-010a244ca3ce-m0.avro rename extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/{snap-5615734881085848694-1-e421fde5-e926-4c4e-a30a-010a244ca3ce.avro => snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro} (92%) 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 3dc2f059779..f8b00a3c121 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 @@ -36,6 +36,7 @@ import java.time.Instant; import java.time.LocalDateTime; +import java.time.OffsetDateTime; import java.util.*; import java.util.stream.Collectors; @@ -108,10 +109,12 @@ private static TableDefinition fromSchema( * @return The converted Deephaven type. */ static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type icebergType) { + final OffsetDateTime now = OffsetDateTime.now().plusSeconds(1); + final Type.TypeID typeId = icebergType.typeId(); switch (typeId) { case BOOLEAN: - return io.deephaven.qst.type.Type.booleanType(); + return io.deephaven.qst.type.Type.booleanType().boxedType(); case DOUBLE: return io.deephaven.qst.type.Type.doubleType(); case FLOAT: diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 22721651d91..cd12dcdea99 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -31,6 +31,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -117,6 +118,8 @@ public void testListNamespaces() { final Table table = adapter.listNamespacesAsTable(); Assert.eq(table.size(), "table.size()", 2, "2 namespace in the catalog"); + Assert.eqTrue(table.getColumnSource("namespace").getType().equals(String.class), "namespace column type"); + Assert.eqTrue(table.getColumnSource("namespace_object").getType().equals(Namespace.class), "namespace_object column type"); } @Test @@ -134,6 +137,9 @@ public void testListTables() { final Table table = adapter.listTablesAsTable(ns); Assert.eq(table.size(), "table.size()", 3, "3 tables in the namespace"); + Assert.eqTrue(table.getColumnSource("namespace").getType().equals(String.class), "namespace column type"); + Assert.eqTrue(table.getColumnSource("table_name").getType().equals(String.class), "table_name column type"); + Assert.eqTrue(table.getColumnSource("table_identifier_object").getType().equals(TableIdentifier.class), "table_identifier_object column type"); } @Test @@ -154,6 +160,11 @@ public void testListSnapshots() { final Table table = adapter.listSnapshotsAsTable(tableIdentifier); Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); + Assert.eqTrue(table.getColumnSource("id").getType().equals(long.class), "id column type"); + Assert.eqTrue(table.getColumnSource("timestamp_ms").getType().equals(long.class), "timestamp_ms column type"); + Assert.eqTrue(table.getColumnSource("operation").getType().equals(String.class), "operation column type"); + Assert.eqTrue(table.getColumnSource("summary").getType().equals(Map.class), "summary column type"); + Assert.eqTrue(table.getColumnSource("snapshot_object").getType().equals(Snapshot.class), "snapshot_object column type"); } @Test diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet new file mode 100644 index 00000000000..6bcedd0c6c5 --- /dev/null +++ b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:1d28097102526b4f6ecbcac552da69348448a28f8909c155527a6b1faccffef6 +size 4239 diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/b648746c-d0a8-4697-8a8b-cdde507ffd02.parquet b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/b648746c-d0a8-4697-8a8b-cdde507ffd02.parquet deleted file mode 100644 index d726ec831f3..00000000000 --- a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/b648746c-d0a8-4697-8a8b-cdde507ffd02.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:6c41d07c03f719700f7ef7bf4f38ec4f9d4f7ac31dc5524ae6f315fdf26d4d9e -size 3904 diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-3edeb19d-2420-4247-88af-3583a772d037.metadata.json b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json similarity index 89% rename from extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-3edeb19d-2420-4247-88af-3583a772d037.metadata.json rename to extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json index 7c96ddd34ef..ec2c53dc065 100644 --- a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-3edeb19d-2420-4247-88af-3583a772d037.metadata.json +++ b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json @@ -1,10 +1,10 @@ { "format-version" : 2, - "table-uuid" : "0eb53bb4-0f8b-403a-8386-7427eb2cd719", + "table-uuid" : "059cfc0e-7f89-4be3-913d-b1d451736783", "location" : "s3://warehouse/sample/all_types", "last-sequence-number" : 0, - "last-updated-ms" : 1717109442450, - "last-column-id" : 12, + "last-updated-ms" : 1717446052895, + "last-column-id" : 13, "current-schema-id" : 0, "schemas" : [ { "type" : "struct", @@ -69,6 +69,11 @@ "name" : "binaryField", "required" : false, "type" : "binary" + }, { + "id" : 13, + "name" : "instantField", + "required" : false, + "type" : "timestamptz" } ] } ], "default-spec-id" : 0, diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-4dbde963-bce4-47f1-b2b8-7a4d90c1c524.metadata.json b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json similarity index 78% rename from extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-4dbde963-bce4-47f1-b2b8-7a4d90c1c524.metadata.json rename to extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json index 6ec810befc0..c9e2ea56748 100644 --- a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-4dbde963-bce4-47f1-b2b8-7a4d90c1c524.metadata.json +++ b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json @@ -1,10 +1,10 @@ { "format-version" : 2, - "table-uuid" : "0eb53bb4-0f8b-403a-8386-7427eb2cd719", + "table-uuid" : "059cfc0e-7f89-4be3-913d-b1d451736783", "location" : "s3://warehouse/sample/all_types", "last-sequence-number" : 1, - "last-updated-ms" : 1717109460647, - "last-column-id" : 12, + "last-updated-ms" : 1717446059625, + "last-column-id" : 13, "current-schema-id" : 0, "schemas" : [ { "type" : "struct", @@ -69,6 +69,11 @@ "name" : "binaryField", "required" : false, "type" : "binary" + }, { + "id" : 13, + "name" : "instantField", + "required" : false, + "type" : "timestamptz" } ] } ], "default-spec-id" : 0, @@ -85,41 +90,41 @@ "properties" : { "write.parquet.compression-codec" : "zstd" }, - "current-snapshot-id" : 5615734881085848694, + "current-snapshot-id" : 6186754175552482648, "refs" : { "main" : { - "snapshot-id" : 5615734881085848694, + "snapshot-id" : 6186754175552482648, "type" : "branch" } }, "snapshots" : [ { "sequence-number" : 1, - "snapshot-id" : 5615734881085848694, - "timestamp-ms" : 1717109460647, + "snapshot-id" : 6186754175552482648, + "timestamp-ms" : 1717446059625, "summary" : { "operation" : "append", "added-data-files" : "1", "added-records" : "10", - "added-files-size" : "3904", + "added-files-size" : "4239", "changed-partition-count" : "1", "total-records" : "10", - "total-files-size" : "3904", + "total-files-size" : "4239", "total-data-files" : "1", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sample/all_types/metadata/snap-5615734881085848694-1-e421fde5-e926-4c4e-a30a-010a244ca3ce.avro", + "manifest-list" : "s3://warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro", "schema-id" : 0 } ], "statistics" : [ ], "partition-statistics" : [ ], "snapshot-log" : [ { - "timestamp-ms" : 1717109460647, - "snapshot-id" : 5615734881085848694 + "timestamp-ms" : 1717446059625, + "snapshot-id" : 6186754175552482648 } ], "metadata-log" : [ { - "timestamp-ms" : 1717109442450, - "metadata-file" : "s3://warehouse/sample/all_types/metadata/00000-3edeb19d-2420-4247-88af-3583a772d037.metadata.json" + "timestamp-ms" : 1717446052895, + "metadata-file" : "s3://warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json" } ] } \ No newline at end of file diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..ebe516d17798c8942989e13e9ba59e803350bc4b GIT binary patch literal 7582 zcmb_gYiu0V6;54BTndPXNg)a1c6cRWHr<(V{8$r63Nf$5m^v;2<8JTH-n9qM&TM98 zof^mw0))6yBOWb}gqPq45=u%D6>U|NK!|9mXlWp7)moxWk(w%1rSwNB2wL^rxik0P zS?|khOID=Jp67SJbM86!=G>iMIO}xZ94AHm{uy^G!BmMT8O0C0vKc6F=QBk*#IRBOp@1wola9Y$T>^SD68w8?c;! zj1jMq0TAQL23NLSXRMJXmEo$LT(w}kIB29B3UTE%TsiNSb2e$D&7`<`7gq<6t;T^f zDX!kl)$=%L>>FgbY7bWptYSk|D#KNK6RLiIi={?sQ~gko8_|G45nuAew2IhnWFrz= z(W|1)mBNnTUBUV;$v3hR3XN##QDH$4FK0Q}n`&f&NeO+yJ{tarF-x4ln}Y8JQGk&Mcn``x{1jC#t&m8LRV}Tx zSk}QMf7}hSP`t6ih~kuO8~z&>$f&C1NdcGbK*`YWvL#Kn1O-kUlVqK8F-N=>tQnHE zf@PDeKW~$rl1V5IoTYYoXf;9hbdZ@1tm#5a3c*Yun|DpF)fR|#= z-66k3%!=%FDSK93S<@}c#~SiJ_Bs&KnVu>W_L0S$P~<91&t|Z77s*;qHa7()&U0sl z6+77M2G|bif_xCC=3{aB&}1^0u3auVSt>59#%0Tq#h!<0hp+-tgd3WFuWTt3n%142 z)U=*B8+M&X9b$q|8wtWoVQJkGnv>Wri|UHAl%;#1(b2v#W|`RD%m#}hE<92UgQ9!jNV8a36=47P{eD>ZML>GL3!L1GKz6aJkyvL4)82 zGz79RWP&XCsOzGWaZFsI^akh%*DBM6z41(l&W7!GIU8kDlf?63AaBzZJ};)|ya?ye zmR@d|1bW+8w2}S#3O+tH_Yfi$TfUBoClGi_5icO}1BO^*{lsD>RFkhKi6;!PfzDo> znld20fxv8AW&LW(iddxOjZ3J?$$q+$_^{cuug^_mw*D>08!&j}eI8J#H zs}aW>7Bk^N9DQ;nYtBcrE1K{pj)}7$s;j!UI>)pC>zRm2Ft}nds~%R|KiP@bU(|#X zL8uUDx>m?5S~IR{u57}Kcv-7h)`a~;DRYSFI;1La7qRO=gV#Me7H1-MhgWU%6Zl{^ zZze{9{VH70T7!j)0o3GbC2|l?e4$;j1PBMsogr{MMN8c)Y#~iaTfJim6FyQFspGCthf#;=*r}T=2V2(NI@TguR zCmpbhbBbPZqYo+Jq+#XA^ScONlzqjhE-W`yGJ5TjF{F}dJ$!kx=HVOS5(4bxA-YER z3J-v_n7Y%na3k)X%E7sYAKo2p%ct**$pICvjpzfk$>=F1mJkLLCt78)7U2mFRt(X? zSoK4!8XXW(M3@^`~qeXQvoSDhFy|I1y-chp@9s4%RR#!{2f~3xt#KDI=q2 zMpIWi9p)DO>&4~6Z@l}be=a@z_ODKVYSk5O2fkE(@8C^ut?O8Pa@%$7e>mr_7hLqU z{8?MiesJl?$i9KsKKbG)>-YcG+ccCqdi$eG_V3-hdWW(h^Rww4X!UUEyj$P=j2ue$QamuLQSeE)wRKYml&qRS62yWpq$QpYkA%gXP)a^)-E zPW3&u;_+MW{QgTH-F9f;()SPT{^e=AuRr|A3)AKf~o4DufR@nzGm?eeZWY99LYu8-HB-1+yuw=dSd-u}tj9}M>I zyk+y?hTTh*6XVZKzj@9j58b=Iqhs*i`%Z1ood2D+!IkGPzVP7T&qvn%?RQh^yGNeg z_r%Xn{^*Y1q=vus;P7PE;x|4WI)S^kOuYV&f4{nSkN(`zS6}=nux3_n{NA2j%brLL z-#^p7buQSn=?dORl~6;NUZ#_J4Ny|DHEj{o=GEANTd0`0|khPrmix%%UGo VkA42~W2g458a#1s$AMXh{~u`@^@IQb literal 0 HcmV?d00001 diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/e421fde5-e926-4c4e-a30a-010a244ca3ce-m0.avro b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/e421fde5-e926-4c4e-a30a-010a244ca3ce-m0.avro deleted file mode 100644 index 0d7f444c9dff1b2784b8aa2e2aef76aaeda9763e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7496 zcmb_gZHyJw8O9YGAX~F2@fR+qQ-1*7l{+*0c^izIE(lbS2y3di-kdx4+}&~K&fJ}u zyRcZ=n6$~NKTIu2f6!F=gNlICh8U|61;2>I1UDK#0)!CbM=Vuat)V|^e9t*E=bX8F zzxM7X8xrQepU-*T_q^}9EBD}49qoZNMvC~&_J_1^szkJm7KDD;3bmBRf9bOGS|**+ z3N~@_ftK0%kVe1gDb2-2DxdQ_hhTRj{T;M?GEugD0wM+M1SA!kjl{IQDQ>_J25dJZ zqr`7y0K|l{DU==09c`pZWrS+CP%Stf4jbt*A)(wOl=EIW=a5F)TuP|-3UvV4ZX7t5 z66$?IJ&(i2zClK)_6ya}E;dx9GD3AAp&Ep^SZb6u)n|gjhz<;j_>w1Iy<2!$JkcWx#wV8N zQ|d^SgWM(vT5mL~;Vc0w&fzpIvX)e@8@aV?T{%f%e=Td-y-)(W;mvcbGfb}J@<);LS; ziqL9;>>D668(7POX2QTsAe;9r=u|)FKL*Gvpm2;J>M4WL>#5rm($|-~zz%KCMS0?o zkOV1YtcU!tV~11mx7aH?Cf!t{XMmJ&I3}W}y%fqE{?7xr!?Q4RFnsrBw95|1(8leE z@rh>4vf*~qX$2K#x8V6jOi3cUv}$=Hq=L8%qP)o<%GDI|e2e~rO4x_9Pm)YetKA;FWjM^M68PJ4JCVCU0KU3 ztH;WCAA8-(=uA(Q3kS&NPAGB}re`zQ-b1pso6SwZiHqEMVZ{!1cp-KeT~Gkx)O;+i z9-2x9%X7*_H%rB()wpa0ve@@A?GScIig34z@0BfOT-W=ule*p?XTvYzs6&hsY9m2- zB`m#9L30v2Wm#Quma23=G&Mr{i_JNQUY#3kr|F&cN`62+Qli0nKHJ@}&tF>q-8 z*ugkNRJ$pq`zvGO2fQ#)4FFU|x)rDfD#MbvMvPOSGSc&ait#xGOtPGE9x55p#*MI> zM_cjM`q-&KQmrgI)CC04&?$h45*N2oeXbd5M`_44!)OUWh7>an9OPZ0Hq&&qle*Yl zaInmN0LvU`foz3a(2wQ*9oOw~P4M9AFNMK-_TV z(bJ4x%Co)}pfh@F7iyKxZ1lB+9m^N7eCvW_^eEAkOQnSciFPxym2PIJjiRC$tAJ#- z*3MYI&48q`9lA)iNJ31fgjr0|cyQvr+2C6KREf<^pod~TlykgX-E%S3ZWNSYjLUk3 zLO{ai*&E-GFc21Du8gKWQ(lJKso4<2E(kRi+pgV?Om(!0XG@su=MWL!6_Nq<%78Y+ z1FPi~VaT!rxR2STg*FdSFLh#5X&iVypylU*%hmP{8iY8Y?H~(7Cdh)1x-Pmo$K)kS zZ-5nSt+G7W_^yTM9N1e|@=>;HlK24(r7ckpaS-+aHB3EE_ zDFGnKCH3epNT7aq=$UIz~G$dyh)X<5U;18ga~FF_RwT(I;23=6p1} zqDg=9m<0Qwx~hAtb4&}cp2?UbgDV!Z>S4wGlb?9~MNK-9gknI`wL)FdnsHTgWs_dy zn^?uNChaFmSwKwJp_B%95x)*}c$cGNaVBCnyK1AKzz4s1GcgkUSLuS*8Z2BKpe9!< znS*?sV|L{dARRQ90~o52g8_xLFgeuUc|=mmTVb{JW{#N|P_?x(g94jxX0w(Nk%lRs zYpd)XbIig7BGt&kgrl2(77e!qk(VJK!7CPmIp$=*b9jxM48ShVDSCg6K6FVZ4J${T z-(~o+>?=lfX}PJAF>05Lsgz9X;j5Fi2;Y>K5MZYc(KW)?L;$?S)SafK8+rFs4$d|F zi0-Ht~l1rOb>678(kKA`|N5|1a`w!eRdZPE~ zt{u;8e)RpfW(M!yr)3`5KR$D6aKiuQu8#XRbewu)P+z(>v-{kxuRi|&?tQnO_|pRy zE`E9Is_Pf^-*oDoX;Lg*Jo3zQZD*HV*>e2Sy)VuDvi;nh>;87;g`$?I72z8I diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/snap-5615734881085848694-1-e421fde5-e926-4c4e-a30a-010a244ca3ce.avro b/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro similarity index 92% rename from extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/snap-5615734881085848694-1-e421fde5-e926-4c4e-a30a-010a244ca3ce.avro rename to extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro index 3f76b00c6dd8c9b3a1cd3b8f70a7a8676b419bb5..cc27bc3f4f23aacbf48b00d5e0784834a0755a2f 100644 GIT binary patch delta 141 zcmZoxY*O5?nO)e-(8A2z)Wp!-)YR0-#KOqT#A5R?c1;c;@tKWt;{F_yUKS>H?&RHV zWlRqyGYaU0=I10%OiG)VwJJL+FKt0~(uAB9von}dRHId9FIhHAEpmor=dDjmNlFl4>@r|rKsOrz?=m}u delta 142 zcmZotY*yT`nO)e_%+S=_*u=uZ(7?jf!ok9a@OR1 z8<`$WW)#o~o1c}GGGp4LtW~qKGEgcaEd851&6k|xc`m@{Y9?Cb^8 lBf_TGFPzfY*!lA8@uR=r$-b|&pFOd&(UC>t5Ca3c;Q&6pK{fyY From e51cf7c3e7bdbe866c037005ef763b9624248045 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 3 Jun 2024 14:25:45 -0700 Subject: [PATCH 22/25] Hadoop gradle version harmonization. --- buildSrc/src/main/groovy/Classpaths.groovy | 11 ++++++++++- extensions/iceberg/build.gradle | 3 +-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/buildSrc/src/main/groovy/Classpaths.groovy b/buildSrc/src/main/groovy/Classpaths.groovy index a09da2269ad..e3a1bc41546 100644 --- a/buildSrc/src/main/groovy/Classpaths.groovy +++ b/buildSrc/src/main/groovy/Classpaths.groovy @@ -123,6 +123,9 @@ class Classpaths { static final String GUAVA_NAME = 'guava' static final String GUAVA_VERSION = '33.2.0-jre' + static final String HADOOP_GROUP = 'org.apache.hadoop' + static final String HADOOP_VERSION = '3.4.0' + static boolean addDependency(Configuration conf, String group, String name, String version, Action configure = Actions.doNothing()) { if (!conf.dependencies.find { it.name == name && it.group == group}) { DefaultExternalModuleDependency dep = dependency group, name, version @@ -292,10 +295,16 @@ class Classpaths { addDependency(config, 'org.apache.parquet', 'parquet-hadoop', '1.14.0') } + static void inheritIcebergHadoop(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { + Configuration config = p.configurations.getByName(configName) + addDependency(config, HADOOP_GROUP, 'hadoop-common', HADOOP_VERSION) + addDependency(config, HADOOP_GROUP, 'hadoop-hdfs-client', HADOOP_VERSION) + } + /** configName controls only the Configuration's classpath, all transitive dependencies are runtimeOnly */ static void inheritParquetHadoopConfiguration(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { Configuration config = p.configurations.getByName(configName) - addDependency(config, 'org.apache.hadoop', 'hadoop-common', '3.4.0') { + addDependency(config, HADOOP_GROUP, 'hadoop-common', HADOOP_VERSION) { it.setTransitive(false) // Do not take any extra dependencies of this project transitively. We just want a few classes for // configuration and compression codecs. For any additional required dependencies, add them separately, as diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle index 5b764989e2c..a086a89663a 100644 --- a/extensions/iceberg/build.gradle +++ b/extensions/iceberg/build.gradle @@ -47,8 +47,7 @@ dependencies { implementation "org.apache.iceberg:iceberg-bundled-guava" runtimeOnly "org.apache.iceberg:iceberg-aws-bundle" - implementation "org.apache.hadoop:hadoop-common:${hadoopVersion}" - implementation "org.apache.hadoop:hadoop-hdfs-client:${hadoopVersion}" + Classpaths.inheritIcebergHadoop(project) // could be downstream configurable ?? testRuntimeOnly might be better // The following lines can be uncommented when we enable support for Azure and GCP From b408f12c2aa0d6c39f1f06e91aa777dc875bd521 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 3 Jun 2024 16:11:39 -0700 Subject: [PATCH 23/25] Iceberg project restructure. --- buildSrc/src/main/groovy/Classpaths.groovy | 44 ++++++++++-- extensions/iceberg-aws/build.gradle | 41 +++++++++++ extensions/iceberg-aws/gradle.properties | 4 ++ .../iceberg/util/IcebergToolsAWS.java | 64 ++++++++++++++++++ .../iceberg/util/IcebergLocalStackTest.java | 0 .../iceberg/util/IcebergMinIOTest.java | 0 .../iceberg/util/IcebergToolsTest.java | 31 +++++---- ...e7d-4830-aedb-14dab6fabdd6-0-00001.parquet | 0 ...f10-4e08-89ae-f1b1b578ce63-0-00001.parquet | 0 ...256-4a04-9248-d7c69bec4881-0-00001.parquet | 0 ...3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet | 0 ...-a414-468b-b7b2-78558b2e8c1f.metadata.json | 0 ...-9c8a-4fa9-911d-e356b2b04061.metadata.json | 0 ...-a010-4afe-8461-f5261787aae9.metadata.json | 0 ...-48c8-4f6d-bd6b-9b452de74dc3.metadata.json | 0 ...f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro | Bin ...be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro | Bin ...bfad1c-123c-452c-814d-298a1483a99f-m0.avro | Bin ...1cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro | Bin ...-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro | Bin ...-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro | Bin ...-77bfad1c-123c-452c-814d-298a1483a99f.avro | Bin ...-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro | Bin ...66f-4474-a2a5-ad5e580e8f54-0-00001.parquet | 0 ...66f-4474-a2a5-ad5e580e8f54-0-00001.parquet | 0 ...-14fb-4a05-b8af-2c9f5986dfe6.metadata.json | 0 ...305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro | Bin ...-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro | Bin ...7ce-4968-b83d-30b13e1e7151-0-00001.parquet | 0 ...-ae4e-42dd-ab04-884f6fd52372.metadata.json | 0 ...95af41-65d8-41f0-8674-13624d6129c6-m0.avro | Bin ...-3395af41-65d8-41f0-8674-13624d6129c6.avro | Bin ...528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet | 0 ...-4659-434a-b422-227b8b15dfb9.metadata.json | 0 ...-5cf6-4266-9f0c-5ae723cd77ba.metadata.json | 0 ...8d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro | Bin ...-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro | Bin extensions/iceberg/build.gradle | 48 +------------ extensions/iceberg/gradle.properties | 3 - .../iceberg/layout/IcebergBaseLayout.java | 4 +- .../iceberg/util/IcebergInstructions.java | 8 +-- .../deephaven/iceberg/util/IcebergTools.java | 58 +--------------- extensions/parquet/table/build.gradle | 4 +- extensions/s3/build.gradle | 9 +-- settings.gradle | 3 + 45 files changed, 179 insertions(+), 142 deletions(-) create mode 100644 extensions/iceberg-aws/build.gradle create mode 100644 extensions/iceberg-aws/gradle.properties create mode 100644 extensions/iceberg-aws/src/main/java/io/deephaven/iceberg/util/IcebergToolsAWS.java rename extensions/{iceberg => iceberg-aws}/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java (100%) rename extensions/{iceberg => iceberg-aws}/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java (100%) rename extensions/{iceberg => iceberg-aws}/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java (96%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro (100%) rename extensions/{iceberg => iceberg-aws}/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro (100%) diff --git a/buildSrc/src/main/groovy/Classpaths.groovy b/buildSrc/src/main/groovy/Classpaths.groovy index e3a1bc41546..197526d739c 100644 --- a/buildSrc/src/main/groovy/Classpaths.groovy +++ b/buildSrc/src/main/groovy/Classpaths.groovy @@ -126,6 +126,12 @@ class Classpaths { static final String HADOOP_GROUP = 'org.apache.hadoop' static final String HADOOP_VERSION = '3.4.0' + static final String ICEBERG_GROUP = 'org.apache.iceberg' + static final String ICEBERG_VERSION = '1.5.0' + + static final String AWSSDK_GROUP = 'software.amazon.awssdk' + static final String AWSSDK_VERSION = '2.23.19' + static boolean addDependency(Configuration conf, String group, String name, String version, Action configure = Actions.doNothing()) { if (!conf.dependencies.find { it.name == name && it.group == group}) { DefaultExternalModuleDependency dep = dependency group, name, version @@ -295,12 +301,6 @@ class Classpaths { addDependency(config, 'org.apache.parquet', 'parquet-hadoop', '1.14.0') } - static void inheritIcebergHadoop(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { - Configuration config = p.configurations.getByName(configName) - addDependency(config, HADOOP_GROUP, 'hadoop-common', HADOOP_VERSION) - addDependency(config, HADOOP_GROUP, 'hadoop-hdfs-client', HADOOP_VERSION) - } - /** configName controls only the Configuration's classpath, all transitive dependencies are runtimeOnly */ static void inheritParquetHadoopConfiguration(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { Configuration config = p.configurations.getByName(configName) @@ -323,4 +323,36 @@ class Classpaths { it.because('hadoop-common required dependency for Configuration') } } + + static void inheritIcebergHadoop(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { + Configuration config = p.configurations.getByName(configName) + addDependency(config, HADOOP_GROUP, 'hadoop-common', HADOOP_VERSION) + addDependency(config, HADOOP_GROUP, 'hadoop-hdfs-client', HADOOP_VERSION) + } + + + static void inheritIcebergCore(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { + Configuration config = p.configurations.getByName(configName) + addDependency(config, p.getDependencies().platform(ICEBERG_GROUP + ":iceberg-bom:" + ICEBERG_VERSION)) + + addDependency(config, ICEBERG_GROUP, 'iceberg-bom', ICEBERG_VERSION) + addDependency(config, ICEBERG_GROUP, 'iceberg-core', ICEBERG_VERSION) + addDependency(config, ICEBERG_GROUP, 'iceberg-bundled-guava', ICEBERG_VERSION) + } + + static void inheritAWSSDK(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { + Configuration config = p.configurations.getByName(configName) + addDependency(config, p.getDependencies().platform(AWSSDK_GROUP + ":bom:" + AWSSDK_VERSION)) + + addDependency(config, AWSSDK_GROUP, 's3', AWSSDK_VERSION) + addDependency(config, AWSSDK_GROUP, 'aws-crt-client', AWSSDK_VERSION) + } + + static void inheritTestContainers(Project p, String configName = JavaPlugin.TEST_IMPLEMENTATION_CONFIGURATION_NAME) { + Configuration config = p.configurations.getByName(configName) + addDependency(config, 'org.testcontainers', 'testcontainers', '1.19.4') + addDependency(config, 'org.testcontainers', 'junit-jupiter', '1.19.4') + addDependency(config, 'org.testcontainers', 'localstack', '1.19.4') + addDependency(config, 'org.testcontainers', 'minio', '1.19.4') + } } diff --git a/extensions/iceberg-aws/build.gradle b/extensions/iceberg-aws/build.gradle new file mode 100644 index 00000000000..c07df457cf7 --- /dev/null +++ b/extensions/iceberg-aws/build.gradle @@ -0,0 +1,41 @@ +plugins { + id 'java-library' + id 'io.deephaven.project.register' +} + +description 'Iceberg: Support to read iceberg catalogs.' + +dependencies { + implementation project(':extensions-iceberg') + + // Bring in the AWS / S3 extensions + Classpaths.inheritIcebergCore(project) + + implementation project(':extensions-s3') + implementation "org.apache.iceberg:iceberg-aws" + runtimeOnly "org.apache.iceberg:iceberg-aws-bundle" + Classpaths.inheritAWSSDK(project) + + Classpaths.inheritTestContainers(project) + + testImplementation TestTools.projectDependency(project, 'extensions-s3') + testImplementation TestTools.projectDependency(project, 'extensions-iceberg') + + testRuntimeOnly project(':test-configs') + testRuntimeOnly project(':log-to-slf4j') + Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly') +} + +test { + useJUnitPlatform { + excludeTags("testcontainers") + } +} + +tasks.register('testOutOfBand', Test) { + useJUnitPlatform { + includeTags("testcontainers") + } + systemProperty 'testcontainers.localstack.image', project.property('testcontainers.localstack.image') + systemProperty 'testcontainers.minio.image', project.property('testcontainers.minio.image') +} diff --git a/extensions/iceberg-aws/gradle.properties b/extensions/iceberg-aws/gradle.properties new file mode 100644 index 00000000000..cfd384f094a --- /dev/null +++ b/extensions/iceberg-aws/gradle.properties @@ -0,0 +1,4 @@ +io.deephaven.project.ProjectType=JAVA_PUBLIC + +testcontainers.localstack.image=localstack/localstack:3.1.0 +testcontainers.minio.image=minio/minio:RELEASE.2024-02-04T22-36-13Z diff --git a/extensions/iceberg-aws/src/main/java/io/deephaven/iceberg/util/IcebergToolsAWS.java b/extensions/iceberg-aws/src/main/java/io/deephaven/iceberg/util/IcebergToolsAWS.java new file mode 100644 index 00000000000..fca90154388 --- /dev/null +++ b/extensions/iceberg-aws/src/main/java/io/deephaven/iceberg/util/IcebergToolsAWS.java @@ -0,0 +1,64 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import com.google.common.base.Strings; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.aws.AwsClientProperties; +import org.apache.iceberg.aws.s3.S3FileIOProperties; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.rest.RESTCatalog; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.HashMap; +import java.util.Map; + +/** + * Tools for accessing tables in the Iceberg table format. + */ +public class IcebergToolsAWS extends IcebergTools { + private static final String S3_FILE_IO_CLASS = "org.apache.iceberg.aws.s3.S3FileIO"; + + public static IcebergCatalogAdapter createS3Rest( + @Nullable final String name, + @NotNull final String catalogURI, + @NotNull final String warehouseLocation, + @Nullable final String region, + @Nullable final String accessKeyId, + @Nullable final String secretAccessKey, + @Nullable final String endpointOverride) { + + // Set up the properties map for the Iceberg catalog + final Map properties = new HashMap<>(); + + final RESTCatalog catalog = new RESTCatalog(); + + properties.put(CatalogProperties.CATALOG_IMPL, catalog.getClass().getName()); + properties.put(CatalogProperties.URI, catalogURI); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + + // Configure the properties map from the Iceberg instructions. + if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) { + properties.put(S3FileIOProperties.ACCESS_KEY_ID, accessKeyId); + properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, secretAccessKey); + } + if (!Strings.isNullOrEmpty(region)) { + properties.put(AwsClientProperties.CLIENT_REGION, region); + } + if (!Strings.isNullOrEmpty(endpointOverride)) { + properties.put(S3FileIOProperties.ENDPOINT, endpointOverride); + } + + // TODO: create a FileIO interface wrapping the Deephaven S3SeekableByteChannel/Provider + final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); + + final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; + catalog.initialize(catalogName, properties); + + return new IcebergCatalogAdapter(catalog, fileIO); + } + +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java b/extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java similarity index 100% rename from extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java rename to extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java b/extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java similarity index 100% rename from extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java rename to extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java similarity index 96% rename from extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java rename to extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index cd12dcdea99..0fd3b3fcf7e 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -69,7 +69,7 @@ void setUp() throws ExecutionException, InterruptedException { final S3Instructions s3Instructions = s3Instructions(S3Instructions.builder()).build(); instructions = IcebergInstructions.builder() - .s3Instructions(s3Instructions) + .dataInstructions(s3Instructions) .build(); } @@ -119,7 +119,8 @@ public void testListNamespaces() { final Table table = adapter.listNamespacesAsTable(); Assert.eq(table.size(), "table.size()", 2, "2 namespace in the catalog"); Assert.eqTrue(table.getColumnSource("namespace").getType().equals(String.class), "namespace column type"); - Assert.eqTrue(table.getColumnSource("namespace_object").getType().equals(Namespace.class), "namespace_object column type"); + Assert.eqTrue(table.getColumnSource("namespace_object").getType().equals(Namespace.class), + "namespace_object column type"); } @Test @@ -139,7 +140,8 @@ public void testListTables() { Assert.eq(table.size(), "table.size()", 3, "3 tables in the namespace"); Assert.eqTrue(table.getColumnSource("namespace").getType().equals(String.class), "namespace column type"); Assert.eqTrue(table.getColumnSource("table_name").getType().equals(String.class), "table_name column type"); - Assert.eqTrue(table.getColumnSource("table_identifier_object").getType().equals(TableIdentifier.class), "table_identifier_object column type"); + Assert.eqTrue(table.getColumnSource("table_identifier_object").getType().equals(TableIdentifier.class), + "table_identifier_object column type"); } @Test @@ -164,7 +166,8 @@ public void testListSnapshots() { Assert.eqTrue(table.getColumnSource("timestamp_ms").getType().equals(long.class), "timestamp_ms column type"); Assert.eqTrue(table.getColumnSource("operation").getType().equals(String.class), "operation column type"); Assert.eqTrue(table.getColumnSource("summary").getType().equals(Map.class), "summary column type"); - Assert.eqTrue(table.getColumnSource("snapshot_object").getType().equals(Snapshot.class), "snapshot_object column type"); + Assert.eqTrue(table.getColumnSource("snapshot_object").getType().equals(Snapshot.class), + "snapshot_object column type"); } @Test @@ -245,7 +248,7 @@ public void testOpenTableDefinition() throws ExecutionException, InterruptedExce final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructions.s3Instructions().get()) + .dataInstructions(instructions.dataInstructions().get()) .build(); final IcebergCatalogAdapter adapter = @@ -272,7 +275,7 @@ public void testOpenTablePartitionTypeException() { final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructions.s3Instructions().get()) + .dataInstructions(instructions.dataInstructions().get()) .build(); final IcebergCatalogAdapter adapter = @@ -305,7 +308,7 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructions.s3Instructions().get()) + .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Region", "RegionName") .putColumnRenames("Item_Type", "ItemType") .putColumnRenames("Units_Sold", "UnitsSold") @@ -342,7 +345,7 @@ public void testSkippedPartitioningColumn() throws ExecutionException, Interrupt final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructions.s3Instructions().get()) + .dataInstructions(instructions.dataInstructions().get()) .build(); final IcebergCatalogAdapter adapter = @@ -372,7 +375,7 @@ public void testReorderedPartitioningColumn() throws ExecutionException, Interru final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructions.s3Instructions().get()) + .dataInstructions(instructions.dataInstructions().get()) .build(); final IcebergCatalogAdapter adapter = @@ -400,7 +403,7 @@ public void testZeroPartitioningColumns() throws ExecutionException, Interrupted final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructions.s3Instructions().get()) + .dataInstructions(instructions.dataInstructions().get()) .build(); final IcebergCatalogAdapter adapter = @@ -427,7 +430,7 @@ public void testIncorrectPartitioningColumns() throws ExecutionException, Interr final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructions.s3Instructions().get()) + .dataInstructions(instructions.dataInstructions().get()) .build(); final IcebergCatalogAdapter adapter = @@ -457,7 +460,7 @@ public void testMissingPartitioningColumns() { final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) - .s3Instructions(instructions.s3Instructions().get()) + .dataInstructions(instructions.dataInstructions().get()) .build(); final IcebergCatalogAdapter adapter = @@ -479,7 +482,7 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx warehousePath); final IcebergInstructions localInstructions = IcebergInstructions.builder() - .s3Instructions(instructions.s3Instructions().get()) + .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("RegionName", "Region") .putColumnRenames("ItemType", "Item_Type") .build(); @@ -502,7 +505,7 @@ public void testOpenTableColumnRenamePartitioningColumns() warehousePath); final IcebergInstructions localInstructions = IcebergInstructions.builder() - .s3Instructions(instructions.s3Instructions().get()) + .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("VendorID", "vendor_id") .putColumnRenames("month", "__month") .putColumnRenames("year", "__year") diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet b/extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet rename to extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json b/extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json rename to extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json b/extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json rename to extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro diff --git a/extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro b/extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro similarity index 100% rename from extensions/iceberg/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro rename to extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle index a086a89663a..1f7db078764 100644 --- a/extensions/iceberg/build.gradle +++ b/extensions/iceberg/build.gradle @@ -5,10 +5,6 @@ plugins { description 'Iceberg: Support to read iceberg catalogs.' -ext { - hadoopVersion = '3.3.6' -} - dependencies { api project(':engine-api') api project(':engine-table') @@ -17,10 +13,6 @@ dependencies { implementation project(':log-factory') implementation project(':Configuration') - implementation platform('software.amazon.awssdk:bom:2.23.19') - implementation 'software.amazon.awssdk:s3' - implementation 'software.amazon.awssdk:aws-crt-client' - Classpaths.inheritAutoService(project) Classpaths.inheritImmutables(project) @@ -35,49 +27,13 @@ dependencies { implementation project(':extensions-parquet-base') implementation project(':extensions-parquet-table') - implementation project(':extensions-s3') - - implementation platform("org.apache.iceberg:iceberg-bom:1.5.0") - implementation "org.apache.iceberg:iceberg-api" - implementation "org.apache.iceberg:iceberg-core" - implementation "org.apache.iceberg:iceberg-aws" - // The following lines can be uncommented when we enable support for Azure and GCP - // implementation "org.apache.iceberg:iceberg-azure" - // implementation "org.apache.iceberg:iceberg-gcp" - implementation "org.apache.iceberg:iceberg-bundled-guava" - runtimeOnly "org.apache.iceberg:iceberg-aws-bundle" + Classpaths.inheritIcebergCore(project) Classpaths.inheritIcebergHadoop(project) - // could be downstream configurable ?? testRuntimeOnly might be better - // The following lines can be uncommented when we enable support for Azure and GCP - // runtimeOnly "org.apache.iceberg:iceberg-azure-bundle" - // runtimeOnly "org.apache.iceberg:iceberg-gcp-bundle" - - testImplementation "org.testcontainers:testcontainers:1.19.4" - testImplementation "org.testcontainers:junit-jupiter:1.19.4" - testImplementation "org.testcontainers:localstack:1.19.4" - testImplementation "org.testcontainers:minio:1.19.4" - - testImplementation TestTools.projectDependency(project, 'extensions-s3') + Classpaths.inheritTestContainers(project) testRuntimeOnly project(':test-configs') testRuntimeOnly project(':log-to-slf4j') Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly') } - -test { - useJUnitPlatform { - excludeTags("testcontainers") - } -} - -tasks.register('testOutOfBand', Test) { - useJUnitPlatform { - includeTags("testcontainers") - } - systemProperty 'testcontainers.localstack.image', project.property('testcontainers.localstack.image') - systemProperty 'testcontainers.minio.image', project.property('testcontainers.minio.image') -} - - diff --git a/extensions/iceberg/gradle.properties b/extensions/iceberg/gradle.properties index cfd384f094a..c186bbfdde1 100644 --- a/extensions/iceberg/gradle.properties +++ b/extensions/iceberg/gradle.properties @@ -1,4 +1 @@ io.deephaven.project.ProjectType=JAVA_PUBLIC - -testcontainers.localstack.image=localstack/localstack:3.1.0 -testcontainers.minio.image=minio/minio:RELEASE.2024-02-04T22-36-13Z 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 27daf87fd34..f5334cf866c 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 @@ -79,8 +79,8 @@ protected IcebergTableLocationKey locationKey( } } - // Add the S3 instructions. - instructions.s3Instructions().ifPresent(builder::setSpecialInstructions); + // Add the data instructions. + instructions.dataInstructions().ifPresent(builder::setSpecialInstructions); parquetInstructions = builder.build(); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 7616241e4d1..4788e0e8714 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -5,7 +5,6 @@ import io.deephaven.annotations.BuildableStyle; import io.deephaven.engine.table.TableDefinition; -import io.deephaven.extensions.s3.S3Instructions; import org.immutables.value.Value.Immutable; import java.util.Map; @@ -28,9 +27,10 @@ public static Builder builder() { public abstract Optional tableDefinition(); /** - * The {@link S3Instructions} to use for reading the Iceberg data files. + * The data instructions to use for reading the Iceberg data files (might be S3Instructions or other cloud + * provider-specific instructions). */ - public abstract Optional s3Instructions(); + public abstract Optional dataInstructions(); /** * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading the Iceberg @@ -43,7 +43,7 @@ public interface Builder { Builder tableDefinition(TableDefinition tableDefinition); @SuppressWarnings("unused") - Builder s3Instructions(S3Instructions s3Instructions); + Builder dataInstructions(Object s3Instructions); @SuppressWarnings("unused") Builder putColumnRenames(String key, String value); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java index 079ef5b17b9..bcdda326dca 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java @@ -3,73 +3,17 @@ // package io.deephaven.iceberg.util; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.aws.AwsClientProperties; -import org.apache.iceberg.aws.s3.S3FileIOProperties; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.rest.RESTCatalog; -import org.apache.parquet.Strings; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.HashMap; -import java.util.Map; /** * Tools for accessing tables in the Iceberg table format. */ -public class IcebergTools { - private static final String S3_FILE_IO_CLASS = "org.apache.iceberg.aws.s3.S3FileIO"; - +public abstract class IcebergTools { @SuppressWarnings("unused") public static IcebergCatalogAdapter createAdapter( final Catalog catalog, final FileIO fileIO) { return new IcebergCatalogAdapter(catalog, fileIO); } - - private IcebergTools() {} - - - public static IcebergCatalogAdapter createS3Rest( - @Nullable final String name, - @NotNull final String catalogURI, - @NotNull final String warehouseLocation, - @Nullable final String region, - @Nullable final String accessKeyId, - @Nullable final String secretAccessKey, - @Nullable final String endpointOverride) { - - // Set up the properties map for the Iceberg catalog - final Map properties = new HashMap<>(); - - final RESTCatalog catalog = new RESTCatalog(); - - properties.put(CatalogProperties.CATALOG_IMPL, catalog.getClass().getName()); - properties.put(CatalogProperties.URI, catalogURI); - properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); - - // Configure the properties map from the Iceberg instructions. - if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) { - properties.put(S3FileIOProperties.ACCESS_KEY_ID, accessKeyId); - properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, secretAccessKey); - } - if (!Strings.isNullOrEmpty(region)) { - properties.put(AwsClientProperties.CLIENT_REGION, region); - } - if (!Strings.isNullOrEmpty(endpointOverride)) { - properties.put(S3FileIOProperties.ENDPOINT, endpointOverride); - } - - // TODO: create a FileIO interface wrapping the Deephaven S3SeekableByteChannel/Provider - final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); - - final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; - catalog.initialize(catalogName, properties); - - return new IcebergCatalogAdapter(catalog, fileIO); - } - } diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 6d594e29a41..1c80142c1d8 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -48,9 +48,7 @@ dependencies { testImplementation TestTools.projectDependency(project, 'extensions-s3') Classpaths.inheritJUnitClassic(project, 'testImplementation') - testImplementation "org.testcontainers:testcontainers:1.19.4" - testImplementation "org.testcontainers:localstack:1.19.4" - testImplementation "org.testcontainers:minio:1.19.4" + Classpaths.inheritTestContainers(project) testRuntimeOnly project(':log-to-slf4j'), project(path: ':configs'), diff --git a/extensions/s3/build.gradle b/extensions/s3/build.gradle index 64954314387..7d1343c052b 100644 --- a/extensions/s3/build.gradle +++ b/extensions/s3/build.gradle @@ -13,9 +13,7 @@ dependencies { implementation project(':Configuration') implementation project(':log-factory') - implementation platform('software.amazon.awssdk:bom:2.23.19') - implementation 'software.amazon.awssdk:s3' - implementation 'software.amazon.awssdk:aws-crt-client' + Classpaths.inheritAWSSDK(project) compileOnly depAnnotations @@ -32,10 +30,7 @@ dependencies { testRuntimeOnly 'org.junit.platform:junit-platform-launcher' testImplementation 'software.amazon.awssdk:s3-transfer-manager' - testImplementation "org.testcontainers:testcontainers:1.19.4" - testImplementation "org.testcontainers:junit-jupiter:1.19.4" - testImplementation "org.testcontainers:localstack:1.19.4" - testImplementation "org.testcontainers:minio:1.19.4" + Classpaths.inheritTestContainers(project) testRuntimeOnly project(':test-configs') testRuntimeOnly project(':log-to-slf4j') diff --git a/settings.gradle b/settings.gradle index 4a74c9bb125..00fe42e2764 100644 --- a/settings.gradle +++ b/settings.gradle @@ -270,6 +270,9 @@ project(':extensions-s3').projectDir = file('extensions/s3') include(':extensions-iceberg') project(':extensions-iceberg').projectDir = file('extensions/iceberg') +include(':extensions-iceberg-aws') +project(':extensions-iceberg-aws').projectDir = file('extensions/iceberg-aws') + include(':plugin') include(':plugin-dagger') From de7d1f368f98b8bb42ce942f36cef41c96dfb410 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 3 Jun 2024 16:35:45 -0700 Subject: [PATCH 24/25] Exposing 'iceberg-aws' in gradle. --- py/embedded-server/java-runtime/build.gradle | 1 + server/jetty-app-custom/build.gradle | 1 + server/jetty-app/build.gradle | 1 + server/netty-app/build.gradle | 1 + 4 files changed, 4 insertions(+) diff --git a/py/embedded-server/java-runtime/build.gradle b/py/embedded-server/java-runtime/build.gradle index 56111b847c5..66790b9d6d7 100644 --- a/py/embedded-server/java-runtime/build.gradle +++ b/py/embedded-server/java-runtime/build.gradle @@ -32,6 +32,7 @@ dependencies { if (!hasProperty('excludeS3')) { runtimeOnly project(':extensions-s3') + runtimeOnly project(':extensions-iceberg-aws') } } diff --git a/server/jetty-app-custom/build.gradle b/server/jetty-app-custom/build.gradle index ed64f1cdab7..c33373e10b8 100644 --- a/server/jetty-app-custom/build.gradle +++ b/server/jetty-app-custom/build.gradle @@ -55,6 +55,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') + runtimeOnly project(':extensions-iceberg-aws') } } diff --git a/server/jetty-app/build.gradle b/server/jetty-app/build.gradle index 407235af4ae..8d6fb13005a 100644 --- a/server/jetty-app/build.gradle +++ b/server/jetty-app/build.gradle @@ -54,6 +54,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') + runtimeOnly project(':extensions-iceberg-aws') } } diff --git a/server/netty-app/build.gradle b/server/netty-app/build.gradle index 15a180d998b..a985f13533f 100644 --- a/server/netty-app/build.gradle +++ b/server/netty-app/build.gradle @@ -54,6 +54,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') + runtimeOnly project(':extensions-iceberg-aws') } } From 23d6e328e9ceadd6c5f32f88fdd2289e05804117 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 4 Jun 2024 14:12:55 -0700 Subject: [PATCH 25/25] Addressing PR comments. --- buildSrc/src/main/groovy/Classpaths.groovy | 12 +++++++----- extensions/iceberg/build.gradle | 14 +++++++------- .../{iceberg-aws => iceberg/s3}/build.gradle | 0 .../{iceberg-aws => iceberg/s3}/gradle.properties | 0 .../deephaven/iceberg/util/IcebergToolsS3.java} | 3 ++- .../iceberg/util/IcebergLocalStackTest.java | 0 .../deephaven/iceberg/util/IcebergMinIOTest.java | 0 .../deephaven/iceberg/util/IcebergToolsTest.java | 0 ...c8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet | 0 ...4b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet | 0 ...3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet | 0 ...c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet | 0 ...65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json | 0 ...2e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json | 0 ...7323-a010-4afe-8461-f5261787aae9.metadata.json | 0 ...4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json | 0 .../02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro | Bin .../51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro | Bin .../77bfad1c-123c-452c-814d-298a1483a99f-m0.avro | Bin .../a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro | Bin ...75-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro | Bin ...48-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro | Bin ...63-1-77bfad1c-123c-452c-814d-298a1483a99f.avro | Bin ...66-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro | Bin ...0e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet | 0 ...0e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet | 0 ...bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json | 0 .../4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro | Bin ...45-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro | Bin ...52-57ce-4968-b83d-30b13e1e7151-0-00001.parquet | 0 ...eb71-ae4e-42dd-ab04-884f6fd52372.metadata.json | 0 .../3395af41-65d8-41f0-8674-13624d6129c6-m0.avro | Bin ...97-1-3395af41-65d8-41f0-8674-13624d6129c6.avro | Bin .../ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet | 0 ...83f2-4659-434a-b422-227b8b15dfb9.metadata.json | 0 ...eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json | 0 .../388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro | Bin ...48-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro | Bin .../iceberg/util/IcebergCatalogAdapter.java | 3 --- py/embedded-server/java-runtime/build.gradle | 2 +- server/jetty-app-custom/build.gradle | 2 +- server/jetty-app/build.gradle | 2 +- server/netty-app/build.gradle | 2 +- settings.gradle | 4 ++-- 44 files changed, 22 insertions(+), 22 deletions(-) rename extensions/{iceberg-aws => iceberg/s3}/build.gradle (100%) rename extensions/{iceberg-aws => iceberg/s3}/gradle.properties (100%) rename extensions/{iceberg-aws/src/main/java/io/deephaven/iceberg/util/IcebergToolsAWS.java => iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java} (96%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro (100%) rename extensions/{iceberg-aws => iceberg/s3}/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro (100%) diff --git a/buildSrc/src/main/groovy/Classpaths.groovy b/buildSrc/src/main/groovy/Classpaths.groovy index 197526d739c..caf4f5769d1 100644 --- a/buildSrc/src/main/groovy/Classpaths.groovy +++ b/buildSrc/src/main/groovy/Classpaths.groovy @@ -132,6 +132,9 @@ class Classpaths { static final String AWSSDK_GROUP = 'software.amazon.awssdk' static final String AWSSDK_VERSION = '2.23.19' + static final String TESTCONTAINER_GROUP = 'org.testcontainers' + static final String TESTCONTAINER_VERSION = '1.19.4' + static boolean addDependency(Configuration conf, String group, String name, String version, Action configure = Actions.doNothing()) { if (!conf.dependencies.find { it.name == name && it.group == group}) { DefaultExternalModuleDependency dep = dependency group, name, version @@ -335,7 +338,6 @@ class Classpaths { Configuration config = p.configurations.getByName(configName) addDependency(config, p.getDependencies().platform(ICEBERG_GROUP + ":iceberg-bom:" + ICEBERG_VERSION)) - addDependency(config, ICEBERG_GROUP, 'iceberg-bom', ICEBERG_VERSION) addDependency(config, ICEBERG_GROUP, 'iceberg-core', ICEBERG_VERSION) addDependency(config, ICEBERG_GROUP, 'iceberg-bundled-guava', ICEBERG_VERSION) } @@ -350,9 +352,9 @@ class Classpaths { static void inheritTestContainers(Project p, String configName = JavaPlugin.TEST_IMPLEMENTATION_CONFIGURATION_NAME) { Configuration config = p.configurations.getByName(configName) - addDependency(config, 'org.testcontainers', 'testcontainers', '1.19.4') - addDependency(config, 'org.testcontainers', 'junit-jupiter', '1.19.4') - addDependency(config, 'org.testcontainers', 'localstack', '1.19.4') - addDependency(config, 'org.testcontainers', 'minio', '1.19.4') + addDependency(config, TESTCONTAINER_GROUP, 'testcontainers', TESTCONTAINER_VERSION) + addDependency(config, TESTCONTAINER_GROUP, 'junit-jupiter', TESTCONTAINER_VERSION) + addDependency(config, TESTCONTAINER_GROUP, 'localstack', TESTCONTAINER_VERSION) + addDependency(config, TESTCONTAINER_GROUP, 'minio', TESTCONTAINER_VERSION) } } diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle index 1f7db078764..2eba0158fb1 100644 --- a/extensions/iceberg/build.gradle +++ b/extensions/iceberg/build.gradle @@ -16,21 +16,21 @@ dependencies { Classpaths.inheritAutoService(project) Classpaths.inheritImmutables(project) - Classpaths.inheritJUnitPlatform(project) - Classpaths.inheritAssertJ(project) - Classpaths.inheritParquetHadoop(project) - testImplementation 'org.junit.jupiter:junit-jupiter' - testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine' - testRuntimeOnly 'org.junit.platform:junit-platform-launcher' - implementation project(':extensions-parquet-base') implementation project(':extensions-parquet-table') Classpaths.inheritIcebergCore(project) Classpaths.inheritIcebergHadoop(project) + Classpaths.inheritJUnitPlatform(project) + Classpaths.inheritAssertJ(project) + + testImplementation 'org.junit.jupiter:junit-jupiter' + testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine' + testRuntimeOnly 'org.junit.platform:junit-platform-launcher' + Classpaths.inheritTestContainers(project) testRuntimeOnly project(':test-configs') diff --git a/extensions/iceberg-aws/build.gradle b/extensions/iceberg/s3/build.gradle similarity index 100% rename from extensions/iceberg-aws/build.gradle rename to extensions/iceberg/s3/build.gradle diff --git a/extensions/iceberg-aws/gradle.properties b/extensions/iceberg/s3/gradle.properties similarity index 100% rename from extensions/iceberg-aws/gradle.properties rename to extensions/iceberg/s3/gradle.properties diff --git a/extensions/iceberg-aws/src/main/java/io/deephaven/iceberg/util/IcebergToolsAWS.java b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java similarity index 96% rename from extensions/iceberg-aws/src/main/java/io/deephaven/iceberg/util/IcebergToolsAWS.java rename to extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java index fca90154388..6f7845c43eb 100644 --- a/extensions/iceberg-aws/src/main/java/io/deephaven/iceberg/util/IcebergToolsAWS.java +++ b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java @@ -19,7 +19,8 @@ /** * Tools for accessing tables in the Iceberg table format. */ -public class IcebergToolsAWS extends IcebergTools { +@SuppressWarnings("unused") +public class IcebergToolsS3 extends IcebergTools { private static final String S3_FILE_IO_CLASS = "org.apache.iceberg.aws.s3.S3FileIO"; public static IcebergCatalogAdapter createS3Rest( diff --git a/extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java similarity index 100% rename from extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java rename to extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java diff --git a/extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java similarity index 100% rename from extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java rename to extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java diff --git a/extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java similarity index 100% rename from extensions/iceberg-aws/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java rename to extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet rename to extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro diff --git a/extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro similarity index 100% rename from extensions/iceberg-aws/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro 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 f8b00a3c121..c379c715c6d 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 @@ -36,7 +36,6 @@ import java.time.Instant; import java.time.LocalDateTime; -import java.time.OffsetDateTime; import java.util.*; import java.util.stream.Collectors; @@ -109,8 +108,6 @@ private static TableDefinition fromSchema( * @return The converted Deephaven type. */ static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type icebergType) { - final OffsetDateTime now = OffsetDateTime.now().plusSeconds(1); - final Type.TypeID typeId = icebergType.typeId(); switch (typeId) { case BOOLEAN: diff --git a/py/embedded-server/java-runtime/build.gradle b/py/embedded-server/java-runtime/build.gradle index 66790b9d6d7..b64ddaf2958 100644 --- a/py/embedded-server/java-runtime/build.gradle +++ b/py/embedded-server/java-runtime/build.gradle @@ -32,7 +32,7 @@ dependencies { if (!hasProperty('excludeS3')) { runtimeOnly project(':extensions-s3') - runtimeOnly project(':extensions-iceberg-aws') + runtimeOnly project(':extensions-iceberg:s3') } } diff --git a/server/jetty-app-custom/build.gradle b/server/jetty-app-custom/build.gradle index c33373e10b8..11917d41326 100644 --- a/server/jetty-app-custom/build.gradle +++ b/server/jetty-app-custom/build.gradle @@ -55,7 +55,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') - runtimeOnly project(':extensions-iceberg-aws') + runtimeOnly project(':extensions-iceberg:s3') } } diff --git a/server/jetty-app/build.gradle b/server/jetty-app/build.gradle index 8d6fb13005a..39bb137af16 100644 --- a/server/jetty-app/build.gradle +++ b/server/jetty-app/build.gradle @@ -54,7 +54,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') - runtimeOnly project(':extensions-iceberg-aws') + runtimeOnly project(':extensions-iceberg:s3') } } diff --git a/server/netty-app/build.gradle b/server/netty-app/build.gradle index a985f13533f..cd5b23ea1bb 100644 --- a/server/netty-app/build.gradle +++ b/server/netty-app/build.gradle @@ -54,7 +54,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') - runtimeOnly project(':extensions-iceberg-aws') + runtimeOnly project(':extensions-iceberg:s3') } } diff --git a/settings.gradle b/settings.gradle index 00fe42e2764..9a240a942db 100644 --- a/settings.gradle +++ b/settings.gradle @@ -270,8 +270,8 @@ project(':extensions-s3').projectDir = file('extensions/s3') include(':extensions-iceberg') project(':extensions-iceberg').projectDir = file('extensions/iceberg') -include(':extensions-iceberg-aws') -project(':extensions-iceberg-aws').projectDir = file('extensions/iceberg-aws') +include(':extensions-iceberg:s3') +project(':extensions-iceberg:s3').projectDir = file('extensions/iceberg/s3') include(':plugin')