Skip to content

Conversation

@gengliangwang
Copy link
Member

What changes were proposed in this pull request?

Migrate Avro to File source V2.

How was this patch tested?

Unit test

@gengliangwang
Copy link
Member Author

This is the last migration for file source V2. It is a relatively simple one. Please help review it.
@cloud-fan @dongjoon-hyun

@SparkQA
Copy link

SparkQA commented Jun 30, 2019

Test build #107051 has finished for PR 25017 at commit 747eeb5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class AvroDataSourceV2 extends FileDataSourceV2
  • case class AvroPartitionReaderFactory(
  • case class AvroScan(
  • class AvroScanBuilder (
  • case class AvroTable(
  • class AvroWriteBuilder(

@dongjoon-hyun
Copy link
Member

Thank you for pinging me, @gengliangwang .
cc @dbtsai , too.

@dongjoon-hyun
Copy link
Member

Retest this please.

@SparkQA
Copy link

SparkQA commented Jul 3, 2019

Test build #107144 has finished for PR 25017 at commit 747eeb5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class AvroDataSourceV2 extends FileDataSourceV2
  • case class AvroPartitionReaderFactory(
  • case class AvroScan(
  • class AvroScanBuilder (
  • case class AvroTable(
  • class AvroWriteBuilder(

@SparkQA
Copy link

SparkQA commented Jul 4, 2019

Test build #107217 has finished for PR 25017 at commit 92d2f47.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class AvroV1LogicalTypeSuite extends AvroLogicalTypeSuite
  • class AvroV2LogicalTypeSuite extends AvroLogicalTypeSuite

job: Job,
options: Map[String, String],
dataSchema: StructType): OutputWriterFactory = {
val parsedOptions = new AvroOptions(options, job.getConfiguration)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Previously, this was the following (sharedState.sparkContext.hadoopConfiguration + SQLConf). Is job.getConfiguration enough for Avro?

val parsedOptions = new AvroOptions(options, spark.sessionState.newHadoopConf())

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, it is enough. Orc/Parquet also use the configuration from job.

val parsedOptions = new AvroOptions(options, conf)
val userProvidedSchema = parsedOptions.schema.map(new Schema.Parser().parse)

if (parsedOptions.ignoreExtension || partitionedFile.filePath.endsWith(".avro")) {
Copy link
Member

@dongjoon-hyun dongjoon-hyun Jul 5, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shall we have the same comment above this line in order not to forget that?

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

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, there is an option pathGlobFilter for it. I have marked it as deprecated in #24518.
I think we can still support it in 3.0. So I am not sure what to comment here.

paths: Seq[String],
userSpecifiedSchema: Option[StructType],
fallbackFileFormat: Class[_ <: FileFormat])
extends FileTable(sparkSession, options, paths, userSpecifiedSchema) with Logging {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's remove with Logging and line 23.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I left a few comments. Could you update the PR, @gengliangwang ?

@dongjoon-hyun dongjoon-hyun changed the title [SPARK-28218][SQL] Migrate Avro to File source V2 [SPARK-28218][SQL] Migrate Avro to File Data Source V2 Jul 5, 2019
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._
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

-import org.apache.spark.sql.execution.datasources.v2._
+import org.apache.spark.sql.execution.datasources.v2.{EmptyPartitionReader, FilePartitionReaderFactory, PartitionReaderWithPartitionValues}

@gengliangwang
Copy link
Member Author

@dongjoon-hyun I have updated the code. Thanks for reviewing this in your vacation!

@SparkQA
Copy link

SparkQA commented Jul 5, 2019

Test build #107263 has finished for PR 25017 at commit fc98bd5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM. Merged to master.
Thank you, @gengliangwang !

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants