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

    https://github.com/apache/spark/pull/9774#discussion_r45235636
  
    --- Diff: 
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java
 ---
    @@ -0,0 +1,613 @@
    +/*
    + * 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.Preconditions;
    +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.OriginalType;
    +import org.apache.parquet.schema.PrimitiveType;
    +import org.apache.parquet.schema.Type;
    +import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
    +import org.apache.spark.sql.types.Decimal;
    +import org.apache.spark.unsafe.Platform;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.util.*;
    +
    +/**
    + * A specialized RecordReader that reads into UnsafeRows directly using 
the Parquet column APIs.
    + *
    + * This is somewhat based on parquet-mr's ColumnReader.
    + *
    + * TODO: handle complex types, decimal requiring more than 8 bytes, INT96. 
Schema mismatch.
    + * All of these can be handled efficiently and easily with codegen.
    + */
    +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;
    +
    +  /**
    +   * Used to write variable lenght columns. Same length as `rows`.
    +   */
    +  private RowWriter[] rowWriters = null;
    +
    +  /**
    +   * True if the row contains variable length fields.
    +   */
    +  private boolean containsVarLenFields;
    +
    +  /**
    +   * 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;
    +
    +  /**
    +   * For each column, the annotated original type.
    +   */
    +  OriginalType[] originalTypes;
    +
    +  /**
    +   * Implementation of RecordReader API.
    +   */
    +  @Override
    +  public void initialize(InputSplit inputSplit, TaskAttemptContext 
taskAttemptContext)
    +      throws IOException, InterruptedException {
    +    super.initialize(inputSplit, taskAttemptContext);
    +
    +    /**
    +     * Check that the requested schema is supported.
    +     */
    +    int numVarLenFields = 0;
    +    originalTypes = new OriginalType[requestedSchema.getFieldCount()];
    +    for (int i = 0; i < requestedSchema.getFieldCount(); ++i) {
    +      Type t = requestedSchema.getFields().get(i);
    +      if (!t.isPrimitive()) throw new IOException("Complex types not 
supported.");
    +      PrimitiveType primitiveType = t.asPrimitiveType();
    +
    +      originalTypes[i] = t.getOriginalType();
    +
    +      // TODO: Be extremely cautious in what is supported. Expand this.
    --- End diff --
    
    Could we move this logic to where we are deciding which reader to 
instantiate instead of using exceptions for control flow?


---
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