Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix: Use makeCopy to change relation in FileSourceScanExec #207

Merged
merged 1 commit into from
Mar 14, 2024
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ package org.apache.spark.sql.comet

import scala.collection.mutable.HashMap
import scala.concurrent.duration.NANOSECONDS
import scala.reflect.ClassTag

import org.apache.hadoop.fs.Path
import org.apache.spark.rdd.RDD
Expand Down Expand Up @@ -439,8 +440,29 @@ case class CometScanExec(

object CometScanExec {
Copy link
Contributor

Choose a reason for hiding this comment

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

perhaps we can also consider CometScanExec extends TreeNode to remove mapProductIterator method copy?

Copy link
Member Author

Choose a reason for hiding this comment

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

We want to run mapProductIterator on FileSourceScanExec. I'm not sure how object CometScanExec extends TreeNode can help us?

Copy link
Contributor

Choose a reason for hiding this comment

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

You are right, its needed to run mapProductIterator on FileSourceScanExec, I wrongly assumed this method needed for CometScanExec .

def apply(scanExec: FileSourceScanExec, session: SparkSession): CometScanExec = {
val wrapped = scanExec.copy(relation =
scanExec.relation.copy(fileFormat = new CometParquetFileFormat)(session))
// TreeNode.mapProductIterator is protected method.
def mapProductIterator[B: ClassTag](product: Product, f: Any => B): Array[B] = {
val arr = Array.ofDim[B](product.productArity)
var i = 0
while (i < arr.length) {
arr(i) = f(product.productElement(i))
i += 1
}
arr
}

// Replacing the relation in FileSourceScanExec by `copy` seems causing some issues
// on other Spark distributions if FileSourceScanExec constructor is changed.
// Using `makeCopy` to avoid the issue.
// https://github.com/apache/arrow-datafusion-comet/issues/190
def transform(arg: Any): AnyRef = arg match {
case _: HadoopFsRelation =>
scanExec.relation.copy(fileFormat = new CometParquetFileFormat)(session)
case other: AnyRef => other
case null => null
}
val newArgs = mapProductIterator(scanExec, transform(_))
val wrapped = scanExec.makeCopy(newArgs).asInstanceOf[FileSourceScanExec]
val batchScanExec = CometScanExec(
wrapped.relation,
wrapped.output,
Expand Down
Loading