Skip to content

Commit

Permalink
fix: Use makeCopy to change relation in FileSourceScanExec
Browse files Browse the repository at this point in the history
  • Loading branch information
viirya committed Mar 13, 2024
1 parent 9aa42cd commit bc12dc3
Showing 1 changed file with 12 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -439,8 +439,18 @@ case class CometScanExec(

object CometScanExec {
def apply(scanExec: FileSourceScanExec, session: SparkSession): CometScanExec = {
val wrapped = scanExec.copy(relation =
scanExec.relation.copy(fileFormat = new CometParquetFileFormat)(session))
// 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 = scanExec.mapProductIterator(transform)
val wrapped = scanExec.makeCopy(newArgs).asInstanceOf[FileSourceScanExec]
val batchScanExec = CometScanExec(
wrapped.relation,
wrapped.output,
Expand Down

0 comments on commit bc12dc3

Please sign in to comment.