diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala index 4bf01f0f43..af99eea1af 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala @@ -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,