Github user julienledem commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9774#discussion_r45162142
  
    --- Diff: 
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java
 ---
    @@ -0,0 +1,428 @@
    +/*
    + * 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 static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL;
    +import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL;
    +import static org.apache.parquet.column.ValuesType.VALUES;
    +
    +import org.apache.hadoop.mapreduce.InputSplit;
    +import org.apache.hadoop.mapreduce.TaskAttemptContext;
    +import org.apache.parquet.column.ColumnDescriptor;
    +import org.apache.parquet.column.Dictionary;
    +import org.apache.parquet.column.page.*;
    +import org.apache.parquet.column.Encoding;
    +import org.apache.parquet.column.values.ValuesReader;
    +import org.apache.parquet.io.api.Binary;
    +import org.apache.parquet.schema.Type;
    +import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
    +import org.apache.spark.unsafe.Platform;
    +
    +import java.io.IOException;
    +import java.util.*;
    +
    +/**
    + * A specialized RecordReader that reads into UnsafeRows directly using 
the Parquet column APIs.
    + *
    + * TODO: handle complex types.
    + */
    +public class UnsafeRowParquetRecordReader extends 
SpecificParquetRecordReaderBase<UnsafeRow> {
    +  /**
    +   * Batch of unsafe rows that we assemble and the current index we've 
returned. Everytime this
    +   * batch is used up (batchIdx == numBatched), we populated the batch.
    +   */
    +  private UnsafeRow[] rows = new UnsafeRow[64];
    +  private int batchIdx = 0;
    +  private int numBatched = 0;
    +
    +  /**
    +   * For each request column, the reader to read this column.
    +   * columnsReaders[i] populated the UnsafeRow's attribute at i.
    +   */
    +  private ColumnReader[] columnReaders;
    +
    +  /**
    +   * The number of rows that have been returned.
    +   */
    +  private long rowsReturned;
    +
    +  /**
    +   * The number of rows that have been reading, including the current in 
flight row group.
    +   */
    +  private long totalCountLoadedSoFar = 0;
    +
    +  /**
    +   * Implementation of RecordReader API.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext 
taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    super.initialize(inputSplit, taskAttemptContext);
    +
    +    int rowByteSize = 
UnsafeRow.calculateBitSetWidthInBytes(requestedSchema.getFieldCount());
    +    rowByteSize += 8 * requestedSchema.getFieldCount();
    +
    +    byte[] buffer = new byte[rowByteSize * rows.length];
    +    for (int i = 0; i < rows.length; ++i) {
    +      rows[i] = new UnsafeRow();
    +      rows[i].pointTo(buffer, Platform.BYTE_ARRAY_OFFSET + i * rowByteSize,
    +          requestedSchema.getFieldCount(), rowByteSize);
    +    }
    +
    +    /**
    +     * Check that the requested schema is supported.
    +     */
    +    for (Type t: requestedSchema.getFields()) {
    +      if (!t.isPrimitive()) throw new IOException("Unsupported schema");
    +    }
    +
    +    // TODO: this needs to handle the schema resolution rules
    +  }
    +
    +  @Override
    +  public boolean nextKeyValue() throws IOException, InterruptedException {
    +    if (batchIdx >= numBatched) {
    +      if (!loadBatch()) return false;
    +    }
    +    ++batchIdx;
    +    return true;
    +  }
    +
    +  @Override
    +  public UnsafeRow getCurrentValue() throws IOException, 
