Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
23 changes: 21 additions & 2 deletions core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package org.apache.iceberg.hadoop;

import java.io.IOException;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
Expand All @@ -29,9 +30,17 @@
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.util.SerializableSupplier;

public class HadoopFileIO implements FileIO {
public class HadoopFileIO implements FileIO, Configurable {

private final SerializableSupplier<Configuration> hadoopConf;
private SerializableSupplier<Configuration> hadoopConf;

/**
* Constructor used for dynamic FileIO loading.
* <p>
* {@link Configuration Hadoop configuration} must be set through {@link HadoopFileIO#setConf(Configuration)}
*/
public HadoopFileIO() {
}

public HadoopFileIO(Configuration hadoopConf) {
this(new SerializableConfiguration(hadoopConf)::get);
Expand Down Expand Up @@ -65,4 +74,14 @@ public void deleteFile(String path) {
throw new RuntimeIOException(e, "Failed to delete file: %s", path);
}
}

@Override
public void setConf(Configuration conf) {
this.hadoopConf = new SerializableConfiguration(conf)::get;
}

@Override
public Configuration getConf() {
return hadoopConf.get();
}
}
10 changes: 10 additions & 0 deletions core/src/test/java/org/apache/iceberg/TestCatalogUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.hadoop.HadoopFileIO;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.io.OutputFile;
Expand Down Expand Up @@ -123,6 +124,15 @@ public void loadCustomFileIO_noArg() {
Assert.assertEquals(properties, ((TestFileIONoArg) fileIO).map);
}

@Test
public void loadCustomFileIO_hadoopConfigConstructor() {
Configuration configuration = new Configuration();
configuration.set("key", "val");
FileIO fileIO = CatalogUtil.loadFileIO(HadoopFileIO.class.getName(), Maps.newHashMap(), configuration);
Assert.assertTrue(fileIO instanceof HadoopFileIO);
Assert.assertEquals("val", ((HadoopFileIO) fileIO).conf().get("key"));
}

@Test
public void loadCustomFileIO_configurable() {
Configuration configuration = new Configuration();
Expand Down