dongjoon-hyun commented on a change in pull request #25017: [SPARK-28218][SQL] Migrate Avro to File source V2 URL: https://github.com/apache/spark/pull/25017#discussion_r300525553
########## File path: external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala ########## @@ -0,0 +1,128 @@ +/* + * 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.v2.avro + +import java.net.URI + +import scala.util.control.NonFatal + +import org.apache.avro.Schema +import org.apache.avro.file.DataFileReader +import org.apache.avro.generic.{GenericDatumReader, GenericRecord} +import org.apache.avro.mapred.FsInput +import org.apache.hadoop.fs.Path + +import org.apache.spark.TaskContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +/** + * A factory used to create AVRO readers. + * + * @param sqlConf SQL configuration. + * @param broadcastedConf Broadcast serializable Hadoop Configuration. + * @param dataSchema Schema of AVRO files. + * @param readDataSchema Required schema of AVRO files. + * @param partitionSchema Schema of partitions. + * @param options Options for parsing AVRO files. + */ +case class AvroPartitionReaderFactory( + sqlConf: SQLConf, + broadcastedConf: Broadcast[SerializableConfiguration], + dataSchema: StructType, + readDataSchema: StructType, + partitionSchema: StructType, + options: Map[String, String]) extends FilePartitionReaderFactory with Logging { + + override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { + val conf = broadcastedConf.value.value + val parsedOptions = new AvroOptions(options, conf) + val userProvidedSchema = parsedOptions.schema.map(new Schema.Parser().parse) + + if (parsedOptions.ignoreExtension || partitionedFile.filePath.endsWith(".avro")) { Review comment: Shall we have the same comment in order not to forget that? ```scala // TODO Removes this check once `FileFormat` gets a general file filtering interface method. // Doing input file filtering is improper because we may generate empty tasks that process no // input files but stress the scheduler. We should probably add a more general input file // filtering mechanism for `FileFormat` data sources. See SPARK-16317. ``` ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
