diff --git a/core/src/main/java/org/apache/iceberg/InternalData.java b/core/src/main/java/org/apache/iceberg/InternalData.java index 8f9aaa03bb96..f07b0f3d85e1 100644 --- a/core/src/main/java/org/apache/iceberg/InternalData.java +++ b/core/src/main/java/org/apache/iceberg/InternalData.java @@ -22,6 +22,8 @@ import java.util.Map; import java.util.function.Function; import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.InternalReader; +import org.apache.iceberg.avro.InternalWriter; import org.apache.iceberg.common.DynMethods; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; @@ -41,12 +43,14 @@ private InternalData() {} Maps.newConcurrentMap(); static { - Avro.register(); + InternalData.register(FileFormat.AVRO, + outputFile -> Avro.write(outputFile).createWriterFunc(InternalWriter::create), + inputFile -> Avro.read(inputFile).createResolvingReader(InternalReader::create)); try { DynMethods.StaticMethod registerParquet = DynMethods.builder("register") - .impl("org.apache.iceberg.parquet.Parquet") + .impl("org.apache.iceberg.InternalParquet") .buildStaticChecked(); registerParquet.invoke(); @@ -57,7 +61,7 @@ private InternalData() {} } } - public static void register( + static void register( FileFormat format, Function writeBuilder, Function readBuilder) { diff --git a/core/src/main/java/org/apache/iceberg/avro/Avro.java b/core/src/main/java/org/apache/iceberg/avro/Avro.java index 5907028fb68a..b1814d707f9e 100644 --- a/core/src/main/java/org/apache/iceberg/avro/Avro.java +++ b/core/src/main/java/org/apache/iceberg/avro/Avro.java @@ -72,18 +72,6 @@ public class Avro { private Avro() {} - public static void register() { - InternalData.register(FileFormat.AVRO, Avro::writeInternal, Avro::readInternal); - } - - private static WriteBuilder writeInternal(OutputFile outputFile) { - return write(outputFile).createWriterFunc(InternalWriter::create); - } - - private static ReadBuilder readInternal(InputFile inputFile) { - return read(inputFile).createResolvingReader(InternalReader::create); - } - private enum Codec { UNCOMPRESSED, SNAPPY, diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 8e59e21c4c0a..d8de89de3cc5 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -74,8 +74,6 @@ import org.apache.iceberg.Table; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.data.parquet.GenericParquetWriter; -import org.apache.iceberg.data.parquet.InternalReader; -import org.apache.iceberg.data.parquet.InternalWriter; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; import org.apache.iceberg.encryption.EncryptedOutputFile; @@ -129,18 +127,6 @@ public class Parquet { private Parquet() {} - public static void register() { - InternalData.register(FileFormat.PARQUET, Parquet::writeInternal, Parquet::readInternal); - } - - private static WriteBuilder writeInternal(OutputFile outputFile) { - return write(outputFile).createWriterFunc(InternalWriter::create); - } - - private static ReadBuilder readInternal(InputFile inputFile) { - return read(inputFile).createReaderFunc(InternalReader::create); - } - private static final Collection READ_PROPERTIES_TO_REMOVE = Sets.newHashSet( "parquet.read.filter", @@ -1158,7 +1144,7 @@ public ReadBuilder createReaderFunc( return this; } - private ReadBuilder createReaderFunc( + public ReadBuilder createReaderFunc( BiFunction> newReaderFunction) { Preconditions.checkArgument( this.readerFunc == null,