This is an automated email from the ASF dual-hosted git repository. bchapuis pushed a commit to branch geoparquet-writer in repository https://gitbox.apache.org/repos/asf/incubator-baremaps.git
commit 65ec5a9e67b0cab1ff6d120f149aec0e179f9f28 Author: Bertil Chapuis <[email protected]> AuthorDate: Mon Oct 21 21:53:33 2024 +0200 Implement the builder --- .../baremaps/geoparquet/GeoParquetWriter.java | 73 +++++++++++++++++----- 1 file changed, 58 insertions(+), 15 deletions(-) diff --git a/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetWriter.java b/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetWriter.java index 39fef2085..7d23d8857 100644 --- a/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetWriter.java +++ b/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetWriter.java @@ -18,11 +18,14 @@ package org.apache.baremaps.geoparquet; import java.io.IOException; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.parquet.column.ParquetProperties.WriterVersion; +import org.apache.parquet.conf.ParquetConfiguration; import org.apache.parquet.hadoop.ParquetWriter; -import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.hadoop.api.WriteSupport; +import org.apache.parquet.hadoop.example.ExampleParquetWriter.Builder; +import org.apache.parquet.io.OutputFile; import org.apache.parquet.schema.MessageType; /** @@ -35,24 +38,17 @@ public class GeoParquetWriter implements AutoCloseable { /** * Constructs a new GeoParquetWriter. * - * @param outputFile the output file + * @param path the output file * @param schema the Parquet schema * @param metadata the GeoParquet metadata * @throws IOException if an I/O error occurs */ - public GeoParquetWriter(Path outputFile, MessageType schema, GeoParquetMetadata metadata) + public GeoParquetWriter(Path path, MessageType schema, GeoParquetMetadata metadata) throws IOException { - this.parquetWriter = new ParquetWriter<>( - outputFile, - new GeoParquetWriteSupport(schema, metadata), - CompressionCodecName.UNCOMPRESSED, - ParquetWriter.DEFAULT_BLOCK_SIZE, - ParquetWriter.DEFAULT_PAGE_SIZE, - ParquetWriter.DEFAULT_PAGE_SIZE, - ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, - ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, - WriterVersion.PARQUET_2_0, - new Configuration()); + parquetWriter = new Builder(path) + .withType(schema) + .withMetadata(metadata) + .build(); } /** @@ -70,7 +66,54 @@ public class GeoParquetWriter implements AutoCloseable { * * @throws IOException if an I/O error occurs */ + @Override public void close() throws IOException { parquetWriter.close(); } + + public static class Builder + extends ParquetWriter.Builder<GeoParquetGroup, GeoParquetWriter.Builder> { + + private MessageType type = null; + + private GeoParquetMetadata metadata = null; + + private Builder(Path file) { + super(file); + } + + private Builder(OutputFile file) { + super(file); + } + + public GeoParquetWriter.Builder withType(MessageType type) { + this.type = type; + return this; + } + + public GeoParquetWriter.Builder withMetadata(GeoParquetMetadata metadata) { + this.metadata = metadata; + return this; + } + + @Override + protected GeoParquetWriter.Builder self() { + return this; + } + + @Override + protected WriteSupport<GeoParquetGroup> getWriteSupport(Configuration conf) { + return getWriteSupport((ParquetConfiguration) null); + } + + @Override + protected WriteSupport<GeoParquetGroup> getWriteSupport(ParquetConfiguration conf) { + return new GeoParquetWriteSupport(type, metadata); + } + + @Override + public GeoParquetWriter.Builder withExtraMetaData(Map<String, String> extraMetaData) { + return super.withExtraMetaData(extraMetaData); + } + } }
