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

Pass-through IcebergIO catalog properties #31726

Merged
merged 9 commits into from
Jul 8, 2024

Conversation

ahmedabu98
Copy link
Contributor

@ahmedabu98 ahmedabu98 commented Jun 29, 2024

The current IcebergIO implementation supports only a fixed set of catalog properties. This is an unnecessary limitation for users and in some cases makes the connector unusable (e.g. if an unsupported property is required).

The changes in this PR opens it up so that properties can be specified in a key-store fashion and then simply be passed through to the catalog.

Note: This is a breaking change, but it's worth going ahead with it considering the big upside. This is also a new connector with not many users.

@github-actions github-actions bot added the build label Jun 30, 2024
Copy link
Contributor

Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment assign set of reviewers

@Pure
public abstract long getAuthSessionTimeoutMillis();
public abstract Properties getProperties();
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@kennknowles any benefits to using java.util.Properties as opposed to a Map<String, String>?

@ahmedabu98
Copy link
Contributor Author

R: @kennknowles

CC: @BhupiSindhwani
CC: @emkornfield

Copy link
Contributor

github-actions bot commented Jul 1, 2024

Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control

@liferoad liferoad requested a review from kennknowles July 3, 2024 13:59
Copy link
Member

@kennknowles kennknowles left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Java change LGTM (I didn't look line by line but viewed a number of sample lines and this is a good change)

The dependencies I'm a little more worried about. Need to think hard about what becomes a required dependency. We could also put some of the "optional" deps in the ManagedIO layer instead of this internal module.

@@ -98,7 +98,7 @@ dependencies {
permitUnusedDeclared enforcedPlatform(library.java.google_cloud_platform_libraries_bom)
provided library.java.json_org
implementation library.java.everit_json_schema
implementation library.java.snake_yaml
shadow library.java.snake_yaml
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm afraid of this getting changed again. Is it necessarily part of this change? Is it safe now?

@@ -23,6 +23,8 @@ import java.util.stream.Collectors
plugins { id 'org.apache.beam.module' }
applyJavaNature(
automaticModuleName: 'org.apache.beam.sdk.io.iceberg',
shadowClosure: {},
validateShadowJar: false,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why false? I think we want to validate it. Otherwise we end up having duplicate classes in multiple jars. Basically we want everything in the jar to be in our namespace, not any other namespace.

@@ -54,11 +56,12 @@ dependencies {
implementation "org.apache.iceberg:iceberg-parquet:$iceberg_version"
implementation "org.apache.iceberg:iceberg-orc:$iceberg_version"
implementation library.java.hadoop_common
// Hadoop GCS filesystem dependencies
runtimeOnly library.java.hadoop_client
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think I'm OK with this... it isn't very Java-like to put optional dependencies as mandatory dependencies though. Typically these should also be resolvable via transitive dependencies if we depend on e.g. GCS core which provides the GCS filesystem.

@ahmedabu98
Copy link
Contributor Author

Thanks for taking a look @kennknowles. I see what you mean about including dependencies only if they're mandatory. These dependency changes aren't necessary for this PR so I reverted them so we can get this in.

I wonder if we can improve IcebergIO's dependency story in a future PR though. Currently a user writing to a GCS Iceberg table has to include these dependencies:

    runtimeOnly "org.yaml:snakeyaml:2.0"
    runtimeOnly "com.google.cloud.bigdataoss:gcsio:2.2.16"
    runtimeOnly "com.google.cloud.bigdataoss:gcs-connector:hadoop2-2.2.16"
    runtimeOnly "org.apache.hadoop:hadoop-client:2.10.2"

Maybe we can provide an uber jar just for these dependencies (e.g. beam-sdks-java-io-iceberg-gcs-deps) so users can simply include it along with the core beam-sdks-java-io-iceberg jar. We may similarly need other jars for different catalogs we want to support. Let me know if there's a more Java-esque way of doing this.

@ahmedabu98 ahmedabu98 merged commit ac423af into apache:master Jul 8, 2024
22 checks passed
acrites pushed a commit to acrites/beam that referenced this pull request Jul 17, 2024
* Pass-through iceberg catalog properties

* add to CHANGES.md

* trigger integration test

* remove custom configuration; pass catalog name
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants