nastra commented on code in PR #10926: URL: https://github.com/apache/iceberg/pull/10926#discussion_r1719928348
########## 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: > I am wondering if we should change FileIOParser to serialize and deserialize Hadoop Configuration when the FileIO is HadoopConfigurable. We can probably only serialize the key-value string pairs from the Configuration as a JSON object (kind of a read only copy). I am also a bit undecided on this topic, but was also wondering whether we should ser/de the Hadoop config in the `FileIOParser`. I did a quick check and I think something like the below might work, where we ser/de the actual bytes. `SerializationUtil` currently takes the entire `FileIO` object rather than just the `Configuration` in the below example, but if we would want to go down that route we can most likely improve this. ``` --- a/core/src/main/java/org/apache/iceberg/io/FileIOParser.java +++ b/core/src/main/java/org/apache/iceberg/io/FileIOParser.java @@ -21,10 +21,16 @@ package org.apache.iceberg.io; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Map; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hadoop.HadoopConfigurable; +import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; +import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.SerializationUtil; public class FileIOParser { private FileIOParser() {} @@ -61,6 +67,12 @@ public class FileIOParser { generator.writeStringField(FILE_IO_IMPL, impl); JsonUtil.writeStringMap(PROPERTIES, properties, generator); + if (io instanceof HadoopFileIO) { + ByteBuffer wrap = ByteBuffer.wrap(SerializationUtil.serializeToBytes(io)); + generator.writeStringField( + "hadoopConf", BaseEncoding.base16().encode(ByteBuffers.toByteArray(wrap))); + } + generator.writeEndObject(); } @@ -76,6 +88,14 @@ public class FileIOParser { Preconditions.checkArgument(json.isObject(), "Cannot parse FileIO from non-object: %s", json); String impl = JsonUtil.getString(FILE_IO_IMPL, json); Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, json); - return CatalogUtil.loadFileIO(impl, properties, conf); + FileIO fileIO = CatalogUtil.loadFileIO(impl, properties, conf); + + if (json.hasNonNull("hadoopConf") && fileIO instanceof HadoopFileIO) { + ByteBuffer buffer = JsonUtil.getByteBufferOrNull("hadoopConf", json); + HadoopConfigurable o = SerializationUtil.deserializeFromBytes(buffer.array()); + HadoopFileIO hadoopFileIO = (HadoopFileIO) fileIO; + hadoopFileIO.setConf(o.getConf()); + } + return fileIO; ``` -- 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