Skip to content

Commit fcbc64e

Browse files
committed
Core, Hive, Nessie: Use ResolvingFileIO as default instead of HadoopFileIO
1 parent f5f543a commit fcbc64e

File tree

9 files changed

+98
-36
lines changed

9 files changed

+98
-36
lines changed

core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java

+2-3
Original file line numberDiff line numberDiff line change
@@ -67,6 +67,7 @@ public class JdbcCatalog extends BaseMetastoreCatalog
6767
private static final String NAMESPACE_EXISTS_PROPERTY = "exists";
6868
private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
6969
private static final Joiner SLASH = Joiner.on("/");
70+
private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
7071

7172
private FileIO io;
7273
private String catalogName = "jdbc";
@@ -112,9 +113,7 @@ public void initialize(String name, Map<String, String> properties) {
112113
if (null != ioBuilder) {
113114
this.io = ioBuilder.apply(properties);
114115
} else {
115-
String ioImpl =
116-
properties.getOrDefault(
117-
CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");
116+
String ioImpl = properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, DEFAULT_FILE_IO_IMPL);
118117
this.io = CatalogUtil.loadFileIO(ioImpl, properties, conf);
119118
}
120119

flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java

+15-4
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import java.io.ObjectInputStream;
2828
import java.io.ObjectOutputStream;
2929
import java.util.Map;
30+
import org.apache.hadoop.conf.Configuration;
3031
import org.apache.hadoop.fs.FileSystem;
3132
import org.apache.hadoop.fs.Path;
3233
import org.apache.iceberg.CatalogProperties;
@@ -35,6 +36,7 @@
3536
import org.apache.iceberg.catalog.TableIdentifier;
3637
import org.apache.iceberg.hadoop.HadoopFileIO;
3738
import org.apache.iceberg.io.FileIO;
39+
import org.apache.iceberg.io.ResolvingFileIO;
3840
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
3941
import org.apache.iceberg.types.Types;
4042
import org.assertj.core.api.Assertions;
@@ -97,10 +99,19 @@ private static void validateCatalogLoader(CatalogLoader loader)
9799
private static void validateHadoopConf(Table table) {
98100
FileIO io = table.io();
99101
Assertions.assertThat(io)
100-
.as("FileIO should be a HadoopFileIO")
101-
.isInstanceOf(HadoopFileIO.class);
102-
HadoopFileIO hadoopIO = (HadoopFileIO) io;
103-
Assert.assertEquals("my_value", hadoopIO.conf().get("my_key"));
102+
.as("FileIO should be a HadoopFileIO or ResolvingFileIO")
103+
.isInstanceOfAny(HadoopFileIO.class, ResolvingFileIO.class);
104+
105+
Configuration conf;
106+
if (io instanceof ResolvingFileIO) {
107+
ResolvingFileIO resolvingFileIO = (ResolvingFileIO) io;
108+
conf = resolvingFileIO.getConf();
109+
} else {
110+
HadoopFileIO hadoopIO = (HadoopFileIO) io;
111+
conf = hadoopIO.conf();
112+
}
113+
114+
Assert.assertEquals("my_value", conf.get("my_key"));
104115
}
105116

106117
@SuppressWarnings("unchecked")

flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java

+15-4
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import java.io.IOException;
2525
import java.io.ObjectInputStream;
2626
import java.io.ObjectOutputStream;
27+
import org.apache.hadoop.conf.Configuration;
2728
import org.apache.hadoop.fs.FileSystem;
2829
import org.apache.hadoop.fs.Path;
2930
import org.apache.iceberg.Schema;
@@ -32,6 +33,7 @@
3233
import org.apache.iceberg.hadoop.HadoopFileIO;
3334
import org.apache.iceberg.hadoop.HadoopTables;
3435
import org.apache.iceberg.io.FileIO;
36+
import org.apache.iceberg.io.ResolvingFileIO;
3537
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
3638
import org.apache.iceberg.types.Types;
3739
import org.assertj.core.api.Assertions;
@@ -94,10 +96,19 @@ private static void validateTableLoader(TableLoader loader)
9496
private static void validateHadoopConf(Table table) {
9597
FileIO io = table.io();
9698
Assertions.assertThat(io)
97-
.as("FileIO should be a HadoopFileIO")
98-
.isInstanceOf(HadoopFileIO.class);
99-
HadoopFileIO hadoopIO = (HadoopFileIO) io;
100-
Assert.assertEquals("my_value", hadoopIO.conf().get("my_key"));
99+
.as("FileIO should be a HadoopFileIO or ResolvingFileIO")
100+
.isInstanceOfAny(HadoopFileIO.class, ResolvingFileIO.class);
101+
102+
Configuration conf;
103+
if (io instanceof ResolvingFileIO) {
104+
ResolvingFileIO resolvingFileIO = (ResolvingFileIO) io;
105+
conf = resolvingFileIO.getConf();
106+
} else {
107+
HadoopFileIO hadoopIO = (HadoopFileIO) io;
108+
conf = hadoopIO.conf();
109+
}
110+
111+
Assert.assertEquals("my_value", conf.get("my_key"));
101112
}
102113

103114
@SuppressWarnings("unchecked")

flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java

+15-4
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import java.io.ObjectInputStream;
2828
import java.io.ObjectOutputStream;
2929
import java.util.Map;
30+
import org.apache.hadoop.conf.Configuration;
3031
import org.apache.hadoop.fs.FileSystem;
3132
import org.apache.hadoop.fs.Path;
3233
import org.apache.iceberg.CatalogProperties;
@@ -35,6 +36,7 @@
3536
import org.apache.iceberg.catalog.TableIdentifier;
3637
import org.apache.iceberg.hadoop.HadoopFileIO;
3738
import org.apache.iceberg.io.FileIO;
39+
import org.apache.iceberg.io.ResolvingFileIO;
3840
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
3941
import org.apache.iceberg.types.Types;
4042
import org.assertj.core.api.Assertions;
@@ -97,10 +99,19 @@ private static void validateCatalogLoader(CatalogLoader loader)
9799
private static void validateHadoopConf(Table table) {
98100
FileIO io = table.io();
99101
Assertions.assertThat(io)
100-
.as("FileIO should be a HadoopFileIO")
101-
.isInstanceOf(HadoopFileIO.class);
102-
HadoopFileIO hadoopIO = (HadoopFileIO) io;
103-
Assert.assertEquals("my_value", hadoopIO.conf().get("my_key"));
102+
.as("FileIO should be a HadoopFileIO or ResolvingFileIO")
103+
.isInstanceOfAny(HadoopFileIO.class, ResolvingFileIO.class);
104+
105+
Configuration conf;
106+
if (io instanceof ResolvingFileIO) {
107+
ResolvingFileIO resolvingFileIO = (ResolvingFileIO) io;
108+
conf = resolvingFileIO.getConf();
109+
} else {
110+
HadoopFileIO hadoopIO = (HadoopFileIO) io;
111+
conf = hadoopIO.conf();
112+
}
113+
114+
Assert.assertEquals("my_value", conf.get("my_key"));
104115
}
105116

106117
@SuppressWarnings("unchecked")

flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java

+15-4
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import java.io.IOException;
2525
import java.io.ObjectInputStream;
2626
import java.io.ObjectOutputStream;
27+
import org.apache.hadoop.conf.Configuration;
2728
import org.apache.hadoop.fs.FileSystem;
2829
import org.apache.hadoop.fs.Path;
2930
import org.apache.iceberg.Schema;
@@ -32,6 +33,7 @@
3233
import org.apache.iceberg.hadoop.HadoopFileIO;
3334
import org.apache.iceberg.hadoop.HadoopTables;
3435
import org.apache.iceberg.io.FileIO;
36+
import org.apache.iceberg.io.ResolvingFileIO;
3537
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
3638
import org.apache.iceberg.types.Types;
3739
import org.assertj.core.api.Assertions;
@@ -94,10 +96,19 @@ private static void validateTableLoader(TableLoader loader)
9496
private static void validateHadoopConf(Table table) {
9597
FileIO io = table.io();
9698
Assertions.assertThat(io)
97-
.as("FileIO should be a HadoopFileIO")
98-
.isInstanceOf(HadoopFileIO.class);
99-
HadoopFileIO hadoopIO = (HadoopFileIO) io;
100-
Assert.assertEquals("my_value", hadoopIO.conf().get("my_key"));
99+
.as("FileIO should be a HadoopFileIO or ResolvingFileIO")
100+
.isInstanceOfAny(HadoopFileIO.class, ResolvingFileIO.class);
101+
102+
Configuration conf;
103+
if (io instanceof ResolvingFileIO) {
104+
ResolvingFileIO resolvingFileIO = (ResolvingFileIO) io;
105+
conf = resolvingFileIO.getConf();
106+
} else {
107+
HadoopFileIO hadoopIO = (HadoopFileIO) io;
108+
conf = hadoopIO.conf();
109+
}
110+
111+
Assert.assertEquals("my_value", conf.get("my_key"));
101112
}
102113

103114
@SuppressWarnings("unchecked")

flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java

+15-4
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import java.io.ObjectInputStream;
2828
import java.io.ObjectOutputStream;
2929
import java.util.Map;
30+
import org.apache.hadoop.conf.Configuration;
3031
import org.apache.hadoop.fs.FileSystem;
3132
import org.apache.hadoop.fs.Path;
3233
import org.apache.iceberg.CatalogProperties;
@@ -35,6 +36,7 @@
3536
import org.apache.iceberg.catalog.TableIdentifier;
3637
import org.apache.iceberg.hadoop.HadoopFileIO;
3738
import org.apache.iceberg.io.FileIO;
39+
import org.apache.iceberg.io.ResolvingFileIO;
3840
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
3941
import org.apache.iceberg.types.Types;
4042
import org.assertj.core.api.Assertions;
@@ -97,10 +99,19 @@ private static void validateCatalogLoader(CatalogLoader loader)
9799
private static void validateHadoopConf(Table table) {
98100
FileIO io = table.io();
99101
Assertions.assertThat(io)
100-
.as("FileIO should be a HadoopFileIO")
101-
.isInstanceOf(HadoopFileIO.class);
102-
HadoopFileIO hadoopIO = (HadoopFileIO) io;
103-
Assert.assertEquals("my_value", hadoopIO.conf().get("my_key"));
102+
.as("FileIO should be a HadoopFileIO or ResolvingFileIO")
103+
.isInstanceOfAny(HadoopFileIO.class, ResolvingFileIO.class);
104+
105+
Configuration conf;
106+
if (io instanceof ResolvingFileIO) {
107+
ResolvingFileIO resolvingFileIO = (ResolvingFileIO) io;
108+
conf = resolvingFileIO.getConf();
109+
} else {
110+
HadoopFileIO hadoopIO = (HadoopFileIO) io;
111+
conf = hadoopIO.conf();
112+
}
113+
114+
Assert.assertEquals("my_value", conf.get("my_key"));
104115
}
105116

106117
@SuppressWarnings("unchecked")

flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java

+15-4
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import java.io.IOException;
2525
import java.io.ObjectInputStream;
2626
import java.io.ObjectOutputStream;
27+
import org.apache.hadoop.conf.Configuration;
2728
import org.apache.hadoop.fs.FileSystem;
2829
import org.apache.hadoop.fs.Path;
2930
import org.apache.iceberg.Schema;
@@ -32,6 +33,7 @@
3233
import org.apache.iceberg.hadoop.HadoopFileIO;
3334
import org.apache.iceberg.hadoop.HadoopTables;
3435
import org.apache.iceberg.io.FileIO;
36+
import org.apache.iceberg.io.ResolvingFileIO;
3537
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
3638
import org.apache.iceberg.types.Types;
3739
import org.assertj.core.api.Assertions;
@@ -94,10 +96,19 @@ private static void validateTableLoader(TableLoader loader)
9496
private static void validateHadoopConf(Table table) {
9597
FileIO io = table.io();
9698
Assertions.assertThat(io)
97-
.as("FileIO should be a HadoopFileIO")
98-
.isInstanceOf(HadoopFileIO.class);
99-
HadoopFileIO hadoopIO = (HadoopFileIO) io;
100-
Assert.assertEquals("my_value", hadoopIO.conf().get("my_key"));
99+
.as("FileIO should be a HadoopFileIO or ResolvingFileIO")
100+
.isInstanceOfAny(HadoopFileIO.class, ResolvingFileIO.class);
101+
102+
Configuration conf;
103+
if (io instanceof ResolvingFileIO) {
104+
ResolvingFileIO resolvingFileIO = (ResolvingFileIO) io;
105+
conf = resolvingFileIO.getConf();
106+
} else {
107+
HadoopFileIO hadoopIO = (HadoopFileIO) io;
108+
conf = hadoopIO.conf();
109+
}
110+
111+
Assert.assertEquals("my_value", conf.get("my_key"));
101112
}
102113

103114
@SuppressWarnings("unchecked")

hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java

+4-6
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,6 @@
4747
import org.apache.iceberg.exceptions.NoSuchNamespaceException;
4848
import org.apache.iceberg.exceptions.NoSuchTableException;
4949
import org.apache.iceberg.exceptions.NotFoundException;
50-
import org.apache.iceberg.hadoop.HadoopFileIO;
5150
import org.apache.iceberg.io.FileIO;
5251
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
5352
import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
@@ -72,6 +71,7 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
7271
static final String HIVE_CONF_CATALOG = "metastore.catalog.default";
7372

7473
private static final Logger LOG = LoggerFactory.getLogger(HiveCatalog.class);
74+
private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
7575

7676
private String name;
7777
private Configuration conf;
@@ -104,11 +104,9 @@ public void initialize(String inputName, Map<String, String> properties) {
104104
this.listAllTables =
105105
Boolean.parseBoolean(properties.getOrDefault(LIST_ALL_TABLES, LIST_ALL_TABLES_DEFAULT));
106106

107-
String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
108-
this.fileIO =
109-
fileIOImpl == null
110-
? new HadoopFileIO(conf)
111-
: CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
107+
String fileIOImpl =
108+
properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, DEFAULT_FILE_IO_IMPL);
109+
this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
112110

113111
this.clients = new CachedClientPool(conf, properties);
114112
}

nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java

+2-3
Original file line numberDiff line numberDiff line change
@@ -63,6 +63,7 @@ public class NessieCatalog extends BaseMetastoreCatalog
6363
implements AutoCloseable, SupportsNamespaces, Configurable<Object> {
6464

6565
private static final Logger LOG = LoggerFactory.getLogger(NessieCatalog.class);
66+
private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
6667
private static final Joiner SLASH = Joiner.on("/");
6768
private static final String NAMESPACE_LOCATION_PROPS = "location";
6869
private NessieIcebergClient client;
@@ -79,9 +80,7 @@ public NessieCatalog() {}
7980
@Override
8081
public void initialize(String name, Map<String, String> options) {
8182
Map<String, String> catalogOptions = ImmutableMap.copyOf(options);
82-
String fileIOImpl =
83-
options.getOrDefault(
84-
CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.hadoop.HadoopFileIO");
83+
String fileIOImpl = options.getOrDefault(CatalogProperties.FILE_IO_IMPL, DEFAULT_FILE_IO_IMPL);
8584
// remove nessie prefix
8685
final Function<String, String> removePrefix =
8786
x -> x.replace(NessieUtil.NESSIE_CONFIG_PREFIX, "");

0 commit comments

Comments
 (0)