This is an automated email from the ASF dual-hosted git repository.

amoghj pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/main by this push:
     new cbd71ebd12 Core,AWS: Fix NPE in ResolvingFileIO when HadoopConf is not 
set (#10872)
cbd71ebd12 is described below

commit cbd71ebd12f70da9449a6a1755b141054bbfb389
Author: S N Munendra <[email protected]>
AuthorDate: Thu Aug 22 09:35:23 2024 +0530

    Core,AWS: Fix NPE in ResolvingFileIO when HadoopConf is not set (#10872)
---
 .../test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java    | 12 ++++++++++++
 .../src/main/java/org/apache/iceberg/io/ResolvingFileIO.java | 10 ++++++----
 2 files changed, 18 insertions(+), 4 deletions(-)

diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java 
b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java
index 38489e3674..1cc34a59cc 100644
--- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java
+++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java
@@ -384,6 +384,18 @@ public class TestS3FileIO {
     assertThat(result).isInstanceOf(S3FileIO.class);
   }
 
+  @Test
+  public void testResolvingFileIOLoadWithoutConf() {
+    ResolvingFileIO resolvingFileIO = new ResolvingFileIO();
+    resolvingFileIO.initialize(ImmutableMap.of());
+    FileIO result =
+        DynMethods.builder("io")
+            .hiddenImpl(ResolvingFileIO.class, String.class)
+            .build(resolvingFileIO)
+            .invoke("s3://foo/bar");
+    assertThat(result).isInstanceOf(S3FileIO.class);
+  }
+
   @Test
   public void testInputFileWithDataFile() throws IOException {
     String location = "s3://bucket/path/to/data-file.parquet";
diff --git a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java 
b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java
index b5232960dc..a858045aab 100644
--- a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java
+++ b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java
@@ -21,8 +21,10 @@ package org.apache.iceberg.io;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Function;
+import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.iceberg.CatalogUtil;
@@ -141,7 +143,7 @@ public class ResolvingFileIO implements HadoopConfigurable, 
DelegateFileIO {
   @Override
   public void serializeConfWith(
       Function<Configuration, SerializableSupplier<Configuration>> 
confSerializer) {
-    this.hadoopConf = confSerializer.apply(hadoopConf.get());
+    this.hadoopConf = confSerializer.apply(getConf());
   }
 
   @Override
@@ -151,7 +153,7 @@ public class ResolvingFileIO implements HadoopConfigurable, 
DelegateFileIO {
 
   @Override
   public Configuration getConf() {
-    return hadoopConf.get();
+    return Optional.ofNullable(hadoopConf).map(Supplier::get).orElse(null);
   }
 
   @VisibleForTesting
@@ -163,7 +165,7 @@ public class ResolvingFileIO implements HadoopConfigurable, 
DelegateFileIO {
         synchronized (io) {
           if (((HadoopConfigurable) io).getConf() == null) {
             // re-apply the config in case it's null after Kryo serialization
-            ((HadoopConfigurable) io).setConf(hadoopConf.get());
+            ((HadoopConfigurable) io).setConf(getConf());
           }
         }
       }
@@ -174,7 +176,7 @@ public class ResolvingFileIO implements HadoopConfigurable, 
DelegateFileIO {
     return ioInstances.computeIfAbsent(
         impl,
         key -> {
-          Configuration conf = hadoopConf.get();
+          Configuration conf = getConf();
           FileIO fileIO;
 
           try {

Reply via email to