You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by sh...@apache.org on 2021/03/16 15:34:55 UTC

[parquet-mr] branch master updated: PARQUET-1999: NPE might occur if OutputFile is implemented by the client (#881)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 616c3d6  PARQUET-1999: NPE might occur if OutputFile is implemented by the client (#881)
616c3d6 is described below

commit 616c3d681c5506ebad83a7d3724e85da2154eac9
Author: Gabor Szadovszky <ga...@apache.org>
AuthorDate: Tue Mar 16 16:34:45 2021 +0100

    PARQUET-1999: NPE might occur if OutputFile is implemented by the client (#881)
---
 .../org/apache/parquet/hadoop/ParquetWriter.java   |  3 +-
 .../apache/parquet/hadoop/TestParquetWriter.java   | 49 ++++++++++++++++++++--
 2 files changed, 47 insertions(+), 5 deletions(-)

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 550cd33..e1afaca 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
@@ -281,8 +281,9 @@ public class ParquetWriter<T> implements Closeable {
 
     // encryptionProperties could be built from the implementation of EncryptionPropertiesFactory when it is attached.
     if (encryptionProperties == null) {
+      String path = file == null ? null : file.getPath();
       encryptionProperties = ParquetOutputFormat.createEncryptionProperties(conf,
-          file == null ? null : new Path(file.getPath()), writeContext);
+          path == null ? null : new Path(path), writeContext);
     }
 
     ParquetFileWriter fileWriter = new ParquetFileWriter(
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 868af8e..de53e96 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
@@ -49,6 +49,7 @@ import org.apache.parquet.column.values.bloomfilter.BloomFilter;
 import org.apache.parquet.example.data.GroupFactory;
 import org.apache.parquet.hadoop.example.ExampleParquetWriter;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
 import org.apache.parquet.schema.GroupType;
 import org.apache.parquet.schema.InvalidSchemaException;
 import org.apache.parquet.schema.Types;
@@ -65,12 +66,46 @@ import org.apache.parquet.hadoop.example.GroupWriteSupport;
 import org.apache.parquet.hadoop.metadata.BlockMetaData;
 import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.io.OutputFile;
+import org.apache.parquet.io.PositionOutputStream;
 import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.schema.MessageType;
 import org.junit.rules.TemporaryFolder;
 
 public class TestParquetWriter {
 
+  /**
+   * A test OutputFile implementation to validate the scenario of an OutputFile is implemented by an API client.
+   */
+  private static class TestOutputFile implements OutputFile {
+
+    private final OutputFile outputFile;
+
+    TestOutputFile(Path path, Configuration conf) throws IOException {
+      outputFile = HadoopOutputFile.fromPath(path, conf);
+    }
+
+    @Override
+    public PositionOutputStream create(long blockSizeHint) throws IOException {
+      return outputFile.create(blockSizeHint);
+    }
+
+    @Override
+    public PositionOutputStream createOrOverwrite(long blockSizeHint) throws IOException {
+      return outputFile.createOrOverwrite(blockSizeHint);
+    }
+
+    @Override
+    public boolean supportsBlockSize() {
+      return outputFile.supportsBlockSize();
+    }
+
+    @Override
+    public long defaultBlockSize() {
+      return outputFile.defaultBlockSize();
+    }
+  }
+
   @Test
   public void test() throws Exception {
     Configuration conf = new Configuration();
@@ -97,10 +132,16 @@ public class TestParquetWriter {
     for (int modulo : asList(10, 1000)) {
       for (WriterVersion version : WriterVersion.values()) {
         Path file = new Path(root, version.name() + "_" + modulo);
-        ParquetWriter<Group> writer = new ParquetWriter<Group>(
-            file,
-            new GroupWriteSupport(),
-            UNCOMPRESSED, 1024, 1024, 512, true, false, version, conf);
+        ParquetWriter<Group> writer = ExampleParquetWriter.builder(new TestOutputFile(file, conf))
+            .withCompressionCodec(UNCOMPRESSED)
+            .withRowGroupSize(1024)
+            .withPageSize(1024)
+            .withDictionaryPageSize(512)
+            .enableDictionaryEncoding()
+            .withValidation(false)
+            .withWriterVersion(version)
+            .withConf(conf)
+            .build();
         for (int i = 0; i < 1000; i++) {
           writer.write(
               f.newGroup()