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

    https://github.com/apache/spark/pull/11709#discussion_r56428259
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
 ---
    @@ -268,6 +275,140 @@ private[sql] class DefaultSource extends FileFormat 
with DataSourceRegister with
             file.getName == ParquetFileWriter.PARQUET_METADATA_FILE
       }
     
    +  /**
    +   * Returns a function that can be used to read a single file in as an 
Iterator of InternalRow.
    +   *
    +   * @param partitionSchema The schema of the partition column row that 
will be present in each
    +   *                        PartitionedFile.  These columns should be 
prepended to the rows that
    +   *                        are produced by the iterator.
    +   * @param dataSchema The schema of the data that should be output for 
each row.  This may be a
    +   *                   subset of the columns that are present in the file 
if  column pruning has
    +   *                   occurred.
    +   * @param filters A set of filters than can optionally be used to reduce 
the number of rows output
    +   * @param options A set of string -> string configuration options.
    +   * @return
    +   */
    +  override def buildReader(
    +      sqlContext: SQLContext,
    +      partitionSchema: StructType,
    +      dataSchema: StructType,
    +      filters: Seq[Filter],
    +      options: Map[String, String]): PartitionedFile => 
Iterator[InternalRow] = {
    +    val parquetConf = new 
Configuration(sqlContext.sparkContext.hadoopConfiguration)
    +    parquetConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, 
classOf[CatalystReadSupport].getName)
    +    parquetConf.set(
    +      CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
    +      CatalystSchemaConverter.checkFieldNames(dataSchema).json)
    +    parquetConf.set(
    +      CatalystWriteSupport.SPARK_ROW_SCHEMA,
    +      CatalystSchemaConverter.checkFieldNames(dataSchema).json)
    +
    +    // We want to clear this temporary metadata from saving into Parquet 
file.
    +    // This metadata is only useful for detecting optional columns when 
pushdowning filters.
    +    val dataSchemaToWrite = 
StructType.removeMetadata(StructType.metadataKeyForOptionalField,
    +      dataSchema).asInstanceOf[StructType]
    +    CatalystWriteSupport.setSchema(dataSchemaToWrite, parquetConf)
    +
    +    // Sets flags for `CatalystSchemaConverter`
    +    parquetConf.setBoolean(
    +      SQLConf.PARQUET_BINARY_AS_STRING.key,
    +      sqlContext.conf.getConf(SQLConf.PARQUET_BINARY_AS_STRING))
    +    parquetConf.setBoolean(
    +      SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
    +      sqlContext.conf.getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP))
    +
    +    // Try to push down filters when filter push-down is enabled.
    +    val pushed = if 
(sqlContext.getConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key).toBoolean) {
    +      filters
    +          // Collects all converted Parquet filter predicates. Notice that 
not all predicates can be
    +          // converted (`ParquetFilters.createFilter` returns an 
`Option`). That's why a `flatMap`
    +          // is used here.
    +          .flatMap(ParquetFilters.createFilter(dataSchema, _))
    +          .reduceOption(FilterApi.and)
    +    } else {
    +      None
    +    }
    +
    +    val broadcastedConf =
    +      sqlContext.sparkContext.broadcast(new 
SerializableConfiguration(parquetConf))
    +
    +    // TODO: if you move this into the closure it reverts to the default 
values.
    +    val useUnsafeReader: Boolean =
    +      sqlContext.getConf(SQLConf.PARQUET_UNSAFE_ROW_RECORD_READER_ENABLED)
    +
    +    // If true, enable using the custom RecordReader for parquet. This 
only works for
    +    // a subset of the types (no complex types).
    +    val enableUnsafeRowParquetReader: Boolean =
    +      
sqlContext.getConf(SQLConf.PARQUET_UNSAFE_ROW_RECORD_READER_ENABLED.key).toBoolean
    +    val enableVectorizedParquetReader: Boolean =
    +      
sqlContext.getConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key).toBoolean
    +    val enableWholestageCodegen: Boolean =
    +      sqlContext.getConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key).toBoolean
    +
    +    (file: PartitionedFile) => {
    +      val fileSplit =
    +        new FileSplit(new Path(new URI(file.filePath)), file.start, 
file.length, Array.empty)
    +
    +      val split =
    +        new org.apache.parquet.hadoop.ParquetInputSplit(
    +          fileSplit.getPath,
    +          fileSplit.getStart,
    +          fileSplit.getStart + fileSplit.getLength,
    +          fileSplit.getLength,
    +          fileSplit.getLocations,
    +          null)
    +
    +      val attemptId = new TaskAttemptID(new TaskID(new JobID(), 
TaskType.MAP, 0), 0)
    +      val hadoopAttemptContext = new 
TaskAttemptContextImpl(broadcastedConf.value.value, attemptId)
    +
    +      val parquetReader = try {
    +        if (!useUnsafeReader) sys.error("Unsafe reader turned off.")
    +        val unsafeReader = new UnsafeRowParquetRecordReader()
    +        unsafeReader.initialize(split, hadoopAttemptContext)
    +
    +        if (enableVectorizedParquetReader) {
    +          unsafeReader.initBatch(partitionSchema, file.partitionValues)
    +          // Whole stage codegen (PhysicalRDD) is able to deal with 
batches directly
    +          // TODO: fix column appending
    +          if (enableWholestageCodegen) {
    +            unsafeReader.enableReturningBatches()
    +          }
    +        }
    +        unsafeReader
    +      } catch {
    +        case NonFatal(e) =>
    +          logError(s"Falling back to parquet-mr: $e", e)
    +          val reader = pushed match {
    +            case Some(filter) =>
    +              new ParquetRecordReader[InternalRow](
    +                new CatalystReadSupport,
    +                FilterCompat.get(filter, null))
    +            case _ =>
    +              new ParquetRecordReader[InternalRow](new CatalystReadSupport)
    +          }
    +          reader.initialize(split, hadoopAttemptContext)
    +          reader
    +      }
    +
    +      val iter = new RecordReaderIterator(parquetReader)
    +      val fullSchema = dataSchema.toAttributes ++ 
partitionSchema.toAttributes
    +      val joinedRow = new JoinedRow()
    +      val appendPartitionColumns = 
GenerateUnsafeProjection.generate(fullSchema, fullSchema)
    +
    +      // UnsafeRowParquetRecordReader appends the columns internally to 
avoid another copy.
    +      if (parquetReader.isInstanceOf[UnsafeRowParquetRecordReader]) {
    --- End diff --
    
    UnsafeRowParquetRecordReader could still produce UnsafeRow (without 
partition values), if enableVectorizedParquetReader is false


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