You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by ga...@apache.org on 2021/04/19 08:23:35 UTC

[parquet-mr] branch master updated: PARQUET-2022: ZstdDecompressorStream should close `zstdInputStream` (#889)

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-mr.git


The following commit(s) were added to refs/heads/master by this push:
     new 8c08403  PARQUET-2022: ZstdDecompressorStream should close `zstdInputStream` (#889)
8c08403 is described below

commit 8c0840365ab76070da0efc8bfe0559bc4e525e49
Author: Dongjoon Hyun <do...@apache.org>
AuthorDate: Mon Apr 19 01:22:39 2021 -0700

    PARQUET-2022: ZstdDecompressorStream should close `zstdInputStream` (#889)
---
 .../org/apache/parquet/hadoop/codec/ZstdDecompressorStream.java  | 9 +++++++++
 .../test/java/org/apache/parquet/hadoop/TestZstandardCodec.java  | 9 ++++-----
 2 files changed, 13 insertions(+), 5 deletions(-)

diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/ZstdDecompressorStream.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/ZstdDecompressorStream.java
index a505e7b..a24612f 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/ZstdDecompressorStream.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/ZstdDecompressorStream.java
@@ -50,4 +50,13 @@ public class ZstdDecompressorStream extends CompressionInputStream {
   public void resetState() throws IOException {
     // no-opt, doesn't apply to ZSTD
   }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      zstdInputStream.close();
+    } finally {
+      super.close();
+    }
+  }
 }
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestZstandardCodec.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestZstandardCodec.java
index c0d9826..b6bab26 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestZstandardCodec.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestZstandardCodec.java
@@ -79,8 +79,8 @@ public class TestZstandardCodec {
     byte[] data = new byte[dataSize];
     (new Random()).nextBytes(data);
     BytesInput compressedData = compress(codec,  BytesInput.from(data));
-    BytesInput decompressedData = decompress(codec, compressedData, data.length);
-    Assert.assertArrayEquals(data, decompressedData.toByteArray());
+    byte[] decompressedData = decompress(codec, compressedData, data.length);
+    Assert.assertArrayEquals(data, decompressedData);
   }
 
   private BytesInput compress(ZstandardCodec codec, BytesInput bytes) throws IOException {
@@ -91,10 +91,9 @@ public class TestZstandardCodec {
     return BytesInput.from(compressedOutBuffer);
   }
 
-  private BytesInput decompress(ZstandardCodec codec, BytesInput bytes, int uncompressedSize) throws IOException {
-    BytesInput decompressed;
+  private byte[] decompress(ZstandardCodec codec, BytesInput bytes, int uncompressedSize) throws IOException {
     InputStream is = codec.createInputStream(bytes.toInputStream(), null);
-    decompressed = BytesInput.from(is, uncompressedSize);
+    byte[] decompressed = BytesInput.from(is, uncompressedSize).toByteArray();
     is.close();
     return decompressed;
   }