11/*
2- * Copyright (c) 2025, NVIDIA CORPORATION.
2+ * Copyright (c) 2025-2026 , NVIDIA CORPORATION.
33 *
44 * Licensed under the Apache License, Version 2.0 (the "License");
55 * you may not use this file except in compliance with the License.
@@ -25,11 +25,12 @@ import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableSeq
2525import com .nvidia .spark .rapids .SpillPriorities .ACTIVE_ON_DECK_PRIORITY
2626import com .nvidia .spark .rapids .fileio .iceberg .IcebergFileIO
2727import com .nvidia .spark .rapids .iceberg .GpuIcebergSpecPartitioner
28+ import com .nvidia .spark .rapids .shims .parquet .ParquetFieldIdShims
2829import org .apache .hadoop .mapreduce .Job
2930import org .apache .hadoop .shaded .org .apache .commons .lang3 .reflect .{FieldUtils , MethodUtils }
3031import org .apache .iceberg ._
3132import org .apache .iceberg .io ._
32- import org .apache .iceberg .spark .{Spark3Util , SparkSchemaUtil }
33+ import org .apache .iceberg .spark .{GpuTypeToSparkType , Spark3Util , SparkSchemaUtil }
3334import org .apache .iceberg .spark .functions .{GpuFieldTransform , GpuTransform }
3435import org .apache .iceberg .spark .source .GpuWriteContext .positionDeleteSparkType
3536import org .apache .iceberg .spark .source .SparkWrite .TaskCommit
@@ -42,8 +43,10 @@ import org.apache.spark.sql.connector.distributions.Distribution
4243import org .apache .spark .sql .connector .expressions .SortOrder
4344import org .apache .spark .sql .connector .write .{DataWriter , _ }
4445import org .apache .spark .sql .connector .write .streaming .StreamingWrite
46+ import org .apache .spark .sql .execution .SparkPlan
4547import org .apache .spark .sql .execution .datasources .v2 .{AtomicCreateTableAsSelectExec , AtomicReplaceTableAsSelectExec }
4648import org .apache .spark .sql .rapids .GpuWriteJobStatsTracker
49+ import org .apache .spark .sql .rapids .shims .SparkSessionUtils
4750import org .apache .spark .sql .types .StructType
4851import org .apache .spark .sql .vectorized .ColumnarBatch
4952import org .apache .spark .util .SerializableConfiguration
@@ -104,7 +107,12 @@ class GpuSparkWrite(cpu: SparkWrite) extends GpuWrite with RequiresDistributionA
104107 val outputSpecId = FieldUtils .readField(cpu, " outputSpecId" , true ).asInstanceOf [Int ]
105108 val targetFileSize = FieldUtils .readField(cpu, " targetFileSize" , true ).asInstanceOf [Long ]
106109 val writeSchema = FieldUtils .readField(cpu, " writeSchema" , true ).asInstanceOf [Schema ]
107- val dsSchema = FieldUtils .readField(cpu, " dsSchema" , true ).asInstanceOf [StructType ]
110+ // Convert writeSchema to Spark StructType with Iceberg field IDs (PARQUET:field_id).
111+ // The CPU path uses Iceberg's own Parquet writer which natively embeds field IDs, but
112+ // the GPU path uses Spark's Parquet infrastructure which requires field IDs in the
113+ // StructType metadata. Without them, Iceberg's ParquetMetrics cannot extract file-level
114+ // statistics, causing StrictMetricsEvaluator to fail during overwrite validation.
115+ val dsSchema = GpuTypeToSparkType .toSparkType(writeSchema)
108116 val useFanout = FieldUtils .readField(cpu, " useFanoutWriter" , true ).asInstanceOf [Boolean ]
109117 val writeProps = FieldUtils .readField(cpu, " writeProperties" , true )
110118 .asInstanceOf [java.util.Map [String , String ]]
@@ -115,6 +123,7 @@ class GpuSparkWrite(cpu: SparkWrite) extends GpuWrite with RequiresDistributionA
115123 }
116124
117125 val hadoopConf = sparkContext.hadoopConfiguration
126+
118127 val job = {
119128 val tmpJob = Job .getInstance(hadoopConf)
120129 tmpJob.setOutputKeyClass(classOf [Void ])
@@ -180,6 +189,16 @@ object GpuSparkWrite {
180189 partitionSpec : PartitionSpec ,
181190 meta : SparkPlanMeta [_]): Unit = {
182191
192+ // Iceberg requires Parquet field IDs for correct file-level metrics. Without them,
193+ // StrictMetricsEvaluator fails during overwrite validation.
194+ val spark = SparkSessionUtils .sessionFromPlan(meta.wrapped.asInstanceOf [SparkPlan ])
195+ val hadoopConf = spark.sparkContext.hadoopConfiguration
196+ val sqlConf = spark.sessionState.conf
197+ if (! ParquetFieldIdShims .getParquetIdWriteEnabled(hadoopConf, sqlConf)) {
198+ meta.willNotWorkOnGpu(" Iceberg requires Parquet field IDs to be written for correct " +
199+ " file-level metrics. Set spark.sql.parquet.fieldId.write.enabled=true" )
200+ }
201+
183202 // Check file format support
184203 if (dataFormat.exists(! _.equals(FileFormat .PARQUET ))) {
185204 meta.willNotWorkOnGpu(s " GpuSparkWrite only supports Parquet, but got: ${dataFormat.get}" )
@@ -292,6 +311,7 @@ object GpuSparkWrite {
292311 def convert (cpuWrite : Write ): GpuSparkWrite = {
293312 new GpuSparkWrite (cpuWrite.asInstanceOf [SparkWrite ])
294313 }
314+
295315}
296316
297317class GpuWriterFactory (val tableBroadcast : Broadcast [Table ],
0 commit comments