@@ -24,11 +24,13 @@ import java.util.Locale
2424import scala .jdk .CollectionConverters ._
2525
2626import org .apache .spark .SparkException
27+ import org .apache .spark .sql .{SaveMode , SparkSession }
2728import org .apache .spark .sql .comet .{CometNativeExec , CometNativeWriteExec }
2829import org .apache .spark .sql .execution .command .DataWritingCommandExec
2930import org .apache .spark .sql .execution .datasources .{InsertIntoHadoopFsRelationCommand , WriteFilesExec }
3031import org .apache .spark .sql .execution .datasources .parquet .ParquetFileFormat
3132import org .apache .spark .sql .internal .SQLConf
33+ import org .apache .spark .sql .internal .SQLConf .PartitionOverwriteMode
3234
3335import org .apache .comet .{CometConf , ConfigEntry , DataTypeSupport }
3436import org .apache .comet .CometSparkSessionExtensions .withInfo
@@ -61,6 +63,10 @@ object CometDataWritingCommand extends CometOperatorSerde[DataWritingCommandExec
6163 return Unsupported (Some (" Bucketed writes are not supported" ))
6264 }
6365
66+ if (SQLConf .get.partitionOverwriteMode == PartitionOverwriteMode .DYNAMIC ) {
67+ return Unsupported (Some (" Dynamic partition overwrite is not supported" ))
68+ }
69+
6470 if (cmd.partitionColumns.nonEmpty || cmd.staticPartitions.nonEmpty) {
6571 return Incompatible (Some (" Partitioned writes are highly experimental" ))
6672 }
@@ -158,6 +164,14 @@ object CometDataWritingCommand extends CometOperatorSerde[DataWritingCommandExec
158164 val cmd = op.cmd.asInstanceOf [InsertIntoHadoopFsRelationCommand ]
159165 val outputPath = cmd.outputPath.toString
160166
167+ // TODO : support dynamic partition overwrite
168+ if (cmd.mode == SaveMode .Overwrite ) {
169+ val fs = cmd.outputPath.getFileSystem(SparkSession .active.sparkContext.hadoopConfiguration)
170+ if (fs.exists(cmd.outputPath)) {
171+ fs.delete(cmd.outputPath, true )
172+ }
173+ }
174+
161175 // Get the child plan from the WriteFilesExec or use the child directly
162176 val childPlan = op.child match {
163177 case writeFiles : WriteFilesExec =>
@@ -168,8 +182,6 @@ object CometDataWritingCommand extends CometOperatorSerde[DataWritingCommandExec
168182 other
169183 }
170184
171- val isDynamicOverWriteMode = cmd.partitionColumns.nonEmpty
172-
173185 // Create FileCommitProtocol for atomic writes
174186 val jobId = java.util.UUID .randomUUID().toString
175187 val committer =
0 commit comments