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

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

commit c88a3f8ab0dd2f4041b6249c807f43ed6e6d052a
Author: joyCurry30 <[email protected]>
AuthorDate: Wed Jul 17 22:53:58 2024 +0800

    GH-2948: Fix NPE when using the AvroParquetReader.Builder with 
LocalInputFile (#2949)
---
 .../org/apache/parquet/avro/TestReadWrite.java     | 22 ++++++++++++++++++++++
 .../org/apache/parquet/hadoop/ParquetReader.java   |  7 +++----
 2 files changed, 25 insertions(+), 4 deletions(-)

diff --git 
a/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java 
b/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java
index d0b2a7dba..29f3e7409 100644
--- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java
+++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java
@@ -29,6 +29,7 @@ import java.io.File;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.math.BigInteger;
+import java.net.URI;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Paths;
@@ -59,10 +60,13 @@ import org.apache.parquet.column.ParquetProperties;
 import org.apache.parquet.conf.HadoopParquetConfiguration;
 import org.apache.parquet.conf.ParquetConfiguration;
 import org.apache.parquet.conf.PlainParquetConfiguration;
+import org.apache.parquet.example.data.Group;
 import org.apache.parquet.hadoop.ParquetReader;
 import org.apache.parquet.hadoop.ParquetWriter;
 import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
 import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.io.InputFile;
 import org.apache.parquet.io.LocalInputFile;
 import org.apache.parquet.io.LocalOutputFile;
 import org.apache.parquet.io.api.Binary;
@@ -919,6 +923,24 @@ public class TestReadWrite {
     Assert.assertEquals("Content should match", expected, records);
   }
 
+  @Test
+  public void testConstructor() throws IOException {
+    String testFile =
+        
URI.create(Resources.getResource("strings-2.parquet").getFile()).getRawPath();
+    InputFile inputFile = new LocalInputFile(Paths.get(testFile));
+    ParquetReader<Group> reader =
+        AvroParquetReader.<Group>builder(inputFile).build();
+    assertNotNull(reader);
+
+    reader = AvroParquetReader.<Group>builder(inputFile, new 
HadoopParquetConfiguration(new Configuration()))
+        .build();
+    assertNotNull(reader);
+
+    reader = AvroParquetReader.builder(new GroupReadSupport(), new 
Path(testFile))
+        .build();
+    assertNotNull(reader);
+  }
+
   private File createTempFile() throws IOException {
     File tmp = File.createTempFile(getClass().getSimpleName(), ".tmp");
     tmp.deleteOnExit();
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 9ca1202eb..be599ba56 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
@@ -229,12 +229,11 @@ public class ParquetReader<T> implements Closeable {
       this.path = null;
       if (file instanceof HadoopInputFile) {
         HadoopInputFile hadoopFile = (HadoopInputFile) file;
-        this.conf = hadoopFile.getConfiguration();
-        this.configuration = new HadoopParquetConfiguration(this.conf);
-        optionsBuilder = HadoopReadOptions.builder(this.conf, 
hadoopFile.getPath());
+        this.configuration = new 
HadoopParquetConfiguration(hadoopFile.getConfiguration());
       } else {
-        optionsBuilder = ParquetReadOptions.builder(new 
HadoopParquetConfiguration());
+        this.configuration = new HadoopParquetConfiguration();
       }
+      optionsBuilder = HadoopReadOptions.builder(this.configuration);
     }
 
     protected Builder(InputFile file, ParquetConfiguration conf) {

Reply via email to