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 1 commit
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
6 changes: 4 additions & 2 deletions extensions/iceberg/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -38,16 +38,18 @@ 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}"

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}"
// 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}"
malhotrashivam marked this conversation as resolved.
Show resolved Hide resolved

testImplementation depCommonsCompress
testImplementation depCommonsIo
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
*/
Expand All @@ -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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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);
malhotrashivam marked this conversation as resolved.
Show resolved Hide resolved
} else {
Expand All @@ -91,6 +97,22 @@ public class IcebergCatalog {
catalog.initialize(catalogName, properties);
}

private static S3Instructions buildS3Instructions(final Map<String, String> properties) {
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
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<TableIdentifier> listTables(final Namespace namespace) {
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
// TODO: have this return a Deephaven Table of table identifiers
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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<String> s3AccessKeyId();

/**
* The AWS secret access key, used to authenticate the user interacting with services.
*/
public abstract String s3SecretAccessKey();
public abstract Optional<String> s3SecretAccessKey();

/**
* The AWS region to use for this connection.
*/
public abstract String s3Region();
public abstract Optional<String> s3Region();

/**
* The {@link io.deephaven.extensions.s3.S3Instructions} to use for reading the Iceberg data files.
*/
public abstract Optional<Object> readInstructions();

public interface Builder {
Builder catalogImpl(CATALOG_IMPL catalogImpl);
Expand All @@ -110,6 +115,9 @@ public interface Builder {

Builder s3Region(String s3Region);

@SuppressWarnings("unused")
Builder readInstructions(Object readInstructions);

IcebergInstructions build();
}

Expand All @@ -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");
}
}
}

}
Loading