pvary commented on code in PR #10926: URL: https://github.com/apache/iceberg/pull/10926#discussion_r1735204259
########## core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java: ########## @@ -63,7 +63,11 @@ public class HadoopFileIO implements HadoopConfigurable, DelegateFileIO { * <p>{@link Configuration Hadoop configuration} must be set through {@link * HadoopFileIO#setConf(Configuration)} */ - public HadoopFileIO() {} + public HadoopFileIO() { + // Create a default hadoopConf as it is required for the object to be valid. + // E.g. newInputFile would throw NPE with hadoopConf.get() otherwise. + this.hadoopConf = new SerializableConfiguration(new Configuration())::get; Review Comment: If we need to do the FileIO serialization, then we will add a serious amount of data for the tasks: ``` @Test public void testA() throws IOException { Configuration confWithDefaults = new Configuration(); Configuration confWithoutDefaults = new Configuration(false); SerializableConfiguration sconfWithDefaults = new SerializableConfiguration(confWithDefaults); SerializableConfiguration sconfWithoutDefaults = new SerializableConfiguration(confWithoutDefaults); byte[] sconfWithDefaultBytes = SerializationUtil.serializeToBytes(sconfWithDefaults); byte[] sconfWithoutDefaultBytes = SerializationUtil.serializeToBytes(sconfWithoutDefaults); String jsonWithDefaults = String.join(", " , Lists.newArrayList(confWithDefaults.iterator()).stream().map(e -> e.getKey() + ":" + e.getValue()).collect(Collectors.toList())); ByteArrayOutputStream out = new ByteArrayOutputStream(); DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); view.writeUTF(jsonWithDefaults); byte[] jsonWithDefaultsBytes = out.toByteArray(); String jsonWithoutDefaults = String.join(", " , Lists.newArrayList(confWithDefaults.iterator()).stream().map(e -> e.getKey() + ":" + e.getValue()).collect(Collectors.toList())); out = new ByteArrayOutputStream(); view = new DataOutputViewStreamWrapper(out); view.writeUTF(jsonWithoutDefaults); byte[] jsonWithoutDefaultsBytes = out.toByteArray(); System.out.println("Binary serialization length with defaults: " + sconfWithDefaultBytes.length); System.out.println("Binary serialization length without defaults: " + sconfWithoutDefaultBytes.length); System.out.println("JSON serialization length with defaults: " + jsonWithDefaultsBytes.length); System.out.println("JSON serialization length without defaults: " + jsonWithoutDefaultsBytes.length); } ``` Results in: ``` Binary serialization length with defaults: 68638 Binary serialization length without defaults: 76 JSON serialization length with defaults: 36282 JSON serialization length without defaults: 36282 ``` I have seen that we use `new Configuration(false)` in the code, so we allow for the user to provide a trimmed configuration, and in this case the serialized config is quite small for the binary serialization. We might have to do something similar for the JSON serialization. We use the serialized FileIO for: - ManifestListReadTask - ManifestReadTask If we step back one more time. We might consider why do we need the FileIO in the `ManifestListReadTask` at all? The `ManifestListReadTask.rows()` and `ManifestListReadTask.file()` is using the `io` to get the new input file like `io.newInputFile(manifestListLocation)`. I think this FileIO serialization will become even more complicated when the manifest file encryption arrives here. We will need to apply the encryption for the FileIO like this: ``` EncryptingFileIO.combine(table().io(), table().encryption()) ``` We could serialize/deserialize the DataFile to return the `ManifestListReadTask.file()` like we do it for the `BaseFileScanTask`, but I don't see a better option for the `ManifestListReadTask.rows()` yet 😢 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org