You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by su...@apache.org on 2022/08/17 23:21:55 UTC

[spark] branch master updated: [SPARK-40128][SQL] Make the VectorizedColumnReader recognize DELTA_LENGTH_BYTE_ARRAY as a standalone column encoding

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

sunchao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 01f9d270ea1 [SPARK-40128][SQL] Make the VectorizedColumnReader recognize DELTA_LENGTH_BYTE_ARRAY as a standalone column encoding
01f9d270ea1 is described below

commit 01f9d270ea14d4a9b3a5f326fc3f721ddd23e3f4
Author: Dennis Huo <de...@snowflake.com>
AuthorDate: Wed Aug 17 16:21:43 2022 -0700

    [SPARK-40128][SQL] Make the VectorizedColumnReader recognize DELTA_LENGTH_BYTE_ARRAY as a standalone column encoding
    
    ### What changes were proposed in this pull request?
    Add DELTA_LENGTH_BYTE_ARRAY as a recognized encoding in VectorizedColumnReader so that
    vectorized reads succeed when there are columns using DELTA_LENGTH_BYTE_ARRAY as a standalone
    encoding.
    
    ### Why are the changes needed?
    
    Spark currently throws an exception for DELTA_LENGTH_BYTE_ARRAY columns when vectorized
    reads are enabled and trying to read `delta_length_byte_array.parquet` from https://github.com/apache/parquet-testing:
    
        java.lang.UnsupportedOperationException: Unsupported encoding: DELTA_LENGTH_BYTE_ARRAY
    
    ### Does this PR introduce _any_ user-facing change?
    Yes - previously throw UNSUPPORTED exception. Now reads the encoding same as if vectorized reads are disabled.
    
    ### How was this patch tested?
    Added test case to ParquetIOSuite; made sure it fails without the fix to VectorizedColumnReader and passes after.
    
    Closes #37557 from sfc-gh-dhuo/support-parquet-delta-length-byte-array.
    
    Authored-by: Dennis Huo <de...@snowflake.com>
    Signed-off-by: Chao Sun <su...@apple.com>
---
 .../datasources/parquet/VectorizedColumnReader.java      |   2 ++
 .../resources/test-data/delta_length_byte_array.parquet  | Bin 0 -> 3072 bytes
 .../execution/datasources/parquet/ParquetIOSuite.scala   |  10 ++++++++++
 3 files changed, 12 insertions(+)

diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java
index c2e85da3884..64178fdd72d 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java
@@ -329,6 +329,8 @@ public class VectorizedColumnReader {
         return new VectorizedPlainValuesReader();
       case DELTA_BYTE_ARRAY:
         return new VectorizedDeltaByteArrayReader();
+      case DELTA_LENGTH_BYTE_ARRAY:
+        return new VectorizedDeltaLengthByteArrayReader();
       case DELTA_BINARY_PACKED:
         return new VectorizedDeltaBinaryPackedReader();
       case RLE:
diff --git a/sql/core/src/test/resources/test-data/delta_length_byte_array.parquet b/sql/core/src/test/resources/test-data/delta_length_byte_array.parquet
new file mode 100644
index 00000000000..ead505a1a1f
Binary files /dev/null and b/sql/core/src/test/resources/test-data/delta_length_byte_array.parquet differ
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
index 5a8f4563756..0458e5a1a14 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
@@ -1307,6 +1307,16 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession
     }
   }
 
+  test("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") {
+    withAllParquetReaders {
+      checkAnswer(
+        // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY.
+        // The file comes from https://github.com/apache/parquet-testing
+        readResourceParquetFile("test-data/delta_length_byte_array.parquet"),
+        (0 to 999).map(i => Row("apple_banana_mango" + Integer.toString(i * i))))
+    }
+  }
+
   test("SPARK-12589 copy() on rows returned from reader works for strings") {
     withTempPath { dir =>
       val data = (1, "abc") ::(2, "helloabcde") :: Nil


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org