rdblue commented on a change in pull request #1640:
URL: https://github.com/apache/iceberg/pull/1640#discussion_r513117643
##########
File path: flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java
##########
@@ -105,4 +113,54 @@ public String toString() {
.toString();
}
}
+
+ class CustomCatalogLoader implements CatalogLoader {
+
+ private final SerializableConfiguration hadoopConf;
+ private final Map<String, String> properties;
+ private final String name;
+ private final String impl;
+
+ private CustomCatalogLoader(
+ String name,
+ Map<String, String> properties,
+ Configuration conf,
+ String impl) {
+ this.hadoopConf = new SerializableConfiguration(conf);
+ this.properties = new HashMap<>(properties); // use hashmap for
serialization
+ this.name = name;
+ this.impl = Preconditions.checkNotNull(impl,
+ "Cannot initialize custom Catalog because impl property is not set");
+ }
+
+ @Override
+ public Catalog loadCatalog() {
+ DynConstructors.Ctor<Catalog> ctor;
+ try {
+ ctor = DynConstructors.builder(Catalog.class)
+ .impl(impl, Map.class, Configuration.class) // take in flink
properties and hadoop configs
+ .impl(impl) // fall back to no-arg constructor
Review comment:
We definitely need an option to create a catalog without passing
`Configuration` but still passing config properties. I originally thought that
it would make sense to use another constructor, but then I thought about how
`name` is passed... and I think the number of possible constructors may get out
of hand.
Instead of adding a lot of constructors, I think we should do this:
1. Use a no-arg constructor for all catalogs
2. Add an `initialize` method to catalogs that is called to pass the catalog
name and a string map of config (this matches what Spark does)
3. If the catalog implements Hadoop's `Configurable` interface, also call
`setConf` to set the Hadoop config.
That way, we avoid having `Configuration` in any of our APIs and minimize
the number of constructors that we need to support. What do you think,
@jackye1995?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]