|
24 | 24 | import java.io.IOException;
|
25 | 25 | import java.io.ObjectInputStream;
|
26 | 26 | import java.io.ObjectOutputStream;
|
| 27 | +import org.apache.hadoop.conf.Configuration; |
27 | 28 | import org.apache.hadoop.fs.FileSystem;
|
28 | 29 | import org.apache.hadoop.fs.Path;
|
29 | 30 | import org.apache.iceberg.Schema;
|
|
32 | 33 | import org.apache.iceberg.hadoop.HadoopFileIO;
|
33 | 34 | import org.apache.iceberg.hadoop.HadoopTables;
|
34 | 35 | import org.apache.iceberg.io.FileIO;
|
| 36 | +import org.apache.iceberg.io.ResolvingFileIO; |
35 | 37 | import org.apache.iceberg.relocated.com.google.common.collect.Maps;
|
36 | 38 | import org.apache.iceberg.types.Types;
|
37 | 39 | import org.assertj.core.api.Assertions;
|
@@ -94,10 +96,19 @@ private static void validateTableLoader(TableLoader loader)
|
94 | 96 | private static void validateHadoopConf(Table table) {
|
95 | 97 | FileIO io = table.io();
|
96 | 98 | 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")); |
101 | 112 | }
|
102 | 113 |
|
103 | 114 | @SuppressWarnings("unchecked")
|
|
0 commit comments