-
Notifications
You must be signed in to change notification settings - Fork 3.7k
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
AWS Glue Catalog for Iceberg ingest extension #17392
Open
Shekharrajak
wants to merge
26
commits into
apache:master
Choose a base branch
from
Shekharrajak:feature-gluecatalog
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from 8 commits
Commits
Show all changes
26 commits
Select commit
Hold shift + click to select a range
8d700da
iceberg glue catalog dependencies added
shekhar-rajak c84ce7b
GlueIcebergCatalog added in druid module
shekhar-rajak c66a39f
default version of iceberg glue catalog implementation - basics
shekhar-rajak e6efb4a
basic tests added
shekhar-rajak f311706
removed dependecy iceberg-aws-bundle
shekhar-rajak f211f8e
glue catalog support - docs update for iceberg
shekhar-rajak fc68a48
Update IcebergDruidModule.java
shekhar-rajak efeac84
Update IcebergDruidModule.java
shekhar-rajak 8cec695
updates in dependencies and warehousePath must be under catalogProp
shekhar-rajak 78c53ab
removed some dependencies - which not required
shekhar-rajak 4df3857
only glue sdk added
shekhar-rajak 4bae037
update license
shekhar-rajak a617d4d
avro exclusion removed
shekhar-rajak 02e18e2
doc update
shekhar-rajak ead345d
doc update
shekhar-rajak bc7a6ab
set the type to glue
shekhar-rajak 928c3ce
minor change
shekhar-rajak 2b0cfba
minor change
shekhar-rajak 9cc2423
fixing codestyle
shekhar-rajak 9cf321f
checkstyle fixes
shekhar-rajak 67be023
checkstyle fixes
shekhar-rajak 6145f5f
checkstyle fixes
shekhar-rajak 81aaa14
dependency check fixes
shekhar-rajak a7a4daf
update pom for ignore warning for glue catalog
shekhar-rajak eb94233
compile scope needed - iceberg-aws and awssdk
shekhar-rajak b0edb4e
updates pom with comment
shekhar-rajak File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
70 changes: 70 additions & 0 deletions
70
...d-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/GlueIcebergCatalog.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,70 @@ | ||
package org.apache.druid.iceberg.input; | ||
|
||
import com.fasterxml.jackson.annotation.JacksonInject; | ||
import com.fasterxml.jackson.annotation.JsonCreator; | ||
import com.fasterxml.jackson.annotation.JsonProperty; | ||
import com.fasterxml.jackson.databind.ObjectMapper; | ||
import com.google.common.base.Preconditions; | ||
import org.apache.druid.guice.annotations.Json; | ||
import org.apache.druid.java.util.common.logger.Logger; | ||
import org.apache.druid.utils.DynamicConfigProviderUtils; | ||
import org.apache.iceberg.CatalogProperties; | ||
import org.apache.iceberg.catalog.Catalog; | ||
import org.apache.iceberg.aws.glue.GlueCatalog; | ||
import javax.annotation.Nullable; | ||
import java.util.Map; | ||
|
||
public class GlueIcebergCatalog extends IcebergCatalog { | ||
private static final String CATALOG_NAME = "glue"; | ||
private Catalog catalog; | ||
|
||
public static final String TYPE_KEY = "glue"; | ||
|
||
@JsonProperty | ||
private String warehousePath; | ||
|
||
@JsonProperty | ||
private Map<String, String> catalogProperties; | ||
|
||
@JsonProperty | ||
private final Boolean caseSensitive; | ||
private static final Logger log = new Logger(GlueIcebergCatalog.class); | ||
|
||
@JsonCreator | ||
public GlueIcebergCatalog( | ||
@JsonProperty("warehousePath") String warehousePath, | ||
@JsonProperty("catalogProperties") @Nullable | ||
Map<String, Object> catalogProperties, | ||
@JsonProperty("caseSensitive") Boolean caseSensitive, | ||
@JacksonInject @Json ObjectMapper mapper | ||
) | ||
{ | ||
this.warehousePath = Preconditions.checkNotNull(warehousePath, "warehousePath cannot be null"); | ||
this.catalogProperties = DynamicConfigProviderUtils.extraConfigAndSetStringMap(catalogProperties, DRUID_DYNAMIC_CONFIG_PROVIDER_KEY, mapper); | ||
this.caseSensitive = caseSensitive == null ? true : caseSensitive; | ||
this.catalog = retrieveCatalog(); | ||
} | ||
|
||
@Override | ||
public Catalog retrieveCatalog() { | ||
if (catalog == null) { | ||
log.info("catalog is null, setting up default glue catalog."); | ||
catalog = setupGlueCatalog(); | ||
} | ||
log.info("Glue catalog set [%s].", catalog.toString()); | ||
return catalog; | ||
} | ||
|
||
private Catalog setupGlueCatalog() { | ||
catalog = new GlueCatalog(); | ||
catalogProperties.put(CatalogProperties.WAREHOUSE_LOCATION, warehousePath); | ||
catalog.initialize(CATALOG_NAME, catalogProperties); | ||
return catalog; | ||
} | ||
|
||
@Override | ||
public boolean isCaseSensitive() | ||
{ | ||
return caseSensitive; | ||
} | ||
} |
41 changes: 41 additions & 0 deletions
41
...eberg-extensions/src/test/java/org/apache/druid/iceberg/input/GlueIcebergCatalogTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,41 @@ | ||
package org.apache.druid.iceberg.input; | ||
|
||
|
||
import com.fasterxml.jackson.databind.ObjectMapper; | ||
import org.apache.druid.jackson.DefaultObjectMapper; | ||
import org.apache.druid.java.util.common.FileUtils; | ||
import org.junit.Assert; | ||
import org.junit.Test; | ||
|
||
import java.io.File; | ||
import java.util.HashMap; | ||
|
||
public class GlueIcebergCatalogTest { | ||
private final ObjectMapper mapper = new DefaultObjectMapper(); | ||
public void setUp() throws Exception { | ||
} | ||
|
||
public void tearDown() throws Exception { | ||
} | ||
|
||
@Test | ||
public void testCatalogCreate() { | ||
GlueIcebergCatalog glueCatalog = new GlueIcebergCatalog( | ||
"s3://testbucket/testpath", | ||
new HashMap<>(), | ||
true, | ||
mapper | ||
); | ||
Assert.assertEquals("glue", glueCatalog.retrieveCatalog().name()); | ||
} | ||
@Test | ||
public void testIsCaseSensitive() { | ||
GlueIcebergCatalog glueCatalog = new GlueIcebergCatalog( | ||
"s3://testbucket/testpath", | ||
new HashMap<>(), | ||
true, | ||
mapper | ||
); | ||
Assert.assertEquals(true, glueCatalog.isCaseSensitive()); | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
catalog properties must have these key value pairs
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
warehouse path must be s3://bucket/path
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
AWS related env variables must be available where druid cluster is running.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could we add more information related to this in the docs specific to the glue catalog?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, I will do that. Recently figured out that there is simpler approach in iceberg API itself to choose the catalog. I am spending sometime to check if that would drastically make it modular & work for all available iceberg catalog support on the fly.