You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/12/13 20:25:05 UTC

[GitHub] [spark] sunchao commented on a change in pull request #34471: [SPARK-36879][SQL] Support Parquet v2 data page encoding (DELTA_BINARY_PACKED) for the vectorized path

sunchao commented on a change in pull request #34471:
URL: https://github.com/apache/spark/pull/34471#discussion_r768089566



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetDeltaEncodingSuite.scala
##########
@@ -0,0 +1,356 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import java.io.IOException
+import java.nio.ByteBuffer
+import java.util.Random
+
+import org.apache.parquet.bytes.{ByteBufferInputStream, DirectByteBufferAllocator}
+import org.apache.parquet.column.values.ValuesWriter
+import org.apache.parquet.column.values.delta.{DeltaBinaryPackingValuesWriterForInteger, DeltaBinaryPackingValuesWriterForLong}
+import org.apache.parquet.io.ParquetDecodingException
+
+import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, IntegralType, LongType}
+
+/**
+ * Read tests for vectorized Delta binary packed reader.
+ * Translated from
+ *  org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriterForIntegerTest
+ *  org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriterForLongTest
+ */
+abstract class ParquetDeltaEncodingSuite[T] extends ParquetCompatibilityTest
+  with SharedSparkSession {
+  protected var blockSize = 128
+  protected var miniBlockNum = 4
+  protected var reader: VectorizedDeltaBinaryPackedReader = _
+  protected var writableColumnVector: WritableColumnVector = _
+  protected var writer: ValuesWriter = _
+  protected var random: Random = _
+
+  protected def getSparkSqlType: IntegralType
+
+  protected def writeData(data: Array[T]): Unit
+
+  protected def writeData(data: Array[T], length: Int): Unit
+
+  protected def initValuesWriter (): Unit
+
+//  protected def allocArrayOfType[T: ClassTag](size: Int): Array[T] = new Array[T](size)

Review comment:
       nit: remove?

##########
File path: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedDeltaBinaryPackedReader.java
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.values.bitpacking.BytePackerForLong;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.apache.spark.sql.catalyst.util.RebaseDateTime;
+import org.apache.spark.sql.execution.datasources.DataSourceUtils;
+import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
+
+/**
+ * An implementation of the Parquet DELTA_BINARY_PACKED decoder that supports the vectorized
+ * interface. DELTA_BINARY_PACKED is a delta encoding for integer and long types that stores values
+ * as a delta between consecutive values. Delta values are themselves bit packed. Similar to RLE but
+ * is more effective in the case of large variation of values in the encoded column.
+ * <p>
+ * DELTA_BINARY_PACKED is the default encoding for integer and long columns in Parquet V2.
+ * <p>
+ * Supported Types: INT32, INT64
+ * <p>
+ *
+ * @see <a href="https://github.com/apache/parquet-format/blob/master/Encodings.md#delta-encoding-delta_binary_packed--5">
+ * Parquet format encodings: DELTA_BINARY_PACKED</a>
+ */
+public class VectorizedDeltaBinaryPackedReader extends VectorizedReaderBase {
+
+  // header data
+  private int blockSizeInValues;
+  private int miniBlockNumInABlock;
+  private int totalValueCount;
+  private long firstValue;
+
+  private int miniBlockSizeInValues;
+
+  // values read by the caller
+  private int valuesRead = 0;
+
+  // variables to keep state of the current block and miniblock
+  private long lastValueRead;  // needed to compute the next value
+  private long minDeltaInCurrentBlock; // needed to compute the next value
+  // currentMiniBlock keeps track of the mini block within the current block that
+  // we read and decoded most recently. Only used as an index into
+  // bitWidths array
+  private int currentMiniBlock = 0;
+  private int[] bitWidths; // bit widths for each miniBlock in the current block
+  private int remainingInBlock = 0; // values in current block still to be read
+  private int remainingInMiniBlock = 0; // values in current mini block still to be read
+  private long[] unpackedValuesBuffer;
+
+  private ByteBufferInputStream in;
+
+  // temporary buffers used by readByte, readShort, readInteger, and readLong
+  byte byteVal;
+  short shortVal;
+  int intVal;
+  long longVal;
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    Preconditions.checkArgument(valueCount >= 1,
+        "Page must have at least one value, but it has " + valueCount);
+    this.in = in;
+    // Read the header
+    this.blockSizeInValues = BytesUtils.readUnsignedVarInt(in);
+    this.miniBlockNumInABlock = BytesUtils.readUnsignedVarInt(in);
+    double miniSize = (double) blockSizeInValues / miniBlockNumInABlock;
+    Preconditions.checkArgument(miniSize % 8 == 0,
+        "miniBlockSize must be multiple of 8, but it's " + miniSize);
+    this.miniBlockSizeInValues = (int) miniSize;
+    this.totalValueCount = BytesUtils.readUnsignedVarInt(in);
+    this.bitWidths = new int[miniBlockNumInABlock];
+    this.unpackedValuesBuffer = new long[miniBlockSizeInValues];
+    // read the first value
+    firstValue = BytesUtils.readZigZagVarLong(in);
+  }
+
+  @Override
+  public byte readByte() {
+    readValues(1, null, 0, (w, r, v) -> byteVal = (byte) v);
+    return byteVal;
+  }
+
+  @Override
+  public short readShort() {
+    readValues(1, null, 0, (w, r, v) -> shortVal = (short) v);
+    return shortVal;
+  }
+
+  @Override
+  public int readInteger() {
+    readValues(1, null, 0, (w, r, v) -> intVal = (int) v);
+    return intVal;
+  }
+
+  @Override
+  public long readLong() {
+    readValues(1, null, 0, (w, r, v) -> longVal = v);
+    return longVal;
+  }
+
+  @Override
+  public void readBytes(int total, WritableColumnVector c, int rowId) {
+    readValues(total, c, rowId, (w, r, v) -> w.putByte(r, (byte) v));
+  }
+
+  @Override
+  public void readShorts(int total, WritableColumnVector c, int rowId) {
+    readValues(total, c, rowId, (w, r, v) -> w.putShort(r, (short) v));
+  }
+
+  @Override
+  public void readIntegers(int total, WritableColumnVector c, int rowId) {
+    readValues(total, c, rowId, (w, r, v) -> w.putInt(r, (int) v));
+  }
+
+  // Based on VectorizedPlainValuesReader.readIntegersWithRebase
+  @Override
+  public final void readIntegersWithRebase(
+      int total, WritableColumnVector c, int rowId, boolean failIfRebase) {
+    readValues(total, c, rowId, (w, r, v) -> {
+      if (v < RebaseDateTime.lastSwitchJulianDay()) {
+        if (failIfRebase) {
+          throw DataSourceUtils.newRebaseExceptionInRead("Parquet");
+        } else {
+          w.putInt(r, RebaseDateTime.rebaseJulianToGregorianDays((int) v));
+        }
+      } else {
+        w.putInt(r, (int) v);
+      }
+    });
+  }
+
+  @Override
+  public void readUnsignedIntegers(int total, WritableColumnVector c, int rowId) {
+    readValues(total, c, rowId, (w, r, v) -> {
+      w.putLong(r, Integer.toUnsignedLong((int) v));
+    });
+  }
+
+  @Override
+  public void readUnsignedLongs(int total, WritableColumnVector c, int rowId) {
+    readValues(total, c, rowId, (w, r, v) -> {
+      w.putByteArray(r, new BigInteger(Long.toUnsignedString(v)).toByteArray());
+    });
+  }
+
+  @Override
+  public void readLongs(int total, WritableColumnVector c, int rowId) {
+    readValues(total, c, rowId, WritableColumnVector::putLong);
+  }
+
+  @Override
+  public final void readLongsWithRebase(
+      int total, WritableColumnVector c, int rowId, boolean failIfRebase) {
+    readValues(total, c, rowId, (w, r, v) -> {
+      if (v < RebaseDateTime.lastSwitchJulianTs()) {
+        if (failIfRebase) {
+          throw DataSourceUtils.newRebaseExceptionInRead("Parquet");
+        } else {
+          w.putLong(r, RebaseDateTime.rebaseJulianToGregorianMicros(v));
+        }
+      } else {
+        w.putLong(r, v);
+      }
+    });
+  }
+
+  @Override
+  public void skipBytes(int total) {
+    skipValues(total);
+  }
+
+  @Override
+  public void skipShorts(int total) {
+    skipValues(total);
+  }
+
+  @Override
+  public void skipIntegers(int total) {
+    skipValues(total);
+  }
+
+  @Override
+  public void skipLongs(int total) {
+    skipValues(total);
+  }
+
+  private void readValues(int total, WritableColumnVector c, int rowId,
+      IntegerOutputWriter outputWriter) {
+    if (valuesRead + total > totalValueCount) {
+      throw new ParquetDecodingException(
+          "No more values to read, total value count is " + valuesRead);

Review comment:
       nit: maybe revise this message a bit, since "total value count is + valuesRead" looks a bit confusing.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetDeltaEncodingSuite.scala
##########
@@ -0,0 +1,356 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import java.io.IOException
+import java.nio.ByteBuffer
+import java.util.Random
+
+import org.apache.parquet.bytes.{ByteBufferInputStream, DirectByteBufferAllocator}
+import org.apache.parquet.column.values.ValuesWriter
+import org.apache.parquet.column.values.delta.{DeltaBinaryPackingValuesWriterForInteger, DeltaBinaryPackingValuesWriterForLong}
+import org.apache.parquet.io.ParquetDecodingException
+
+import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, IntegralType, LongType}
+
+/**
+ * Read tests for vectorized Delta binary packed reader.
+ * Translated from
+ *  org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriterForIntegerTest
+ *  org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriterForLongTest
+ */
+abstract class ParquetDeltaEncodingSuite[T] extends ParquetCompatibilityTest
+  with SharedSparkSession {
+  protected var blockSize = 128
+  protected var miniBlockNum = 4
+  protected var reader: VectorizedDeltaBinaryPackedReader = _
+  protected var writableColumnVector: WritableColumnVector = _
+  protected var writer: ValuesWriter = _
+  protected var random: Random = _
+
+  protected def getSparkSqlType: IntegralType
+
+  protected def writeData(data: Array[T]): Unit
+
+  protected def writeData(data: Array[T], length: Int): Unit
+
+  protected def initValuesWriter (): Unit
+
+//  protected def allocArrayOfType[T: ClassTag](size: Int): Array[T] = new Array[T](size)
+
+  protected def allocDataArray(size: Int): Array[T]
+
+  protected def getNextRandom: T
+
+  protected def getTypeMinValue: T
+
+  protected def getTypeMaxValue: T
+
+  protected def readData(total: Int, columnVector : WritableColumnVector, rowId: Int): Unit
+
+  protected def skip(total: Int): Unit
+
+  protected def readDataFromVector(columnVector : WritableColumnVector, rowId: Int): T
+
+  protected def estimatedSize(length: Int) : Double
+
+  protected def setValue(arr: Array[T], index: Int, value: Int): Unit
+  protected def compareValues(expected: Int, actual: T) : Boolean
+
+
+  protected override def beforeEach(): Unit = {
+    blockSize = 128
+    miniBlockNum = 4
+    random = new Random(0)
+    initValuesWriter()
+    super.beforeAll()
+  }
+
+  test("read when data is aligned with block") {
+    val data = allocDataArray(5 * blockSize)
+    for (i <- 0 until blockSize * 5) {
+      data(i) = getNextRandom
+    }
+    shouldWriteAndRead(data)
+  }
+
+  test("read when block is not fully written") {
+    val data = allocDataArray(blockSize - 3)
+    for (i <- data.indices) {
+      data(i) = getNextRandom
+    }
+    shouldWriteAndRead(data)
+  }
+
+  test("read when mini block is not fully written") {
+    val miniBlockSize = blockSize / miniBlockNum
+    val data = allocDataArray(miniBlockSize - 3)
+    for (i <- data.indices) {
+      data(i) = getNextRandom
+    }
+    shouldWriteAndRead(data)
+  }
+
+  test("read with negative deltas") {
+    val data = allocDataArray(blockSize)
+    for (i <- data.indices) {
+      setValue(data, i, 10 - (i * 32 - random.nextInt(6)))
+    }
+    shouldWriteAndRead(data)
+  }
+
+  test("read when deltas are same") {
+    val data = allocDataArray(2 * blockSize)
+    for (i <- 0 until blockSize) {
+      setValue(data, i, i * 32)
+    }
+    for (i <- blockSize until 2 * blockSize) {
+      setValue(data, i, 0)
+    }
+    shouldWriteAndRead(data)
+  }
+
+  test("read when values are same") {
+    val data = allocDataArray(2 * blockSize)
+    for (i <- 0 until blockSize) {
+      setValue(data, i, 3)
+    }
+    for (i <- blockSize until 2 * blockSize) {
+      setValue(data, i, 0)
+    }
+    shouldWriteAndRead(data)
+  }
+
+  test("read when delta is 0 for each block") {
+    val data = allocDataArray(5 * blockSize + 1)
+    for (i <- data.indices) {
+      setValue(data, i, (i - 1) / blockSize)
+    }
+    shouldWriteAndRead(data)
+  }
+
+  test("read when data is not aligned with block") {
+    val data = allocDataArray(5 * blockSize + 3)
+    for (i <- data.indices) {
+      setValue(data, i, random.nextInt(20) - 10)
+    }
+    shouldWriteAndRead(data)
+  }
+
+  test("read max min value") {
+    val data = allocDataArray(10)
+    for (i <- data.indices) {
+      if (i % 2 == 0) data(i) = getTypeMinValue
+      else data(i) = getTypeMaxValue
+    }
+    shouldWriteAndRead(data)
+  }
+
+  test("throw exception when read more than written") {
+    val data = allocDataArray(5 * blockSize + 1)
+    for (i <- data.indices) {
+      setValue(data, i, i * 32)
+    }
+    shouldWriteAndRead(data)
+    try readData(1, writableColumnVector, data.length)
+    catch {
+      case e: ParquetDecodingException =>
+        assert("no more values to read, total value count is " + data.length == e.getMessage)
+    }
+  }
+
+  test("skip()") {
+    val data = allocDataArray(5 * blockSize + 1)
+    for (i <- data.indices) {
+      setValue(data, i, i * 32)
+    }
+    writeData(data)
+    reader = new VectorizedDeltaBinaryPackedReader
+    reader.initFromPage(100, writer.getBytes.toInputStream)
+    writableColumnVector = new OnHeapColumnVector(data.length, getSparkSqlType)
+    for (i <- data.indices) {
+      if (i % 3 == 0) {
+        skip(1)
+      } else {
+        readData(1, writableColumnVector, i)
+        assert(compareValues(i * 32, readDataFromVector(writableColumnVector, i)))
+      }
+    }
+  }
+
+  test("SkipN()") {
+    val data = allocDataArray(5 * blockSize + 1)
+    for (i <- data.indices) {
+      setValue(data, i, i * 32)
+    }
+    writeData(data)
+    reader = new VectorizedDeltaBinaryPackedReader
+    reader.initFromPage(100, writer.getBytes.toInputStream)
+    writableColumnVector = new OnHeapColumnVector(data.length, getSparkSqlType)
+    var skipCount = 0
+    var i = 0
+    while (i < data.length) {
+      skipCount = (data.length - i) / 2
+      readData(1, writableColumnVector, i)
+      assert(compareValues(i * 32, readDataFromVector(writableColumnVector, i)))
+      skip(skipCount)
+
+      i += skipCount + 1
+    }
+  }
+
+  test("random data test") {
+    val maxSize = 1000
+    val data = allocDataArray(maxSize)
+    for (round <- 0 until 100000) {
+      val size = random.nextInt(maxSize)
+      for (i <- 0 until size) {
+        data(i) = getNextRandom
+      }
+      shouldReadAndWrite(data, size)
+      writer.reset()
+    }
+  }
+
+  @throws[IOException]
+  private def shouldWriteAndRead(data: Array[T]): Unit = {
+    shouldReadAndWrite(data, data.length)
+  }
+
+  private def shouldReadAndWrite(data: Array[T], length: Int): Unit = {
+    writeData(data, length)
+    reader = new VectorizedDeltaBinaryPackedReader
+    val page = writer.getBytes.toByteArray
+
+    assert(estimatedSize(length) >= page.length)
+    writableColumnVector = new OnHeapColumnVector(data.length, getSparkSqlType)
+    reader.initFromPage(100, ByteBufferInputStream.wrap(ByteBuffer.wrap(page)))
+    readData(length, writableColumnVector, 0)
+    for (i <- 0 until length) {
+      assert(data(i) == readDataFromVector(writableColumnVector, i))
+    }
+  }
+
+}
+
+class ParquetDeltaEncodingInteger extends ParquetDeltaEncodingSuite[Int] {
+
+  override protected def getSparkSqlType: IntegralType = IntegerType
+  override protected def writeData(data: Array[Int]): Unit = writeData(data, data.length)
+
+  override protected def writeData(data: Array[Int], length: Int): Unit = {
+    for (i <- 0 until length) {
+      writer.writeInteger(data(i))
+    }
+  }
+
+  override protected def initValuesWriter (): Unit = {
+    writer = new DeltaBinaryPackingValuesWriterForInteger(
+      blockSize,
+      miniBlockNum,
+      100,
+      200,
+      new DirectByteBufferAllocator())
+  }
+
+  override protected def allocDataArray(size: Int): Array[Int] = new Array[Int](size)
+
+  override protected def getNextRandom: Int = random.nextInt
+  override protected def getTypeMinValue: Int = Int.MinValue
+  override protected def getTypeMaxValue: Int = Int.MaxValue
+
+  override protected def readData(total: Int, columnVector : WritableColumnVector, rowId: Int): Unit
+  = reader.readIntegers(total, columnVector, rowId)
+
+  override protected def skip(total: Int): Unit = reader.skipIntegers(total)
+
+  override protected def readDataFromVector(columnVector: WritableColumnVector, rowId: Int): Int =
+    columnVector.getInt(rowId)
+
+  override protected def estimatedSize(length: Int) : Double = {
+    val miniBlockSize = blockSize / miniBlockNum
+    val miniBlockFlushed = Math.ceil((length.toDouble - 1) / miniBlockSize)
+    val blockFlushed = Math.ceil((length.toDouble - 1) / blockSize)
+    4 * 5 /* blockHeader */ +
+      4 * miniBlockFlushed * miniBlockSize /* data(aligned to miniBlock) */ +
+      blockFlushed * miniBlockNum /* bitWidth of mini blocks */ +
+      (5.0 * blockFlushed) /* min delta for each block */
+  }
+
+  override protected def setValue(arr: Array[Int], index: Int, value: Int): Unit =
+    arr(index) = value
+
+  override protected def compareValues(expected: Int, actual: Int) : Boolean =
+    expected == actual
+
+}
+
+class ParquetDeltaEncodingLong extends ParquetDeltaEncodingSuite[Long] {
+
+  override protected def getSparkSqlType: IntegralType = LongType
+  override protected def writeData(data: Array[Long]): Unit = writeData(data, data.length)
+
+  override protected def writeData(data: Array[Long], length: Int): Unit = {
+    for (i <- 0 until length) {
+      writer.writeLong(data(i))
+    }
+  }
+
+  override protected def initValuesWriter (): Unit = {
+    writer = new DeltaBinaryPackingValuesWriterForLong(
+      blockSize,
+      miniBlockNum,
+      100,
+      200,
+      new DirectByteBufferAllocator())
+  }
+
+  override protected def allocDataArray(size: Int): Array[Long] = new Array[Long](size)
+
+  override protected def getNextRandom: Long = random.nextLong
+  override protected def getTypeMinValue: Long = Long.MinValue
+  override protected def getTypeMaxValue: Long = Long.MaxValue
+
+  override protected def readData(total: Int, columnVector: WritableColumnVector, rowId: Int): Unit
+  = reader.readLongs(total, columnVector, rowId)

Review comment:
       nit: indentation

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala
##########
@@ -142,13 +154,23 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark {
           }
         }
 
+        sqlBenchmark.addCase("SQL Parquet Vectorized (Delta Binary)") { _ =>
+          spark.sql("select sum(id) from parquetV2Table").noop()
+        }
+
+        sqlBenchmark.addCase("SQL Parquet MR (Delta Binary)") { _ =>
+          withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") {
+            spark.sql("select sum(id) from parquetV2Table").noop()
+          }
+        }
+
         sqlBenchmark.addCase("SQL ORC Vectorized") { _ =>
-          spark.sql(s"SELECT $query FROM orcTable").noop()
+          spark.sql("SELECT sum(id) FROM orcTable").noop()

Review comment:
       hmm why this change?

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala
##########
@@ -147,6 +159,16 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark {
           }
         }
 
+        sqlBenchmark.addCase("SQL Parquet Vectorized (Delta Binary)") { _ =>

Review comment:
       This won't work yet because of the `BooleanType` added recently.
   
   Error:
   
   ```
   [error] Exception in thread "main" org.apache.spark.sql.AnalysisException: cannot resolve 'sum(parquetv2table.id)' due to data type mismatch: function sum requires numeric or interval types, not boolean; line 1 pos 7;
   [error] 'Aggregate [unresolvedalias(sum(id#40), None)]
   [error] +- SubqueryAlias parquetv2table
   [error]    +- View (`parquetV2Table`, [id#40])
   [error]       +- Relation [id#40] parquet
   ```

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetDeltaEncodingSuite.scala
##########
@@ -0,0 +1,356 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import java.io.IOException
+import java.nio.ByteBuffer
+import java.util.Random
+
+import org.apache.parquet.bytes.{ByteBufferInputStream, DirectByteBufferAllocator}
+import org.apache.parquet.column.values.ValuesWriter
+import org.apache.parquet.column.values.delta.{DeltaBinaryPackingValuesWriterForInteger, DeltaBinaryPackingValuesWriterForLong}
+import org.apache.parquet.io.ParquetDecodingException
+
+import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, IntegralType, LongType}
+
+/**
+ * Read tests for vectorized Delta binary packed reader.
+ * Translated from
+ *  org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriterForIntegerTest
+ *  org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriterForLongTest
+ */
+abstract class ParquetDeltaEncodingSuite[T] extends ParquetCompatibilityTest
+  with SharedSparkSession {
+  protected var blockSize = 128
+  protected var miniBlockNum = 4
+  protected var reader: VectorizedDeltaBinaryPackedReader = _
+  protected var writableColumnVector: WritableColumnVector = _
+  protected var writer: ValuesWriter = _
+  protected var random: Random = _
+
+  protected def getSparkSqlType: IntegralType
+
+  protected def writeData(data: Array[T]): Unit
+
+  protected def writeData(data: Array[T], length: Int): Unit
+
+  protected def initValuesWriter (): Unit
+
+//  protected def allocArrayOfType[T: ClassTag](size: Int): Array[T] = new Array[T](size)
+
+  protected def allocDataArray(size: Int): Array[T]
+
+  protected def getNextRandom: T
+
+  protected def getTypeMinValue: T
+
+  protected def getTypeMaxValue: T
+
+  protected def readData(total: Int, columnVector : WritableColumnVector, rowId: Int): Unit
+
+  protected def skip(total: Int): Unit
+
+  protected def readDataFromVector(columnVector : WritableColumnVector, rowId: Int): T
+
+  protected def estimatedSize(length: Int) : Double
+
+  protected def setValue(arr: Array[T], index: Int, value: Int): Unit
+  protected def compareValues(expected: Int, actual: T) : Boolean
+
+
+  protected override def beforeEach(): Unit = {
+    blockSize = 128

Review comment:
       nit: these seem redundant




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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