Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Initial commit of Iceberg integration. #5277

Merged
merged 31 commits into from
Jun 5, 2024
Merged
Show file tree
Hide file tree
Changes from 8 commits
Commits
Show all changes
31 commits
Select commit Hold shift + click to select a range
54e0a80
Initial commit of Iceberg integration.
lbooker42 Mar 22, 2024
780f2b1
Ready for auto-spotless anytime!
lbooker42 Mar 22, 2024
585a26e
Refactored to use internal parquet objects instead of re-implementing…
lbooker42 Apr 2, 2024
816f6b3
Merge branch 'main' into lab-iceberg
lbooker42 Apr 2, 2024
c2323d2
Rebased to main.
lbooker42 Apr 2, 2024
62952f4
update to use faster URI creation
lbooker42 Apr 2, 2024
e070b9f
Address PR comments.
lbooker42 Apr 2, 2024
c7487fe
Fix gradle broken-ness.
lbooker42 Apr 2, 2024
d4a80b8
Gradle comments, a few changes to IcebergInstructions
lbooker42 Apr 2, 2024
4f2ba28
Final gradle fix (uncomment the correct lines)
lbooker42 Apr 3, 2024
3bd98f9
Addressed PR comments, more testing needed.
lbooker42 Apr 18, 2024
de402ed
PR comments, improved testing.
lbooker42 Apr 19, 2024
f45e7d8
Merged with main.
lbooker42 Apr 22, 2024
f2900aa
merged with main
lbooker42 May 1, 2024
9762093
WIP
lbooker42 May 3, 2024
d7c2604
WIP, but test code implemented.
lbooker42 May 14, 2024
905c8ac
merged with main
lbooker42 May 15, 2024
a507402
Tests simplified and passing.
lbooker42 May 20, 2024
8db7923
Merge branch 'main' into lab-iceberg
lbooker42 May 20, 2024
0390102
Gradle cleanup.
lbooker42 May 22, 2024
4539bee
Simplified Iceberg instructions.
lbooker42 May 28, 2024
c5d6be1
Addressed many PR comments.
lbooker42 May 30, 2024
23e4a18
Attempted to handle partitioning columns correctly.
lbooker42 May 31, 2024
fa2e79d
Getting close to final.
lbooker42 May 31, 2024
d6065e4
Another rev from comments.
lbooker42 May 31, 2024
ea5ca0e
WIP, some updates.
lbooker42 Jun 3, 2024
35861c1
Merge branch 'main' into lab-iceberg
lbooker42 Jun 3, 2024
e51cf7c
Hadoop gradle version harmonization.
lbooker42 Jun 3, 2024
b408f12
Iceberg project restructure.
lbooker42 Jun 3, 2024
de7d1f3
Exposing 'iceberg-aws' in gradle.
lbooker42 Jun 3, 2024
23d6e32
Addressing PR comments.
lbooker42 Jun 4, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
65 changes: 65 additions & 0 deletions extensions/iceberg/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
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')
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
api project(':engine-table')

implementation project(':engine-table')
rcaudy marked this conversation as resolved.
Show resolved Hide resolved
implementation project(':engine-base')
implementation project(':log-factory')
implementation project(':Configuration')
implementation depTrove3
rcaudy marked this conversation as resolved.
Show resolved Hide resolved

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}"
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
implementation "org.apache.iceberg:iceberg-bundled-guava:${icebergVersion}"

implementation "org.apache.hadoop:hadoop-common:${hadoopVersion}"
rcaudy marked this conversation as resolved.
Show resolved Hide resolved
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}"

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)
1 change: 1 addition & 0 deletions extensions/iceberg/gradle.properties
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
io.deephaven.project.ProjectType=JAVA_PUBLIC
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
//
// 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.impl.locations.TableDataException;
import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
import io.deephaven.iceberg.location.IcebergTableLocationKey;
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 org.apache.iceberg.Snapshot}
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
*/
public final class IcebergFlatLayout implements TableLocationKeyFinder<IcebergTableLocationKey> {
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
private final Snapshot tableSnapshot;
private final FileIO fileIO;
private final Map<URI, IcebergTableLocationKey> cache;
private final Object readInstructions;

private static IcebergTableLocationKey locationKey(
final FileFormat format,
final URI fileUri,
@NotNull final Object readInstructions) {
return new IcebergTableLocationKey(format, 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 Object readInstructions) {
this.tableSnapshot = tableSnapshot;
this.fileIO = fileIO;
this.readInstructions = readInstructions;

this.cache = new HashMap<>();
}

public String toString() {
return IcebergFlatLayout.class.getSimpleName() + '[' + tableSnapshot + ']';
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
}

@Override
public synchronized void findKeys(@NotNull final Consumer<IcebergTableLocationKey> locationKeyObserver) {
try {
// Retrieve the manifest files from the snapshot
final List<ManifestFile> manifestFiles = tableSnapshot.allManifests(fileIO);
for (final ManifestFile manifestFile : manifestFiles) {
final ManifestReader<DataFile> reader = ManifestFiles.read(manifestFile, fileIO);
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
for (DataFile df : reader) {
final URI fileUri = FileUtils.convertToURI(df.path().toString(), false);
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
IcebergTableLocationKey locationKey = cache.get(fileUri);
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
if (locationKey == null) {
locationKey = locationKey(df.format(), 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);
}
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
//
// 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.impl.locations.TableDataException;
import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
import io.deephaven.iceberg.location.IcebergTableLocationKey;
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.function.Consumer;

/**
* Iceberg {@link TableLocationKeyFinder location finder} for tables without partitions that will discover data files
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
* from a {@link org.apache.iceberg.Snapshot}
*/
public final class IcebergKeyValuePartitionedLayout implements TableLocationKeyFinder<IcebergTableLocationKey> {
private final Snapshot tableSnapshot;
private final FileIO fileIO;
private final String[] partitionColumns;
private final Object readInstructions;
private final Map<URI, IcebergTableLocationKey> cache;

private static IcebergTableLocationKey locationKey(
final FileFormat format,
final URI fileUri,
final Map<String, Comparable<?>> partitions,
@NotNull final Object readInstructions) {
return new IcebergTableLocationKey(format, 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 IcebergKeyValuePartitionedLayout(
@NotNull final Snapshot tableSnapshot,
@NotNull final FileIO fileIO,
@NotNull final String[] partitionColumns,
@NotNull final Object 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 + ']';
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
}

@Override
public synchronized void findKeys(@NotNull final Consumer<IcebergTableLocationKey> locationKeyObserver) {
final Map<String, Comparable<?>> partitions = new LinkedHashMap<>();
try {
// Retrieve the manifest files from the snapshot
final List<ManifestFile> manifestFiles = tableSnapshot.allManifests(fileIO);
for (final ManifestFile manifestFile : manifestFiles) {
final ManifestReader<DataFile> reader = ManifestFiles.read(manifestFile, fileIO);
for (DataFile df : reader) {
final URI fileUri = FileUtils.convertToURI(df.path().toString(), false);
IcebergTableLocationKey locationKey = cache.get(fileUri);
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
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));
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
}
locationKey = locationKey(df.format(), 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);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
//
// 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 {
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved

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<SortColumn> getSortedColumns() {
return internalTableLocation.getSortedColumns();
}

@Override
@NotNull
public List<String[]> 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();
}
}
Loading
Loading