Github user marmbrus commented on a diff in the pull request: https://github.com/apache/spark/pull/11709#discussion_r57363361 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala --- @@ -269,6 +276,137 @@ 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. + // If true, enable using the custom RecordReader for parquet. This only works for + // a subset of the types (no complex types). + 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) => { + assert(file.partitionValues.numFields == partitionSchema.size) + + 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 (!enableVectorizedParquetReader) sys.error("Vectorized reader turned off.") + val vectorizedReader = new VectorizedParquetRecordReader() + vectorizedReader.initialize(split, hadoopAttemptContext) + logDebug(s"Appending $partitionSchema ${file.partitionValues}") + vectorizedReader.initBatch(partitionSchema, file.partitionValues) --- End diff -- Basically, I'd just take the non-vectorized version below, put it in a utility function and use it everywhere. If we vectorize all the sources, that will be the only part we have to remove and then this can be done in FileScanRDD. I think that you do not want to do the actually partition appending in the planner like we were before, because you can't have Spark Partitions (splits) that read from different partitions very easily. This is what was making the bucking logic so convoluted in the old code path. This makes bucketing and collapsing of small files into a single partition much simpler.
--- 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