InterruptedException {
    +    return rows[batchIdx - 1];
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException, InterruptedException {
    +    return (float) rowsReturned / totalRowCount;
    +  }
    +
    +  /**
    +   * Decodes a batch of values into `rows`. This function is the hot path.
    +   */
    +  private boolean loadBatch() throws IOException {
    +    // no more records left
    +    if (rowsReturned >= totalRowCount) { return false; }
    +    checkEndOfRowGroup();
    +
    +    int num = (int)Math.min(rows.length, totalCountLoadedSoFar - 
rowsReturned);
    +    rowsReturned += num;
    +
    +    for (int i = 0; i < columnReaders.length; ++i) {
    +      switch (columnReaders[i].descriptor.getType()) {
    +        case BOOLEAN:
    +          decodeBooleanBatch(i, num);
    +          break;
    +        case INT32:
    +          decodeIntBatch(i, num);
    +          break;
    +        case INT64:
    +          decodeLongBatch(i, num);
    +          break;
    +        case FLOAT:
    +          decodeFloatBatch(i, num);
    +          break;
    +        case DOUBLE:
    +          decodeDoubleBatch(i, num);
    +          break;
    +      }
    +      numBatched = num;
    +      batchIdx = 0;
    +    }
    +    return true;
    +  }
    +
    +  private void decodeBooleanBatch(int col, int num) throws IOException {
    +    for (int n = 0; n < num; ++n) {
    +      if (columnReaders[col].next() != 0) {
    +        rows[n].setBoolean(col, columnReaders[col].nextBoolean());
    +      } else {
    +        rows[n].setNullAt(col);
    +      }
    +    }
    +  }
    +
    +  private void decodeIntBatch(int col, int num) throws IOException {
    +    for (int n = 0; n < num; ++n) {
    +      if (columnReaders[col].next() != 0) {
    +        rows[n].setInt(col, columnReaders[col].nextInt());
    +      } else {
    +        rows[n].setNullAt(col);
    +      }
    +    }
    +  }
    +
    +  private void decodeLongBatch(int col, int num) throws IOException {
    +    for (int n = 0; n < num; ++n) {
    +      if (columnReaders[col].next() != 0) {
    +        rows[n].setLong(col, columnReaders[col].nextLong());
    +      } else {
    +        rows[n].setNullAt(col);
    +      }
    +    }
    +  }
    +
    +  private void decodeFloatBatch(int col, int num) throws IOException {
    +    for (int n = 0; n < num; ++n) {
    +      if (columnReaders[col].next() != 0) {
    +        rows[n].setFloat(col, columnReaders[col].nextFloat());
    +      } else {
    +        rows[n].setNullAt(col);
    +      }
    +    }
    +  }
    +
    +  private void decodeDoubleBatch(int col, int num) throws IOException {
    +    for (int n = 0; n < num; ++n) {
    +      if (columnReaders[col].next() != 0) {
    +        rows[n].setDouble(col, columnReaders[col].nextDouble());
    +      } else {
    +        rows[n].setNullAt(col);
    +      }
    +    }
    +  }
    +
    +  /**
    +   *
    +   * Decoder to return values from a single column.
    +   */
    +  private static final class ColumnReader {
    +    /**
    +     * Total number of values read.
    +     */
    +    private long valuesRead;
    +
    +    /**
    +     * value that indicates the end of the current page. That is,
    +     * if valuesRead == endOfPageValueCount, we are at the end of the page.
    +     */
    +    private long endOfPageValueCount;
    +
    +    /**
    +     * The dictionary, if this column has dictionary encoding.
    +     */
    +    private final Dictionary dictionary;
    +
    +    /**
    +     * If true, the current page is dictionary encoded.
    +     */
    +    private boolean useDictionary;
    +
    +    /**
    +     * Repetition/Definition/Value readers.
    +     */
    +    private IntIterator repetitionLevelColumn;
    +    private IntIterator definitionLevelColumn;
    +    private ValuesReader dataColumn;
    +
    +    /**
    +     * Total number of values in this column (in this row group).
    +     */
    +    private final long totalValueCount;
    +
    +    /**
    +     * Total values in the current page.
    +     */
    +    private int pageValueCount;
    +
    +    private PageReader pageReader;
    +    private ColumnDescriptor descriptor;
    +
    +    public ColumnReader(ColumnDescriptor descriptor, PageReader 
pageReader) throws IOException {
    +      this.descriptor = descriptor;
    +      this.pageReader = pageReader;
    +
    +      DictionaryPage dictionaryPage = pageReader.readDictionaryPage();
    +      if (dictionaryPage != null) {
    +        try {
    +          this.dictionary = 
dictionaryPage.getEncoding().initDictionary(descriptor, dictionaryPage);
    +          this.useDictionary = true;
    +        } catch (IOException e) {
    +          throw new IOException("could not decode the dictionary for " + 
descriptor, e);
    +        }
    +      } else {
    +        this.dictionary = null;
    +        this.useDictionary = false;
    +      }
    +      this.totalValueCount = pageReader.getTotalValueCount();
    +      if (totalValueCount == 0) {
    +        throw new IOException("totalValueCount == 0");
    +      }
    +    }
    +
    +    /**
    +     * TODO: Hoist the useDictionary branch to decode*Batch and make the 
batch page aligned.
    +     */
    +    public boolean nextBoolean() {
    +      if (!useDictionary) {
    +        return dataColumn.readBoolean();
    +      } else {
    +        return 
dictionary.decodeToBoolean(dataColumn.readValueDictionaryId());
    +      }
    +    }
    +
    +    public int nextInt() {
    +      if (!useDictionary) {
    +        return dataColumn.readInteger();
    +      } else {
    +        return dictionary.decodeToInt(dataColumn.readValueDictionaryId());
    +      }
    +    }
    +
    +    public long nextLong() {
    +      if (!useDictionary) {
    +        return dataColumn.readLong();
    +      } else {
    +        return dictionary.decodeToLong(dataColumn.readValueDictionaryId());
    +      }
    +    }
    +
    +    public float nextFloat() {
    +      if (!useDictionary) {
    +        return dataColumn.readFloat();
    +      } else {
    +        return 
dictionary.decodeToFloat(dataColumn.readValueDictionaryId());
    +      }
    +    }
    +
    +    public double nextDouble() {
    +      if (!useDictionary) {
    +        return dataColumn.readDouble();
    +      } else {
    +        return 
dictionary.decodeToDouble(dataColumn.readValueDictionaryId());
    +      }
    +    }
    +
    +    public Binary nextBinary() {
    +      if (!useDictionary) {
    +        return dataColumn.readBytes();
    +      } else {
    +        return 
dictionary.decodeToBinary(dataColumn.readValueDictionaryId());
    +      }
    +    }
    +
    +    private int next() throws IOException {
    +      if (valuesRead >= endOfPageValueCount) {
    +        if (valuesRead >= totalValueCount) {
    +          // How do we get here? Throw end of stream exception?
    +          return 0;
    +        }
    +        readPage();
    +      }
    +      ++valuesRead;
    +      // TODO: Don't read for flat schemas
    +      //repetitionLevel = repetitionLevelColumn.nextInt();
    +      return definitionLevelColumn.nextInt();
    +    }
    +
    +    private void readPage() throws IOException {
    +      DataPage page = pageReader.readPage();
    +      // TODO: Why is this a visitor?
    --- End diff --
    
    well, you can use instanceof and casts, is that what you are getting at?
    the visitor is type safe ;)
    In Scala you would just use patter matching:
    ```
    page match {
      case dataPageV1: DataPageV1 => readPageV1(dataPageV1)
      case dataPageV2: DataPageV2 => readPageV2(dataPageV2)
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to