From a8508d2feabcd409a4a70a95a4e8d297287e38c7 Mon Sep 17 00:00:00 2001 From: joy Date: Thu, 11 Jul 2024 09:17:17 +0800 Subject: [PATCH 1/8] PARQUET-2948: Fix the npe when using the AvroParquetReader.Builder with a LocalInputFile. --- .../src/main/java/org/apache/parquet/hadoop/ParquetReader.java | 3 ++- 1 file changed, 2 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 9ca1202eb9..2e0c4c9559 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 @@ -230,11 +230,12 @@ protected Builder(InputFile file) { 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()); } else { + this.conf = new Configuration(); optionsBuilder = ParquetReadOptions.builder(new HadoopParquetConfiguration()); } + this.configuration = new HadoopParquetConfiguration(this.conf); } protected Builder(InputFile file, ParquetConfiguration conf) { From 6bb6b15baf71f81cd0d71bdaf10b5e299e206a53 Mon Sep 17 00:00:00 2001 From: joy Date: Thu, 11 Jul 2024 09:52:50 +0800 Subject: [PATCH 2/8] PARQUET-2948: Fix the npe when using the AvroParquetReader.Builder with a LocalInputFile. --- .../parquet/avro/TestAvroParquetReader.java | 51 +++++++++++++++++++ .../parquet/hadoop/TestParquetReader.java | 4 +- 2 files changed, 53 insertions(+), 2 deletions(-) create mode 100644 parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java diff --git a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java new file mode 100644 index 0000000000..66ec2a4450 --- /dev/null +++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.parquet.avro; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.conf.HadoopParquetConfiguration; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.example.GroupReadSupport; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.LocalInputFile; +import org.junit.Test; +import java.io.IOException; +import java.nio.file.Paths; + +import static org.apache.parquet.hadoop.TestParquetReader.FILE_V1; +import static org.junit.Assert.assertNotNull; + +public class TestAvroParquetReader { + + @Test + public void testConstructor() throws IOException { + InputFile inputFile = new LocalInputFile(Paths.get(FILE_V1.toUri().getRawPath())); + ParquetReader reader = AvroParquetReader.builder(inputFile).build(); + assertNotNull(reader); + + reader = AvroParquetReader.builder(inputFile, new HadoopParquetConfiguration(new Configuration())).build(); + assertNotNull(reader); + + reader = AvroParquetReader.builder(new GroupReadSupport(), new Path(FILE_V1.toUri().getRawPath())).build(); + assertNotNull(reader); + } +} 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 db14f69150..ccad848d24 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 @@ -54,8 +54,8 @@ @RunWith(Parameterized.class) public class TestParquetReader { - private static final Path FILE_V1 = createTempFile(); - private static final Path FILE_V2 = createTempFile(); + public static final Path FILE_V1 = createTempFile(); + public static final Path FILE_V2 = createTempFile(); private static final Path STATIC_FILE_WITHOUT_COL_INDEXES = createPathFromCP("/test-file-with-no-column-indexes-1.parquet"); private static final List DATA = Collections.unmodifiableList(makeUsers(1000)); From fd5ca9bca6afaf69b0a8b914d141be8b8aa48749 Mon Sep 17 00:00:00 2001 From: joy Date: Thu, 11 Jul 2024 09:57:30 +0800 Subject: [PATCH 3/8] PARQUET-2948: Format the code. --- .../parquet/avro/TestAvroParquetReader.java | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java index 66ec2a4450..1a533fe23d 100644 --- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java +++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java @@ -18,7 +18,11 @@ */ package org.apache.parquet.avro; -import org.apache.avro.generic.GenericRecord; +import static org.apache.parquet.hadoop.TestParquetReader.FILE_V1; +import static org.junit.Assert.assertNotNull; + +import java.io.IOException; +import java.nio.file.Paths; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.conf.HadoopParquetConfiguration; @@ -28,24 +32,23 @@ import org.apache.parquet.io.InputFile; import org.apache.parquet.io.LocalInputFile; import org.junit.Test; -import java.io.IOException; -import java.nio.file.Paths; - -import static org.apache.parquet.hadoop.TestParquetReader.FILE_V1; -import static org.junit.Assert.assertNotNull; public class TestAvroParquetReader { @Test public void testConstructor() throws IOException { InputFile inputFile = new LocalInputFile(Paths.get(FILE_V1.toUri().getRawPath())); - ParquetReader reader = AvroParquetReader.builder(inputFile).build(); + ParquetReader reader = + AvroParquetReader.builder(inputFile).build(); assertNotNull(reader); - reader = AvroParquetReader.builder(inputFile, new HadoopParquetConfiguration(new Configuration())).build(); + reader = AvroParquetReader.builder(inputFile, new HadoopParquetConfiguration(new Configuration())) + .build(); assertNotNull(reader); - reader = AvroParquetReader.builder(new GroupReadSupport(), new Path(FILE_V1.toUri().getRawPath())).build(); + reader = AvroParquetReader.builder( + new GroupReadSupport(), new Path(FILE_V1.toUri().getRawPath())) + .build(); assertNotNull(reader); } } From 97c3d3469d843072252fb1da34bb1287c222ac5c Mon Sep 17 00:00:00 2001 From: joy Date: Thu, 11 Jul 2024 16:21:33 +0800 Subject: [PATCH 4/8] PARQUET-2948: Modify the test case. --- .../parquet/avro/TestAvroParquetReader.java | 54 ------------------- .../apache/parquet/avro/TestReadWrite.java | 22 ++++++++ 2 files changed, 22 insertions(+), 54 deletions(-) delete mode 100644 parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java diff --git a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java deleted file mode 100644 index 1a533fe23d..0000000000 --- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroParquetReader.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.parquet.avro; - -import static org.apache.parquet.hadoop.TestParquetReader.FILE_V1; -import static org.junit.Assert.assertNotNull; - -import java.io.IOException; -import java.nio.file.Paths; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.parquet.conf.HadoopParquetConfiguration; -import org.apache.parquet.example.data.Group; -import org.apache.parquet.hadoop.ParquetReader; -import org.apache.parquet.hadoop.example.GroupReadSupport; -import org.apache.parquet.io.InputFile; -import org.apache.parquet.io.LocalInputFile; -import org.junit.Test; - -public class TestAvroParquetReader { - - @Test - public void testConstructor() throws IOException { - InputFile inputFile = new LocalInputFile(Paths.get(FILE_V1.toUri().getRawPath())); - ParquetReader reader = - AvroParquetReader.builder(inputFile).build(); - assertNotNull(reader); - - reader = AvroParquetReader.builder(inputFile, new HadoopParquetConfiguration(new Configuration())) - .build(); - assertNotNull(reader); - - reader = AvroParquetReader.builder( - new GroupReadSupport(), new Path(FILE_V1.toUri().getRawPath())) - .build(); - assertNotNull(reader); - } -} 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 d0b2a7dba3..29f3e74091 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.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.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 void testParsesDataModelFromConf() throws Exception { 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 reader = + AvroParquetReader.builder(inputFile).build(); + assertNotNull(reader); + + reader = AvroParquetReader.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(); From 0f29e576340ef47265b18db9598f51c9c43782bc Mon Sep 17 00:00:00 2001 From: joyCurry30 <149778446+joyCurry30@users.noreply.github.com> Date: Thu, 11 Jul 2024 19:25:22 +0800 Subject: [PATCH 5/8] initialize a Hadoop configuration while it is not a Hadoop file. Co-authored-by: Fokko Driesprong --- .../main/java/org/apache/parquet/hadoop/ParquetReader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 2e0c4c9559..991c5b954b 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 @@ -232,8 +232,8 @@ protected Builder(InputFile file) { this.conf = hadoopFile.getConfiguration(); optionsBuilder = HadoopReadOptions.builder(this.conf, hadoopFile.getPath()); } else { - this.conf = new Configuration(); - optionsBuilder = ParquetReadOptions.builder(new HadoopParquetConfiguration()); + this.configuration = new HadoopParquetConfiguration(); + optionsBuilder = ParquetReadOptions.builder(this.configuration); } this.configuration = new HadoopParquetConfiguration(this.conf); } From 1a3ef3ab08f9193a62772ab2229c32c2916b61e5 Mon Sep 17 00:00:00 2001 From: joy Date: Sat, 13 Jul 2024 09:47:17 +0800 Subject: [PATCH 6/8] PARQUET-2948: Fix the npe when using the AvroParquetReader.Builder with a LocalInputFile. And revert the unnecessary change. --- .../main/java/org/apache/parquet/hadoop/ParquetReader.java | 1 - .../java/org/apache/parquet/hadoop/TestParquetReader.java | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) 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 991c5b954b..3bf570daee 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 @@ -235,7 +235,6 @@ protected Builder(InputFile file) { this.configuration = new HadoopParquetConfiguration(); optionsBuilder = ParquetReadOptions.builder(this.configuration); } - this.configuration = new HadoopParquetConfiguration(this.conf); } protected Builder(InputFile file, ParquetConfiguration conf) { 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 ccad848d24..db14f69150 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 @@ -54,8 +54,8 @@ @RunWith(Parameterized.class) public class TestParquetReader { - public static final Path FILE_V1 = createTempFile(); - public static final Path FILE_V2 = createTempFile(); + private static final Path FILE_V1 = createTempFile(); + private static final Path FILE_V2 = createTempFile(); private static final Path STATIC_FILE_WITHOUT_COL_INDEXES = createPathFromCP("/test-file-with-no-column-indexes-1.parquet"); private static final List DATA = Collections.unmodifiableList(makeUsers(1000)); From beaa7ce83325e9c885c91fdfe26708003d95fba6 Mon Sep 17 00:00:00 2001 From: joy Date: Sat, 13 Jul 2024 23:25:21 +0800 Subject: [PATCH 7/8] PARQUET-2948: Fix the npe when using the AvroParquetReader.Builder with a LocalInputFile. --- .../main/java/org/apache/parquet/hadoop/ParquetReader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 3bf570daee..d1dd445e18 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 @@ -230,11 +230,11 @@ protected Builder(InputFile file) { if (file instanceof HadoopInputFile) { HadoopInputFile hadoopFile = (HadoopInputFile) file; this.conf = hadoopFile.getConfiguration(); - optionsBuilder = HadoopReadOptions.builder(this.conf, hadoopFile.getPath()); + this.configuration = new HadoopParquetConfiguration(this.conf); } else { this.configuration = new HadoopParquetConfiguration(); - optionsBuilder = ParquetReadOptions.builder(this.configuration); } + optionsBuilder = HadoopReadOptions.builder(this.configuration); } protected Builder(InputFile file, ParquetConfiguration conf) { From 5cbe2241b957659ae907618e5bc8125721a923dd Mon Sep 17 00:00:00 2001 From: joy Date: Sun, 14 Jul 2024 00:00:53 +0800 Subject: [PATCH 8/8] PARQUET-2948: Remove unnecessary code. --- .../src/main/java/org/apache/parquet/hadoop/ParquetReader.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 d1dd445e18..be599ba569 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,8 +229,7 @@ protected Builder(InputFile file) { this.path = null; if (file instanceof HadoopInputFile) { HadoopInputFile hadoopFile = (HadoopInputFile) file; - this.conf = hadoopFile.getConfiguration(); - this.configuration = new HadoopParquetConfiguration(this.conf); + this.configuration = new HadoopParquetConfiguration(hadoopFile.getConfiguration()); } else { this.configuration = new HadoopParquetConfiguration(); }