|
19 | 19 |
|
20 | 20 | package org.apache.spark.sql.comet |
21 | 21 |
|
| 22 | +import java.io.File |
| 23 | + |
22 | 24 | import scala.collection.mutable |
23 | 25 |
|
24 | | -import org.apache.spark.SparkConf |
| 26 | +import org.apache.spark.{SparkConf, SparkContext} |
25 | 27 | import org.apache.spark.executor.ShuffleReadMetrics |
26 | 28 | import org.apache.spark.executor.ShuffleWriteMetrics |
27 | 29 | import org.apache.spark.scheduler.SparkListener |
| 30 | +import org.apache.spark.scheduler.SparkListenerJobStart |
28 | 31 | import org.apache.spark.scheduler.SparkListenerTaskEnd |
29 | 32 | import org.apache.spark.sql.CometTestBase |
30 | 33 | import org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffle |
31 | 34 | import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec |
32 | 35 | import org.apache.spark.sql.execution.SparkPlan |
33 | 36 | import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper |
| 37 | +import org.apache.spark.sql.execution.command.DataWritingCommandExec |
| 38 | +import org.apache.spark.sql.internal.SQLConf |
34 | 39 |
|
35 | 40 | import org.apache.comet.CometConf |
36 | 41 |
|
@@ -100,6 +105,91 @@ class CometTaskMetricsSuite extends CometTestBase with AdaptiveSparkPlanHelper { |
100 | 105 | } |
101 | 106 | } |
102 | 107 |
|
| 108 | + test("native parquet write reports task-level output metrics") { |
| 109 | + withParquetTable((0 until 5000).map(i => (i, (i + 1).toLong)), "tbl") { |
| 110 | + withTempPath { dir => |
| 111 | + val outPath = new File(dir, "written").getAbsolutePath |
| 112 | + val expectedRows = 5000L |
| 113 | + val outputBytes = mutable.ArrayBuffer.empty[Long] |
| 114 | + val outputRecords = mutable.ArrayBuffer.empty[Long] |
| 115 | + val targetStageIds = mutable.HashSet.empty[Int] |
| 116 | + val jobGroupId = s"native-write-metrics-${java.util.UUID.randomUUID().toString}" |
| 117 | + |
| 118 | + val listener = new SparkListener { |
| 119 | + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { |
| 120 | + val isTargetJob = Option(jobStart.properties) |
| 121 | + .flatMap(props => Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID))) |
| 122 | + .contains(jobGroupId) |
| 123 | + if (isTargetJob) { |
| 124 | + targetStageIds.synchronized { |
| 125 | + targetStageIds ++= jobStart.stageInfos.map(_.stageId) |
| 126 | + } |
| 127 | + } |
| 128 | + } |
| 129 | + |
| 130 | + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { |
| 131 | + val isTargetStage = targetStageIds.synchronized { |
| 132 | + targetStageIds.contains(taskEnd.stageId) |
| 133 | + } |
| 134 | + if (isTargetStage) { |
| 135 | + val om = taskEnd.taskMetrics.outputMetrics |
| 136 | + if (om.bytesWritten > 0) { |
| 137 | + outputBytes.synchronized { |
| 138 | + outputBytes += om.bytesWritten |
| 139 | + outputRecords += om.recordsWritten |
| 140 | + } |
| 141 | + } |
| 142 | + } |
| 143 | + } |
| 144 | + } |
| 145 | + spark.sparkContext.addSparkListener(listener) |
| 146 | + |
| 147 | + try { |
| 148 | + spark.sparkContext.listenerBus.waitUntilEmpty() |
| 149 | + |
| 150 | + withSQLConf( |
| 151 | + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", |
| 152 | + CometConf.COMET_EXEC_ENABLED.key -> "true", |
| 153 | + CometConf.getOperatorAllowIncompatConfigKey( |
| 154 | + classOf[DataWritingCommandExec]) -> "true", |
| 155 | + SQLConf.SESSION_LOCAL_TIMEZONE.key -> "America/Halifax") { |
| 156 | + spark.sparkContext.setJobGroup(jobGroupId, "native parquet write output metrics") |
| 157 | + try { |
| 158 | + sql("SELECT * FROM tbl").write.parquet(outPath) |
| 159 | + } finally { |
| 160 | + spark.sparkContext.clearJobGroup() |
| 161 | + } |
| 162 | + } |
| 163 | + |
| 164 | + spark.sparkContext.listenerBus.waitUntilEmpty() |
| 165 | + |
| 166 | + assert(outputBytes.nonEmpty, "No task reported outputMetrics.bytesWritten") |
| 167 | + val totalOutputBytes = outputBytes.sum |
| 168 | + val totalOutputRecords = outputRecords.sum |
| 169 | + |
| 170 | + assert( |
| 171 | + totalOutputRecords == expectedRows, |
| 172 | + s"recordsWritten mismatch: metrics=$totalOutputRecords, expected=$expectedRows") |
| 173 | + |
| 174 | + val outputDir = new File(outPath) |
| 175 | + val fileBytes = Option(outputDir.listFiles()) |
| 176 | + .getOrElse(Array.empty) |
| 177 | + .filter(f => f.isFile && f.getName.startsWith("part-")) |
| 178 | + .map(_.length()) |
| 179 | + .sum |
| 180 | + |
| 181 | + assert(fileBytes > 0L, s"Expected written parquet bytes should be > 0, got $fileBytes") |
| 182 | + val ratio = totalOutputBytes.toDouble / fileBytes.toDouble |
| 183 | + assert( |
| 184 | + ratio >= 0.7 && ratio <= 1.3, |
| 185 | + s"bytesWritten ratio out of range: metrics=$totalOutputBytes, files=$fileBytes, ratio=$ratio") |
| 186 | + } finally { |
| 187 | + spark.sparkContext.removeSparkListener(listener) |
| 188 | + } |
| 189 | + } |
| 190 | + } |
| 191 | + } |
| 192 | + |
103 | 193 | test("native_datafusion scan reports task-level input metrics matching Spark") { |
104 | 194 | val totalRows = 10000 |
105 | 195 | withTempPath { dir => |
|
0 commit comments