This is an automated email from the ASF dual-hosted git repository.

gabor pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-java.git


The following commit(s) were added to refs/heads/master by this push:
     new 43c597678 GH-3300: add ParquetWriter and ParquetReader builders 
constructor without params (#3301)
43c597678 is described below

commit 43c59767876ca04e45d135cc070a36322fd42077
Author: Jeronimo López <[email protected]>
AuthorDate: Wed Sep 10 09:31:12 2025 +0200

    GH-3300: add ParquetWriter and ParquetReader builders constructor without 
params (#3301)
---
 .../org/apache/parquet/hadoop/ParquetReader.java   | 26 +++++++++-
 .../org/apache/parquet/hadoop/ParquetWriter.java   | 21 ++++++++
 .../hadoop/example/ExampleParquetWriter.java       | 15 ++++++
 .../apache/parquet/hadoop/TestParquetReader.java   | 43 ++++++++++++++++
 .../apache/parquet/hadoop/TestParquetWriter.java   | 58 ++++++++++++++++++++++
 5 files changed, 162 insertions(+), 1 deletion(-)

diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetReader.java 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetReader.java
index f08e45ced..4514a829c 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetReader.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetReader.java
@@ -190,9 +190,13 @@ public class ParquetReader<T> implements Closeable {
     return new Builder<>(readSupport, path);
   }
 
+  public static <T> Builder<T> builder() {
+    return new Builder<T>();
+  }
+
   public static class Builder<T> {
     private final ReadSupport<T> readSupport;
-    private final InputFile file;
+    private InputFile file;
     private final Path path;
     private Filter filter = null;
     private ByteBufferAllocator allocator = new HeapByteBufferAllocator();
@@ -223,6 +227,14 @@ public class ParquetReader<T> implements Closeable {
       this.optionsBuilder = HadoopReadOptions.builder(this.conf, path);
     }
 
+    protected Builder() {
+      this.readSupport = null;
+      this.file = null;
+      this.path = null;
+      this.configuration = new HadoopParquetConfiguration();
+      this.optionsBuilder = HadoopReadOptions.builder(this.configuration);
+    }
+
     protected Builder(InputFile file) {
       this.readSupport = null;
       this.file = Objects.requireNonNull(file, "file cannot be null");
@@ -250,6 +262,14 @@ public class ParquetReader<T> implements Closeable {
       }
     }
 
+    public Builder<T> withFile(InputFile file) {
+      this.file = Objects.requireNonNull(file, "file cannot be null");
+      if (this.path != null) {
+        throw new IllegalStateException("Path is already set");
+      }
+      return this;
+    }
+
     // when called, resets options to the defaults from conf
     public Builder<T> withConf(Configuration conf) {
       this.conf = Objects.requireNonNull(conf, "conf cannot be null");
@@ -384,6 +404,10 @@ public class ParquetReader<T> implements Closeable {
     }
 
     public ParquetReader<T> build() throws IOException {
+      if (file == null && path == null) {
+        throw new IllegalStateException("File or Path must be set");
+      }
+
       ParquetReadOptions options = 
optionsBuilder.withAllocator(allocator).build();
 
       if (path != null) {
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
index 7789cad5c..05f0e2e3a 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
@@ -22,6 +22,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.parquet.bytes.ByteBufferAllocator;
@@ -476,6 +477,9 @@ public class ParquetWriter<T> implements Closeable {
     private boolean enableValidation = DEFAULT_IS_VALIDATING_ENABLED;
     private ParquetProperties.Builder encodingPropsBuilder = 
ParquetProperties.builder();
 
+    protected Builder() {}
+
+    @Deprecated
     protected Builder(Path path) {
       this.path = path;
     }
@@ -527,6 +531,20 @@ public class ParquetWriter<T> implements Closeable {
       return self();
     }
 
+    /**
+     * Set the {@link OutputFile} to be written by the constructed writer.
+     *
+     * @param file a {@code OutputFile}
+     * @return this builder for method chaining.
+     */
+    public SELF withFile(OutputFile file) {
+      this.file = Objects.requireNonNull(file, "file cannot be null");
+      if (this.path != null) {
+        throw new IllegalStateException("Cannot set both path and file");
+      }
+      return self();
+    }
+
     /**
      * Set the {@link ParquetFileWriter.Mode write mode} used when creating the
      * backing file for this writer.
@@ -960,6 +978,9 @@ public class ParquetWriter<T> implements Closeable {
      * @throws IOException if there is an error while creating the writer
      */
     public ParquetWriter<T> build() throws IOException {
+      if (file == null && path == null) {
+        throw new IllegalStateException("File or Path must be set");
+      }
       if (conf == null) {
         conf = new HadoopParquetConfiguration();
       }
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/ExampleParquetWriter.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/ExampleParquetWriter.java
index e6b71a49d..cd161469d 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/ExampleParquetWriter.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/ExampleParquetWriter.java
@@ -37,6 +37,17 @@ import org.apache.parquet.schema.MessageType;
  */
 public class ExampleParquetWriter extends ParquetWriter<Group> {
 
+  /**
+   * Creates a Builder without preconfigured Output File for configuring
+   * ParquetWriter with the example object model.
+   * THIS IS AN EXAMPLE ONLY AND NOT INTENDED FOR USE.
+   *
+   * @return a {@link Builder} to create a {@link ParquetWriter}
+   */
+  public static Builder builder() {
+    return new Builder();
+  }
+
   /**
    * Creates a Builder for configuring ParquetWriter with the example object
    * model. THIS IS AN EXAMPLE ONLY AND NOT INTENDED FOR USE.
@@ -106,6 +117,10 @@ public class ExampleParquetWriter extends 
ParquetWriter<Group> {
       super(file);
     }
 
+    private Builder() {
+      super();
+    }
+
     public Builder withType(MessageType type) {
       this.type = type;
       return this;
diff --git 
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReader.java 
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReader.java
index db14f6915..36a8707c9 100644
--- 
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReader.java
+++ 
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReader.java
@@ -33,6 +33,7 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.Callable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.parquet.bytes.HeapByteBufferAllocator;
@@ -41,8 +42,12 @@ import org.apache.parquet.column.ParquetProperties;
 import org.apache.parquet.example.data.Group;
 import org.apache.parquet.filter2.compat.FilterCompat;
 import org.apache.parquet.filter2.recordlevel.PhoneBookWriter;
+import org.apache.parquet.hadoop.ParquetReader.Builder;
+import org.apache.parquet.hadoop.api.ReadSupport;
 import org.apache.parquet.hadoop.example.ExampleParquetWriter;
 import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.InputFile;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -230,4 +235,42 @@ public class TestParquetReader {
     assertEquals(DATA, readUsers(FilterCompat.NOOP, false, true));
     assertEquals(DATA, readUsers(FilterCompat.NOOP, true, true));
   }
+
+  private static class TestParquetReaderBuilder extends 
ParquetReader.Builder<Group> {
+
+    @Override
+    protected ReadSupport<Group> getReadSupport() {
+      return new GroupReadSupport();
+    }
+  }
+
+  @Test
+  public void testParquetReaderBuilderWithInputFile() throws Exception {
+    InputFile inputFile = HadoopInputFile.fromPath(file, new Configuration());
+    Builder<Group> builder = new 
TestParquetReaderBuilder().withFile(inputFile);
+    assertEquals(DATA, PhoneBookWriter.readUsers(builder, false));
+  }
+
+  @Test
+  public void testParquetReaderBuilderValidatesThatInputFileCanNotBeNull() 
throws Exception {
+    TestUtils.assertThrows("file cannot be null", NullPointerException.class, 
(Callable<ParquetReader<Group>>)
+        () -> new TestParquetReaderBuilder().withFile(null).build());
+  }
+
+  @Test
+  public void testParquetReaderBuilderValidatesThatInputFileIsSet() throws 
Exception {
+    TestUtils.assertThrows("File or Path must be set", 
IllegalStateException.class, (Callable<ParquetReader<Group>>)
+        () -> new TestParquetReaderBuilder().build());
+  }
+
+  @Test
+  public void testParquetReaderBuilderCanNotConfigurePathAndFile() throws 
Exception {
+    TestUtils.assertThrows(
+        "Path is already set", IllegalStateException.class, 
(Callable<ParquetReader<Group>>) () -> {
+          InputFile inputFile = HadoopInputFile.fromPath(file, new 
Configuration());
+          return ParquetReader.<Group>builder(new GroupReadSupport(), file)
+              .withFile(inputFile)
+              .build();
+        });
+  }
 }
diff --git 
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java 
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
index 2cd83624f..d73079c92 100644
--- 
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
+++ 
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
@@ -722,4 +722,62 @@ public class TestParquetWriter {
       }
     }
   }
+
+  @Test
+  public void testParquetWriterConfiguringOutputFile() throws IOException {
+    MessageType schema = Types.buildMessage()
+        .required(BINARY)
+        .as(stringType())
+        .named("name")
+        .named("msg");
+
+    Configuration conf = new Configuration();
+    GroupWriteSupport.setSchema(schema, conf);
+
+    GroupFactory factory = new SimpleGroupFactory(schema);
+    File file = temp.newFile();
+    file.delete();
+    Path path = new Path(file.getAbsolutePath());
+    OutputFile outputFile = new TestOutputFile(path, conf);
+
+    String[] testNames = {"new", "writer", "builder", "without", "file"};
+    try (ParquetWriter<Group> writer = ExampleParquetWriter.builder()
+        .withFile(outputFile)
+        .withConf(conf)
+        .build()) {
+      for (String testName : testNames) {
+        writer.write(factory.newGroup().append("name", testName));
+      }
+    }
+    ParquetReader<Group> reader =
+        ParquetReader.builder(new GroupReadSupport(), path).build();
+    assertEquals("new", reader.read().getBinary("name", 
0).toStringUsingUTF8());
+    assertEquals("writer", reader.read().getBinary("name", 
0).toStringUsingUTF8());
+    assertEquals("builder", reader.read().getBinary("name", 
0).toStringUsingUTF8());
+    assertEquals("without", reader.read().getBinary("name", 
0).toStringUsingUTF8());
+    assertEquals("file", reader.read().getBinary("name", 
0).toStringUsingUTF8());
+  }
+
+  @Test
+  public void testParquetWriterBuilderOutputFileCanNotBeNull() throws 
IOException {
+    TestUtils.assertThrows("file cannot be null", NullPointerException.class, 
(Callable<ParquetWriter<Group>>)
+        () -> ExampleParquetWriter.builder().withFile(null).build());
+  }
+
+  @Test
+  public void testParquetWriterBuilderValidatesThatOutputFileIsSet() throws 
IOException {
+    TestUtils.assertThrows("File or Path must be set", 
IllegalStateException.class, (Callable<ParquetWriter<Group>>)
+        () -> ExampleParquetWriter.builder().build());
+  }
+
+  @Test
+  public void testParquetWriterBuilderCanNotConfigurePathAndFile() throws 
IOException {
+    File file = temp.newFile();
+    Path path = new Path(file.getAbsolutePath());
+    Configuration conf = new Configuration();
+    OutputFile outputFile = new TestOutputFile(path, conf);
+    TestUtils.assertThrows(
+        "Cannot set both path and file", IllegalStateException.class, 
(Callable<ParquetWriter<Group>>) () ->
+            ExampleParquetWriter.builder(path).withFile(outputFile).build());
+  }
 }

Reply via email to