Skip to content

Commit

Permalink
reflection cleanup
Browse files Browse the repository at this point in the history
  • Loading branch information
Bryan Keller committed Jan 21, 2024
1 parent ea95f21 commit 8892606
Showing 1 changed file with 8 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;

import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.net.URL;
import java.nio.file.Files;
import java.nio.file.Path;
Expand All @@ -39,6 +38,7 @@
import org.apache.iceberg.Table;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.common.DynClasses;
import org.apache.iceberg.common.DynConstructors;
import org.apache.iceberg.common.DynMethods;
import org.apache.iceberg.common.DynMethods.BoundMethod;
import org.apache.iceberg.connect.IcebergSinkConfig;
Expand Down Expand Up @@ -74,19 +74,19 @@ public static Catalog loadCatalog(IcebergSinkConfig config) {
// use reflection here to avoid requiring Hadoop as a dependency
private static Object loadHadoopConfig(IcebergSinkConfig config) {
Class<?> configClass =
DynClasses.builder().impl("org.apache.hadoop.hdfs.HdfsConfiguration").orNull().build();
if (configClass == null) {
configClass =
DynClasses.builder().impl("org.apache.hadoop.conf.Configuration").orNull().build();
}
DynClasses.builder()
.impl("org.apache.hadoop.hdfs.HdfsConfiguration")
.impl("org.apache.hadoop.conf.Configuration")
.orNull()
.build();

if (configClass == null) {
LOG.info("Hadoop not found on classpath, not creating Hadoop config");
return null;
}

try {
Object result = configClass.getDeclaredConstructor().newInstance();
Object result = DynConstructors.builder().hiddenImpl(configClass).build().newInstance();
BoundMethod addResourceMethod =
DynMethods.builder("addResource").impl(configClass, URL.class).build(result);
BoundMethod setMethod =
Expand All @@ -113,10 +113,7 @@ private static Object loadHadoopConfig(IcebergSinkConfig config) {

LOG.info("Hadoop config initialized: {}", configClass.getName());
return result;
} catch (InstantiationException
| IllegalAccessException
| NoSuchMethodException
| InvocationTargetException e) {
} catch (Exception e) {
LOG.warn(
"Hadoop found on classpath but could not create config, proceeding without config", e);
}
Expand Down

0 comments on commit 8892606

Please sign in to comment.