From 5b0cb1b52fb117fac9c38d28ac4c828ddb01fbc8 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Wed, 11 Mar 2026 16:15:36 +0200 Subject: [PATCH 01/15] updated testing --- .../dataflint/DataFlintWindowExecSpec.scala | 114 +++++--- .../sql/execution/ExplicitRepartition.scala | 245 ++++++++++++++++++ 2 files changed, 330 insertions(+), 29 deletions(-) create mode 100644 spark-plugin/pluginspark3/src/test/scala/org/apache/spark/sql/execution/ExplicitRepartition.scala diff --git a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala index d452e37..e1ec978 100644 --- a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala +++ b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala @@ -1,22 +1,29 @@ package org.apache.spark.dataflint -import org.apache.spark.sql.{Encoder, Encoders, SparkSession} +import org.apache.spark.sql.execution.{ExplicitRepartitionExtension, ExplicitRepartitionOps} +import org.apache.spark.sql.{DataFrame, Encoder, Encoders, SparkSession} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.window.DataFlintWindowExec -import org.apache.spark.sql.expressions.Aggregator -import org.apache.spark.sql.functions.udaf +import org.apache.spark.sql.expressions.{Aggregator, Window} +import org.apache.spark.sql.functions.{col, rank, udaf} import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers import java.util.concurrent.TimeUnit.NANOSECONDS -private class SlowSumAggregator(sleep: Long) extends Aggregator[Long, Long, Long] { +private class SlowSumAggregator(fromSleep: Long, toSleep: Long) extends Aggregator[Long, Long, Long] { def zero: Long = 0L def reduce(b: Long, a: Long): Long = { - Thread.sleep(sleep); b + a + b + a } def merge(b1: Long, b2: Long): Long = b1 + b2 - def finish(r: Long): Long = r + def finish(r: Long): Long = { + val sleep = fromSleep + (math.random() * (toSleep - fromSleep)).toLong + Thread.sleep(sleep) + println(s"SlowSumAggregator finished with sleep of $sleep") + r + } def bufferEncoder: Encoder[Long] = Encoders.scalaLong def outputEncoder: Encoder[Long] = Encoders.scalaLong } @@ -29,11 +36,15 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf spark = SparkSession.builder() .master("local[1]") .appName("DataFlintWindowExecSpec") - .config("spark.sql.extensions", "org.apache.spark.dataflint.DataFlintInstrumentationExtension") +// .config("spark.sql.extensions", "org.apache.spark.dataflint.DataFlintInstrumentationExtension") .config(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, "true") .config(DataflintSparkUICommonLoader.INSTRUMENT_WINDOW_ENABLED, "true") .config("spark.ui.enabled", "true") - .config("spark.sql.adaptive.enabled", "false") + .config("spark.sql.adaptive.enabled", "true") + .config("spark.sql.adaptive.coalescePartitions.enabled", "true") +// .config("spark.sql.adaptive.advisoryPartitionSizeInBytes", "64MB") + .withExtensions(new ExplicitRepartitionExtension) + .withExtensions(new DataFlintInstrumentationExtension) .getOrCreate() } @@ -41,6 +52,14 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf if (spark != null) spark.stop() } + // With AQE, executedPlan is AdaptiveSparkPlanExec. After collect(), finalPhysicalPlan holds + // the fully optimised plan. For plan-structure tests that don't execute the query, use + // queryExecution.sparkPlan instead (our strategy runs before AQE wraps the plan). + private def finalPlan(df: DataFrame) = df.queryExecution.executedPlan match { + case aqe: AdaptiveSparkPlanExec => aqe.finalPhysicalPlan + case p => p + } + test("DataFlintWindowPlannerStrategy replaces WindowExec with DataFlintWindowExec for SQL window") { val session = spark import session.implicits._ @@ -50,12 +69,13 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf val result = spark.sql( "SELECT id, cat, rank() OVER (PARTITION BY cat ORDER BY id) AS r FROM test_window_plan" ) - val windowNodes = result.queryExecution.executedPlan.collect { + // Use sparkPlan (pre-AQE) — our strategy injects DataFlintWindowExec at planning time + val windowNodes = finalPlan(result).collect { case w: DataFlintWindowExec => w } withClue("Expected DataFlintWindowExec in physical plan but found: " + - result.queryExecution.executedPlan.treeString) { + finalPlan(result).treeString) { windowNodes should not be empty } } @@ -72,7 +92,7 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf ) result.collect() - val windowNode = result.queryExecution.executedPlan.collect { + val windowNode = finalPlan(result).collect { case w: DataFlintWindowExec => w }.head val duration = windowNode.metrics("duration").value @@ -93,7 +113,7 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf result.collect() val wallMs = NANOSECONDS.toMillis(System.nanoTime() - wallStart) - val windowNode = result.queryExecution.executedPlan.collect { + val windowNode = finalPlan(result).collect { case w: DataFlintWindowExec => w }.head val duration = windowNode.metrics("duration").value @@ -112,7 +132,7 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf "SELECT id, cat, rank() OVER (PARTITION BY cat ORDER BY id) AS r FROM test_window_small" ) resultSmall.collect() - val durationSmall = resultSmall.queryExecution.executedPlan.collect { + val durationSmall = finalPlan(resultSmall).collect { case w: DataFlintWindowExec => w }.head.metrics("duration").value @@ -122,7 +142,7 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf "SELECT id, cat, rank() OVER (PARTITION BY cat ORDER BY id) AS r FROM test_window_large" ) resultLarge.collect() - val durationLarge = resultLarge.queryExecution.executedPlan.collect { + val durationLarge = finalPlan(resultLarge).collect { case w: DataFlintWindowExec => w }.head.metrics("duration").value @@ -133,39 +153,52 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf val session = spark import session.implicits._ - // A UDAF whose reduce() sleeps 1ms per row — all work happens inside the window operator. - // With 200 rows / 5 categories = 40 rows per partition, reduce() is called 200 times total - // → at least 200ms of window-internal computation that the metric must capture. + // A UDAF whose finish() sleeps 1ms per partition — all work happens inside the window operator. + // With 200 rows / 5 categories = 40 rows per partition, finish() is called 5 times total + // → at least 20ms of window-internal computation that the metric must capture. val sleepTime=4 - spark.udf.register("slow_sum", udaf(new SlowSumAggregator(sleepTime))) + spark.udf.register("slow_sum", udaf(new SlowSumAggregator(sleepTime, 10))) val rows = 200 val partitions = 5 - val df = (1 to rows).map(i => (i, i % partitions)).toDF("id", "cat") + // Pre-repartition by cat so Spark reuses the existing HashPartitioning(cat, 5) for the + // window exchange (skips the shuffle), guaranteeing exactly `partitions` tasks. + val dforg = (1 to rows).map(i => (i, i % partitions)).toDF("id", "cat") + + //repartition by exact number of partitions (require adaptive and ExplicitRepartitionExtension) + import ExplicitRepartitionOps._ + val df = dforg.adaptiveRepartition(col("cat")) + df.createOrReplaceTempView("test_window_udaf") // Fast baseline: rank() has negligible per-row computation - val resultFast = spark.sql( - "SELECT id, cat, rank() OVER (PARTITION BY cat ORDER BY id) AS r FROM test_window_udaf" - ) + val resultFast = df.withColumn("r", rank().over(Window.partitionBy("cat").orderBy("id"))) +// val resultFast = spark.sql( +// "SELECT id, cat, rank() OVER (PARTITION BY cat ORDER BY id) AS r FROM test_window_udaf" +// ) resultFast.collect() - val durationFast = resultFast.queryExecution.executedPlan.collect { + val durationFast = finalPlan(resultFast).collect { case w: DataFlintWindowExec => w }.head.metrics("duration").value - // Slow: UDAF sleeps 1ms per row inside the window operator — 200ms+ of window-internal work + // Set shuffle partitions = partitions so each cat group runs as its own Spark task. + // Without this the default 200 shuffle partitions leave 195 empty tasks with ~0ms duration, + // making min trivially 0 and the min/max variation check meaningless. + + // Slow: UDAF finish() sleeps a random time per partition inside the window operator val resultSlow = spark.sql( "SELECT id, cat, slow_sum(CAST(id AS BIGINT)) OVER (PARTITION BY cat) AS r FROM test_window_udaf" ) val wallSlowStart = System.nanoTime() resultSlow.collect() val wallSlowMs = NANOSECONDS.toMillis(System.nanoTime() - wallSlowStart) - val durationSlow = resultSlow.queryExecution.executedPlan.collect { + val windowNode = finalPlan(resultSlow).collect { case w: DataFlintWindowExec => w - }.head.metrics("duration").value - // The metric must capture the UDAF's sleep time (at least rows*sleepTime ms = 1ms × 200 rows) - withClue(s"durationSlow=$durationSlow ms should be >= ${rows*sleepTime}ms (UDAF sleep captured in metric)") { - durationSlow should be >= rows.toLong + }.head + val durationSlow = windowNode.metrics("duration").value + // The metric must capture the UDAF's sleep time (at least partitions * fromSleep ms) + withClue(s"durationSlow=$durationSlow ms should be >= ${partitions*sleepTime}ms (UDAF sleep captured in metric)") { + durationSlow should be >= partitions.toLong * sleepTime } // The metric must reflect actual window computation — slow >> fast @@ -177,6 +210,29 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf withClue(s"durationSlow=$durationSlow ms should be <= wallSlowMs=$wallSlowMs ms") { durationSlow should be <= wallSlowMs } + + // Per-task breakdown: createTimingMetric records each task's value individually. + // spark.sharedState.statusStore exposes the formatted string "X ms (N tasks: min Y ms, med Z ms, max W ms)". + // Randomized sleep must produce variation — min and max across partitions must differ. + val sqlStore = spark.sharedState.statusStore + val execData = sqlStore.executionsList().maxBy(_.executionId) + val metricStr = sqlStore.executionMetrics(execData.executionId) + .getOrElse(windowNode.metrics("duration").id, "") + // Format: "total (min, med, max (stageId: taskId))\nX ms (min ms, med ms, max ms (stage A.B: task C))" + val minMaxPattern = """\((\d+) ms, \d+ ms, (\d+) ms""".r + val (minDuration, maxDuration) = minMaxPattern.findFirstMatchIn(metricStr) match { + case Some(m) => (m.group(1).toLong, m.group(2).toLong) + case None => fail(s"Expected per-task timing breakdown but got: '$metricStr'") + } + withClue(s"minDuration=$minDuration ms should differ from maxDuration=$maxDuration ms (randomized sleep)") { + minDuration should be < maxDuration + } + withClue(s"maxDuration=$maxDuration ms should be >= sleepTime=$sleepTime ms (sleep was captured in metric)") { + maxDuration should be >= sleepTime.toLong + } + withClue(s"minDuration=$minDuration ms should be > sleepTime=$sleepTime ms (sleep was captured in metric)") { + minDuration should be >= sleepTime.toLong + } } } \ No newline at end of file diff --git a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/sql/execution/ExplicitRepartition.scala b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/sql/execution/ExplicitRepartition.scala new file mode 100644 index 0000000..c62bea8 --- /dev/null +++ b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/sql/execution/ExplicitRepartition.scala @@ -0,0 +1,245 @@ +package org.apache.spark.sql.execution + +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.exchange._ +import org.apache.spark.sql.execution.metric.{SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.{MapOutputStatistics, ShuffleDependency} + +import scala.concurrent.Future + +// 1. Logical Plan Node for Adaptive Repartition +case class ExplicitRepartition( + partitionExprs: Seq[Expression], + child: LogicalPlan + ) extends UnaryNode { + override def output: Seq[Attribute] = child.output + override protected def withNewChildInternal(newChild: LogicalPlan): ExplicitRepartition = + copy(child = newChild) +} + +// 2. DataFrame API Extension +object ExplicitRepartitionOps { + implicit class AdaptiveRepartitionDataFrame(df: DataFrame) { + @scala.annotation.varargs + def adaptiveRepartition(cols: Column*): DataFrame = { + val exprs = cols.map(_.expr) + Dataset[Row](df.sparkSession, ExplicitRepartition(exprs, df.logicalPlan))(ExpressionEncoder.apply(df.schema)) + } + } +} + +// 3. Custom Shuffle Exchange that marks itself for adaptive coalescing +case class ExplicitShuffleExchangeExec( + override val outputPartitioning: Partitioning, + child: SparkPlan + ) extends ShuffleExchangeLike { + + override def nodeName: String = "AdaptiveExchange" + + private lazy val writeMetrics = + SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) + private[sql] lazy val readMetrics = + SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) + override lazy val metrics = Map( + "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), + "numPartitions" -> SQLMetrics.createMetric(sparkContext, "number of partitions") + ) ++ readMetrics ++ writeMetrics + + private lazy val serializer: Serializer = + new UnsafeRowSerializer(child.output.size, longMetric("dataSize")) + + @transient lazy val inputRDD: RDD[InternalRow] = child.execute() + + @transient + override lazy val mapOutputStatisticsFuture: Future[MapOutputStatistics] = { + if (inputRDD.getNumPartitions == 0) { + Future.successful(null) + } else { + sparkContext.submitMapStage(shuffleDependency) + } + } + + override def numMappers: Int = shuffleDependency.rdd.getNumPartitions + override def numPartitions: Int = shuffleDependency.partitioner.numPartitions + override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[InternalRow] = { + new ShuffledRowRDD(shuffleDependency, readMetrics, partitionSpecs) + } + override def runtimeStatistics: Statistics = { + val dataSize = metrics("dataSize").value + val rowCount = metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_RECORDS_WRITTEN).value + Statistics(dataSize, Some(rowCount)) + } + + @transient + lazy val shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow] = { + val dep = ShuffleExchangeExec.prepareShuffleDependency( + inputRDD, + child.output, + outputPartitioning, + serializer, + writeMetrics) + metrics("numPartitions").set(dep.partitioner.numPartitions) + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates( + sparkContext, executionId, metrics("numPartitions") :: Nil) + dep + } + + private var cachedShuffleRDD: ShuffledRowRDD = null + override protected def doExecute(): RDD[InternalRow] = { + if (cachedShuffleRDD == null) { + cachedShuffleRDD = new ShuffledRowRDD(shuffleDependency, readMetrics) + } + cachedShuffleRDD + } + + override protected def withNewChildInternal(newChild: SparkPlan): ExplicitShuffleExchangeExec = + copy(child = newChild) + + // Override to indicate this exchange should be adaptively coalesced + override def advisoryPartitionSize: Option[Long] = Some(1L) // Force minimal partition size for coalescing + override def shuffleOrigin: ShuffleOrigin = REPARTITION_BY_COL +} + +// 4. Custom AQE Rule to coalesce only ExplicitShuffleExchangeExec +object CoalesceExplicitShufflePartitions extends Rule[SparkPlan] { + override val ruleName = "CoalesceAdaptiveShufflePartitions" + + def apply(plan: SparkPlan): SparkPlan = { + if (!SQLConf.get.adaptiveExecutionEnabled) { + return plan + } + + plan transformUp { + case aqeRead: AQEShuffleReadExec if aqeRead.child.isInstanceOf[ShuffleQueryStageExec] => + val stage = aqeRead.child.asInstanceOf[ShuffleQueryStageExec] + if (stage.shuffle.isInstanceOf[ExplicitShuffleExchangeExec]) { + coalesceAQEShuffleRead(aqeRead, stage) + } else { + aqeRead + } + case other => other + } + } + + private def coalesceAQEShuffleRead(aqeRead: AQEShuffleReadExec, stage: ShuffleQueryStageExec): SparkPlan = { + stage.mapStats match { + case Some(mapStats) => + val nonEmptyPartitions = mapStats.bytesByPartitionId.zipWithIndex + .filter(_._1 > 0) + .map(_._2) + + if (nonEmptyPartitions.length < mapStats.bytesByPartitionId.length) { + // Create coalesced specs that only include non-empty partitions + val coalescedSpecs = nonEmptyPartitions.map { partitionId => + CoalescedPartitionSpec( + startReducerIndex = partitionId, + endReducerIndex = partitionId + 1, + dataSize = mapStats.bytesByPartitionId(partitionId) + ) + }.toArray + + // Replace the existing AQEShuffleReadExec with one using coalesced specs + AQEShuffleReadExec(stage, coalescedSpecs.map(_.asInstanceOf[ShufflePartitionSpec])) + } else { + aqeRead + } + case None => aqeRead + } + } +} + +// Custom partitioning that uses raw hash values as partition IDs +class ExplicitHashPartitioning( + expressions: Seq[Expression], + ) extends HashPartitioning(expressions, 100000) { + + override def partitionIdExpression: Expression = { + // Use absolute value of hash to ensure positive partition IDs + // Each distinct value gets its own partition based on hash + Pmod(new Murmur3Hash(expressions), Literal(100000)) + } + + override def toString: String = s"DynamicDistinctPartitioning(${expressions.mkString(", ")})" +} + + +// 5. Planning Strategy +object ExplicitRepartitionStrategy extends SparkStrategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case ExplicitRepartition(partitionExprs, child) if partitionExprs.forall(_.resolved) => + val physicalChild = planLater(child) + val partitioning = new ExplicitHashPartitioning(partitionExprs) + // Create adaptive shuffle exchange + val adaptiveExchange = ExplicitShuffleExchangeExec(partitioning, physicalChild) + + adaptiveExchange :: Nil + case _ => Nil + } +} + +// 7. Extension to register all components +class ExplicitRepartitionExtension extends (SparkSessionExtensions => Unit) { + override def apply(extensions: SparkSessionExtensions): Unit = { + // Add physical planning strategy + extensions.injectPlannerStrategy(_ => ExplicitRepartitionStrategy) + + // Add AQE rule for coalescing + extensions.injectQueryStageOptimizerRule(_ => CoalesceExplicitShufflePartitions) + } +} + +// 8. Usage Example +object ExplicitRepartitionExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession.builder() + .appName("ExplicitRepartitionTest") + .master("local[*]") + .config("spark.sql.adaptive.enabled", "true") + .config("spark.sql.adaptive.coalescePartitions.enabled", "true") + .config("spark.sql.adaptive.advisoryPartitionSizeInBytes", "64MB") + .getOrCreate() + + // Register the extension + new ExplicitRepartitionExtension().apply(spark.extensions) + + import ExplicitRepartitionOps._ + import org.apache.spark.sql.functions._ + import spark.implicits._ + + val df = spark.range(10000).toDF("id") + .withColumn("b", $"id" % 250) + + // Usage: Adaptive repartition that coalesces to unique values + val result = df + .repartition($"b") // Will repartition by 'b' and coalesce to 5 partitions (unique values of b) + .withColumn("partition_ida", spark_partition_id()) + .adaptiveRepartition($"b") // Will repartition by 'b' and coalesce to 5 partitions (unique values of b) + .withColumn("partition_idb", spark_partition_id()) + .groupBy("b") + .agg(count("*").as("count"), min("partition_ida"), max("partition_ida"), min("partition_idb"), max("partition_idb")) + + result.explain(true) + try { + result.show() + } catch { + case e: Throwable => + e.printStackTrace() +// Thread.sleep(300000) + }finally { + Thread.sleep(300000) + spark.stop() + } + + } +} From 8381a9dcb5874cfd1a31be37bb2d9f418730cd22 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Wed, 11 Mar 2026 16:28:42 +0200 Subject: [PATCH 02/15] added test helper for metric stat extraction --- .../dataflint/DataFlintWindowExecSpec.scala | 32 +++++++------------ .../spark/dataflint/SqlMetricTestHelper.scala | 24 ++++++++++++++ 2 files changed, 35 insertions(+), 21 deletions(-) create mode 100644 spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/SqlMetricTestHelper.scala diff --git a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala index e1ec978..d865df9 100644 --- a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala +++ b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala @@ -28,7 +28,7 @@ private class SlowSumAggregator(fromSleep: Long, toSleep: Long) extends Aggregat def outputEncoder: Encoder[Long] = Encoders.scalaLong } -class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAfterAll { +class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAfterAll with SqlMetricTestHelper { private var spark: SparkSession = _ @@ -70,12 +70,12 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf "SELECT id, cat, rank() OVER (PARTITION BY cat ORDER BY id) AS r FROM test_window_plan" ) // Use sparkPlan (pre-AQE) — our strategy injects DataFlintWindowExec at planning time - val windowNodes = finalPlan(result).collect { + val windowNodes = result.queryExecution.sparkPlan.collect { case w: DataFlintWindowExec => w } withClue("Expected DataFlintWindowExec in physical plan but found: " + - finalPlan(result).treeString) { + result.queryExecution.sparkPlan.treeString) { windowNodes should not be empty } } @@ -212,27 +212,17 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf } // Per-task breakdown: createTimingMetric records each task's value individually. - // spark.sharedState.statusStore exposes the formatted string "X ms (N tasks: min Y ms, med Z ms, max W ms)". // Randomized sleep must produce variation — min and max across partitions must differ. - val sqlStore = spark.sharedState.statusStore - val execData = sqlStore.executionsList().maxBy(_.executionId) - val metricStr = sqlStore.executionMetrics(execData.executionId) - .getOrElse(windowNode.metrics("duration").id, "") - // Format: "total (min, med, max (stageId: taskId))\nX ms (min ms, med ms, max ms (stage A.B: task C))" - val minMaxPattern = """\((\d+) ms, \d+ ms, (\d+) ms""".r - val (minDuration, maxDuration) = minMaxPattern.findFirstMatchIn(metricStr) match { - case Some(m) => (m.group(1).toLong, m.group(2).toLong) - case None => fail(s"Expected per-task timing breakdown but got: '$metricStr'") + implicit val sparkImplicit: SparkSession = spark + val stats = metricMinMax(windowNode.metrics("duration")) + withClue(s"min=${stats.min} ms should differ from max=${stats.max} ms (randomized sleep)") { + stats.min should be < stats.max } - withClue(s"minDuration=$minDuration ms should differ from maxDuration=$maxDuration ms (randomized sleep)") { - minDuration should be < maxDuration + withClue(s"max=${stats.max} ms should be >= sleepTime=$sleepTime ms (sleep was captured in metric)") { + stats.max should be >= sleepTime.toLong } - withClue(s"maxDuration=$maxDuration ms should be >= sleepTime=$sleepTime ms (sleep was captured in metric)") { - maxDuration should be >= sleepTime.toLong - } - withClue(s"minDuration=$minDuration ms should be > sleepTime=$sleepTime ms (sleep was captured in metric)") { - minDuration should be >= sleepTime.toLong + withClue(s"min=${stats.min} ms should be >= sleepTime=$sleepTime ms (sleep was captured in metric)") { + stats.min should be >= sleepTime.toLong } } - } \ No newline at end of file diff --git a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/SqlMetricTestHelper.scala b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/SqlMetricTestHelper.scala new file mode 100644 index 0000000..88e6578 --- /dev/null +++ b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/SqlMetricTestHelper.scala @@ -0,0 +1,24 @@ +package org.apache.spark.dataflint + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.metric.SQLMetric +import org.scalatest.Assertions + +case class MetricStats(total: Long, min: Long, med: Long, max: Long) + +trait SqlMetricTestHelper extends Assertions { + + // Reads per-task total/min/med/max from the SQL status store for the given SQLMetric. + // createTimingMetric records each task's elapsed time individually; the store formats them as: + // "total (min, med, max (stageId: taskId))\nX ms (Y ms, Z ms, W ms (stage A.B: task C))" + def metricMinMax(metric: SQLMetric)(implicit spark: SparkSession): MetricStats = { + val sqlStore = spark.sharedState.statusStore + val execData = sqlStore.executionsList().maxBy(_.executionId) + val metricStr = sqlStore.executionMetrics(execData.executionId).getOrElse(metric.id, "") + val pattern = """(\d+) ms \((\d+) ms, (\d+) ms, (\d+) ms""".r + pattern.findFirstMatchIn(metricStr) match { + case Some(m) => MetricStats(m.group(1).toLong, m.group(2).toLong, m.group(3).toLong, m.group(4).toLong) + case None => fail(s"Expected per-task timing breakdown but got: '$metricStr'") + } + } +} \ No newline at end of file From 222c942b849dc33ba6ddf8280e52f925f52f3217 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Thu, 12 Mar 2026 12:28:21 +0200 Subject: [PATCH 03/15] add instrumentation for ArrowEvalPython, FlatMapCoGroupsInPandas, and FlatMapGroupsInPandas exec nodes --- .../DataflintSparkUICommonLoader.scala | 9 +- .../DataFlintInstrumentationExtension.scala | 97 +++++++++++- .../python/DataFlintArrowEvalPythonExec.scala | 62 ++++++++ ...DataFlintFlatMapCoGroupsInPandasExec.scala | 66 ++++++++ .../DataFlintFlatMapGroupsInPandasExec.scala | 61 ++++++++ .../dataflint/DataFlintPythonExecSpec.scala | 144 +++++++++++++++++ .../DataFlintInstrumentationExtension.scala | 147 +++++++++++++----- .../python/DataFlintArrowEvalPythonExec.scala | 60 +++++++ ...DataFlintFlatMapCoGroupsInPandasExec.scala | 65 ++++++++ .../DataFlintFlatMapGroupsInPandasExec.scala | 60 +++++++ 10 files changed, 728 insertions(+), 43 deletions(-) create mode 100644 spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala create mode 100644 spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala create mode 100644 spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala create mode 100644 spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala create mode 100644 spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala create mode 100644 spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala create mode 100644 spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala diff --git a/spark-plugin/plugin/src/main/scala/org/apache/spark/dataflint/DataflintSparkUICommonLoader.scala b/spark-plugin/plugin/src/main/scala/org/apache/spark/dataflint/DataflintSparkUICommonLoader.scala index 5b2bf06..c4a7c5e 100644 --- a/spark-plugin/plugin/src/main/scala/org/apache/spark/dataflint/DataflintSparkUICommonLoader.scala +++ b/spark-plugin/plugin/src/main/scala/org/apache/spark/dataflint/DataflintSparkUICommonLoader.scala @@ -135,6 +135,9 @@ object DataflintSparkUICommonLoader extends Logging { val INSTRUMENT_MAP_IN_PANDAS_ENABLED = "spark.dataflint.instrument.spark.mapInPandas.enabled" val INSTRUMENT_MAP_IN_ARROW_ENABLED = "spark.dataflint.instrument.spark.mapInArrow.enabled" val INSTRUMENT_WINDOW_ENABLED = "spark.dataflint.instrument.spark.window.enabled" + val INSTRUMENT_ARROW_EVAL_PYTHON_ENABLED = "spark.dataflint.instrument.spark.arrowEvalPython.enabled" + val INSTRUMENT_FLAT_MAP_GROUPS_PANDAS_ENABLED = "spark.dataflint.instrument.spark.flatMapGroupsInPandas.enabled" + val INSTRUMENT_FLAT_MAP_COGROUPS_PANDAS_ENABLED = "spark.dataflint.instrument.spark.flatMapCoGroupsInPandas.enabled" def install(context: SparkContext, pageFactory: DataflintPageFactory): String = { new DataflintSparkUICommonInstaller().install(context, pageFactory) @@ -166,7 +169,11 @@ object DataflintSparkUICommonLoader extends Logging { val mapInPandasEnabled = sc.conf.getBoolean(INSTRUMENT_MAP_IN_PANDAS_ENABLED, defaultValue = false) val mapInArrowEnabled = sc.conf.getBoolean(INSTRUMENT_MAP_IN_ARROW_ENABLED, defaultValue = false) val windowEnabled = sc.conf.getBoolean(INSTRUMENT_WINDOW_ENABLED, defaultValue = false) - val anyInstrumentationEnabled = instrumentEnabled || mapInPandasEnabled || mapInArrowEnabled || windowEnabled + val arrowEvalPythonEnabled = sc.conf.getBoolean(INSTRUMENT_ARROW_EVAL_PYTHON_ENABLED, defaultValue = false) + val flatMapGroupsPandasEnabled = sc.conf.getBoolean(INSTRUMENT_FLAT_MAP_GROUPS_PANDAS_ENABLED, defaultValue = false) + val flatMapCogroupsPandasEnabled = sc.conf.getBoolean(INSTRUMENT_FLAT_MAP_COGROUPS_PANDAS_ENABLED, defaultValue = false) + val anyInstrumentationEnabled = instrumentEnabled || mapInPandasEnabled || mapInArrowEnabled || + windowEnabled || arrowEvalPythonEnabled || flatMapGroupsPandasEnabled || flatMapCogroupsPandasEnabled if (!anyInstrumentationEnabled) { logInfo("DataFlint instrumentation extension is disabled (no instrumentation flags enabled)") return diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala index 59e9fc7..7c18147 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala @@ -8,7 +8,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalWindow import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Window => LogicalWindow} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} -import org.apache.spark.sql.execution.python.{DataFlintMapInPandasExec_3_0, DataFlintMapInPandasExec_3_1, DataFlintMapInPandasExec_3_3, DataFlintMapInPandasExec_3_4, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_3, DataFlintPythonMapInArrowExec_3_4, DataFlintPythonMapInArrowExec_3_5, DataFlintWindowInPandasExec, MapInPandasExec, WindowInPandasExec} +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, DataFlintArrowEvalPythonExec_3_2, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_0, DataFlintMapInPandasExec_3_1, DataFlintMapInPandasExec_3_3, DataFlintMapInPandasExec_3_4, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_3, DataFlintPythonMapInArrowExec_3_4, DataFlintPythonMapInArrowExec_3_5, DataFlintWindowInPandasExec, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, WindowInPandasExec} import org.apache.spark.sql.execution.window.DataFlintWindowExec /** @@ -83,8 +83,29 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C globalEnabled || specificEnabled } + private val arrowEvalPythonEnabled: Boolean = { + val conf = session.sparkContext.conf + val globalEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, defaultValue = false) + val specificEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_ARROW_EVAL_PYTHON_ENABLED, defaultValue = false) + globalEnabled || specificEnabled + } + + private val flatMapGroupsEnabled: Boolean = { + val conf = session.sparkContext.conf + val globalEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, defaultValue = false) + val specificEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_FLAT_MAP_GROUPS_PANDAS_ENABLED, defaultValue = false) + globalEnabled || specificEnabled + } + + private val flatMapCoGroupsEnabled: Boolean = { + val conf = session.sparkContext.conf + val globalEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, defaultValue = false) + val specificEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_FLAT_MAP_COGROUPS_PANDAS_ENABLED, defaultValue = false) + globalEnabled || specificEnabled + } + override def preColumnarTransitions: Rule[SparkPlan] = { plan => - if (!mapInPandasEnabled && !mapInArrowEnabled) plan + if (!mapInPandasEnabled && !mapInArrowEnabled && !arrowEvalPythonEnabled && !flatMapGroupsEnabled && !flatMapCoGroupsEnabled) plan else { var result = plan @@ -96,6 +117,18 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C result = replaceMapInArrow(result) } + if (arrowEvalPythonEnabled) { + result = replaceArrowEvalPython(result) + } + + if (flatMapGroupsEnabled) { + result = replaceFlatMapGroupsInPandas(result) + } + + if (flatMapCoGroupsEnabled) { + result = replaceFlatMapCoGroupsInPandas(result) + } + result } } @@ -141,6 +174,66 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C } } + /** + * Replaces ArrowEvalPythonExec nodes (pandas_udf SCALAR) with DataFlint instrumented versions. + * + * Only supported on Spark 3.2+ which uses the 4-param constructor (udfs, resultAttrs, child, + * evalType). Spark 3.0–3.1 used a 3-param constructor incompatible with our wrapper. + * + * Isolated in its own method with a try-catch so that any unexpected class-loading error + * on older Spark versions degrades gracefully. + */ + private def replaceArrowEvalPython(plan: SparkPlan): SparkPlan = { + if (sparkMinorVersion == "3.0" || sparkMinorVersion == "3.1") { + logWarning("ArrowEvalPython instrumentation requires Spark 3.2+ — skipping on Spark " + sparkMinorVersion) + return plan + } + try { + plan.transformUp { + case arrowEval: ArrowEvalPythonExec => + logInfo(s"Replacing ArrowEvalPythonExec with DataFlint version for Spark $sparkMinorVersion") + DataFlintArrowEvalPythonExec_3_2( + udfs = arrowEval.udfs, + resultAttrs = arrowEval.resultAttrs, + child = arrowEval.child, + evalType = arrowEval.evalType + ) + } + } catch { + case _: NoClassDefFoundError => + logWarning("ArrowEvalPythonExec not available in this Spark version — arrowEvalPython instrumentation disabled") + plan + } + } + + private def replaceFlatMapGroupsInPandas(plan: SparkPlan): SparkPlan = { + plan.transformUp { + case exec: FlatMapGroupsInPandasExec => + logInfo(s"Replacing FlatMapGroupsInPandasExec with DataFlint version for Spark $sparkMinorVersion") + DataFlintFlatMapGroupsInPandasExec( + groupingAttributes = exec.groupingAttributes, + func = exec.func, + output = exec.output, + child = exec.child + ) + } + } + + private def replaceFlatMapCoGroupsInPandas(plan: SparkPlan): SparkPlan = { + plan.transformUp { + case exec: FlatMapCoGroupsInPandasExec => + logInfo(s"Replacing FlatMapCoGroupsInPandasExec with DataFlint version for Spark $sparkMinorVersion") + DataFlintFlatMapCoGroupsInPandasExec( + leftGroup = exec.leftGroup, + rightGroup = exec.rightGroup, + func = exec.func, + output = exec.output, + left = exec.left, + right = exec.right + ) + } + } + /** * Replaces PythonMapInArrowExec nodes with DataFlint instrumented versions. * diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala new file mode 100644 index 0000000..e547f96 --- /dev/null +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * DataFlint instrumented version of ArrowEvalPythonExec for Spark 3.2–3.5. + * + * Wraps the parent's doExecute() RDD with a duration metric. + * Supports Spark 3.2+ which uses the 4-param constructor + * (udfs, resultAttrs, child, evalType). Spark 3.0–3.1 lack the evalType param + * and are not instrumented. + */ +class DataFlintArrowEvalPythonExec_3_2 private ( + udfs: Seq[PythonUDF], + resultAttrs: Seq[Attribute], + child: SparkPlan, + evalType: Int) + extends ArrowEvalPythonExec(udfs, resultAttrs, child, evalType) with Logging { + + override def nodeName: String = "DataFlintArrowEvalPython" + + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintArrowEvalPythonExec_3_2 = + DataFlintArrowEvalPythonExec_3_2(udfs, resultAttrs, newChild, evalType) +} + +object DataFlintArrowEvalPythonExec_3_2 { + def apply( + udfs: Seq[PythonUDF], + resultAttrs: Seq[Attribute], + child: SparkPlan, + evalType: Int): DataFlintArrowEvalPythonExec_3_2 = + new DataFlintArrowEvalPythonExec_3_2(udfs, resultAttrs, child, evalType) +} diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala new file mode 100644 index 0000000..8b90c01 --- /dev/null +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * DataFlint instrumented version of FlatMapCoGroupsInPandasExec for Spark 3.x. + * + * Instruments PandasCogroupedOps.applyInPandas() / df.cogroup(df2) operations + * with a duration metric by wrapping the parent's doExecute() RDD. + * The constructor (leftGroup, rightGroup, func, output, left, right) is stable across 3.0–3.5. + */ +class DataFlintFlatMapCoGroupsInPandasExec private ( + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + left: SparkPlan, + right: SparkPlan) + extends FlatMapCoGroupsInPandasExec(leftGroup, rightGroup, func, output, left, right) with Logging { + + override def nodeName: String = "DataFlintFlatMapCoGroupsInPandas" + + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildrenInternal( + newLeft: SparkPlan, newRight: SparkPlan): DataFlintFlatMapCoGroupsInPandasExec = + DataFlintFlatMapCoGroupsInPandasExec(leftGroup, rightGroup, func, output, newLeft, newRight) +} + +object DataFlintFlatMapCoGroupsInPandasExec { + def apply( + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + left: SparkPlan, + right: SparkPlan): DataFlintFlatMapCoGroupsInPandasExec = + new DataFlintFlatMapCoGroupsInPandasExec(leftGroup, rightGroup, func, output, left, right) +} diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala new file mode 100644 index 0000000..70947ee --- /dev/null +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * DataFlint instrumented version of FlatMapGroupsInPandasExec for Spark 3.x. + * + * Instruments GroupedData.applyInPandas() / pandas_udf(GROUPED_MAP) operations + * with a duration metric by wrapping the parent's doExecute() RDD. + * The constructor (groupingAttributes, func, output, child) is stable across 3.0–3.5. + */ +class DataFlintFlatMapGroupsInPandasExec private ( + groupingAttributes: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + child: SparkPlan) + extends FlatMapGroupsInPandasExec(groupingAttributes, func, output, child) with Logging { + + override def nodeName: String = "DataFlintFlatMapGroupsInPandas" + + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintFlatMapGroupsInPandasExec = + DataFlintFlatMapGroupsInPandasExec(groupingAttributes, func, output, newChild) +} + +object DataFlintFlatMapGroupsInPandasExec { + def apply( + groupingAttributes: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + child: SparkPlan): DataFlintFlatMapGroupsInPandasExec = + new DataFlintFlatMapGroupsInPandasExec(groupingAttributes, func, output, child) +} diff --git a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala new file mode 100644 index 0000000..fc159e1 --- /dev/null +++ b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala @@ -0,0 +1,144 @@ +package org.apache.spark.dataflint + +import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.PythonUDF +import org.apache.spark.sql.execution.LocalTableScanExec +import org.apache.spark.sql.execution.python._ +import org.apache.spark.sql.types.LongType +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.matchers.should.Matchers + +class DataFlintPythonExecSpec extends AnyFunSuite with Matchers with BeforeAndAfterAll { + + private var spark: SparkSession = _ + + override def beforeAll(): Unit = { + spark = SparkSession.builder() + .master("local[1]") + .appName("DataFlintPythonExecSpec") + .config(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, "true") + .withExtensions(new DataFlintInstrumentationExtension) + .getOrCreate() + } + + override def afterAll(): Unit = { + if (spark != null) spark.stop() + } + + private def emptyChild = LocalTableScanExec(output = Seq.empty, rows = Seq.empty) + + private def fakePythonUDF(evalType: Int): PythonUDF = { + val func = SimplePythonFunction( + command = Seq.empty[Byte], + envVars = new java.util.HashMap[String, String](), + pythonIncludes = new java.util.ArrayList[String](), + pythonExec = "python3", + pythonVer = "3.8", + broadcastVars = new java.util.ArrayList(), + accumulator = null) + PythonUDF( + name = "test_udf", + func = func, + dataType = LongType, + children = Seq.empty, + evalType = evalType, + udfDeterministic = true) + } + + // ---- MapInPandasExec (mapInPandas) ---- + + test("replaces MapInPandasExec in plan") { + val udf = fakePythonUDF(PythonEvalType.SQL_MAP_PANDAS_ITER_UDF) + val original = MapInPandasExec(udf, Seq.empty, emptyChild, false) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original) + result shouldBe a[DataFlintMapInPandasExec_3_5] + } + + test("MapInPandasExec nodeName and metrics") { + val udf = fakePythonUDF(PythonEvalType.SQL_MAP_PANDAS_ITER_UDF) + val original = MapInPandasExec(udf, Seq.empty, emptyChild, false) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original).asInstanceOf[DataFlintMapInPandasExec_3_5] + result.nodeName shouldBe "DataFlintMapInPandas" + result.metrics should contain key "duration" + } + + // ---- PythonMapInArrowExec (mapInArrow) ---- + + test("replaces PythonMapInArrowExec in plan") { + val udf = fakePythonUDF(PythonEvalType.SQL_MAP_ARROW_ITER_UDF) + val original = PythonMapInArrowExec(udf, Seq.empty, emptyChild, false) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original) + result shouldBe a[DataFlintPythonMapInArrowExec_3_5] + } + + test("PythonMapInArrowExec nodeName and metrics") { + val udf = fakePythonUDF(PythonEvalType.SQL_MAP_ARROW_ITER_UDF) + val original = PythonMapInArrowExec(udf, Seq.empty, emptyChild, false) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original).asInstanceOf[DataFlintPythonMapInArrowExec_3_5] + result.nodeName shouldBe "DataFlintMapInArrow" + result.metrics should contain key "duration" + } + + // ---- ArrowEvalPythonExec (pandas_udf SCALAR) ---- + + test("replaces ArrowEvalPythonExec in plan") { + val udf = fakePythonUDF(PythonEvalType.SQL_SCALAR_PANDAS_UDF) + val original = ArrowEvalPythonExec(Seq(udf), Seq.empty, emptyChild, PythonEvalType.SQL_SCALAR_PANDAS_UDF) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original) + result shouldBe a[DataFlintArrowEvalPythonExec_3_2] + } + + test("ArrowEvalPythonExec nodeName and metrics") { + val udf = fakePythonUDF(PythonEvalType.SQL_SCALAR_PANDAS_UDF) + val original = ArrowEvalPythonExec(Seq(udf), Seq.empty, emptyChild, PythonEvalType.SQL_SCALAR_PANDAS_UDF) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original).asInstanceOf[DataFlintArrowEvalPythonExec_3_2] + result.nodeName shouldBe "DataFlintArrowEvalPython" + result.metrics should contain key "duration" + } + + // ---- FlatMapGroupsInPandasExec (applyInPandas / GROUPED_MAP) ---- + + test("replaces FlatMapGroupsInPandasExec in plan") { + val udf = fakePythonUDF(PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF) + val original = FlatMapGroupsInPandasExec(Seq.empty, udf, Seq.empty, emptyChild) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original) + result shouldBe a[DataFlintFlatMapGroupsInPandasExec] + } + + test("FlatMapGroupsInPandasExec nodeName and metrics") { + val udf = fakePythonUDF(PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF) + val original = FlatMapGroupsInPandasExec(Seq.empty, udf, Seq.empty, emptyChild) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original).asInstanceOf[DataFlintFlatMapGroupsInPandasExec] + result.nodeName shouldBe "DataFlintFlatMapGroupsInPandas" + result.metrics should contain key "duration" + } + + // ---- FlatMapCoGroupsInPandasExec (cogroup / applyInPandas on two DataFrames) ---- + + test("replaces FlatMapCoGroupsInPandasExec in plan") { + val udf = fakePythonUDF(PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF) + val original = FlatMapCoGroupsInPandasExec(Seq.empty, Seq.empty, udf, Seq.empty, emptyChild, emptyChild) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original) + result shouldBe a[DataFlintFlatMapCoGroupsInPandasExec] + } + + test("FlatMapCoGroupsInPandasExec nodeName and metrics") { + val udf = fakePythonUDF(PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF) + val original = FlatMapCoGroupsInPandasExec(Seq.empty, Seq.empty, udf, Seq.empty, emptyChild, emptyChild) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original).asInstanceOf[DataFlintFlatMapCoGroupsInPandasExec] + result.nodeName shouldBe "DataFlintFlatMapCoGroupsInPandas" + result.metrics should contain key "duration" + } +} diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala index e035795..085aba2 100644 --- a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala @@ -10,7 +10,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Window => Logic import org.apache.spark.sql.execution.SparkStrategy import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} -import org.apache.spark.sql.execution.python.{DataFlintArrowWindowPythonExec_4_1, DataFlintMapInPandasExec_4_0, DataFlintMapInPandasExec_4_1, DataFlintPythonMapInArrowExec_4_0, DataFlintPythonMapInArrowExec_4_1, DataFlintWindowInPandasExec_4_0, MapInArrowExec, MapInPandasExec} +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintArrowWindowPythonExec_4_1, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_4_0, DataFlintMapInPandasExec_4_1, DataFlintPythonMapInArrowExec_4_0, DataFlintPythonMapInArrowExec_4_1, DataFlintWindowInPandasExec_4_0, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInArrowExec, MapInPandasExec} import org.apache.spark.sql.execution.window.DataFlintWindowExec /** @@ -74,52 +74,119 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C globalEnabled || specificEnabled } + private val arrowEvalPythonEnabled: Boolean = { + val conf = session.sparkContext.conf + val globalEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, defaultValue = false) + val specificEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_ARROW_EVAL_PYTHON_ENABLED, defaultValue = false) + globalEnabled || specificEnabled + } + + private val flatMapGroupsEnabled: Boolean = { + val conf = session.sparkContext.conf + val globalEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, defaultValue = false) + val specificEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_FLAT_MAP_GROUPS_PANDAS_ENABLED, defaultValue = false) + globalEnabled || specificEnabled + } + + private val flatMapCoGroupsEnabled: Boolean = { + val conf = session.sparkContext.conf + val globalEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, defaultValue = false) + val specificEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_FLAT_MAP_COGROUPS_PANDAS_ENABLED, defaultValue = false) + globalEnabled || specificEnabled + } override def preColumnarTransitions: Rule[SparkPlan] = { plan => - if (!mapInPandasEnabled && !mapInArrowEnabled) plan - else plan.transformUp { - case mapInPandas: MapInPandasExec if mapInPandasEnabled => - logInfo(s"Replacing MapInPandasExec with DataFlint version for Spark $sparkMinorVersion") - sparkMinorVersion match { - case "4.0" => - DataFlintMapInPandasExec_4_0( - func = mapInPandas.func, - output = mapInPandas.output, - child = mapInPandas.child, - isBarrier = mapInPandas.isBarrier, - profile = mapInPandas.profile - ) - case _ => - // Default to 4.1 implementation for 4.1.x and any future 4.x - DataFlintMapInPandasExec_4_1( - func = mapInPandas.func, - output = mapInPandas.output, - child = mapInPandas.child, - isBarrier = mapInPandas.isBarrier, - profile = mapInPandas.profile + if (!mapInPandasEnabled && !mapInArrowEnabled && !arrowEvalPythonEnabled && !flatMapGroupsEnabled && !flatMapCoGroupsEnabled) plan + else { + var result = plan + + if (mapInPandasEnabled || mapInArrowEnabled) { + result = result.transformUp { + case mapInPandas: MapInPandasExec if mapInPandasEnabled => + logInfo(s"Replacing MapInPandasExec with DataFlint version for Spark $sparkMinorVersion") + sparkMinorVersion match { + case "4.0" => + DataFlintMapInPandasExec_4_0( + func = mapInPandas.func, + output = mapInPandas.output, + child = mapInPandas.child, + isBarrier = mapInPandas.isBarrier, + profile = mapInPandas.profile + ) + case _ => + DataFlintMapInPandasExec_4_1( + func = mapInPandas.func, + output = mapInPandas.output, + child = mapInPandas.child, + isBarrier = mapInPandas.isBarrier, + profile = mapInPandas.profile + ) + } + case mapInArrow: MapInArrowExec if mapInArrowEnabled => + logInfo(s"Replacing MapInArrowExec with DataFlint version for Spark $sparkMinorVersion") + sparkMinorVersion match { + case "4.0" => + DataFlintPythonMapInArrowExec_4_0( + func = mapInArrow.func, + output = mapInArrow.output, + child = mapInArrow.child, + isBarrier = mapInArrow.isBarrier, + profile = mapInArrow.profile + ) + case _ => + DataFlintPythonMapInArrowExec_4_1( + func = mapInArrow.func, + output = mapInArrow.output, + child = mapInArrow.child, + isBarrier = mapInArrow.isBarrier, + profile = mapInArrow.profile + ) + } + } + } + + if (arrowEvalPythonEnabled) { + result = result.transformUp { + case arrowEval: ArrowEvalPythonExec => + logInfo(s"Replacing ArrowEvalPythonExec with DataFlint version for Spark $sparkMinorVersion") + DataFlintArrowEvalPythonExec( + udfs = arrowEval.udfs, + resultAttrs = arrowEval.resultAttrs, + child = arrowEval.child, + evalType = arrowEval.evalType ) } - case mapInArrow: MapInArrowExec if mapInArrowEnabled => - logInfo(s"Replacing MapInArrowExec with DataFlint version for Spark $sparkMinorVersion") - sparkMinorVersion match { - case "4.0" => - DataFlintPythonMapInArrowExec_4_0( - func = mapInArrow.func, - output = mapInArrow.output, - child = mapInArrow.child, - isBarrier = mapInArrow.isBarrier, - profile = mapInArrow.profile + } + + if (flatMapGroupsEnabled) { + result = result.transformUp { + case exec: FlatMapGroupsInPandasExec => + logInfo(s"Replacing FlatMapGroupsInPandasExec with DataFlint version for Spark $sparkMinorVersion") + DataFlintFlatMapGroupsInPandasExec( + groupingAttributes = exec.groupingAttributes, + func = exec.func, + output = exec.output, + child = exec.child ) - case _ => - // Default to 4.1 implementation for 4.1.x and any future 4.x - DataFlintPythonMapInArrowExec_4_1( - func = mapInArrow.func, - output = mapInArrow.output, - child = mapInArrow.child, - isBarrier = mapInArrow.isBarrier, - profile = mapInArrow.profile + } + } + + if (flatMapCoGroupsEnabled) { + result = result.transformUp { + case exec: FlatMapCoGroupsInPandasExec => + logInfo(s"Replacing FlatMapCoGroupsInPandasExec with DataFlint version for Spark $sparkMinorVersion") + DataFlintFlatMapCoGroupsInPandasExec( + leftGroup = exec.leftGroup, + rightGroup = exec.rightGroup, + func = exec.func, + output = exec.output, + left = exec.left, + right = exec.right ) } + } + + result } } } diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala new file mode 100644 index 0000000..53c2eea --- /dev/null +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * DataFlint instrumented version of ArrowEvalPythonExec for Spark 4.x. + * + * Instruments pandas_udf(PandasUDFType.SCALAR) operations with a duration metric + * by wrapping the parent's doExecute() RDD. + */ +class DataFlintArrowEvalPythonExec private ( + udfs: Seq[PythonUDF], + resultAttrs: Seq[Attribute], + child: SparkPlan, + evalType: Int) + extends ArrowEvalPythonExec(udfs, resultAttrs, child, evalType) with Logging { + + override def nodeName: String = "DataFlintArrowEvalPython" + + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintArrowEvalPythonExec = + DataFlintArrowEvalPythonExec(udfs, resultAttrs, newChild, evalType) +} + +object DataFlintArrowEvalPythonExec { + def apply( + udfs: Seq[PythonUDF], + resultAttrs: Seq[Attribute], + child: SparkPlan, + evalType: Int): DataFlintArrowEvalPythonExec = + new DataFlintArrowEvalPythonExec(udfs, resultAttrs, child, evalType) +} diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala new file mode 100644 index 0000000..40fcdc4 --- /dev/null +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * DataFlint instrumented version of FlatMapCoGroupsInPandasExec for Spark 4.x. + * + * Instruments PandasCogroupedOps.applyInPandas() / df.cogroup(df2) operations + * with a duration metric by wrapping the parent's doExecute() RDD. + */ +class DataFlintFlatMapCoGroupsInPandasExec private ( + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + left: SparkPlan, + right: SparkPlan) + extends FlatMapCoGroupsInPandasExec(leftGroup, rightGroup, func, output, left, right) with Logging { + + override def nodeName: String = "DataFlintFlatMapCoGroupsInPandas" + + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildrenInternal( + newLeft: SparkPlan, newRight: SparkPlan): DataFlintFlatMapCoGroupsInPandasExec = + DataFlintFlatMapCoGroupsInPandasExec(leftGroup, rightGroup, func, output, newLeft, newRight) +} + +object DataFlintFlatMapCoGroupsInPandasExec { + def apply( + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + left: SparkPlan, + right: SparkPlan): DataFlintFlatMapCoGroupsInPandasExec = + new DataFlintFlatMapCoGroupsInPandasExec(leftGroup, rightGroup, func, output, left, right) +} diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala new file mode 100644 index 0000000..f622a77 --- /dev/null +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * DataFlint instrumented version of FlatMapGroupsInPandasExec for Spark 4.x. + * + * Instruments GroupedData.applyInPandas() / pandas_udf(GROUPED_MAP) operations + * with a duration metric by wrapping the parent's doExecute() RDD. + */ +class DataFlintFlatMapGroupsInPandasExec private ( + groupingAttributes: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + child: SparkPlan) + extends FlatMapGroupsInPandasExec(groupingAttributes, func, output, child) with Logging { + + override def nodeName: String = "DataFlintFlatMapGroupsInPandas" + + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintFlatMapGroupsInPandasExec = + DataFlintFlatMapGroupsInPandasExec(groupingAttributes, func, output, newChild) +} + +object DataFlintFlatMapGroupsInPandasExec { + def apply( + groupingAttributes: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + child: SparkPlan): DataFlintFlatMapGroupsInPandasExec = + new DataFlintFlatMapGroupsInPandasExec(groupingAttributes, func, output, child) +} From 6278f234b655d6ccaa7f48ef16a357aaf97c4000 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Sun, 15 Mar 2026 13:05:15 +0200 Subject: [PATCH 04/15] add instrumentation for ArrowEvalPython, FlatMapCoGroupsInPandas, and FlatMapGroupsInPandas exec nodes now works --- .../DataflintSparkUICommonLoader.scala | 5 +- .../DataFlintInstrumentationExtension.scala | 85 +++++---- .../python/DataFlintArrowEvalPythonExec.scala | 36 +++- .../python/DataFlintBatchEvalPythonExec.scala | 63 +++++++ ...DataFlintFlatMapCoGroupsInPandasExec.scala | 12 ++ .../DataFlintFlatMapGroupsInPandasExec.scala | 7 + .../python/DataFlintMapInBatchExec_3_0.scala | 1 - .../dataflint/DataFlintPythonExecSpec.scala | 25 ++- .../DataFlintInstrumentationExtension.scala | 29 +++- .../python/DataFlintArrowEvalPythonExec.scala | 7 + .../python/DataFlintBatchEvalPythonExec.scala | 50 ++++++ ...DataFlintFlatMapCoGroupsInPandasExec.scala | 7 + .../DataFlintFlatMapGroupsInPandasExec.scala | 7 + .../dataflint_pyspark_example.py | 161 ++++++++++++++++-- 14 files changed, 427 insertions(+), 68 deletions(-) create mode 100644 spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala create mode 100644 spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala diff --git a/spark-plugin/plugin/src/main/scala/org/apache/spark/dataflint/DataflintSparkUICommonLoader.scala b/spark-plugin/plugin/src/main/scala/org/apache/spark/dataflint/DataflintSparkUICommonLoader.scala index c4a7c5e..d2273fa 100644 --- a/spark-plugin/plugin/src/main/scala/org/apache/spark/dataflint/DataflintSparkUICommonLoader.scala +++ b/spark-plugin/plugin/src/main/scala/org/apache/spark/dataflint/DataflintSparkUICommonLoader.scala @@ -136,6 +136,7 @@ object DataflintSparkUICommonLoader extends Logging { val INSTRUMENT_MAP_IN_ARROW_ENABLED = "spark.dataflint.instrument.spark.mapInArrow.enabled" val INSTRUMENT_WINDOW_ENABLED = "spark.dataflint.instrument.spark.window.enabled" val INSTRUMENT_ARROW_EVAL_PYTHON_ENABLED = "spark.dataflint.instrument.spark.arrowEvalPython.enabled" + val INSTRUMENT_BATCH_EVAL_PYTHON_ENABLED = "spark.dataflint.instrument.spark.batchEvalPython.enabled" val INSTRUMENT_FLAT_MAP_GROUPS_PANDAS_ENABLED = "spark.dataflint.instrument.spark.flatMapGroupsInPandas.enabled" val INSTRUMENT_FLAT_MAP_COGROUPS_PANDAS_ENABLED = "spark.dataflint.instrument.spark.flatMapCoGroupsInPandas.enabled" @@ -170,10 +171,12 @@ object DataflintSparkUICommonLoader extends Logging { val mapInArrowEnabled = sc.conf.getBoolean(INSTRUMENT_MAP_IN_ARROW_ENABLED, defaultValue = false) val windowEnabled = sc.conf.getBoolean(INSTRUMENT_WINDOW_ENABLED, defaultValue = false) val arrowEvalPythonEnabled = sc.conf.getBoolean(INSTRUMENT_ARROW_EVAL_PYTHON_ENABLED, defaultValue = false) + val batchEvalPythonEnabled = sc.conf.getBoolean(INSTRUMENT_BATCH_EVAL_PYTHON_ENABLED, defaultValue = false) val flatMapGroupsPandasEnabled = sc.conf.getBoolean(INSTRUMENT_FLAT_MAP_GROUPS_PANDAS_ENABLED, defaultValue = false) val flatMapCogroupsPandasEnabled = sc.conf.getBoolean(INSTRUMENT_FLAT_MAP_COGROUPS_PANDAS_ENABLED, defaultValue = false) val anyInstrumentationEnabled = instrumentEnabled || mapInPandasEnabled || mapInArrowEnabled || - windowEnabled || arrowEvalPythonEnabled || flatMapGroupsPandasEnabled || flatMapCogroupsPandasEnabled + windowEnabled || arrowEvalPythonEnabled || batchEvalPythonEnabled || + flatMapGroupsPandasEnabled || flatMapCogroupsPandasEnabled if (!anyInstrumentationEnabled) { logInfo("DataFlint instrumentation extension is disabled (no instrumentation flags enabled)") return diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala index 7c18147..8296023 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala @@ -2,34 +2,17 @@ package org.apache.spark.dataflint import org.apache.spark.SPARK_VERSION import org.apache.spark.internal.Logging -import org.apache.spark.sql.{SparkSession, SparkSessionExtensions, Strategy, execution} -import org.apache.spark.sql.catalyst.expressions.{NamedExpression, WindowFunctionType} +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions, Strategy} +import org.apache.spark.sql.catalyst.expressions.WindowFunctionType import org.apache.spark.sql.catalyst.planning.PhysicalWindow import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Window => LogicalWindow} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, DataFlintArrowEvalPythonExec_3_2, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_0, DataFlintMapInPandasExec_3_1, DataFlintMapInPandasExec_3_3, DataFlintMapInPandasExec_3_4, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_3, DataFlintPythonMapInArrowExec_3_4, DataFlintPythonMapInArrowExec_3_5, DataFlintWindowInPandasExec, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, WindowInPandasExec} +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_0, DataFlintMapInPandasExec_3_1, DataFlintMapInPandasExec_3_3, DataFlintMapInPandasExec_3_4, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_3, DataFlintPythonMapInArrowExec_3_4, DataFlintPythonMapInArrowExec_3_5, DataFlintWindowInPandasExec, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, WindowInPandasExec} import org.apache.spark.sql.execution.window.DataFlintWindowExec /** - * A SparkSessionExtension that injects DataFlint instrumentation into Spark's physical planning phase. - * This extension replaces: - * - MapInPandasExec with the version-appropriate DataFlintMapInPandasExec (adds duration metric) - * - PythonMapInArrowExec with the version-appropriate DataFlintPythonMapInArrowExec (adds duration metric) - * - * Supports Spark 3.0.x through 3.5.x with version-specific implementations that match - * each version's internal API. - * - * Note: mapInArrow instrumentation is only available on Spark 3.3+ (PythonMapInArrowExec was - * introduced in SPARK-37227). On Spark 3.0–3.2, only mapInPandas instrumentation is supported. - * - * The extension is automatically registered by SparkDataflintPlugin when any instrumentation flag is enabled: - * - spark.dataflint.instrument.spark.enabled (global) - * - spark.dataflint.instrument.spark.mapInPandas.enabled - * - spark.dataflint.instrument.spark.mapInArrow.enabled - * - * Can also be manually registered via: - * .config("spark.sql.extensions", "org.apache.spark.dataflint.DataFlintInstrumentationExtension") + * A SparkSessionExtensions that injects DataFlint instrumentation into Spark's physical planning phase. */ class DataFlintInstrumentationExtension extends (SparkSessionExtensions => Unit) with Logging { @@ -61,6 +44,9 @@ class DataFlintInstrumentationExtension extends (SparkSessionExtensions => Unit) * The mapInPandas and mapInArrow replacements are performed in separate transforms so that * PythonMapInArrowExec (which doesn't exist before Spark 3.3) doesn't cause NoClassDefFoundError * on older Spark versions. + * + * Each replace method guards against double-wrapping with isInstanceOf checks so that + * re-runs (e.g. under AQE prepareForExecution) are idempotent. */ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends ColumnarRule with Logging { @@ -90,6 +76,13 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C globalEnabled || specificEnabled } + private val batchEvalPythonEnabled: Boolean = { + val conf = session.sparkContext.conf + val globalEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, defaultValue = false) + val specificEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_BATCH_EVAL_PYTHON_ENABLED, defaultValue = false) + globalEnabled || specificEnabled + } + private val flatMapGroupsEnabled: Boolean = { val conf = session.sparkContext.conf val globalEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, defaultValue = false) @@ -105,8 +98,11 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C } override def preColumnarTransitions: Rule[SparkPlan] = { plan => - if (!mapInPandasEnabled && !mapInArrowEnabled && !arrowEvalPythonEnabled && !flatMapGroupsEnabled && !flatMapCoGroupsEnabled) plan + if (!mapInPandasEnabled && !mapInArrowEnabled && !arrowEvalPythonEnabled && !batchEvalPythonEnabled && !flatMapGroupsEnabled && !flatMapCoGroupsEnabled) plan else { + val inputNodes = plan.collect { case n => n.getClass.getSimpleName }.mkString(", ") + logWarning(s"DataFlint preColumnarTransitions ENTER — plan nodes: [$inputNodes]") + var result = plan if (mapInPandasEnabled) { @@ -121,6 +117,10 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C result = replaceArrowEvalPython(result) } + if (batchEvalPythonEnabled) { + result = replaceBatchEvalPython(result) + } + if (flatMapGroupsEnabled) { result = replaceFlatMapGroupsInPandas(result) } @@ -129,6 +129,8 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C result = replaceFlatMapCoGroupsInPandas(result) } + val outputNodes = result.collect { case n => n.getClass.getSimpleName }.mkString(", ") + logWarning(s"DataFlint preColumnarTransitions EXIT — plan nodes: [$outputNodes]") result } } @@ -136,7 +138,7 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C private def replaceMapInPandas(plan: SparkPlan): SparkPlan = { plan.transformUp { case mapInPandas: MapInPandasExec => - logInfo(s"Replacing MapInPandasExec with DataFlint version for Spark $sparkMinorVersion") + logWarning(s"Replacing MapInPandasExec with DataFlint version for Spark $sparkMinorVersion") sparkMinorVersion match { case "3.0" => DataFlintMapInPandasExec_3_0( @@ -180,8 +182,7 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C * Only supported on Spark 3.2+ which uses the 4-param constructor (udfs, resultAttrs, child, * evalType). Spark 3.0–3.1 used a 3-param constructor incompatible with our wrapper. * - * Isolated in its own method with a try-catch so that any unexpected class-loading error - * on older Spark versions degrades gracefully. + * The isInstanceOf guard makes the rule idempotent — safe to re-run under AQE. */ private def replaceArrowEvalPython(plan: SparkPlan): SparkPlan = { if (sparkMinorVersion == "3.0" || sparkMinorVersion == "3.1") { @@ -190,9 +191,9 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C } try { plan.transformUp { - case arrowEval: ArrowEvalPythonExec => - logInfo(s"Replacing ArrowEvalPythonExec with DataFlint version for Spark $sparkMinorVersion") - DataFlintArrowEvalPythonExec_3_2( + case arrowEval: ArrowEvalPythonExec if !arrowEval.isInstanceOf[DataFlintArrowEvalPythonExec] => + logWarning(s"Replacing ArrowEvalPythonExec with DataFlint version for Spark $sparkMinorVersion") + DataFlintArrowEvalPythonExec( udfs = arrowEval.udfs, resultAttrs = arrowEval.resultAttrs, child = arrowEval.child, @@ -206,10 +207,22 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C } } + private def replaceBatchEvalPython(plan: SparkPlan): SparkPlan = { + plan.transformUp { + case exec: BatchEvalPythonExec if !exec.isInstanceOf[DataFlintBatchEvalPythonExec] => + logWarning(s"Replacing BatchEvalPythonExec with DataFlint version for Spark $sparkMinorVersion") + DataFlintBatchEvalPythonExec( + udfs = exec.udfs, + resultAttrs = exec.resultAttrs, + child = exec.child + ) + } + } + private def replaceFlatMapGroupsInPandas(plan: SparkPlan): SparkPlan = { plan.transformUp { - case exec: FlatMapGroupsInPandasExec => - logInfo(s"Replacing FlatMapGroupsInPandasExec with DataFlint version for Spark $sparkMinorVersion") + case exec: FlatMapGroupsInPandasExec if !exec.isInstanceOf[DataFlintFlatMapGroupsInPandasExec] => + logWarning(s"Replacing FlatMapGroupsInPandasExec with DataFlint version for Spark $sparkMinorVersion") DataFlintFlatMapGroupsInPandasExec( groupingAttributes = exec.groupingAttributes, func = exec.func, @@ -221,8 +234,8 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C private def replaceFlatMapCoGroupsInPandas(plan: SparkPlan): SparkPlan = { plan.transformUp { - case exec: FlatMapCoGroupsInPandasExec => - logInfo(s"Replacing FlatMapCoGroupsInPandasExec with DataFlint version for Spark $sparkMinorVersion") + case exec: FlatMapCoGroupsInPandasExec if !exec.isInstanceOf[DataFlintFlatMapCoGroupsInPandasExec] => + logWarning(s"Replacing FlatMapCoGroupsInPandasExec with DataFlint version for Spark $sparkMinorVersion") DataFlintFlatMapCoGroupsInPandasExec( leftGroup = exec.leftGroup, rightGroup = exec.rightGroup, @@ -248,7 +261,7 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C try { plan.transformUp { case mapInArrow: PythonMapInArrowExec => - logInfo(s"Replacing PythonMapInArrowExec with DataFlint version for Spark $sparkMinorVersion") + logWarning(s"Replacing PythonMapInArrowExec with DataFlint version for Spark $sparkMinorVersion") sparkMinorVersion match { case "3.3" => DataFlintPythonMapInArrowExec_3_3( @@ -301,16 +314,16 @@ case class DataFlintWindowPlannerStrategy(session: SparkSession) extends Strateg plan match { case PhysicalWindow( WindowFunctionType.SQL, windowExprs, partitionSpec, orderSpec, child) => - logInfo("Replacing logical Window with DataFlintWindowExec") + logWarning("Replacing logical Window with DataFlintWindowExec") DataFlintWindowExec( windowExprs, partitionSpec, orderSpec, planLater(child)) :: Nil case PhysicalWindow( WindowFunctionType.Python, windowExprs, partitionSpec, orderSpec, child) => - logInfo("Replacing logical Window (Python UDF) with DataFlintWindowInPandasExec") + logWarning("Replacing logical Window (Python UDF) with DataFlintWindowInPandasExec") DataFlintWindowInPandasExec( windowExprs, partitionSpec, orderSpec, planLater(child)) :: Nil case _ => Nil } } -} +} \ No newline at end of file diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala index e547f96..d9e8e40 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.python +import org.apache.spark.TaskContext +import org.apache.spark.api.python.ChainedPythonFunctions import org.apache.spark.dataflint.DataFlintRDDUtils import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -23,6 +25,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.types.StructType + +import java.util.concurrent.TimeUnit.NANOSECONDS /** * DataFlint instrumented version of ArrowEvalPythonExec for Spark 3.2–3.5. @@ -32,7 +37,7 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} * (udfs, resultAttrs, child, evalType). Spark 3.0–3.1 lack the evalType param * and are not instrumented. */ -class DataFlintArrowEvalPythonExec_3_2 private ( +class DataFlintArrowEvalPythonExec private( udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan, @@ -45,18 +50,33 @@ class DataFlintArrowEvalPythonExec_3_2 private ( "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") ) - override protected def doExecute(): RDD[InternalRow] = - DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + override protected def evaluate(funcs: Seq[ChainedPythonFunctions], argOffsets: Array[Array[Int]], iter: Iterator[InternalRow], schema: StructType, context: TaskContext): Iterator[InternalRow] = { + val durationMetric = longMetric("duration") + val startTime = System.nanoTime() + val out = super.evaluate(funcs, argOffsets, iter, schema, context) + durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) + out + } + + // override protected def doExecute(): RDD[InternalRow] = +// DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintArrowEvalPythonExec = + DataFlintArrowEvalPythonExec(udfs, resultAttrs, newChild, evalType) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintArrowEvalPythonExec] + + override def equals(other: Any): Boolean = + other.isInstanceOf[DataFlintArrowEvalPythonExec] && super.equals(other) - override protected def withNewChildInternal(newChild: SparkPlan): DataFlintArrowEvalPythonExec_3_2 = - DataFlintArrowEvalPythonExec_3_2(udfs, resultAttrs, newChild, evalType) + override def hashCode: Int = super.hashCode } -object DataFlintArrowEvalPythonExec_3_2 { +object DataFlintArrowEvalPythonExec { def apply( udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan, - evalType: Int): DataFlintArrowEvalPythonExec_3_2 = - new DataFlintArrowEvalPythonExec_3_2(udfs, resultAttrs, child, evalType) + evalType: Int): DataFlintArrowEvalPythonExec = + new DataFlintArrowEvalPythonExec(udfs, resultAttrs, child, evalType) } diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala new file mode 100644 index 0000000..7716c19 --- /dev/null +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala @@ -0,0 +1,63 @@ +package org.apache.spark.sql.execution.python + +import org.apache.spark.TaskContext +import org.apache.spark.api.python.ChainedPythonFunctions +import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.types.StructType + +import java.util.concurrent.TimeUnit.NANOSECONDS + +/** + * DataFlint instrumented version of BatchEvalPythonExec for Spark 3.x. + * + * Instruments regular Python UDF (@udf / SQL_BATCHED_UDF) operations with a + * duration metric by wrapping the parent's doExecute() RDD. + * Constructor (udfs, resultAttrs, child) is stable across Spark 3.0–3.5. + */ +class DataFlintBatchEvalPythonExec private ( + udfs: Seq[PythonUDF], + resultAttrs: Seq[Attribute], + child: SparkPlan) + extends BatchEvalPythonExec(udfs, resultAttrs, child) with Logging { + + override def nodeName: String = "DataFlintBatchEvalPython" + + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def evaluate(funcs: Seq[ChainedPythonFunctions], argOffsets: Array[Array[Int]], iter: Iterator[InternalRow], schema: StructType, context: TaskContext): Iterator[InternalRow] = { + val durationMetric = longMetric("duration") + val startTime = System.nanoTime() + val out = super.evaluate(funcs, argOffsets, iter, schema, context) + durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) + out + } + +// override protected def doExecute(): RDD[InternalRow] = +// DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintBatchEvalPythonExec = + DataFlintBatchEvalPythonExec(udfs, resultAttrs, newChild) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintBatchEvalPythonExec] + + override def equals(other: Any): Boolean = + other.isInstanceOf[DataFlintBatchEvalPythonExec] && super.equals(other) + + override def hashCode: Int = super.hashCode +} + +object DataFlintBatchEvalPythonExec { + def apply( + udfs: Seq[PythonUDF], + resultAttrs: Seq[Attribute], + child: SparkPlan): DataFlintBatchEvalPythonExec = + new DataFlintBatchEvalPythonExec(udfs, resultAttrs, child) +} \ No newline at end of file diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala index 8b90c01..9acf21d 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.python +import org.apache.spark.TaskContext +import org.apache.spark.api.python.ChainedPythonFunctions import org.apache.spark.dataflint.DataFlintRDDUtils import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -23,6 +25,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.types.StructType + +import java.util.concurrent.TimeUnit.NANOSECONDS /** * DataFlint instrumented version of FlatMapCoGroupsInPandasExec for Spark 3.x. @@ -52,6 +57,13 @@ class DataFlintFlatMapCoGroupsInPandasExec private ( override protected def withNewChildrenInternal( newLeft: SparkPlan, newRight: SparkPlan): DataFlintFlatMapCoGroupsInPandasExec = DataFlintFlatMapCoGroupsInPandasExec(leftGroup, rightGroup, func, output, newLeft, newRight) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintFlatMapCoGroupsInPandasExec] + + override def equals(other: Any): Boolean = + other.isInstanceOf[DataFlintFlatMapCoGroupsInPandasExec] && super.equals(other) + + override def hashCode: Int = super.hashCode } object DataFlintFlatMapCoGroupsInPandasExec { diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala index 70947ee..b8fe584 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala @@ -49,6 +49,13 @@ class DataFlintFlatMapGroupsInPandasExec private ( override protected def withNewChildInternal(newChild: SparkPlan): DataFlintFlatMapGroupsInPandasExec = DataFlintFlatMapGroupsInPandasExec(groupingAttributes, func, output, newChild) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintFlatMapGroupsInPandasExec] + + override def equals(other: Any): Boolean = + other.isInstanceOf[DataFlintFlatMapGroupsInPandasExec] && super.equals(other) + + override def hashCode: Int = super.hashCode } object DataFlintFlatMapGroupsInPandasExec { diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_0.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_0.scala index f0cef04..1079f87 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_0.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_0.scala @@ -67,7 +67,6 @@ trait DataFlintMapInBatchExec_3_0 extends SparkPlan with Logging { logInfo("DataFlint MapInBatchExec (Spark 3.0) doExecute is running") val durationMetric = longMetric("duration") - child.execute().mapPartitionsInternal { inputIter => val startTime = System.nanoTime() diff --git a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala index fc159e1..a1e616f 100644 --- a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala +++ b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala @@ -4,7 +4,7 @@ import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.PythonUDF import org.apache.spark.sql.execution.LocalTableScanExec -import org.apache.spark.sql.execution.python._ +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_5, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, PythonMapInArrowExec} import org.apache.spark.sql.types.LongType import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite @@ -92,14 +92,14 @@ class DataFlintPythonExecSpec extends AnyFunSuite with Matchers with BeforeAndAf val original = ArrowEvalPythonExec(Seq(udf), Seq.empty, emptyChild, PythonEvalType.SQL_SCALAR_PANDAS_UDF) val rule = DataFlintInstrumentationColumnarRule(spark) val result = rule.preColumnarTransitions(original) - result shouldBe a[DataFlintArrowEvalPythonExec_3_2] + result shouldBe a[DataFlintArrowEvalPythonExec] } test("ArrowEvalPythonExec nodeName and metrics") { val udf = fakePythonUDF(PythonEvalType.SQL_SCALAR_PANDAS_UDF) val original = ArrowEvalPythonExec(Seq(udf), Seq.empty, emptyChild, PythonEvalType.SQL_SCALAR_PANDAS_UDF) val rule = DataFlintInstrumentationColumnarRule(spark) - val result = rule.preColumnarTransitions(original).asInstanceOf[DataFlintArrowEvalPythonExec_3_2] + val result = rule.preColumnarTransitions(original).asInstanceOf[DataFlintArrowEvalPythonExec] result.nodeName shouldBe "DataFlintArrowEvalPython" result.metrics should contain key "duration" } @@ -123,6 +123,25 @@ class DataFlintPythonExecSpec extends AnyFunSuite with Matchers with BeforeAndAf result.metrics should contain key "duration" } + // ---- BatchEvalPythonExec (regular @udf / SQL_BATCHED_UDF) ---- + + test("replaces BatchEvalPythonExec in plan") { + val udf = fakePythonUDF(PythonEvalType.SQL_BATCHED_UDF) + val original = BatchEvalPythonExec(Seq(udf), Seq.empty, emptyChild) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original) + result shouldBe a[DataFlintBatchEvalPythonExec] + } + + test("BatchEvalPythonExec nodeName and metrics") { + val udf = fakePythonUDF(PythonEvalType.SQL_BATCHED_UDF) + val original = BatchEvalPythonExec(Seq(udf), Seq.empty, emptyChild) + val rule = DataFlintInstrumentationColumnarRule(spark) + val result = rule.preColumnarTransitions(original).asInstanceOf[DataFlintBatchEvalPythonExec] + result.nodeName shouldBe "DataFlintBatchEvalPython" + result.metrics should contain key "duration" + } + // ---- FlatMapCoGroupsInPandasExec (cogroup / applyInPandas on two DataFrames) ---- test("replaces FlatMapCoGroupsInPandasExec in plan") { diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala index 085aba2..c11b609 100644 --- a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala @@ -10,7 +10,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Window => Logic import org.apache.spark.sql.execution.SparkStrategy import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintArrowWindowPythonExec_4_1, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_4_0, DataFlintMapInPandasExec_4_1, DataFlintPythonMapInArrowExec_4_0, DataFlintPythonMapInArrowExec_4_1, DataFlintWindowInPandasExec_4_0, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInArrowExec, MapInPandasExec} +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintArrowWindowPythonExec_4_1, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_4_0, DataFlintMapInPandasExec_4_1, DataFlintPythonMapInArrowExec_4_0, DataFlintPythonMapInArrowExec_4_1, DataFlintWindowInPandasExec_4_0, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInArrowExec, MapInPandasExec} import org.apache.spark.sql.execution.window.DataFlintWindowExec /** @@ -95,8 +95,15 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C globalEnabled || specificEnabled } + private val batchEvalPythonEnabled: Boolean = { + val conf = session.sparkContext.conf + val globalEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, defaultValue = false) + val specificEnabled = conf.getBoolean(DataflintSparkUICommonLoader.INSTRUMENT_BATCH_EVAL_PYTHON_ENABLED, defaultValue = false) + globalEnabled || specificEnabled + } + override def preColumnarTransitions: Rule[SparkPlan] = { plan => - if (!mapInPandasEnabled && !mapInArrowEnabled && !arrowEvalPythonEnabled && !flatMapGroupsEnabled && !flatMapCoGroupsEnabled) plan + if (!mapInPandasEnabled && !mapInArrowEnabled && !arrowEvalPythonEnabled && !batchEvalPythonEnabled && !flatMapGroupsEnabled && !flatMapCoGroupsEnabled) plan else { var result = plan @@ -147,7 +154,7 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C if (arrowEvalPythonEnabled) { result = result.transformUp { - case arrowEval: ArrowEvalPythonExec => + case arrowEval: ArrowEvalPythonExec if !arrowEval.isInstanceOf[DataFlintArrowEvalPythonExec] => logInfo(s"Replacing ArrowEvalPythonExec with DataFlint version for Spark $sparkMinorVersion") DataFlintArrowEvalPythonExec( udfs = arrowEval.udfs, @@ -158,9 +165,21 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C } } + if (batchEvalPythonEnabled) { + result = result.transformUp { + case exec: BatchEvalPythonExec if !exec.isInstanceOf[DataFlintBatchEvalPythonExec] => + logInfo(s"Replacing BatchEvalPythonExec with DataFlint version for Spark $sparkMinorVersion") + DataFlintBatchEvalPythonExec( + udfs = exec.udfs, + resultAttrs = exec.resultAttrs, + child = exec.child + ) + } + } + if (flatMapGroupsEnabled) { result = result.transformUp { - case exec: FlatMapGroupsInPandasExec => + case exec: FlatMapGroupsInPandasExec if !exec.isInstanceOf[DataFlintFlatMapGroupsInPandasExec] => logInfo(s"Replacing FlatMapGroupsInPandasExec with DataFlint version for Spark $sparkMinorVersion") DataFlintFlatMapGroupsInPandasExec( groupingAttributes = exec.groupingAttributes, @@ -173,7 +192,7 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C if (flatMapCoGroupsEnabled) { result = result.transformUp { - case exec: FlatMapCoGroupsInPandasExec => + case exec: FlatMapCoGroupsInPandasExec if !exec.isInstanceOf[DataFlintFlatMapCoGroupsInPandasExec] => logInfo(s"Replacing FlatMapCoGroupsInPandasExec with DataFlint version for Spark $sparkMinorVersion") DataFlintFlatMapCoGroupsInPandasExec( leftGroup = exec.leftGroup, diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala index 53c2eea..854f9e9 100644 --- a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala @@ -48,6 +48,13 @@ class DataFlintArrowEvalPythonExec private ( override protected def withNewChildInternal(newChild: SparkPlan): DataFlintArrowEvalPythonExec = DataFlintArrowEvalPythonExec(udfs, resultAttrs, newChild, evalType) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintArrowEvalPythonExec] + + override def equals(other: Any): Boolean = + other.isInstanceOf[DataFlintArrowEvalPythonExec] && super.equals(other) + + override def hashCode: Int = super.hashCode } object DataFlintArrowEvalPythonExec { diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala new file mode 100644 index 0000000..e72f861 --- /dev/null +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala @@ -0,0 +1,50 @@ +package org.apache.spark.sql.execution.python + +import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * DataFlint instrumented version of BatchEvalPythonExec for Spark 4.x. + * + * Instruments regular Python UDF (@udf / SQL_BATCHED_UDF) operations with a + * duration metric by wrapping the parent's doExecute() RDD. + * Constructor (udfs, resultAttrs, child) is stable across Spark 4.0–4.1. + */ +class DataFlintBatchEvalPythonExec private ( + udfs: Seq[PythonUDF], + resultAttrs: Seq[Attribute], + child: SparkPlan) + extends BatchEvalPythonExec(udfs, resultAttrs, child) with Logging { + + override def nodeName: String = "DataFlintBatchEvalPython" + + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintBatchEvalPythonExec = + DataFlintBatchEvalPythonExec(udfs, resultAttrs, newChild) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintBatchEvalPythonExec] + + override def equals(other: Any): Boolean = + other.isInstanceOf[DataFlintBatchEvalPythonExec] && super.equals(other) + + override def hashCode: Int = super.hashCode +} + +object DataFlintBatchEvalPythonExec { + def apply( + udfs: Seq[PythonUDF], + resultAttrs: Seq[Attribute], + child: SparkPlan): DataFlintBatchEvalPythonExec = + new DataFlintBatchEvalPythonExec(udfs, resultAttrs, child) +} \ No newline at end of file diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala index 40fcdc4..1b8e5ba 100644 --- a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala @@ -51,6 +51,13 @@ class DataFlintFlatMapCoGroupsInPandasExec private ( override protected def withNewChildrenInternal( newLeft: SparkPlan, newRight: SparkPlan): DataFlintFlatMapCoGroupsInPandasExec = DataFlintFlatMapCoGroupsInPandasExec(leftGroup, rightGroup, func, output, newLeft, newRight) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintFlatMapCoGroupsInPandasExec] + + override def equals(other: Any): Boolean = + other.isInstanceOf[DataFlintFlatMapCoGroupsInPandasExec] && super.equals(other) + + override def hashCode: Int = super.hashCode } object DataFlintFlatMapCoGroupsInPandasExec { diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala index f622a77..a97f598 100644 --- a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala @@ -48,6 +48,13 @@ class DataFlintFlatMapGroupsInPandasExec private ( override protected def withNewChildInternal(newChild: SparkPlan): DataFlintFlatMapGroupsInPandasExec = DataFlintFlatMapGroupsInPandasExec(groupingAttributes, func, output, newChild) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintFlatMapGroupsInPandasExec] + + override def equals(other: Any): Boolean = + other.isInstanceOf[DataFlintFlatMapGroupsInPandasExec] && super.equals(other) + + override def hashCode: Int = super.hashCode } object DataFlintFlatMapGroupsInPandasExec { diff --git a/spark-plugin/pyspark-testing/dataflint_pyspark_example.py b/spark-plugin/pyspark-testing/dataflint_pyspark_example.py index 8af12b6..2b7906f 100755 --- a/spark-plugin/pyspark-testing/dataflint_pyspark_example.py +++ b/spark-plugin/pyspark-testing/dataflint_pyspark_example.py @@ -1,8 +1,13 @@ #!/usr/bin/python3 """ -PySpark mapInPandas and mapInArrow example with DataFlint instrumentation. - -This script demonstrates both mapInPandas and mapInArrow operations with DataFlint plugin enabled. +PySpark example with DataFlint instrumentation for all Python exec types: + - mapInPandas → DataFlintMapInPandasExec + - mapInArrow → DataFlintPythonMapInArrowExec + - Window (SQL) → DataFlintWindowExec + - Window (pandas UDF) → DataFlintWindowInPandasExec + - pandas_udf SCALAR → DataFlintArrowEvalPythonExec + - applyInPandas → DataFlintFlatMapGroupsInPandasExec + - cogroup → DataFlintFlatMapCoGroupsInPandasExec Usage: pip install pyspark pandas pyarrow @@ -53,10 +58,14 @@ .config("spark.sql.maxMetadataStringLength", "10000") \ .config("spark.sql.adaptive.enabled", "false") \ .config("spark.dataflint.telemetry.enabled", "false") \ - .config("spark.dataflint.instrument.spark.enable", "true") \ + .config("spark.dataflint.instrument.spark.enabled", "true") \ .config("spark.dataflint.instrument.spark.mapInPandas.enabled", "true") \ .config("spark.dataflint.instrument.spark.mapInArrow.enabled", "true") \ .config("spark.dataflint.instrument.spark.window.enabled", "true") \ + .config("spark.dataflint.instrument.spark.arrowEvalPython.enabled", "true") \ + .config("spark.dataflint.instrument.spark.flatMapGroupsInPandas.enabled", "true") \ + .config("spark.dataflint.instrument.spark.flatMapCoGroupsInPandas.enabled", "true") \ + .config("spark.dataflint.instrument.spark.batchEvalPython.enabled", "true") \ .master("local[*]") \ .getOrCreate() # spark.sparkContext.setLogLevel("INFO") @@ -151,13 +160,12 @@ def compute_discounted_totals_arrow(iterator): df_pandas = df.mapInPandas(compute_discounted_totals_pandas, output_schema) +spark.sparkContext.setJobDescription("mapInPandas: compute discounted totals → DataFlintMapInPandasExec") df_pandas.write \ .mode("overwrite") \ .parquet("/tmp/dataflint_map_in_pandas_example") print("\nResult written to /tmp/dataflint_map_in_pandas_example") -print("\nSample output:") -df_pandas.show(10, truncate=False) # Run mapInArrow (only if Spark version >= 3.3.0) @@ -168,13 +176,12 @@ def compute_discounted_totals_arrow(iterator): df_arrow = df.mapInArrow(compute_discounted_totals_arrow, output_schema) + spark.sparkContext.setJobDescription("mapInArrow: compute discounted totals → DataFlintPythonMapInArrowExec") df_arrow.write \ .mode("overwrite") \ .parquet("/tmp/dataflint_map_in_arrow_example") print("\nResult written to /tmp/dataflint_map_in_arrow_example") - print("\nSample output:") - df_arrow.show(10, truncate=False) else: print("\n" + "="*80) print("Skipping mapInArrow example") @@ -183,7 +190,6 @@ def compute_discounted_totals_arrow(iterator): print(f"Current version: {spark_version}") - print("\n" + "="*80) print("Running Window function example") print("="*80) @@ -199,13 +205,12 @@ def compute_discounted_totals_arrow(iterator): .withColumn("cumulative_revenue", spark_sum("price").over(window_by_category)) \ .withColumn("avg_price_in_category", avg("price").over(window_category_total)) +spark.sparkContext.setJobDescription("Window SQL: rank + cumulative sum + avg → DataFlintWindowExec") df_window.write \ .mode("overwrite") \ .parquet("/tmp/dataflint_window_example") print("\nResult written to /tmp/dataflint_window_example") -print("\nSample output:") -df_window.show(10, truncate=False) print("\n" + "="*80) @@ -227,13 +232,141 @@ def discounted_sum(prices: pd.Series) -> float: discounted_sum("price").over(window_category_total) ) +spark.sparkContext.setJobDescription("Window pandas UDF: discounted sum → DataFlintWindowInPandasExec") df_window_udf.write \ .mode("overwrite") \ .parquet("/tmp/dataflint_window_udf_example") print("\nResult written to /tmp/dataflint_window_udf_example") -print("\nSample output:") -df_window_udf.show(10, truncate=False) + + +print("\n" + "="*80) +print("Running regular @udf example (BatchEvalPython)") +print("="*80) + +# Regular Python @udf → DataFlintBatchEvalPythonExec +# Spark serializes rows to/from Python via pickle (no Arrow); the UDF runs in a Python subprocess +from pyspark.sql.functions import udf +from pyspark.sql.types import StringType + +@udf(StringType()) +def price_tier(price): + """Classify price into tiers without using pandas/Arrow.""" + if price is None: + return "unknown" + if price < 20: + return "budget" + if price < 100: + return "mid-range" + return "premium" + +df_batch_udf = df.withColumn("price_tier", price_tier("price")) + +spark.sparkContext.setJobDescription("@udf: classify price tier → DataFlintBatchEvalPythonExec") +df_batch_udf.write \ + .mode("overwrite") \ + .parquet("/tmp/dataflint_batch_eval_python_example") + +print("\nResult written to /tmp/dataflint_batch_eval_python_example") + + +print("\n" + "="*80) +print("Running pandas_udf SCALAR example (ArrowEvalPython)") +print("="*80) + +# pandas_udf SCALAR → DataFlintArrowEvalPythonExec +# The UDF is applied column-wise; Spark executes it via ArrowEvalPythonExec +@pandas_udf(DoubleType()) +def apply_discount(price: pd.Series, quantity: pd.Series) -> pd.Series: + """Apply a 10% discount when quantity > 3, otherwise no discount.""" + return price * quantity.apply(lambda q: 0.90 if q > 3 else 1.0) + +df_scalar_udf = df.withColumn( + "discounted_price", + apply_discount("price", "quantity") +) + +spark.sparkContext.setJobDescription("pandas_udf SCALAR: apply discount → DataFlintArrowEvalPythonExec") +df_scalar_udf.write \ + .mode("overwrite") \ + .parquet("/tmp/dataflint_arrow_eval_python_example") + +print("\nResult written to /tmp/dataflint_arrow_eval_python_example") + + +print("\n" + "="*80) +print("Running applyInPandas example (FlatMapGroupsInPandas)") +print("="*80) + +# applyInPandas / GROUPED_MAP → DataFlintFlatMapGroupsInPandasExec +# Groups data by category, then applies a pandas function to each group +def enrich_group(pdf): + """Compute per-category stats and add them as new columns.""" + pdf = pdf.copy() + pdf["quantity"] = pdf["quantity"].fillna(0) + pdf["cat_total_revenue"] = (pdf["quantity"] * pdf["price"]).sum() + pdf["cat_avg_price"] = pdf["price"].mean() + pdf["revenue"] = pdf["quantity"] * pdf["price"] + return pdf[["customer", "category", "quantity", "price", "revenue", + "cat_total_revenue", "cat_avg_price"]] + +group_output_schema = ( + "customer string, category string, quantity int, price double, " + "revenue double, cat_total_revenue double, cat_avg_price double" +) + +df_grouped = df.groupby("category").applyInPandas(enrich_group, schema=group_output_schema) + +spark.sparkContext.setJobDescription("applyInPandas: per-category stats → DataFlintFlatMapGroupsInPandasExec") +df_grouped.write \ + .mode("overwrite") \ + .parquet("/tmp/dataflint_flat_map_groups_in_pandas_example") + +print("\nResult written to /tmp/dataflint_flat_map_groups_in_pandas_example") + + +print("\n" + "="*80) +print("Running cogroup applyInPandas example (FlatMapCoGroupsInPandas)") +print("="*80) + +# cogroup / applyInPandas on two DataFrames → DataFlintFlatMapCoGroupsInPandasExec +# Join two DataFrames co-located on the same key and process each co-group together +discounts_data = [ + ("Electronics", 0.15), + ("Books", 0.05), + ("Clothing", 0.10), +] +discounts_schema = "category string, discount_rate double" +df_discounts = spark.createDataFrame(discounts_data, discounts_schema) + +def apply_category_discount(left_pdf, right_pdf): + """Apply per-category discount from the right DataFrame to the left.""" + import pandas as pd + discount_rate = right_pdf["discount_rate"].iloc[0] if len(right_pdf) > 0 else 0.0 + left_pdf = left_pdf.copy() + left_pdf["quantity"] = left_pdf["quantity"].fillna(0) + left_pdf["final_price"] = left_pdf["price"] * (1.0 - discount_rate) + left_pdf["discount_rate"] = discount_rate + return left_pdf[["customer", "category", "quantity", "price", + "discount_rate", "final_price"]] + +cogroup_output_schema = ( + "customer string, category string, quantity int, price double, " + "discount_rate double, final_price double" +) + +df_cogrouped = ( + df.groupby("category") + .cogroup(df_discounts.groupby("category")) + .applyInPandas(apply_category_discount, schema=cogroup_output_schema) +) + +spark.sparkContext.setJobDescription("cogroup applyInPandas: apply category discounts → DataFlintFlatMapCoGroupsInPandasExec") +df_cogrouped.write \ + .mode("overwrite") \ + .parquet("/tmp/dataflint_flat_map_cogroups_in_pandas_example") + +print("\nResult written to /tmp/dataflint_flat_map_cogroups_in_pandas_example") print("\n" + "="*80) @@ -241,4 +374,4 @@ def discounted_sum(prices: pd.Series) -> float: print("="*80) input("\nPress Enter to exit...") -spark.stop() +spark.stop() \ No newline at end of file From 5cbf9b7a335257b06dfae4b22da50890c0ceba8c Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Sun, 15 Mar 2026 16:07:18 +0200 Subject: [PATCH 05/15] updated and merged implementation of DataFlintMapInArrowExec and DataFlintMapInArrowExec --- .../DataFlintInstrumentationExtension.scala | 54 ++--- .../python/DataFlintMapInBatchExec_3_0.scala | 174 +++++--------- .../python/DataFlintMapInBatchExec_3_1.scala | 148 ------------ .../python/DataFlintMapInBatchExec_3_3.scala | 187 +++++---------- .../python/DataFlintMapInBatchExec_3_4.scala | 164 -------------- .../python/DataFlintMapInBatchExec_3_5.scala | 153 +++++-------- .../python/DataFlintMapInPandasExec.scala | 133 ----------- .../DataFlintInstrumentationExtension.scala | 56 ++--- .../python/DataFlintMapInArrowExec.scala | 71 ++++++ .../python/DataFlintMapInBatchExec_4_0.scala | 148 ------------ .../python/DataFlintMapInBatchExec_4_1.scala | 213 ------------------ .../python/DataFlintMapInPandasExec.scala | 108 ++------- 12 files changed, 302 insertions(+), 1307 deletions(-) delete mode 100644 spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_1.scala delete mode 100644 spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_4.scala delete mode 100644 spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec.scala create mode 100644 spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec.scala delete mode 100644 spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_4_0.scala delete mode 100644 spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_4_1.scala diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala index 8296023..5a7a79a 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala @@ -8,7 +8,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalWindow import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Window => LogicalWindow} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_0, DataFlintMapInPandasExec_3_1, DataFlintMapInPandasExec_3_3, DataFlintMapInPandasExec_3_4, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_3, DataFlintPythonMapInArrowExec_3_4, DataFlintPythonMapInArrowExec_3_5, DataFlintWindowInPandasExec, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, WindowInPandasExec} +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_0, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_3, DataFlintPythonMapInArrowExec_3_5, DataFlintWindowInPandasExec, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, WindowInPandasExec} import org.apache.spark.sql.execution.window.DataFlintWindowExec /** @@ -140,37 +140,19 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C case mapInPandas: MapInPandasExec => logWarning(s"Replacing MapInPandasExec with DataFlint version for Spark $sparkMinorVersion") sparkMinorVersion match { - case "3.0" => - DataFlintMapInPandasExec_3_0( - func = mapInPandas.func, - output = mapInPandas.output, - child = mapInPandas.child - ) - case "3.1" | "3.2" => - DataFlintMapInPandasExec_3_1( - func = mapInPandas.func, - output = mapInPandas.output, - child = mapInPandas.child - ) - case "3.3" => - DataFlintMapInPandasExec_3_3( - func = mapInPandas.func, - output = mapInPandas.output, - child = mapInPandas.child - ) - case "3.4" => - DataFlintMapInPandasExec_3_4( + case "3.5" => + DataFlintMapInPandasExec_3_5( func = mapInPandas.func, output = mapInPandas.output, - child = mapInPandas.child + child = mapInPandas.child, + isBarrier = mapInPandas.isBarrier ) case _ => - // Default to 3.5 implementation for 3.5.x and any future 3.x - DataFlintMapInPandasExec_3_5( + // 3.0–3.4: all share the same 3-arg constructor, handled via reflection + DataFlintMapInPandasExec_3_0( func = mapInPandas.func, output = mapInPandas.output, - child = mapInPandas.child, - isBarrier = mapInPandas.isBarrier + child = mapInPandas.child ) } } @@ -263,25 +245,19 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C case mapInArrow: PythonMapInArrowExec => logWarning(s"Replacing PythonMapInArrowExec with DataFlint version for Spark $sparkMinorVersion") sparkMinorVersion match { - case "3.3" => - DataFlintPythonMapInArrowExec_3_3( - func = mapInArrow.func, - output = mapInArrow.output, - child = mapInArrow.child - ) - case "3.4" => - DataFlintPythonMapInArrowExec_3_4( + case "3.5" => + DataFlintPythonMapInArrowExec_3_5( func = mapInArrow.func, output = mapInArrow.output, - child = mapInArrow.child + child = mapInArrow.child, + isBarrier = mapInArrow.isBarrier ) case _ => - // Default to 3.5 implementation for 3.5.x and any future 3.x - DataFlintPythonMapInArrowExec_3_5( + // 3.3–3.4: share the same 3-arg constructor, handled via reflection + DataFlintPythonMapInArrowExec_3_3( func = mapInArrow.func, output = mapInArrow.output, - child = mapInArrow.child, - isBarrier = mapInArrow.isBarrier + child = mapInArrow.child ) } } diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_0.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_0.scala index 1079f87..41d066e 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_0.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_0.scala @@ -1,60 +1,47 @@ /* - * DataFlint instrumented MapInBatchExec for Spark 3.0.x + * DataFlint instrumented MapInBatchExec for Spark 3.0.x / 3.1.x / 3.2.x * - * Spark 3.0 characteristics: - * - MapInPandasExec extends UnaryExecNode directly (no MapInBatchExec trait, introduced in 3.3) - * - No PythonMapInArrowExec (introduced in 3.3 via SPARK-37227) - * - No PythonSQLMetrics mixin (introduced in 3.4 via SPARK-34265) - * - No barrier mode support (introduced in 3.5 via SPARK-42896) - * - No MapInBatchEvaluatorFactory (introduced in 3.5 via SPARK-44361) - * - Inline execution with ArrowPythonRunner (6-arg constructor) - * - ArrowPythonRunner constructor: (funcs, evalType, argOffsets, schema, timeZoneId, confMap) - * - MapInPandasExec fields: (func, output, child) — no isBarrier, no profile - * - confMap via ArrowUtils.getPythonRunnerConfMap: SESSION_LOCAL_TIMEZONE + PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME + * Spark 3.0–3.2 characteristics: + * - MapInPandasExec(func, output, child) — 3-arg constructor + * - No MapInBatchExec trait (introduced in 3.3) + * - No PythonMapInArrowExec (introduced in 3.3) + * - No PythonSQLMetrics (introduced in 3.4) + * + * Uses the DataFlintArrowWindowPythonExec_4_1 delegation pattern: + * creates the original MapInPandasExec at runtime via reflection, delegates + * execution to it, and wraps the result RDD with a duration metric. + * This avoids copying any Spark execution logic. */ package org.apache.spark.sql.execution.python -import scala.collection.JavaConverters._ - -import java.util.concurrent.TimeUnit.NANOSECONDS - -import org.apache.spark.{ContextAwareIterator, TaskContext} -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} -import org.apache.spark.api.python.BasePythonRunner +import org.apache.spark.dataflint.DataFlintRDDUtils import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{PythonUDF, _} -import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} /** - * DataFlint instrumented version of MapInBatchExec for Spark 3.0.x. - * Adds a duration metric to measure time spent in Python UDF execution. - * - * Uses reflection to construct ArrowPythonRunner since the 3.0 constructor - * (6-arg) differs from the 3.5 constructor (9-arg) that this code compiles against. + * DataFlint instrumented MapInPandasExec for Spark 3.0.x / 3.1.x / 3.2.x. + * Replaces the original MapInPandasExec in the physical plan. */ -trait DataFlintMapInBatchExec_3_0 extends SparkPlan with Logging { - protected val func: Expression - protected val pythonEvalType: Int +class DataFlintMapInPandasExec_3_0 private ( + val func: Expression, + override val output: Seq[Attribute], + val child: SparkPlan) + extends UnaryExecNode with Logging { - def child: SparkPlan + override def nodeName: String = "DataFlintMapInPandas" - override def children: Seq[SparkPlan] = Seq(child) + logInfo("DataFlint MapInPandas (Spark 3.0) is connected") override def producedAttributes: AttributeSet = AttributeSet(output) - override def outputPartitioning: Partitioning = child.outputPartitioning - // No PythonSQLMetrics in 3.0 — only our custom duration metric - // Note: Cannot use SQLMetrics.createTimingMetric() because it gained a default parameter - // in 3.5 (size: Long = -1L), and the Scala compiler generates a call to the synthetic - // $default$3() method which doesn't exist in 3.0 at runtime. + // Cannot use SQLMetrics.createTimingMetric() — it gained a default parameter in 3.5 + // which generates a $default$3() call that doesn't exist in 3.0–3.4 at runtime. override lazy val metrics: Map[String, SQLMetric] = Map( "duration" -> { val metric = new SQLMetric("timing", -1L) @@ -64,83 +51,44 @@ trait DataFlintMapInBatchExec_3_0 extends SparkPlan with Logging { ) override protected def doExecute(): RDD[InternalRow] = { - logInfo("DataFlint MapInBatchExec (Spark 3.0) doExecute is running") - - val durationMetric = longMetric("duration") - child.execute().mapPartitionsInternal { inputIter => - val startTime = System.nanoTime() - - val pythonFunction = func.asInstanceOf[PythonUDF].func - val argOffsets = Array(Array(0)) - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pythonFunction))) - val sessionLocalTimeZone = conf.sessionLocalTimeZone - val outputTypes = child.schema - val batchSize = conf.arrowMaxRecordsPerBatch - - // Construct confMap manually (replicates ArrowUtils.getPythonRunnerConfMap for 3.0) - val pythonRunnerConf = Map( - SQLConf.SESSION_LOCAL_TIMEZONE.key -> sessionLocalTimeZone, - SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key -> - conf.pandasGroupedMapAssignColumnsByName.toString - ) - - val context = TaskContext.get() - val contextAwareIterator = new ContextAwareIterator(context, inputIter) - val wrappedIter = contextAwareIterator.map(InternalRow(_)) - val batchIter = - if (batchSize > 0) new BatchIterator(wrappedIter, batchSize) else Iterator(wrappedIter) - - // Use reflection: Spark 3.0 ArrowPythonRunner has a 6-arg constructor - val runnerClass = classOf[ArrowPythonRunner] - val ctor = runnerClass.getConstructors - .find(_.getParameterCount == 6) + logInfo("DataFlint MapInPandas (Spark 3.0) doExecute is running") + + val innerRDD: RDD[InternalRow] = try { + val companionClass = Class.forName( + "org.apache.spark.sql.execution.python.MapInPandasExec$") + val companion = companionClass.getField("MODULE$").get(null) + val applyMethod = companion.getClass.getMethods + .find(m => m.getName == "apply" && m.getParameterCount == 3) .getOrElse(throw new RuntimeException( - "Cannot find ArrowPythonRunner 6-arg constructor. Expected Spark 3.0.x runtime.")) - val runner = ctor.newInstance( - chainedFunc.asInstanceOf[AnyRef], - Int.box(pythonEvalType), - argOffsets.asInstanceOf[AnyRef], - StructType(StructField("struct", outputTypes) :: Nil).asInstanceOf[AnyRef], - sessionLocalTimeZone.asInstanceOf[AnyRef], - pythonRunnerConf.asInstanceOf[AnyRef] - ).asInstanceOf[BasePythonRunner[Iterator[InternalRow], ColumnarBatch]] - - val columnarBatchIter = runner.compute(batchIter, context.partitionId(), context) - - val unsafeProj = UnsafeProjection.create(output, output) - - val result = columnarBatchIter.flatMap { batch => - val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] - val outputVectors = output.indices.map(structVector.getChild) - val flattenedBatch = new ColumnarBatch(outputVectors.toArray) - flattenedBatch.setNumRows(batch.numRows()) - flattenedBatch.rowIterator.asScala - }.map(unsafeProj) - - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result + "MapInPandasExec$.apply(3) not found — Spark 3.0.x required")) + val innerExec = applyMethod.invoke(companion, func, output, child).asInstanceOf[SparkPlan] + innerExec.execute() + } catch { + case e: Exception => + logWarning(s"DataFlint: failed to create MapInPandasExec via reflection: ${e.getMessage}") + throw e } - } -} -/** - * DataFlint instrumented MapInPandasExec for Spark 3.0.x. - * Replaces the original MapInPandasExec in the physical plan. - * - * Note: mapInArrow is not available in Spark 3.0 (introduced in 3.3 via SPARK-37227). - */ -case class DataFlintMapInPandasExec_3_0( - func: Expression, - output: Seq[Attribute], - child: SparkPlan) - extends DataFlintMapInBatchExec_3_0 with Logging { - - override def nodeName: String = "DataFlintMapInPandas" - - logInfo("DataFlint MapInPandas (Spark 3.0) is connected") + DataFlintRDDUtils.withDurationMetric(innerRDD, longMetric("duration")) + } - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF + override def canEqual(that: Any): Boolean = that.isInstanceOf[DataFlintMapInPandasExec_3_0] + override def productArity: Int = 3 + override def productElement(n: Int): Any = n match { + case 0 => func + case 1 => output + case 2 => child + case _ => throw new IndexOutOfBoundsException(s"$n") + } - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintMapInPandasExec_3_0 = + new DataFlintMapInPandasExec_3_0(func, output, newChild) } + +object DataFlintMapInPandasExec_3_0 { + def apply( + func: Expression, + output: Seq[Attribute], + child: SparkPlan): DataFlintMapInPandasExec_3_0 = + new DataFlintMapInPandasExec_3_0(func, output, child) +} \ No newline at end of file diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_1.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_1.scala deleted file mode 100644 index 3f2cb6f..0000000 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_1.scala +++ /dev/null @@ -1,148 +0,0 @@ -/* - * DataFlint instrumented MapInBatchExec for Spark 3.1.x - * - * Spark 3.1 characteristics: - * - MapInPandasExec extends UnaryExecNode directly (no MapInBatchExec trait, introduced in 3.3) - * - No PythonMapInArrowExec (introduced in 3.3 via SPARK-37227) - * - No PythonSQLMetrics mixin (introduced in 3.4 via SPARK-34265) - * - No barrier mode support (introduced in 3.5 via SPARK-42896) - * - No MapInBatchEvaluatorFactory (introduced in 3.5 via SPARK-44361) - * - Inline execution with ArrowPythonRunner (6-arg constructor) - * - ArrowPythonRunner constructor: (funcs, evalType, argOffsets, schema, timeZoneId, confMap) - * - MapInPandasExec fields: (func, output, child) — no isBarrier, no profile - * - confMap via ArrowUtils.getPythonRunnerConfMap: SESSION_LOCAL_TIMEZONE + PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME - * - Added simplifiedTraceback on ArrowPythonRunner (SPARK-33407) - */ -package org.apache.spark.sql.execution.python - -import scala.collection.JavaConverters._ - -import java.util.concurrent.TimeUnit.NANOSECONDS - -import org.apache.spark.{ContextAwareIterator, TaskContext} -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} -import org.apache.spark.api.python.BasePythonRunner -import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{PythonUDF, _} -import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} - -/** - * DataFlint instrumented version of MapInBatchExec for Spark 3.1.x. - * Adds a duration metric to measure time spent in Python UDF execution. - * - * Uses reflection to construct ArrowPythonRunner since the 3.1 constructor - * (6-arg) differs from the 3.5 constructor (9-arg) that this code compiles against. - */ -trait DataFlintMapInBatchExec_3_1 extends SparkPlan with Logging { - protected val func: Expression - protected val pythonEvalType: Int - - def child: SparkPlan - - override def children: Seq[SparkPlan] = Seq(child) - - override def producedAttributes: AttributeSet = AttributeSet(output) - - override def outputPartitioning: Partitioning = child.outputPartitioning - - // No PythonSQLMetrics in 3.1 — only our custom duration metric - // Note: Cannot use SQLMetrics.createTimingMetric() because it gained a default parameter - // in 3.5 (size: Long = -1L), and the Scala compiler generates a call to the synthetic - // $default$3() method which doesn't exist in 3.1 at runtime. - override lazy val metrics: Map[String, SQLMetric] = Map( - "duration" -> { - val metric = new SQLMetric("timing", -1L) - metric.register(sparkContext, Some("duration"), false) - metric - } - ) - - override protected def doExecute(): RDD[InternalRow] = { - logInfo("DataFlint MapInBatchExec (Spark 3.1) doExecute is running") - - val durationMetric = longMetric("duration") - - child.execute().mapPartitionsInternal { inputIter => - val startTime = System.nanoTime() - - val pythonFunction = func.asInstanceOf[PythonUDF].func - val argOffsets = Array(Array(0)) - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pythonFunction))) - val sessionLocalTimeZone = conf.sessionLocalTimeZone - val outputTypes = child.schema - val batchSize = conf.arrowMaxRecordsPerBatch - - // Construct confMap manually (replicates ArrowUtils.getPythonRunnerConfMap for 3.1) - val pythonRunnerConf = Map( - SQLConf.SESSION_LOCAL_TIMEZONE.key -> sessionLocalTimeZone, - SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key -> - conf.pandasGroupedMapAssignColumnsByName.toString - ) - - val context = TaskContext.get() - val contextAwareIterator = new ContextAwareIterator(context, inputIter) - val wrappedIter = contextAwareIterator.map(InternalRow(_)) - val batchIter = - if (batchSize > 0) new BatchIterator(wrappedIter, batchSize) else Iterator(wrappedIter) - - // Use reflection: Spark 3.1 ArrowPythonRunner has a 6-arg constructor - val runnerClass = classOf[ArrowPythonRunner] - val ctor = runnerClass.getConstructors - .find(_.getParameterCount == 6) - .getOrElse(throw new RuntimeException( - "Cannot find ArrowPythonRunner 6-arg constructor. Expected Spark 3.1.x runtime.")) - val runner = ctor.newInstance( - chainedFunc.asInstanceOf[AnyRef], - Int.box(pythonEvalType), - argOffsets.asInstanceOf[AnyRef], - StructType(StructField("struct", outputTypes) :: Nil).asInstanceOf[AnyRef], - sessionLocalTimeZone.asInstanceOf[AnyRef], - pythonRunnerConf.asInstanceOf[AnyRef] - ).asInstanceOf[BasePythonRunner[Iterator[InternalRow], ColumnarBatch]] - - val columnarBatchIter = runner.compute(batchIter, context.partitionId(), context) - - val unsafeProj = UnsafeProjection.create(output, output) - - val result = columnarBatchIter.flatMap { batch => - val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] - val outputVectors = output.indices.map(structVector.getChild) - val flattenedBatch = new ColumnarBatch(outputVectors.toArray) - flattenedBatch.setNumRows(batch.numRows()) - flattenedBatch.rowIterator.asScala - }.map(unsafeProj) - - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result - } - } -} - -/** - * DataFlint instrumented MapInPandasExec for Spark 3.1.x. - * Replaces the original MapInPandasExec in the physical plan. - * - * Note: mapInArrow is not available in Spark 3.1 (introduced in 3.3 via SPARK-37227). - */ -case class DataFlintMapInPandasExec_3_1( - func: Expression, - output: Seq[Attribute], - child: SparkPlan) - extends DataFlintMapInBatchExec_3_1 with Logging { - - override def nodeName: String = "DataFlintMapInPandas" - - logInfo("DataFlint MapInPandas (Spark 3.1) is connected") - - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) -} diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_3.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_3.scala index d01270a..d6e5bb3 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_3.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_3.scala @@ -1,58 +1,45 @@ /* - * DataFlint instrumented MapInBatchExec for Spark 3.3.x + * DataFlint instrumented PythonMapInArrowExec for Spark 3.3.x – 3.4.x * - * Spark 3.3 characteristics: - * - MapInBatchExec trait was introduced (extracted from MapInPandasExec) - * - No PythonSQLMetrics (added in 3.4 via SPARK-34265) - * - No barrier mode support (added in 3.5 via SPARK-42896) - * - No MapInBatchEvaluatorFactory (added in 3.5 via SPARK-44361) - * - Inline execution with ArrowPythonRunner (6-arg constructor) - * - ArrowPythonRunner constructor: (funcs, evalType, argOffsets, schema, timeZoneId, confMap) - * - MapInPandasExec fields: (func, output, child) — no isBarrier, no profile + * PythonMapInArrowExec was introduced in Spark 3.3 (SPARK-37227). + * Both 3.3 and 3.4 share the same constructor: PythonMapInArrowExec(func, output, child) — 3 args. + * (isBarrier was added in 3.5) + * + * Uses the delegation-via-reflection pattern — see DataFlintMapInBatchExec_3_0.scala. + * This single class handles 3.3 and 3.4 without version-specific subclasses. + * + * For MapInPandasExec (all 3.0–3.4), see DataFlintMapInBatchExec_3_0.scala. */ package org.apache.spark.sql.execution.python -import scala.collection.JavaConverters._ - -import java.util.concurrent.TimeUnit.NANOSECONDS - -import org.apache.spark.{ContextAwareIterator, TaskContext} -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} -import org.apache.spark.api.python.BasePythonRunner +import org.apache.spark.dataflint.DataFlintRDDUtils import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{PythonUDF, _} -import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} /** - * DataFlint instrumented version of MapInBatchExec for Spark 3.3.x. - * Adds a duration metric to measure time spent in Python UDF execution. - * - * Uses reflection to construct ArrowPythonRunner since the 3.3 constructor - * (6-arg) differs from the 3.5 constructor (9-arg) that this code compiles against. + * DataFlint instrumented PythonMapInArrowExec for Spark 3.3.x – 3.4.x. + * Replaces the original PythonMapInArrowExec in the physical plan. */ -trait DataFlintMapInBatchExec_3_3 extends SparkPlan with Logging { - protected val func: Expression - protected val pythonEvalType: Int +class DataFlintPythonMapInArrowExec_3_3 private ( + val func: Expression, + override val output: Seq[Attribute], + val child: SparkPlan) + extends UnaryExecNode with Logging { - def child: SparkPlan + override def nodeName: String = "DataFlintMapInArrow" - override def children: Seq[SparkPlan] = Seq(child) + logInfo("DataFlint MapInArrow (Spark 3.3–3.4) is connected") override def producedAttributes: AttributeSet = AttributeSet(output) - override def outputPartitioning: Partitioning = child.outputPartitioning - // No PythonSQLMetrics in 3.3 — only our custom duration metric - // Note: Cannot use SQLMetrics.createTimingMetric() because it gained a default parameter - // in 3.5 (size: Long = -1L), and the Scala compiler generates a call to the synthetic - // $default$3() method which doesn't exist in 3.3/3.4 at runtime. + // Cannot use SQLMetrics.createTimingMetric() — it gained a default parameter in 3.5 + // which generates a $default$3() call that doesn't exist in 3.3–3.4 at runtime. override lazy val metrics: Map[String, SQLMetric] = Map( "duration" -> { val metric = new SQLMetric("timing", -1L) @@ -62,102 +49,44 @@ trait DataFlintMapInBatchExec_3_3 extends SparkPlan with Logging { ) override protected def doExecute(): RDD[InternalRow] = { - logInfo("DataFlint MapInBatchExec (Spark 3.3) doExecute is running") - - val durationMetric = longMetric("duration") - - child.execute().mapPartitionsInternal { inputIter => - val startTime = System.nanoTime() - - val pythonFunction = func.asInstanceOf[PythonUDF].func - val argOffsets = Array(Array(0)) - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pythonFunction))) - val sessionLocalTimeZone = conf.sessionLocalTimeZone - val outputTypes = child.schema - val batchSize = conf.arrowMaxRecordsPerBatch - - // Construct confMap manually (replicates ArrowUtils.getPythonRunnerConfMap for 3.3) - val pythonRunnerConf = Map( - SQLConf.SESSION_LOCAL_TIMEZONE.key -> sessionLocalTimeZone, - SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key -> - conf.pandasGroupedMapAssignColumnsByName.toString - ) - - val context = TaskContext.get() - val contextAwareIterator = new ContextAwareIterator(context, inputIter) - val wrappedIter = contextAwareIterator.map(InternalRow(_)) - val batchIter = - if (batchSize > 0) new BatchIterator(wrappedIter, batchSize) else Iterator(wrappedIter) - - // Use reflection: Spark 3.3 ArrowPythonRunner has a 6-arg constructor - val runnerClass = classOf[ArrowPythonRunner] - val ctor = runnerClass.getConstructors - .find(_.getParameterCount == 6) + logInfo("DataFlint MapInArrow (Spark 3.3–3.4) doExecute is running") + + val innerRDD: RDD[InternalRow] = try { + val companionClass = Class.forName( + "org.apache.spark.sql.execution.python.PythonMapInArrowExec$") + val companion = companionClass.getField("MODULE$").get(null) + val applyMethod = companion.getClass.getMethods + .find(m => m.getName == "apply" && m.getParameterCount == 3) .getOrElse(throw new RuntimeException( - "Cannot find ArrowPythonRunner 6-arg constructor. Expected Spark 3.3.x runtime.")) - val runner = ctor.newInstance( - chainedFunc.asInstanceOf[AnyRef], - Int.box(pythonEvalType), - argOffsets.asInstanceOf[AnyRef], - StructType(StructField("struct", outputTypes) :: Nil).asInstanceOf[AnyRef], - sessionLocalTimeZone.asInstanceOf[AnyRef], - pythonRunnerConf.asInstanceOf[AnyRef] - ).asInstanceOf[BasePythonRunner[Iterator[InternalRow], ColumnarBatch]] - - val columnarBatchIter = runner.compute(batchIter, context.partitionId(), context) - - val unsafeProj = UnsafeProjection.create(output, output) - - val result = columnarBatchIter.flatMap { batch => - val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] - val outputVectors = output.indices.map(structVector.getChild) - val flattenedBatch = new ColumnarBatch(outputVectors.toArray) - flattenedBatch.setNumRows(batch.numRows()) - flattenedBatch.rowIterator.asScala - }.map(unsafeProj) - - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result + "PythonMapInArrowExec$.apply(3) not found — Spark 3.3–3.4 required")) + val innerExec = applyMethod.invoke(companion, func, output, child).asInstanceOf[SparkPlan] + innerExec.execute() + } catch { + case e: Exception => + logWarning(s"DataFlint: failed to create PythonMapInArrowExec via reflection: ${e.getMessage}") + throw e } - } -} - -/** - * DataFlint instrumented MapInPandasExec for Spark 3.3.x. - * Replaces the original MapInPandasExec in the physical plan. - */ -case class DataFlintMapInPandasExec_3_3( - func: Expression, - output: Seq[Attribute], - child: SparkPlan) - extends DataFlintMapInBatchExec_3_3 with Logging { - override def nodeName: String = "DataFlintMapInPandas" - - logInfo("DataFlint MapInPandas (Spark 3.3) is connected") + DataFlintRDDUtils.withDurationMetric(innerRDD, longMetric("duration")) + } - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF + override def canEqual(that: Any): Boolean = that.isInstanceOf[DataFlintPythonMapInArrowExec_3_3] + override def productArity: Int = 3 + override def productElement(n: Int): Any = n match { + case 0 => func + case 1 => output + case 2 => child + case _ => throw new IndexOutOfBoundsException(s"$n") + } - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintPythonMapInArrowExec_3_3 = + new DataFlintPythonMapInArrowExec_3_3(func, output, newChild) } -/** - * DataFlint instrumented PythonMapInArrowExec for Spark 3.3.x. - * Replaces the original PythonMapInArrowExec in the physical plan. - */ -case class DataFlintPythonMapInArrowExec_3_3( - func: Expression, - output: Seq[Attribute], - child: SparkPlan) - extends DataFlintMapInBatchExec_3_3 with Logging { - - override def nodeName: String = "DataFlintMapInArrow" - - logInfo("DataFlint MapInArrow (Spark 3.3) is connected") - - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_ARROW_ITER_UDF - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) -} +object DataFlintPythonMapInArrowExec_3_3 { + def apply( + func: Expression, + output: Seq[Attribute], + child: SparkPlan): DataFlintPythonMapInArrowExec_3_3 = + new DataFlintPythonMapInArrowExec_3_3(func, output, child) +} \ No newline at end of file diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_4.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_4.scala deleted file mode 100644 index a04bc40..0000000 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_4.scala +++ /dev/null @@ -1,164 +0,0 @@ -/* - * DataFlint instrumented MapInBatchExec for Spark 3.4.x - * - * Spark 3.4 characteristics: - * - Added PythonSQLMetrics mixin (SPARK-34265) for Python UDF SQL metrics - * - No barrier mode support (added in 3.5) - * - No MapInBatchEvaluatorFactory (added in 3.5) - * - Inline execution with ArrowPythonRunner (7-arg constructor, adds pythonMetrics) - * - ArrowPythonRunner constructor: (funcs, evalType, argOffsets, schema, timeZoneId, workerConf, pythonMetrics) - * - MapInPandasExec fields: (func, output, child) — no isBarrier, no profile - */ -package org.apache.spark.sql.execution.python - -import scala.collection.JavaConverters._ - -import java.util.concurrent.TimeUnit.NANOSECONDS - -import org.apache.spark.{ContextAwareIterator, TaskContext} -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} -import org.apache.spark.api.python.BasePythonRunner -import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{PythonUDF, _} -import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} - -/** - * DataFlint instrumented version of MapInBatchExec for Spark 3.4.x. - * Adds a duration metric alongside the standard PythonSQLMetrics. - * - * Uses reflection to construct ArrowPythonRunner since the 3.4 constructor - * (7-arg) differs from the 3.5 constructor (9-arg) that this code compiles against. - */ -trait DataFlintMapInBatchExec_3_4 extends SparkPlan with PythonSQLMetrics with Logging { - protected val func: Expression - protected val pythonEvalType: Int - - def child: SparkPlan - - override def children: Seq[SparkPlan] = Seq(child) - - override def producedAttributes: AttributeSet = AttributeSet(output) - - override def outputPartitioning: Partitioning = child.outputPartitioning - - // Note: Cannot use SQLMetrics.createTimingMetric() because it gained a default parameter - // in 3.5 (size: Long = -1L), and the Scala compiler generates a call to the synthetic - // $default$3() method which doesn't exist in 3.3/3.4 at runtime. - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "duration" -> { - val metric = new SQLMetric("timing", -1L) - metric.register(sparkContext, Some("duration"), false) - metric - } - ) - - override protected def doExecute(): RDD[InternalRow] = { - logInfo("DataFlint MapInBatchExec (Spark 3.4) doExecute is running") - - val durationMetric = longMetric("duration") - - child.execute().mapPartitionsInternal { inputIter => - val startTime = System.nanoTime() - - val pythonFunction = func.asInstanceOf[PythonUDF].func - val argOffsets = Array(Array(0)) - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pythonFunction))) - val sessionLocalTimeZone = conf.sessionLocalTimeZone - val outputTypes = child.schema - val batchSize = conf.arrowMaxRecordsPerBatch - - // Construct confMap manually (replicates ArrowUtils.getPythonRunnerConfMap for 3.4) - val pythonRunnerConf = Map( - SQLConf.SESSION_LOCAL_TIMEZONE.key -> sessionLocalTimeZone, - SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key -> - conf.pandasGroupedMapAssignColumnsByName.toString, - SQLConf.PANDAS_ARROW_SAFE_TYPE_CONVERSION.key -> - conf.arrowSafeTypeConversion.toString - ) - - val context = TaskContext.get() - val contextAwareIterator = new ContextAwareIterator(context, inputIter) - val wrappedIter = contextAwareIterator.map(InternalRow(_)) - val batchIter = - if (batchSize > 0) new BatchIterator(wrappedIter, batchSize) else Iterator(wrappedIter) - - // Use reflection: Spark 3.4 ArrowPythonRunner has a 7-arg constructor (adds pythonMetrics) - val runnerClass = classOf[ArrowPythonRunner] - val ctor = runnerClass.getConstructors - .find(_.getParameterCount == 7) - .getOrElse(throw new RuntimeException( - "Cannot find ArrowPythonRunner 7-arg constructor. Expected Spark 3.4.x runtime.")) - val runner = ctor.newInstance( - chainedFunc.asInstanceOf[AnyRef], - Int.box(pythonEvalType), - argOffsets.asInstanceOf[AnyRef], - StructType(Array(StructField("struct", outputTypes))).asInstanceOf[AnyRef], - sessionLocalTimeZone.asInstanceOf[AnyRef], - pythonRunnerConf.asInstanceOf[AnyRef], - pythonMetrics.asInstanceOf[AnyRef] - ).asInstanceOf[BasePythonRunner[Iterator[InternalRow], ColumnarBatch]] - - val columnarBatchIter = runner.compute(batchIter, context.partitionId(), context) - - val unsafeProj = UnsafeProjection.create(output, output) - - val result = columnarBatchIter.flatMap { batch => - val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] - val outputVectors = output.indices.map(structVector.getChild) - val flattenedBatch = new ColumnarBatch(outputVectors.toArray) - flattenedBatch.setNumRows(batch.numRows()) - flattenedBatch.rowIterator.asScala - }.map(unsafeProj) - - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result - } - } -} - -/** - * DataFlint instrumented MapInPandasExec for Spark 3.4.x. - * Replaces the original MapInPandasExec in the physical plan. - */ -case class DataFlintMapInPandasExec_3_4( - func: Expression, - output: Seq[Attribute], - child: SparkPlan) - extends DataFlintMapInBatchExec_3_4 with Logging { - - override def nodeName: String = "DataFlintMapInPandas" - - logInfo("DataFlint MapInPandas (Spark 3.4) is connected") - - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) -} - -/** - * DataFlint instrumented PythonMapInArrowExec for Spark 3.4.x. - * Replaces the original PythonMapInArrowExec in the physical plan. - */ -case class DataFlintPythonMapInArrowExec_3_4( - func: Expression, - output: Seq[Attribute], - child: SparkPlan) - extends DataFlintMapInBatchExec_3_4 with Logging { - - override def nodeName: String = "DataFlintMapInArrow" - - logInfo("DataFlint MapInArrow (Spark 3.4) is connected") - - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_ARROW_ITER_UDF - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) -} diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_5.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_5.scala index f82ac64..89bf87b 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_5.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_5.scala @@ -8,135 +8,102 @@ * - JobArtifactSet for Spark Connect artifact management * - arrowUseLargeVarTypes config (SPARK-39979) * - ArrowPythonRunner.getPythonRunnerConfMap replaces ArrowUtils.getPythonRunnerConfMap (SPARK-44532) - * - ArrowPythonRunner constructor: 9-arg (adds largeVarTypes, pythonMetrics, jobArtifactUUID) * - MapInPandasExec fields: (func, output, child, isBarrier) + * - PythonMapInArrowExec fields: (func, output, child, isBarrier) + * + * Both classes extend the original exec nodes directly and wrap doExecute() with + * DataFlintRDDUtils.withDurationMetric, delegating all execution logic to Spark. + * This is safe on 3.5 since the constructor signatures match the compile-time artifacts. + * On older Spark versions these classes are never instantiated (version-guarded by the extension). */ package org.apache.spark.sql.execution.python -import java.util.concurrent.TimeUnit.NANOSECONDS - -import org.apache.spark.JobArtifactSet -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} +import org.apache.spark.dataflint.DataFlintRDDUtils import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{PythonUDF, _} -import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -/** - * DataFlint instrumented version of MapInBatchExec for Spark 3.5.x. - * Adds a duration metric alongside PythonSQLMetrics. - * - * Uses MapInBatchEvaluatorFactory directly (introduced in 3.5). - */ -trait DataFlintMapInBatchExec_3_5 extends SparkPlan with PythonSQLMetrics with Logging { - protected val func: Expression - protected val pythonEvalType: Int - protected val isBarrier: Boolean - - def child: SparkPlan - - override def children: Seq[SparkPlan] = Seq(child) - - override def producedAttributes: AttributeSet = AttributeSet(output) - - private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid) - - override def outputPartitioning: Partitioning = child.outputPartitioning - - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") - ) - - override protected def doExecute(): RDD[InternalRow] = { - logInfo("DataFlint MapInBatchExec (Spark 3.5) doExecute is running") - - val pythonRunnerConf = ArrowPythonRunner.getPythonRunnerConfMap(conf) - val pythonFunction = func.asInstanceOf[PythonUDF].func - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pythonFunction))) - - val evaluatorFactory = new MapInBatchEvaluatorFactory( - output, - chainedFunc, - child.schema, - conf.arrowMaxRecordsPerBatch, - pythonEvalType, - conf.sessionLocalTimeZone, - conf.arrowUseLargeVarTypes, - pythonRunnerConf, - pythonMetrics, - jobArtifactUUID) - - val durationMetric = longMetric("duration") - - if (isBarrier) { - val rddBarrier = child.execute().barrier() - if (conf.usePartitionEvaluator) { - rddBarrier.mapPartitionsWithEvaluator(evaluatorFactory) - } else { - rddBarrier.mapPartitionsWithIndex { (index, iter) => - val startTime = System.nanoTime() - val result = evaluatorFactory.createEvaluator().eval(index, iter) - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result - } - } - } else { - val inputRdd = child.execute() - if (conf.usePartitionEvaluator) { - inputRdd.mapPartitionsWithEvaluator(evaluatorFactory) - } else { - inputRdd.mapPartitionsWithIndexInternal { (index, iter) => - val startTime = System.nanoTime() - val result = evaluatorFactory.createEvaluator().eval(index, iter) - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result - } - } - } - } -} - /** * DataFlint instrumented MapInPandasExec for Spark 3.5.x. * Replaces the original MapInPandasExec in the physical plan. */ -case class DataFlintMapInPandasExec_3_5( +class DataFlintMapInPandasExec_3_5 private ( func: Expression, output: Seq[Attribute], child: SparkPlan, - override val isBarrier: Boolean) - extends DataFlintMapInBatchExec_3_5 with Logging { + isBarrier: Boolean) + extends MapInPandasExec(func, output, child, isBarrier) with Logging { override def nodeName: String = "DataFlintMapInPandas" logInfo("DataFlint MapInPandas (Spark 3.5) is connected") - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintMapInPandasExec_3_5 = + DataFlintMapInPandasExec_3_5(func, output, newChild, isBarrier) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintMapInPandasExec_3_5] + override def equals(other: Any): Boolean = other.isInstanceOf[DataFlintMapInPandasExec_3_5] && super.equals(other) + override def hashCode: Int = super.hashCode +} - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) +object DataFlintMapInPandasExec_3_5 { + def apply( + func: Expression, + output: Seq[Attribute], + child: SparkPlan, + isBarrier: Boolean): DataFlintMapInPandasExec_3_5 = + new DataFlintMapInPandasExec_3_5(func, output, child, isBarrier) } /** * DataFlint instrumented PythonMapInArrowExec for Spark 3.5.x. * Replaces the original PythonMapInArrowExec in the physical plan. + * + * Note: PythonMapInArrowExec was introduced in Spark 3.3 (SPARK-37227). + * This class is only loaded/instantiated when the Spark version is 3.5+, + * guarded by the try-catch(NoClassDefFoundError) in DataFlintInstrumentationExtension. */ -case class DataFlintPythonMapInArrowExec_3_5( +class DataFlintPythonMapInArrowExec_3_5 private ( func: Expression, output: Seq[Attribute], child: SparkPlan, - override val isBarrier: Boolean) - extends DataFlintMapInBatchExec_3_5 with Logging { + isBarrier: Boolean) + extends PythonMapInArrowExec(func, output, child, isBarrier) with Logging { override def nodeName: String = "DataFlintMapInArrow" logInfo("DataFlint MapInArrow (Spark 3.5) is connected") - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_ARROW_ITER_UDF + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintPythonMapInArrowExec_3_5 = + DataFlintPythonMapInArrowExec_3_5(func, output, newChild, isBarrier) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintPythonMapInArrowExec_3_5] + override def equals(other: Any): Boolean = other.isInstanceOf[DataFlintPythonMapInArrowExec_3_5] && super.equals(other) + override def hashCode: Int = super.hashCode +} - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) +object DataFlintPythonMapInArrowExec_3_5 { + def apply( + func: Expression, + output: Seq[Attribute], + child: SparkPlan, + isBarrier: Boolean): DataFlintPythonMapInArrowExec_3_5 = + new DataFlintPythonMapInArrowExec_3_5(func, output, child, isBarrier) } diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec.scala deleted file mode 100644 index 14c3257..0000000 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec.scala +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.execution.python - -import org.apache.spark.JobArtifactSet -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} -import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{PythonUDF, _} -import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} - -import java.util.concurrent.TimeUnit.NANOSECONDS - -/** - * DataFlint instrumented version of MapInPandasExec for Spark 3.x. - * - * A relation produced by applying a function that takes an iterator of pandas DataFrames - * and outputs an iterator of pandas DataFrames. Adds a duration metric. - */ -case class DataFlintMapInPandasExec( - func: Expression, - output: Seq[Attribute], - child: SparkPlan, - override val isBarrier: Boolean) - extends DataFlintMapInBatchExec with Logging { - - logInfo("DataFlint MapInPandas is connected") - - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) -} - -/** - * DataFlint instrumented version of MapInBatchExec trait for Spark 3.x. - * - * Adds a duration metric to measure the time spent in Python UDF execution. - */ -trait DataFlintMapInBatchExec extends SparkPlan with PythonSQLMetrics with Logging { - protected val func: Expression - protected val pythonEvalType: Int - protected val isBarrier: Boolean - - def child: SparkPlan - - override def children: Seq[SparkPlan] = Seq(child) - - override def producedAttributes: AttributeSet = AttributeSet(output) - - private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid) - - override def outputPartitioning: Partitioning = child.outputPartitioning - - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") - ) - - override protected def doExecute(): RDD[InternalRow] = { - logInfo("DataFlint MapInBatchExec doExecute is running") - - val pythonRunnerConf = ArrowPythonRunner.getPythonRunnerConfMap(conf) - val pythonFunction = func.asInstanceOf[PythonUDF].func - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pythonFunction))) - - val evaluatorFactory = new MapInBatchEvaluatorFactory( - output, - chainedFunc, - child.schema, - conf.arrowMaxRecordsPerBatch, - pythonEvalType, - conf.sessionLocalTimeZone, - conf.arrowUseLargeVarTypes, - pythonRunnerConf, - pythonMetrics, - jobArtifactUUID) - - val durationMetric = longMetric("duration") - - if (isBarrier) { - val rddBarrier = child.execute().barrier() - if (conf.usePartitionEvaluator) { - rddBarrier.mapPartitionsWithEvaluator(evaluatorFactory) - } else { - rddBarrier.mapPartitionsWithIndex { (index, iter) => - val startTime = System.nanoTime() - val result = evaluatorFactory.createEvaluator().eval(index, iter).toList - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result.iterator - } - } - } else { - val inputRdd = child.execute() - if (conf.usePartitionEvaluator) { - inputRdd.mapPartitionsWithEvaluator(evaluatorFactory) - } else { - inputRdd.mapPartitionsWithIndexInternal { (index, iter) => - val startTime = System.nanoTime() - val result = evaluatorFactory.createEvaluator().eval(index, iter).toList - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result.iterator - } - } - } - } -} - -object DataFlintMapInPandasExec { - def apply( - func: Expression, - output: Seq[Attribute], - child: SparkPlan, - isBarrier: Boolean): DataFlintMapInPandasExec = { - new DataFlintMapInPandasExec(func, output, child, isBarrier) - } -} diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala index c11b609..2bfe1e5 100644 --- a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala @@ -10,7 +10,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Window => Logic import org.apache.spark.sql.execution.SparkStrategy import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintArrowWindowPythonExec_4_1, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_4_0, DataFlintMapInPandasExec_4_1, DataFlintPythonMapInArrowExec_4_0, DataFlintPythonMapInArrowExec_4_1, DataFlintWindowInPandasExec_4_0, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInArrowExec, MapInPandasExec} +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintArrowWindowPythonExec_4_1, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInArrowExec, DataFlintMapInPandasExec, DataFlintWindowInPandasExec_4_0, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInArrowExec, MapInPandasExec} import org.apache.spark.sql.execution.window.DataFlintWindowExec /** @@ -109,46 +109,24 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C if (mapInPandasEnabled || mapInArrowEnabled) { result = result.transformUp { - case mapInPandas: MapInPandasExec if mapInPandasEnabled => + case mapInPandas: MapInPandasExec if mapInPandasEnabled && !mapInPandas.isInstanceOf[DataFlintMapInPandasExec] => logInfo(s"Replacing MapInPandasExec with DataFlint version for Spark $sparkMinorVersion") - sparkMinorVersion match { - case "4.0" => - DataFlintMapInPandasExec_4_0( - func = mapInPandas.func, - output = mapInPandas.output, - child = mapInPandas.child, - isBarrier = mapInPandas.isBarrier, - profile = mapInPandas.profile - ) - case _ => - DataFlintMapInPandasExec_4_1( - func = mapInPandas.func, - output = mapInPandas.output, - child = mapInPandas.child, - isBarrier = mapInPandas.isBarrier, - profile = mapInPandas.profile - ) - } - case mapInArrow: MapInArrowExec if mapInArrowEnabled => + DataFlintMapInPandasExec( + func = mapInPandas.func, + output = mapInPandas.output, + child = mapInPandas.child, + isBarrier = mapInPandas.isBarrier, + profile = mapInPandas.profile + ) + case mapInArrow: MapInArrowExec if mapInArrowEnabled && !mapInArrow.isInstanceOf[DataFlintMapInArrowExec] => logInfo(s"Replacing MapInArrowExec with DataFlint version for Spark $sparkMinorVersion") - sparkMinorVersion match { - case "4.0" => - DataFlintPythonMapInArrowExec_4_0( - func = mapInArrow.func, - output = mapInArrow.output, - child = mapInArrow.child, - isBarrier = mapInArrow.isBarrier, - profile = mapInArrow.profile - ) - case _ => - DataFlintPythonMapInArrowExec_4_1( - func = mapInArrow.func, - output = mapInArrow.output, - child = mapInArrow.child, - isBarrier = mapInArrow.isBarrier, - profile = mapInArrow.profile - ) - } + DataFlintMapInArrowExec( + func = mapInArrow.func, + output = mapInArrow.output, + child = mapInArrow.child, + isBarrier = mapInArrow.isBarrier, + profile = mapInArrow.profile + ) } } diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec.scala new file mode 100644 index 0000000..a4325db --- /dev/null +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.resource.ResourceProfile +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * DataFlint instrumented version of MapInArrowExec for Spark 4.x. + * + * Instruments map_in_arrow UDF operations with a duration metric by wrapping the + * parent's doExecute() RDD. Works for both Spark 4.0 and 4.1 — super.doExecute() + * dispatches to the runtime jar's implementation via JVM invokespecial semantics, + * so no version-specific subclasses are needed. + */ +class DataFlintMapInArrowExec private ( + func: Expression, + output: Seq[Attribute], + child: SparkPlan, + isBarrier: Boolean, + profile: Option[ResourceProfile]) + extends MapInArrowExec(func, output, child, isBarrier, profile) with Logging { + + override def nodeName: String = "DataFlintMapInArrow" + + logInfo("DataFlint MapInArrow is connected") + + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintMapInArrowExec = + DataFlintMapInArrowExec(func, output, newChild, isBarrier, profile) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintMapInArrowExec] + override def equals(other: Any): Boolean = other.isInstanceOf[DataFlintMapInArrowExec] && super.equals(other) + override def hashCode: Int = super.hashCode +} + +object DataFlintMapInArrowExec { + def apply( + func: Expression, + output: Seq[Attribute], + child: SparkPlan, + isBarrier: Boolean, + profile: Option[ResourceProfile]): DataFlintMapInArrowExec = + new DataFlintMapInArrowExec(func, output, child, isBarrier, profile) +} diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_4_0.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_4_0.scala deleted file mode 100644 index 2410129..0000000 --- a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_4_0.scala +++ /dev/null @@ -1,148 +0,0 @@ -/* - * DataFlint instrumented MapInBatchExec for Spark 4.0.x - * - * Spark 4.0 characteristics: - * - ResourceProfile support (SPARK-46812) — profile field on MapInPandasExec - * - resultId tracking: chainedFunc becomes Seq[(ChainedPythonFunctions, Long)] - * - ArrowPythonRunner now 10-arg constructor (adds profiler: Option[String]) - * - MapInBatchEvaluatorFactory chainedFunc type changed to Seq[(ChainedPythonFunctions, Long)] - * - MapInPandasExec fields: (func, output, child, isBarrier, profile) - */ -package org.apache.spark.sql.execution.python - -import java.util.concurrent.TimeUnit.NANOSECONDS - -import org.apache.spark.JobArtifactSet -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} -import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.resource.ResourceProfile -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{PythonUDF, _} -import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} - -/** - * DataFlint instrumented version of MapInBatchExec for Spark 4.0.x. - * Adds a duration metric alongside PythonSQLMetrics. - * - * Key 4.0 change: chainedFunc includes resultId for metrics/profiling correlation, - * and ResourceProfile support allows custom resource allocation for Python UDFs. - */ -trait DataFlintMapInBatchExec_4_0 extends SparkPlan with PythonSQLMetrics with Logging { - protected val func: Expression - protected val pythonEvalType: Int - protected val isBarrier: Boolean - protected val profile: Option[ResourceProfile] - - def child: SparkPlan - - override def children: Seq[SparkPlan] = Seq(child) - - override def producedAttributes: AttributeSet = AttributeSet(output) - - private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid) - - override def outputPartitioning: Partitioning = child.outputPartitioning - - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") - ) - - override protected def doExecute(): RDD[InternalRow] = { - logInfo("DataFlint MapInBatchExec (Spark 4.0) doExecute is running") - - val pythonRunnerConf = ArrowPythonRunner.getPythonRunnerConfMap(conf) - val pythonUDF = func.asInstanceOf[PythonUDF] - val pythonFunction = pythonUDF.func - // 4.0 change: chainedFunc includes resultId for profiling correlation - val chainedFunc = Seq((ChainedPythonFunctions(Seq(pythonFunction)), pythonUDF.resultId.id)) - - val evaluatorFactory = new MapInBatchEvaluatorFactory( - output, - chainedFunc, - child.schema, - conf.arrowMaxRecordsPerBatch, - pythonEvalType, - conf.sessionLocalTimeZone, - conf.arrowUseLargeVarTypes, - pythonRunnerConf, - pythonMetrics, - jobArtifactUUID) - - val durationMetric = longMetric("duration") - - val rdd = if (isBarrier) { - val rddBarrier = child.execute().barrier() - if (conf.usePartitionEvaluator) { - rddBarrier.mapPartitionsWithEvaluator(evaluatorFactory) - } else { - rddBarrier.mapPartitionsWithIndex { (index, iter) => - val startTime = System.nanoTime() - val result = evaluatorFactory.createEvaluator().eval(index, iter) - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result - } - } - } else { - val inputRdd = child.execute() - if (conf.usePartitionEvaluator) { - inputRdd.mapPartitionsWithEvaluator(evaluatorFactory) - } else { - inputRdd.mapPartitionsWithIndexInternal { (index, iter) => - val startTime = System.nanoTime() - val result = evaluatorFactory.createEvaluator().eval(index, iter) - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result - } - } - } - // 4.0 change: apply ResourceProfile if specified - profile.map(rp => rdd.withResources(rp)).getOrElse(rdd) - } -} - -/** - * DataFlint instrumented MapInPandasExec for Spark 4.0.x. - * Replaces the original MapInPandasExec in the physical plan. - */ -case class DataFlintMapInPandasExec_4_0( - func: Expression, - output: Seq[Attribute], - child: SparkPlan, - override val isBarrier: Boolean, - override val profile: Option[ResourceProfile]) - extends DataFlintMapInBatchExec_4_0 with Logging { - - override def nodeName: String = "DataFlintMapInPandas" - - logInfo("DataFlint MapInPandas (Spark 4.0) is connected") - - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) -} - -/** - * DataFlint instrumented PythonMapInArrowExec for Spark 4.0.x. - * Replaces the original PythonMapInArrowExec in the physical plan. - */ -case class DataFlintPythonMapInArrowExec_4_0( - func: Expression, - output: Seq[Attribute], - child: SparkPlan, - override val isBarrier: Boolean, - override val profile: Option[ResourceProfile]) - extends DataFlintMapInBatchExec_4_0 with Logging { - - override def nodeName: String = "DataFlintMapInArrow" - - logInfo("DataFlint MapInArrow (Spark 4.0) is connected") - - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_ARROW_ITER_UDF - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) -} diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_4_1.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_4_1.scala deleted file mode 100644 index e1622f1..0000000 --- a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_4_1.scala +++ /dev/null @@ -1,213 +0,0 @@ -/* - * DataFlint instrumented MapInBatchExec for Spark 4.1.x - * - * Spark 4.1 characteristics: - * - Python worker logging support (SPARK-53976) — sessionUUID for log correlation - * - Python UDF profiler support (SPARK-54153, SPARK-54559) — conf.pythonUDFProfiler - * - Arrow schema validation (SPARK-51739) — pythonUDF.dataType passed as outputSchema - * - MapInBatchEvaluatorFactory now 13-arg constructor (adds outputSchema, sessionUUID, profiler) - * - ArrowPythonRunner now 11-arg constructor (adds sessionUUID) - * - MapInPandasExec fields: (func, output, child, isBarrier, profile) — same as 4.0 - * - * Since this code compiles against Spark 4.0.1, reflection is used for the - * MapInBatchEvaluatorFactory constructor which changed between 4.0 and 4.1. - */ -package org.apache.spark.sql.execution.python - -import java.util.concurrent.TimeUnit.NANOSECONDS - -import org.apache.spark.{JobArtifactSet, PartitionEvaluatorFactory} -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} -import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.resource.ResourceProfile -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{PythonUDF, _} -import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} - -/** - * DataFlint instrumented version of MapInBatchExec for Spark 4.1.x. - * Adds a duration metric alongside PythonSQLMetrics. - * - * Key 4.1 changes over 4.0: - * - sessionUUID passed for Python worker log correlation - * - pythonUDFProfiler config for UDF profiling - * - outputSchema (pythonUDF.dataType) for Arrow schema validation - * - * Uses reflection for MapInBatchEvaluatorFactory since the 4.1 constructor (13-arg) - * differs from the 4.0 constructor (10-arg) that this code compiles against. - */ -trait DataFlintMapInBatchExec_4_1 extends SparkPlan with PythonSQLMetrics with Logging { - protected val func: Expression - protected val pythonEvalType: Int - protected val isBarrier: Boolean - protected val profile: Option[ResourceProfile] - - def child: SparkPlan - - override def children: Seq[SparkPlan] = Seq(child) - - override def producedAttributes: AttributeSet = AttributeSet(output) - - private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid) - - // Get sessionUUID via reflection (4.1-specific API: session.sessionUUID) - private[this] val sessionUUID: Option[String] = { - try { - val sess = session - if (sess != null) { - val sessionConf = sess.sessionState.conf - val loggingEnabled = sessionConf.getClass - .getMethod("pythonWorkerLoggingEnabled") - .invoke(sessionConf).asInstanceOf[Boolean] - if (loggingEnabled) { - Some(sess.getClass.getMethod("sessionUUID").invoke(sess).asInstanceOf[String]) - } else None - } else None - } catch { - case _: Throwable => None - } - } - - // Get pythonUDFProfiler via reflection (4.1-specific config) - private[this] val pythonUDFProfiler: Option[String] = { - try { - conf.getClass.getMethod("pythonUDFProfiler").invoke(conf) match { - case s: Option[_] => s.map(_.toString) - case _ => None - } - } catch { - case _: Throwable => None - } - } - - override def outputPartitioning: Partitioning = child.outputPartitioning - - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") - ) - - override protected def doExecute(): RDD[InternalRow] = { - logInfo("DataFlint MapInBatchExec (Spark 4.1) doExecute is running") - - val pythonRunnerConf = ArrowPythonRunner.getPythonRunnerConfMap(conf) - val pythonUDF = func.asInstanceOf[PythonUDF] - val pythonFunction = pythonUDF.func - val chainedFunc = Seq((ChainedPythonFunctions(Seq(pythonFunction)), pythonUDF.resultId.id)) - - // Use reflection for MapInBatchEvaluatorFactory: 4.1 has 13-arg constructor - // vs 4.0's 10-arg constructor - val evaluatorFactory = createEvaluatorFactory( - chainedFunc, pythonUDF, pythonRunnerConf) - - val durationMetric = longMetric("duration") - - val rdd = if (isBarrier) { - val rddBarrier = child.execute().barrier() - if (conf.usePartitionEvaluator) { - rddBarrier.mapPartitionsWithEvaluator(evaluatorFactory) - } else { - rddBarrier.mapPartitionsWithIndex { (index, iter) => - val startTime = System.nanoTime() - val result = evaluatorFactory.createEvaluator().eval(index, iter) - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result - } - } - } else { - val inputRdd = child.execute() - if (conf.usePartitionEvaluator) { - inputRdd.mapPartitionsWithEvaluator(evaluatorFactory) - } else { - inputRdd.mapPartitionsWithIndexInternal { (index, iter) => - val startTime = System.nanoTime() - val result = evaluatorFactory.createEvaluator().eval(index, iter) - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result - } - } - } - profile.map(rp => rdd.withResources(rp)).getOrElse(rdd) - } - - /** - * Create MapInBatchEvaluatorFactory via reflection for Spark 4.1's 13-arg constructor. - * - * 4.1 constructor: (output, chainedFunc, inputSchema, outputSchema, batchSize, - * pythonEvalType, sessionLocalTimeZone, largeVarTypes, pythonRunnerConf, - * pythonMetrics, jobArtifactUUID, sessionUUID, profiler) - */ - private def createEvaluatorFactory( - chainedFunc: Seq[(ChainedPythonFunctions, Long)], - pythonUDF: PythonUDF, - pythonRunnerConf: Map[String, String] - ): PartitionEvaluatorFactory[InternalRow, InternalRow] = { - val clazz = classOf[MapInBatchEvaluatorFactory] - val ctor = clazz.getConstructors - .find(_.getParameterCount == 13) - .getOrElse(throw new RuntimeException( - "Cannot find MapInBatchEvaluatorFactory 13-arg constructor. Expected Spark 4.1.x runtime.")) - - ctor.newInstance( - output.asInstanceOf[AnyRef], - chainedFunc.asInstanceOf[AnyRef], - child.schema.asInstanceOf[AnyRef], - pythonUDF.dataType.asInstanceOf[AnyRef], // outputSchema (4.1 addition) - Int.box(conf.arrowMaxRecordsPerBatch), - Int.box(pythonEvalType), - conf.sessionLocalTimeZone.asInstanceOf[AnyRef], - Boolean.box(conf.arrowUseLargeVarTypes), - pythonRunnerConf.asInstanceOf[AnyRef], - pythonMetrics.asInstanceOf[AnyRef], - jobArtifactUUID.asInstanceOf[AnyRef], - sessionUUID.asInstanceOf[AnyRef], // sessionUUID (4.1 addition) - pythonUDFProfiler.asInstanceOf[AnyRef] // profiler (4.1 addition) - ).asInstanceOf[PartitionEvaluatorFactory[InternalRow, InternalRow]] - } -} - -/** - * DataFlint instrumented MapInPandasExec for Spark 4.1.x. - * Replaces the original MapInPandasExec in the physical plan. - */ -case class DataFlintMapInPandasExec_4_1( - func: Expression, - output: Seq[Attribute], - child: SparkPlan, - override val isBarrier: Boolean, - override val profile: Option[ResourceProfile]) - extends DataFlintMapInBatchExec_4_1 with Logging { - - override def nodeName: String = "DataFlintMapInPandas" - - logInfo("DataFlint MapInPandas (Spark 4.1) is connected") - - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) -} - -/** - * DataFlint instrumented PythonMapInArrowExec for Spark 4.1.x. - * Replaces the original PythonMapInArrowExec in the physical plan. - */ -case class DataFlintPythonMapInArrowExec_4_1( - func: Expression, - output: Seq[Attribute], - child: SparkPlan, - override val isBarrier: Boolean, - override val profile: Option[ResourceProfile]) - extends DataFlintMapInBatchExec_4_1 with Logging { - - override def nodeName: String = "DataFlintMapInArrow" - - logInfo("DataFlint MapInArrow (Spark 4.1) is connected") - - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_ARROW_ITER_UDF - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) -} diff --git a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec.scala b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec.scala index e9b0d39..8e084c1 100644 --- a/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec.scala +++ b/spark-plugin/pluginspark4/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec.scala @@ -16,115 +16,48 @@ */ package org.apache.spark.sql.execution.python -import org.apache.spark.JobArtifactSet -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} +import org.apache.spark.dataflint.DataFlintRDDUtils import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.resource.ResourceProfile import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{PythonUDF, _} -import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import java.util.concurrent.TimeUnit.NANOSECONDS - /** * DataFlint instrumented version of MapInPandasExec for Spark 4.x. * - * A relation produced by applying a function that takes an iterator of pandas DataFrames - * and outputs an iterator of pandas DataFrames. Adds a duration metric. + * Instruments map_in_pandas UDF operations with a duration metric by wrapping the + * parent's doExecute() RDD. Works for both Spark 4.0 and 4.1 — super.doExecute() + * dispatches to the runtime jar's implementation via JVM invokespecial semantics, + * so no version-specific subclasses are needed. */ -case class DataFlintMapInPandasExec( +class DataFlintMapInPandasExec private ( func: Expression, output: Seq[Attribute], child: SparkPlan, - override val isBarrier: Boolean, - override val profile: Option[ResourceProfile]) - extends DataFlintMapInBatchExec with Logging { - - logInfo("DataFlint MapInPandas is connected") - - override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = - copy(child = newChildren.head) -} - -/** - * DataFlint instrumented version of MapInBatchExec trait for Spark 4.x. - * - * Adds a duration metric to measure the time spent in Python UDF execution. - */ -trait DataFlintMapInBatchExec extends SparkPlan with PythonSQLMetrics with Logging { - protected val func: Expression - protected val pythonEvalType: Int - protected val isBarrier: Boolean - protected val profile: Option[ResourceProfile] - - def child: SparkPlan - - override def children: Seq[SparkPlan] = Seq(child) + isBarrier: Boolean, + profile: Option[ResourceProfile]) + extends MapInPandasExec(func, output, child, isBarrier, profile) with Logging { - override def producedAttributes: AttributeSet = AttributeSet(output) + override def nodeName: String = "DataFlintMapInPandas" - private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid) - - override def outputPartitioning: Partitioning = child.outputPartitioning + logInfo("DataFlint MapInPandas is connected") override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") ) - override protected def doExecute(): RDD[InternalRow] = { - logInfo("DataFlint MapInBatchExec doExecute is running") - - val pythonRunnerConf = ArrowPythonRunner.getPythonRunnerConfMap(conf) - val pythonUDF = func.asInstanceOf[PythonUDF] - val pythonFunction = pythonUDF.func - val chainedFunc = Seq((ChainedPythonFunctions(Seq(pythonFunction)), pythonUDF.resultId.id)) - - val evaluatorFactory = new MapInBatchEvaluatorFactory( - output, - chainedFunc, - child.schema, - conf.arrowMaxRecordsPerBatch, - pythonEvalType, - conf.sessionLocalTimeZone, - conf.arrowUseLargeVarTypes, - pythonRunnerConf, - pythonMetrics, - jobArtifactUUID) + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) - val durationMetric = longMetric("duration") + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintMapInPandasExec = + DataFlintMapInPandasExec(func, output, newChild, isBarrier, profile) - val rdd = if (isBarrier) { - val rddBarrier = child.execute().barrier() - if (conf.usePartitionEvaluator) { - rddBarrier.mapPartitionsWithEvaluator(evaluatorFactory) - } else { - rddBarrier.mapPartitionsWithIndex { (index, iter) => - val startTime = System.nanoTime() - val result = evaluatorFactory.createEvaluator().eval(index, iter).toList - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result.iterator - } - } - } else { - val inputRdd = child.execute() - if (conf.usePartitionEvaluator) { - inputRdd.mapPartitionsWithEvaluator(evaluatorFactory) - } else { - inputRdd.mapPartitionsWithIndexInternal { (index, iter) => - val startTime = System.nanoTime() - val result = evaluatorFactory.createEvaluator().eval(index, iter).toList - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - result.iterator - } - } - } - profile.map(rp => rdd.withResources(rp)).getOrElse(rdd) - } + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintMapInPandasExec] + override def equals(other: Any): Boolean = other.isInstanceOf[DataFlintMapInPandasExec] && super.equals(other) + override def hashCode: Int = super.hashCode } object DataFlintMapInPandasExec { @@ -133,7 +66,6 @@ object DataFlintMapInPandasExec { output: Seq[Attribute], child: SparkPlan, isBarrier: Boolean, - profile: Option[ResourceProfile]): DataFlintMapInPandasExec = { + profile: Option[ResourceProfile]): DataFlintMapInPandasExec = new DataFlintMapInPandasExec(func, output, child, isBarrier, profile) - } } From 7d5cd4d1e589d1caa7bf096cf31bfb56cb46de31 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Sun, 15 Mar 2026 17:12:48 +0200 Subject: [PATCH 06/15] using a simple internal member to overcome override of metrics --- .../python/DataFlintArrowEvalPythonExec.scala | 26 ++++++++++--------- .../python/DataFlintBatchEvalPythonExec.scala | 26 ++++++++++--------- ...DataFlintFlatMapCoGroupsInPandasExec.scala | 14 ++++++++-- .../DataFlintFlatMapGroupsInPandasExec.scala | 14 ++++++++-- .../python/DataFlintWindowInPandasExec.scala | 18 ++++++++++--- .../window/DataFlintWindowExec.scala | 16 +++++++++--- 6 files changed, 79 insertions(+), 35 deletions(-) diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala index d9e8e40..8f571a5 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala @@ -46,20 +46,22 @@ class DataFlintArrowEvalPythonExec private( override def nodeName: String = "DataFlintArrowEvalPython" - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") - ) + // Cannot use super.metrics in a lazy val override — Scala 2 does not generate super + // accessors for trait lazy vals (PythonSQLMetrics). Use a sibling instance instead. + // Cannot use SQLMetrics.createTimingMetric() — it gained a default parameter in 3.5 + // which generates a $default$3() call that doesn't exist in 3.0–3.4 at runtime. + private val internal = ArrowEvalPythonExec(udfs, resultAttrs, child, evalType) - override protected def evaluate(funcs: Seq[ChainedPythonFunctions], argOffsets: Array[Array[Int]], iter: Iterator[InternalRow], schema: StructType, context: TaskContext): Iterator[InternalRow] = { - val durationMetric = longMetric("duration") - val startTime = System.nanoTime() - val out = super.evaluate(funcs, argOffsets, iter, schema, context) - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - out - } + override lazy val metrics: Map[String, SQLMetric] = internal.metrics ++ Map( + "duration" -> { + val metric = new SQLMetric("timing", -1L) + metric.register(sparkContext, Some("duration"), false) + metric + } + ) - // override protected def doExecute(): RDD[InternalRow] = -// DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) override protected def withNewChildInternal(newChild: SparkPlan): DataFlintArrowEvalPythonExec = DataFlintArrowEvalPythonExec(udfs, resultAttrs, newChild, evalType) diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala index 7716c19..8ffee3b 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala @@ -28,20 +28,22 @@ class DataFlintBatchEvalPythonExec private ( override def nodeName: String = "DataFlintBatchEvalPython" - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") - ) + // Cannot use super.metrics in a lazy val override — Scala 2 does not generate super + // accessors for trait lazy vals (PythonSQLMetrics). Use a sibling instance instead. + // Cannot use SQLMetrics.createTimingMetric() — it gained a default parameter in 3.5 + // which generates a $default$3() call that doesn't exist in 3.0–3.4 at runtime. + private val internal = BatchEvalPythonExec(udfs, resultAttrs, child) - override protected def evaluate(funcs: Seq[ChainedPythonFunctions], argOffsets: Array[Array[Int]], iter: Iterator[InternalRow], schema: StructType, context: TaskContext): Iterator[InternalRow] = { - val durationMetric = longMetric("duration") - val startTime = System.nanoTime() - val out = super.evaluate(funcs, argOffsets, iter, schema, context) - durationMetric += NANOSECONDS.toMillis(System.nanoTime() - startTime) - out - } + override lazy val metrics: Map[String, SQLMetric] = internal.metrics ++ Map( + "duration" -> { + val metric = new SQLMetric("timing", -1L) + metric.register(sparkContext, Some("duration"), false) + metric + } + ) -// override protected def doExecute(): RDD[InternalRow] = -// DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) override protected def withNewChildInternal(newChild: SparkPlan): DataFlintBatchEvalPythonExec = DataFlintBatchEvalPythonExec(udfs, resultAttrs, newChild) diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala index 9acf21d..28605eb 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala @@ -47,8 +47,18 @@ class DataFlintFlatMapCoGroupsInPandasExec private ( override def nodeName: String = "DataFlintFlatMapCoGroupsInPandas" - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + // Cannot use super.metrics in a lazy val override — Scala 2 does not generate super + // accessors for trait lazy vals (PythonSQLMetrics). Use a sibling instance instead. + // Cannot use SQLMetrics.createTimingMetric() — it gained a default parameter in 3.5 + // which generates a $default$3() call that doesn't exist in 3.0–3.4 at runtime. + private val internal = FlatMapCoGroupsInPandasExec(leftGroup, rightGroup, func, output, left, right) + + override lazy val metrics: Map[String, SQLMetric] = internal.metrics ++ Map( + "duration" -> { + val metric = new SQLMetric("timing", -1L) + metric.register(sparkContext, Some("duration"), false) + metric + } ) override protected def doExecute(): RDD[InternalRow] = diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala index b8fe584..af0ada1 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala @@ -40,8 +40,18 @@ class DataFlintFlatMapGroupsInPandasExec private ( override def nodeName: String = "DataFlintFlatMapGroupsInPandas" - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + // Cannot use super.metrics in a lazy val override — Scala 2 does not generate super + // accessors for trait lazy vals (PythonSQLMetrics). Use a sibling instance instead. + // Cannot use SQLMetrics.createTimingMetric() — it gained a default parameter in 3.5 + // which generates a $default$3() call that doesn't exist in 3.0–3.4 at runtime. + private val internal = FlatMapGroupsInPandasExec(groupingAttributes, func, output, child) + + override lazy val metrics: Map[String, SQLMetric] = internal.metrics ++ Map( + "duration" -> { + val metric = new SQLMetric("timing", -1L) + metric.register(sparkContext, Some("duration"), false) + metric + } ) override protected def doExecute(): RDD[InternalRow] = diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintWindowInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintWindowInPandasExec.scala index c757288..0bb54b0 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintWindowInPandasExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintWindowInPandasExec.scala @@ -22,7 +22,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.metric.SQLMetric class DataFlintWindowInPandasExec private ( windowExpression: Seq[NamedExpression], @@ -30,14 +30,24 @@ class DataFlintWindowInPandasExec private ( orderSpec: Seq[SortOrder], child: SparkPlan) extends WindowInPandasExec(windowExpression, partitionSpec, orderSpec, child) with Logging { + val internal = WindowInPandasExec(windowExpression, partitionSpec, orderSpec, child) override def nodeName: String = "DataFlintWindowInPandas" - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size"), - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + // Scala 2 forbids calling super directly in a lazy val override. Route through a private + // def so the super call resolves correctly, then use it from the lazy val. + // Cannot use SQLMetrics.createTimingMetric() — it gained a default parameter in 3.5 + // which generates a $default$3() call that doesn't exist in 3.0–3.4 at runtime. + override lazy val metrics: Map[String, SQLMetric] = internal.metrics ++ Map( + "duration" -> { + val metric = new SQLMetric("timing", -1L) + metric.register(sparkContext, Some("duration"), countFailedValues = false) + metric + } ) + override def resetMetrics(): Unit = super.resetMetrics() + override protected def doExecute(): RDD[InternalRow] = DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/window/DataFlintWindowExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/window/DataFlintWindowExec.scala index 1ad2522..d2529de 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/window/DataFlintWindowExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/window/DataFlintWindowExec.scala @@ -43,7 +43,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.metric.SQLMetric class DataFlintWindowExec private ( windowExpression: Seq[NamedExpression], @@ -54,9 +54,19 @@ class DataFlintWindowExec private ( override def nodeName: String = "DataFlintWindow" + // Cannot use SQLMetrics.createSizeMetric/createTimingMetric() — they gained a default + // parameter in 3.5 which generates $default$3() calls that don't exist in 3.0–3.4 at runtime. override lazy val metrics: Map[String, SQLMetric] = Map( - "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size"), - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + "spillSize" -> { + val metric = new SQLMetric("size", 0L) + metric.register(sparkContext, Some("spill size"), false) + metric + }, + "duration" -> { + val metric = new SQLMetric("timing", -1L) + metric.register(sparkContext, Some("duration"), false) + metric + } ) override protected def doExecute(): RDD[InternalRow] = From f5e15dd78164c330d9f4627f77842c03c73f5dbc Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Sun, 15 Mar 2026 18:17:59 +0200 Subject: [PATCH 07/15] updated dataflint ui with new instrumentation --- .../pyspark-testing/dataflint_pyspark_example.py | 1 - spark-ui/src/components/SqlFlow/MetricProcessors.tsx | 2 ++ spark-ui/src/reducers/SQLNodeStageReducer.ts | 9 +++++++-- spark-ui/src/reducers/SqlReducer.ts | 10 ++++++++++ spark-ui/src/reducers/SqlReducerUtils.ts | 10 ++++++++++ 5 files changed, 29 insertions(+), 3 deletions(-) diff --git a/spark-plugin/pyspark-testing/dataflint_pyspark_example.py b/spark-plugin/pyspark-testing/dataflint_pyspark_example.py index 2b7906f..304309b 100755 --- a/spark-plugin/pyspark-testing/dataflint_pyspark_example.py +++ b/spark-plugin/pyspark-testing/dataflint_pyspark_example.py @@ -58,7 +58,6 @@ .config("spark.sql.maxMetadataStringLength", "10000") \ .config("spark.sql.adaptive.enabled", "false") \ .config("spark.dataflint.telemetry.enabled", "false") \ - .config("spark.dataflint.instrument.spark.enabled", "true") \ .config("spark.dataflint.instrument.spark.mapInPandas.enabled", "true") \ .config("spark.dataflint.instrument.spark.mapInArrow.enabled", "true") \ .config("spark.dataflint.instrument.spark.window.enabled", "true") \ diff --git a/spark-ui/src/components/SqlFlow/MetricProcessors.tsx b/spark-ui/src/components/SqlFlow/MetricProcessors.tsx index 7b23966..8cea06c 100644 --- a/spark-ui/src/components/SqlFlow/MetricProcessors.tsx +++ b/spark-ui/src/components/SqlFlow/MetricProcessors.tsx @@ -18,6 +18,8 @@ import { MetricWithTooltip } from "./MetricDisplay"; const INSTRUMENTED_NODE_NAMES = new Set([ "DataFlintWindow", "DataFlintWindowInPandas", "DataFlintArrowWindowPython", "DataFlintMapInPandas", "DataFlintMapInArrow", + "DataFlintBatchEvalPython", "DataFlintArrowEvalPython", + "DataFlintFlatMapGroupsInPandas", "DataFlintFlatMapCoGroupsInPandas", ]); export const processBaseMetrics = (node: EnrichedSqlNode): MetricWithTooltip[] => { diff --git a/spark-ui/src/reducers/SQLNodeStageReducer.ts b/spark-ui/src/reducers/SQLNodeStageReducer.ts index 4dc8b24..a5e557b 100644 --- a/spark-ui/src/reducers/SQLNodeStageReducer.ts +++ b/spark-ui/src/reducers/SQLNodeStageReducer.ts @@ -109,9 +109,12 @@ export function calculateSQLNodeStage(sql: EnrichedSparkSQL, sqlStages: SparkSta }); nodes = nodes.map((node) => { if (node.nodeName === "AQEShuffleRead" || node.nodeName === "Coalesce" || - node.nodeName === "BatchEvalPython" || node.nodeName === "MapInPandas" || node.nodeName === "DataFlintMapInPandas" || + node.nodeName === "BatchEvalPython" || node.nodeName === "DataFlintBatchEvalPython" || + node.nodeName === "MapInPandas" || node.nodeName === "DataFlintMapInPandas" || node.nodeName === "MapInArrow" || node.nodeName === "PythonMapInArrow" || node.nodeName === "DataFlintMapInArrow" || - node.nodeName === "ArrowEvalPython" || node.nodeName === "FlatMapGroupsInPandas" || + node.nodeName === "ArrowEvalPython" || node.nodeName === "DataFlintArrowEvalPython" || + node.nodeName === "FlatMapGroupsInPandas" || node.nodeName === "DataFlintFlatMapGroupsInPandas" || + node.nodeName === "FlatMapCoGroupsInPandas" || node.nodeName === "DataFlintFlatMapCoGroupsInPandas" || node.nodeName === "WindowInPandas" || node.nodeName === "DataFlintWindowInPandas" || node.nodeName === "DataFlintArrowWindowPython" || node.nodeName === "Window" || node.nodeName === "DataFlintWindow") { const nextNode = findNextNode(node.nodeId); @@ -496,6 +499,8 @@ export function calculateSqlStage( const instrumentedNodeNames = [ "DataFlintWindow", "DataFlintWindowInPandas", "DataFlintArrowWindowPython", "DataFlintMapInPandas", "DataFlintMapInArrow", + "DataFlintBatchEvalPython", "DataFlintArrowEvalPython", + "DataFlintFlatMapGroupsInPandas", "DataFlintFlatMapCoGroupsInPandas", ]; for (const node of calculatedStageSql.nodes) { diff --git a/spark-ui/src/reducers/SqlReducer.ts b/spark-ui/src/reducers/SqlReducer.ts index 7fe9eb7..3d143c9 100644 --- a/spark-ui/src/reducers/SqlReducer.ts +++ b/spark-ui/src/reducers/SqlReducer.ts @@ -173,13 +173,18 @@ export function parseNodePlan( plan: parseWindow(plan.planDescription), }; case "BatchEvalPython": + case "DataFlintBatchEvalPython": case "ArrowEvalPython": + case "DataFlintArrowEvalPython": case "MapInPandas": case "DataFlintMapInPandas": case "MapInArrow": case "PythonMapInArrow": case "DataFlintMapInArrow": case "FlatMapGroupsInPandas": + case "DataFlintFlatMapGroupsInPandas": + case "FlatMapCoGroupsInPandas": + case "DataFlintFlatMapCoGroupsInPandas": return { type: "BatchEvalPython", plan: parseBatchEvalPython(plan.planDescription), @@ -681,13 +686,18 @@ function updateNodeEnrichedName( const PYTHON_EVAL_NODE_NAMES = new Set([ "BatchEvalPython", + "DataFlintBatchEvalPython", "ArrowEvalPython", + "DataFlintArrowEvalPython", "MapInPandas", "DataFlintMapInPandas", "MapInArrow", "PythonMapInArrow", "DataFlintMapInArrow", "FlatMapGroupsInPandas", + "DataFlintFlatMapGroupsInPandas", + "FlatMapCoGroupsInPandas", + "DataFlintFlatMapCoGroupsInPandas", ]); function isPythonEvalNode(nodeName: string): boolean { diff --git a/spark-ui/src/reducers/SqlReducerUtils.ts b/spark-ui/src/reducers/SqlReducerUtils.ts index 98987d6..6d33384 100644 --- a/spark-ui/src/reducers/SqlReducerUtils.ts +++ b/spark-ui/src/reducers/SqlReducerUtils.ts @@ -190,8 +190,13 @@ const nodeTypeDict: Record = { PythonMapInArrow: "transformation", DataFlintMapInArrow: "transformation", ArrowEvalPython: "transformation", + DataFlintArrowEvalPython: "transformation", FlatMapGroupsInPandas: "transformation", + DataFlintFlatMapGroupsInPandas: "transformation", + FlatMapCoGroupsInPandas: "transformation", + DataFlintFlatMapCoGroupsInPandas: "transformation", BatchEvalPython: "transformation", + DataFlintBatchEvalPython: "transformation", WindowInPandas: "transformation", DataFlintWindowInPandas: "transformation", DataFlintArrowWindowPython: "transformation", @@ -274,8 +279,13 @@ const nodeRenamerDict: Record = { PythonMapInArrow: "Select (with Arrow UDF)", DataFlintMapInArrow: "Select (with Arrow UDF)", ArrowEvalPython: "Select (with Arrow)", + DataFlintArrowEvalPython: "Select (with Arrow)", FlatMapGroupsInPandas: "Select Flat (with Pandas)", + DataFlintFlatMapGroupsInPandas: "Select Flat (with Pandas)", + FlatMapCoGroupsInPandas: "Join (with Pandas)", + DataFlintFlatMapCoGroupsInPandas: "Join (with Pandas)", BatchEvalPython: "Run Python UDF", + DataFlintBatchEvalPython: "Run Python UDF", Window: "Window", WindowInPandas: "Window (with Pandas UDF)", DataFlintWindow: "Window", From 0da56afcb419bced9d90aaa1ec1103904c5e82fa Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Sun, 15 Mar 2026 18:31:47 +0200 Subject: [PATCH 08/15] updated dataflint ui with new instrumentation tooltips --- .../components/SqlFlow/MetricProcessors.tsx | 2 +- spark-ui/src/components/SqlFlow/StageNode.tsx | 22 ++++++++++++++++++- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/spark-ui/src/components/SqlFlow/MetricProcessors.tsx b/spark-ui/src/components/SqlFlow/MetricProcessors.tsx index 8cea06c..09ef267 100644 --- a/spark-ui/src/components/SqlFlow/MetricProcessors.tsx +++ b/spark-ui/src/components/SqlFlow/MetricProcessors.tsx @@ -15,7 +15,7 @@ import { } from "../../utils/FormatUtils"; import { MetricWithTooltip } from "./MetricDisplay"; -const INSTRUMENTED_NODE_NAMES = new Set([ +export const INSTRUMENTED_NODE_NAMES = new Set([ "DataFlintWindow", "DataFlintWindowInPandas", "DataFlintArrowWindowPython", "DataFlintMapInPandas", "DataFlintMapInArrow", "DataFlintBatchEvalPython", "DataFlintArrowEvalPython", diff --git a/spark-ui/src/components/SqlFlow/StageNode.tsx b/spark-ui/src/components/SqlFlow/StageNode.tsx index b5afe29..6abd585 100644 --- a/spark-ui/src/components/SqlFlow/StageNode.tsx +++ b/spark-ui/src/components/SqlFlow/StageNode.tsx @@ -1,6 +1,7 @@ import ErrorIcon from "@mui/icons-material/Error"; +import FlagIcon from "@mui/icons-material/Flag"; import WarningIcon from "@mui/icons-material/Warning"; -import { Alert, AlertTitle, Box, Typography } from "@mui/material"; +import { Alert, AlertTitle, Box, Tooltip, Typography } from "@mui/material"; import React, { FC, memo, useMemo } from "react"; import { useSearchParams } from "react-router-dom"; import { Handle, Position } from "reactflow"; @@ -10,6 +11,7 @@ import { TransperantTooltip } from "../AlertBadge/AlertBadge"; import MetricDisplay, { MetricWithTooltip } from "./MetricDisplay"; import { addTruncatedCodeTooltip, + INSTRUMENTED_NODE_NAMES, processBaseMetrics, processCachedStorageMetrics, processDeltaLakeScanMetrics, @@ -397,6 +399,24 @@ const StageNodeComponent: FC = ({ data }) => { Δ Optimized )} + {INSTRUMENTED_NODE_NAMES.has(data.node.nodeName) && ( + + + + + + )} {/* Subtitle for exchange nodes */} {exchangeVariant && ( From 760750a3cc2f96b144382021e2198b8153fab8b6 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Mon, 16 Mar 2026 10:40:36 +0200 Subject: [PATCH 09/15] updated names for instrumental nodes, updated version. --- examples/distributed_computing_unpacked.ipynb | 2 +- spark-plugin/build.sbt | 2 +- .../DataFlintInstrumentationExtension.scala | 4 +- ...cala => DataFlintMapInArrowExec_3_3.scala} | 14 ++--- ...ala => DataFlintMapInPandasExec_3_0.scala} | 0 ...ala => DataFlintMapInPandasExec_3_5.scala} | 43 --------------- .../DataFlintPythonMapInArrowExec_3_5.scala | 54 +++++++++++++++++++ .../dataflint/DataFlintPythonExecSpec.scala | 3 +- .../dataflint_pyspark_example.py | 6 +-- .../dataflint_pyspark_example_test.py | 4 +- spark-ui/package-lock.json | 4 +- spark-ui/package.json | 2 +- spark-ui/src/reducers/SqlReducerUtils.ts | 16 +++--- 13 files changed, 83 insertions(+), 71 deletions(-) rename spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/{DataFlintMapInBatchExec_3_3.scala => DataFlintMapInArrowExec_3_3.scala} (90%) rename spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/{DataFlintMapInBatchExec_3_0.scala => DataFlintMapInPandasExec_3_0.scala} (100%) rename spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/{DataFlintMapInBatchExec_3_5.scala => DataFlintMapInPandasExec_3_5.scala} (61%) create mode 100644 spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintPythonMapInArrowExec_3_5.scala diff --git a/examples/distributed_computing_unpacked.ipynb b/examples/distributed_computing_unpacked.ipynb index facbd90..c4327f9 100644 --- a/examples/distributed_computing_unpacked.ipynb +++ b/examples/distributed_computing_unpacked.ipynb @@ -32,7 +32,7 @@ "spark = SparkSession \\\n", " .builder \\\n", " .appName(\"Distributed Compute Examples\") \\\n", - " .config(\"spark.jars.packages\", \"io.dataflint:spark_2.12:0.8.5\") \\\n", + " .config(\"spark.jars.packages\", \"io.dataflint:spark_2.12:0.8.7\") \\\n", " .config(\"spark.plugins\", \"io.dataflint.spark.SparkDataflintPlugin\") \\\n", " .config(\"spark.ui.port\", \"11000\") \\\n", " .master(\"local[*]\") \\\n", diff --git a/spark-plugin/build.sbt b/spark-plugin/build.sbt index 9aa8505..19884fc 100644 --- a/spark-plugin/build.sbt +++ b/spark-plugin/build.sbt @@ -1,7 +1,7 @@ import xerial.sbt.Sonatype._ import sbtassembly.AssemblyPlugin.autoImport._ -lazy val versionNum: String = "0.8.5" +lazy val versionNum: String = "0.8.7" lazy val scala212 = "2.12.20" lazy val scala213 = "2.13.16" lazy val supportedScalaVersions = List(scala212, scala213) diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala index 5a7a79a..733a95d 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/DataFlintInstrumentationExtension.scala @@ -8,7 +8,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalWindow import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Window => LogicalWindow} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_0, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_3, DataFlintPythonMapInArrowExec_3_5, DataFlintWindowInPandasExec, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, WindowInPandasExec} +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInArrowExec_3_3, DataFlintMapInPandasExec_3_0, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_5, DataFlintWindowInPandasExec, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, WindowInPandasExec} import org.apache.spark.sql.execution.window.DataFlintWindowExec /** @@ -254,7 +254,7 @@ case class DataFlintInstrumentationColumnarRule(session: SparkSession) extends C ) case _ => // 3.3–3.4: share the same 3-arg constructor, handled via reflection - DataFlintPythonMapInArrowExec_3_3( + DataFlintMapInArrowExec_3_3( func = mapInArrow.func, output = mapInArrow.output, child = mapInArrow.child diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_3.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec_3_3.scala similarity index 90% rename from spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_3.scala rename to spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec_3_3.scala index d6e5bb3..f72efc1 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_3.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec_3_3.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} * DataFlint instrumented PythonMapInArrowExec for Spark 3.3.x – 3.4.x. * Replaces the original PythonMapInArrowExec in the physical plan. */ -class DataFlintPythonMapInArrowExec_3_3 private ( +class DataFlintMapInArrowExec_3_3 private( val func: Expression, override val output: Seq[Attribute], val child: SparkPlan) @@ -70,7 +70,7 @@ class DataFlintPythonMapInArrowExec_3_3 private ( DataFlintRDDUtils.withDurationMetric(innerRDD, longMetric("duration")) } - override def canEqual(that: Any): Boolean = that.isInstanceOf[DataFlintPythonMapInArrowExec_3_3] + override def canEqual(that: Any): Boolean = that.isInstanceOf[DataFlintMapInArrowExec_3_3] override def productArity: Int = 3 override def productElement(n: Int): Any = n match { case 0 => func @@ -79,14 +79,14 @@ class DataFlintPythonMapInArrowExec_3_3 private ( case _ => throw new IndexOutOfBoundsException(s"$n") } - override protected def withNewChildInternal(newChild: SparkPlan): DataFlintPythonMapInArrowExec_3_3 = - new DataFlintPythonMapInArrowExec_3_3(func, output, newChild) + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintMapInArrowExec_3_3 = + new DataFlintMapInArrowExec_3_3(func, output, newChild) } -object DataFlintPythonMapInArrowExec_3_3 { +object DataFlintMapInArrowExec_3_3 { def apply( func: Expression, output: Seq[Attribute], - child: SparkPlan): DataFlintPythonMapInArrowExec_3_3 = - new DataFlintPythonMapInArrowExec_3_3(func, output, child) + child: SparkPlan): DataFlintMapInArrowExec_3_3 = + new DataFlintMapInArrowExec_3_3(func, output, child) } \ No newline at end of file diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_0.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec_3_0.scala similarity index 100% rename from spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_0.scala rename to spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec_3_0.scala diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_5.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec_3_5.scala similarity index 61% rename from spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_5.scala rename to spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec_3_5.scala index 89bf87b..063ce9c 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInBatchExec_3_5.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec_3_5.scala @@ -64,46 +64,3 @@ object DataFlintMapInPandasExec_3_5 { isBarrier: Boolean): DataFlintMapInPandasExec_3_5 = new DataFlintMapInPandasExec_3_5(func, output, child, isBarrier) } - -/** - * DataFlint instrumented PythonMapInArrowExec for Spark 3.5.x. - * Replaces the original PythonMapInArrowExec in the physical plan. - * - * Note: PythonMapInArrowExec was introduced in Spark 3.3 (SPARK-37227). - * This class is only loaded/instantiated when the Spark version is 3.5+, - * guarded by the try-catch(NoClassDefFoundError) in DataFlintInstrumentationExtension. - */ -class DataFlintPythonMapInArrowExec_3_5 private ( - func: Expression, - output: Seq[Attribute], - child: SparkPlan, - isBarrier: Boolean) - extends PythonMapInArrowExec(func, output, child, isBarrier) with Logging { - - override def nodeName: String = "DataFlintMapInArrow" - - logInfo("DataFlint MapInArrow (Spark 3.5) is connected") - - override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( - "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") - ) - - override protected def doExecute(): RDD[InternalRow] = - DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) - - override protected def withNewChildInternal(newChild: SparkPlan): DataFlintPythonMapInArrowExec_3_5 = - DataFlintPythonMapInArrowExec_3_5(func, output, newChild, isBarrier) - - override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintPythonMapInArrowExec_3_5] - override def equals(other: Any): Boolean = other.isInstanceOf[DataFlintPythonMapInArrowExec_3_5] && super.equals(other) - override def hashCode: Int = super.hashCode -} - -object DataFlintPythonMapInArrowExec_3_5 { - def apply( - func: Expression, - output: Seq[Attribute], - child: SparkPlan, - isBarrier: Boolean): DataFlintPythonMapInArrowExec_3_5 = - new DataFlintPythonMapInArrowExec_3_5(func, output, child, isBarrier) -} diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintPythonMapInArrowExec_3_5.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintPythonMapInArrowExec_3_5.scala new file mode 100644 index 0000000..ab409fd --- /dev/null +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintPythonMapInArrowExec_3_5.scala @@ -0,0 +1,54 @@ +package org.apache.spark.sql.execution.python + +import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * DataFlint instrumented PythonMapInArrowExec for Spark 3.5.x. + * Replaces the original PythonMapInArrowExec in the physical plan. + * + * Note: PythonMapInArrowExec was introduced in Spark 3.3 (SPARK-37227). + * This class is only loaded/instantiated when the Spark version is 3.5+, + * guarded by the try-catch(NoClassDefFoundError) in DataFlintInstrumentationExtension. + */ +class DataFlintPythonMapInArrowExec_3_5 private( + func: Expression, + output: Seq[Attribute], + child: SparkPlan, + isBarrier: Boolean) + extends PythonMapInArrowExec(func, output, child, isBarrier) with Logging { + + override def nodeName: String = "DataFlintMapInArrow" + + logInfo("DataFlint MapInArrow (Spark 3.5) is connected") + + override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map( + "duration" -> SQLMetrics.createTimingMetric(sparkContext, "duration") + ) + + override protected def doExecute(): RDD[InternalRow] = + DataFlintRDDUtils.withDurationMetric(super.doExecute(), longMetric("duration")) + + override protected def withNewChildInternal(newChild: SparkPlan): DataFlintPythonMapInArrowExec_3_5 = + DataFlintPythonMapInArrowExec_3_5(func, output, newChild, isBarrier) + + override def canEqual(other: Any): Boolean = other.isInstanceOf[DataFlintPythonMapInArrowExec_3_5] + + override def equals(other: Any): Boolean = other.isInstanceOf[DataFlintPythonMapInArrowExec_3_5] && super.equals(other) + + override def hashCode: Int = super.hashCode +} + +object DataFlintPythonMapInArrowExec_3_5 { + def apply( + func: Expression, + output: Seq[Attribute], + child: SparkPlan, + isBarrier: Boolean): DataFlintPythonMapInArrowExec_3_5 = + new DataFlintPythonMapInArrowExec_3_5(func, output, child, isBarrier) +} \ No newline at end of file diff --git a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala index a1e616f..dda79b3 100644 --- a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala +++ b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala @@ -4,7 +4,8 @@ import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.PythonUDF import org.apache.spark.sql.execution.LocalTableScanExec -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_5, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, PythonMapInArrowExec} +import org.apache.spark.sql.execution.python.DataFlintPythonMapInArrowExec_3_5.DataFlintPythonMapInArrowExec_3_5 +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_5, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, PythonMapInArrowExec} import org.apache.spark.sql.types.LongType import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite diff --git a/spark-plugin/pyspark-testing/dataflint_pyspark_example.py b/spark-plugin/pyspark-testing/dataflint_pyspark_example.py index 304309b..a4c397c 100755 --- a/spark-plugin/pyspark-testing/dataflint_pyspark_example.py +++ b/spark-plugin/pyspark-testing/dataflint_pyspark_example.py @@ -37,10 +37,10 @@ # Select the appropriate plugin JAR based on Spark version if spark_major_version == 4: - _plugin_jar = _project_root / "pluginspark4" / "target" / "scala-2.13" / "dataflint-spark4_2.13-0.8.5.jar" + _plugin_jar = _project_root / "pluginspark4" / "target" / "scala-2.13" / "dataflint-spark4_2.13-0.8.7.jar" _plugin_module = "pluginspark4" else: - _plugin_jar = _project_root / "pluginspark3" / "target" / "scala-2.12" / "spark_2.12-0.8.5.jar" + _plugin_jar = _project_root / "pluginspark3" / "target" / "scala-2.12" / "spark_2.12-0.8.7.jar" _plugin_module = "pluginspark3" if not _plugin_jar.exists(): @@ -62,9 +62,9 @@ .config("spark.dataflint.instrument.spark.mapInArrow.enabled", "true") \ .config("spark.dataflint.instrument.spark.window.enabled", "true") \ .config("spark.dataflint.instrument.spark.arrowEvalPython.enabled", "true") \ + .config("spark.dataflint.instrument.spark.batchEvalPython.enabled", "true") \ .config("spark.dataflint.instrument.spark.flatMapGroupsInPandas.enabled", "true") \ .config("spark.dataflint.instrument.spark.flatMapCoGroupsInPandas.enabled", "true") \ - .config("spark.dataflint.instrument.spark.batchEvalPython.enabled", "true") \ .master("local[*]") \ .getOrCreate() # spark.sparkContext.setLogLevel("INFO") diff --git a/spark-plugin/pyspark-testing/dataflint_pyspark_example_test.py b/spark-plugin/pyspark-testing/dataflint_pyspark_example_test.py index 143df54..78c0bd3 100755 --- a/spark-plugin/pyspark-testing/dataflint_pyspark_example_test.py +++ b/spark-plugin/pyspark-testing/dataflint_pyspark_example_test.py @@ -45,10 +45,10 @@ # Select the appropriate plugin JAR based on Spark version if spark_major_version == 4: - _plugin_jar = _project_root / "pluginspark4" / "target" / "scala-2.13" / "dataflint-spark4_2.13-0.8.5.jar" + _plugin_jar = _project_root / "pluginspark4" / "target" / "scala-2.13" / "dataflint-spark4_2.13-0.8.7.jar" _plugin_module = "pluginspark4" else: - _plugin_jar = _project_root / "pluginspark3" / "target" / "scala-2.12" / "spark_2.12-0.8.5.jar" + _plugin_jar = _project_root / "pluginspark3" / "target" / "scala-2.12" / "spark_2.12-0.8.7.jar" _plugin_module = "pluginspark3" if not _plugin_jar.exists(): diff --git a/spark-ui/package-lock.json b/spark-ui/package-lock.json index daca433..ffc134d 100644 --- a/spark-ui/package-lock.json +++ b/spark-ui/package-lock.json @@ -1,12 +1,12 @@ { "name": "dataflint-ui", - "version": "0.8.5", + "version": "0.8.7", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "dataflint-ui", - "version": "0.8.5", + "version": "0.8.7", "dependencies": { "@chatscope/chat-ui-kit-react": "^1.10.1", "@chatscope/chat-ui-kit-styles": "^1.4.0", diff --git a/spark-ui/package.json b/spark-ui/package.json index 3383407..dfa0b56 100644 --- a/spark-ui/package.json +++ b/spark-ui/package.json @@ -1,6 +1,6 @@ { "name": "dataflint-ui", - "version": "0.8.5", + "version": "0.8.7", "homepage": "./", "private": true, "dependencies": { diff --git a/spark-ui/src/reducers/SqlReducerUtils.ts b/spark-ui/src/reducers/SqlReducerUtils.ts index 6d33384..03713bc 100644 --- a/spark-ui/src/reducers/SqlReducerUtils.ts +++ b/spark-ui/src/reducers/SqlReducerUtils.ts @@ -275,22 +275,22 @@ const nodeRenamerDict: Record = { InMemoryTableScan: "Cache", MapInPandas: "Select (with Pandas)", DataFlintMapInPandas: "Select (with Pandas)", - MapInArrow: "Select (with Arrow UDF)", - PythonMapInArrow: "Select (with Arrow UDF)", - DataFlintMapInArrow: "Select (with Arrow UDF)", + MapInArrow: "Select (with Arrow)", + PythonMapInArrow: "Select (with Arrow)", + DataFlintMapInArrow: "Select (with Arrow)", ArrowEvalPython: "Select (with Arrow)", DataFlintArrowEvalPython: "Select (with Arrow)", FlatMapGroupsInPandas: "Select Flat (with Pandas)", DataFlintFlatMapGroupsInPandas: "Select Flat (with Pandas)", FlatMapCoGroupsInPandas: "Join (with Pandas)", DataFlintFlatMapCoGroupsInPandas: "Join (with Pandas)", - BatchEvalPython: "Run Python UDF", - DataFlintBatchEvalPython: "Run Python UDF", + BatchEvalPython: "Select (with Python UDF)", + DataFlintBatchEvalPython: "Select (with Python UDF)", Window: "Window", - WindowInPandas: "Window (with Pandas UDF)", + WindowInPandas: "Window (with Pandas)", DataFlintWindow: "Window", - DataFlintWindowInPandas: "Window (with Pandas UDF)", - DataFlintArrowWindowPython: "Window (with Pandas UDF)", + DataFlintWindowInPandas: "Window (with Pandas)", + DataFlintArrowWindowPython: "Window (with Arrow)", Expand: "Expand", }; From b987df28ed38ec859e9b4202161c191aab4ace7d Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Mon, 16 Mar 2026 13:04:34 +0200 Subject: [PATCH 10/15] updated node names --- spark-ui/src/reducers/SqlReducerUtils.ts | 32 ++++++++++++------------ 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/spark-ui/src/reducers/SqlReducerUtils.ts b/spark-ui/src/reducers/SqlReducerUtils.ts index 03713bc..3a721a4 100644 --- a/spark-ui/src/reducers/SqlReducerUtils.ts +++ b/spark-ui/src/reducers/SqlReducerUtils.ts @@ -273,24 +273,24 @@ const nodeRenamerDict: Record = { PhotonBroadcastHashJoin: "Join (Broadcast Hash) (Photon)", CartesianProduct: "Join (Cartesian Product)", InMemoryTableScan: "Cache", - MapInPandas: "Select (with Pandas)", - DataFlintMapInPandas: "Select (with Pandas)", - MapInArrow: "Select (with Arrow)", - PythonMapInArrow: "Select (with Arrow)", - DataFlintMapInArrow: "Select (with Arrow)", - ArrowEvalPython: "Select (with Arrow)", - DataFlintArrowEvalPython: "Select (with Arrow)", - FlatMapGroupsInPandas: "Select Flat (with Pandas)", - DataFlintFlatMapGroupsInPandas: "Select Flat (with Pandas)", - FlatMapCoGroupsInPandas: "Join (with Pandas)", - DataFlintFlatMapCoGroupsInPandas: "Join (with Pandas)", - BatchEvalPython: "Select (with Python UDF)", - DataFlintBatchEvalPython: "Select (with Python UDF)", + MapInPandas: "Select (with Pandas UDF)", + DataFlintMapInPandas: "Select (with Pandas UDF)", + MapInArrow: "Select (with Arrow UDF)", + PythonMapInArrow: "Select (with Arrow UDF)", + DataFlintMapInArrow: "Select (with Arrow UDF)", + ArrowEvalPython: "Select (with Arrow UDF)", + DataFlintArrowEvalPython: "Select (with Arrow UDF)", + FlatMapGroupsInPandas: "Select Flat (with Pandas UDF)", + DataFlintFlatMapGroupsInPandas: "Select Flat (with Pandas UDF)", + FlatMapCoGroupsInPandas: "Join (with Pandas UDF)", + DataFlintFlatMapCoGroupsInPandas: "Join (with Pandas UDF)", + BatchEvalPython: "Python UDF", + DataFlintBatchEvalPython: "Python UDF", Window: "Window", - WindowInPandas: "Window (with Pandas)", + WindowInPandas: "Window (with Pandas UDF)", DataFlintWindow: "Window", - DataFlintWindowInPandas: "Window (with Pandas)", - DataFlintArrowWindowPython: "Window (with Arrow)", + DataFlintWindowInPandas: "Window (with Pandas UDF)", + DataFlintArrowWindowPython: "Window (with Arrow UDF)", Expand: "Expand", }; From 3b39b106a61dc89dbee0a5fbc24576abee96aec7 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Mon, 16 Mar 2026 13:18:49 +0200 Subject: [PATCH 11/15] create MetricsUtils with method to generate a timing metric and use wherever needed --- .../org/apache/spark/dataflint/MetricsUtils.scala | 14 ++++++++++++++ .../python/DataFlintArrowEvalPythonExec.scala | 8 ++------ .../python/DataFlintBatchEvalPythonExec.scala | 8 ++------ .../DataFlintFlatMapCoGroupsInPandasExec.scala | 8 ++------ .../DataFlintFlatMapGroupsInPandasExec.scala | 8 ++------ .../python/DataFlintMapInArrowExec_3_3.scala | 8 ++------ .../python/DataFlintMapInPandasExec_3_0.scala | 8 ++------ .../python/DataFlintWindowInPandasExec.scala | 8 ++------ 8 files changed, 28 insertions(+), 42 deletions(-) create mode 100644 spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/MetricsUtils.scala diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/MetricsUtils.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/MetricsUtils.scala new file mode 100644 index 0000000..e8cfafb --- /dev/null +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/dataflint/MetricsUtils.scala @@ -0,0 +1,14 @@ +package org.apache.spark.dataflint + +import org.apache.spark.SparkContext +import org.apache.spark.sql.execution.metric.SQLMetric + +object MetricsUtils { + def getTimingMetric(name: String)(implicit sparkContext:SparkContext): (String, SQLMetric) = { + name -> { + val metric = new SQLMetric("timing", -1L) + metric.register(sparkContext, Some(name), countFailedValues = false) + metric + } + } +} diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala index 8f571a5..a3a375a 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintArrowEvalPythonExec.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.python import org.apache.spark.TaskContext import org.apache.spark.api.python.ChainedPythonFunctions -import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.dataflint.{DataFlintRDDUtils, MetricsUtils} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -53,11 +53,7 @@ class DataFlintArrowEvalPythonExec private( private val internal = ArrowEvalPythonExec(udfs, resultAttrs, child, evalType) override lazy val metrics: Map[String, SQLMetric] = internal.metrics ++ Map( - "duration" -> { - val metric = new SQLMetric("timing", -1L) - metric.register(sparkContext, Some("duration"), false) - metric - } + MetricsUtils.getTimingMetric("duration")(sparkContext) ) override protected def doExecute(): RDD[InternalRow] = diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala index 8ffee3b..1cf4596 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintBatchEvalPythonExec.scala @@ -2,7 +2,7 @@ package org.apache.spark.sql.execution.python import org.apache.spark.TaskContext import org.apache.spark.api.python.ChainedPythonFunctions -import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.dataflint.{DataFlintRDDUtils, MetricsUtils} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -35,11 +35,7 @@ class DataFlintBatchEvalPythonExec private ( private val internal = BatchEvalPythonExec(udfs, resultAttrs, child) override lazy val metrics: Map[String, SQLMetric] = internal.metrics ++ Map( - "duration" -> { - val metric = new SQLMetric("timing", -1L) - metric.register(sparkContext, Some("duration"), false) - metric - } + MetricsUtils.getTimingMetric("duration")(sparkContext) ) override protected def doExecute(): RDD[InternalRow] = diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala index 28605eb..547fcbd 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapCoGroupsInPandasExec.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.python import org.apache.spark.TaskContext import org.apache.spark.api.python.ChainedPythonFunctions -import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.dataflint.{DataFlintRDDUtils, MetricsUtils} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -54,11 +54,7 @@ class DataFlintFlatMapCoGroupsInPandasExec private ( private val internal = FlatMapCoGroupsInPandasExec(leftGroup, rightGroup, func, output, left, right) override lazy val metrics: Map[String, SQLMetric] = internal.metrics ++ Map( - "duration" -> { - val metric = new SQLMetric("timing", -1L) - metric.register(sparkContext, Some("duration"), false) - metric - } + MetricsUtils.getTimingMetric("duration")(sparkContext) ) override protected def doExecute(): RDD[InternalRow] = diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala index af0ada1..1a54254 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintFlatMapGroupsInPandasExec.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.python -import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.dataflint.{DataFlintRDDUtils, MetricsUtils} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -47,11 +47,7 @@ class DataFlintFlatMapGroupsInPandasExec private ( private val internal = FlatMapGroupsInPandasExec(groupingAttributes, func, output, child) override lazy val metrics: Map[String, SQLMetric] = internal.metrics ++ Map( - "duration" -> { - val metric = new SQLMetric("timing", -1L) - metric.register(sparkContext, Some("duration"), false) - metric - } + MetricsUtils.getTimingMetric("duration")(sparkContext) ) override protected def doExecute(): RDD[InternalRow] = diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec_3_3.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec_3_3.scala index f72efc1..c38fca2 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec_3_3.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInArrowExec_3_3.scala @@ -12,7 +12,7 @@ */ package org.apache.spark.sql.execution.python -import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.dataflint.{DataFlintRDDUtils, MetricsUtils} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -41,11 +41,7 @@ class DataFlintMapInArrowExec_3_3 private( // Cannot use SQLMetrics.createTimingMetric() — it gained a default parameter in 3.5 // which generates a $default$3() call that doesn't exist in 3.3–3.4 at runtime. override lazy val metrics: Map[String, SQLMetric] = Map( - "duration" -> { - val metric = new SQLMetric("timing", -1L) - metric.register(sparkContext, Some("duration"), false) - metric - } + MetricsUtils.getTimingMetric("duration")(sparkContext) ) override protected def doExecute(): RDD[InternalRow] = { diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec_3_0.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec_3_0.scala index 41d066e..fe8712b 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec_3_0.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintMapInPandasExec_3_0.scala @@ -14,7 +14,7 @@ */ package org.apache.spark.sql.execution.python -import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.dataflint.{DataFlintRDDUtils, MetricsUtils} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -43,11 +43,7 @@ class DataFlintMapInPandasExec_3_0 private ( // Cannot use SQLMetrics.createTimingMetric() — it gained a default parameter in 3.5 // which generates a $default$3() call that doesn't exist in 3.0–3.4 at runtime. override lazy val metrics: Map[String, SQLMetric] = Map( - "duration" -> { - val metric = new SQLMetric("timing", -1L) - metric.register(sparkContext, Some("duration"), false) - metric - } + MetricsUtils.getTimingMetric("duration")(sparkContext) ) override protected def doExecute(): RDD[InternalRow] = { diff --git a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintWindowInPandasExec.scala b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintWindowInPandasExec.scala index 0bb54b0..4716190 100644 --- a/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintWindowInPandasExec.scala +++ b/spark-plugin/pluginspark3/src/main/scala/org/apache/spark/sql/execution/python/DataFlintWindowInPandasExec.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.python -import org.apache.spark.dataflint.DataFlintRDDUtils +import org.apache.spark.dataflint.{DataFlintRDDUtils, MetricsUtils} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -39,11 +39,7 @@ class DataFlintWindowInPandasExec private ( // Cannot use SQLMetrics.createTimingMetric() — it gained a default parameter in 3.5 // which generates a $default$3() call that doesn't exist in 3.0–3.4 at runtime. override lazy val metrics: Map[String, SQLMetric] = internal.metrics ++ Map( - "duration" -> { - val metric = new SQLMetric("timing", -1L) - metric.register(sparkContext, Some("duration"), countFailedValues = false) - metric - } + MetricsUtils.getTimingMetric("duration")(sparkContext) ) override def resetMetrics(): Unit = super.resetMetrics() From 692f66ca350ec09c587c44a4251da8d92d84e803 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Mon, 16 Mar 2026 13:29:56 +0200 Subject: [PATCH 12/15] fixed testing --- .../org/apache/spark/dataflint/DataFlintPythonExecSpec.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala index dda79b3..a1e616f 100644 --- a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala +++ b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala @@ -4,8 +4,7 @@ import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.PythonUDF import org.apache.spark.sql.execution.LocalTableScanExec -import org.apache.spark.sql.execution.python.DataFlintPythonMapInArrowExec_3_5.DataFlintPythonMapInArrowExec_3_5 -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_5, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, PythonMapInArrowExec} +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec, DataFlintArrowEvalPythonExec, DataFlintBatchEvalPythonExec, DataFlintFlatMapCoGroupsInPandasExec, DataFlintFlatMapGroupsInPandasExec, DataFlintMapInPandasExec_3_5, DataFlintPythonMapInArrowExec_3_5, FlatMapCoGroupsInPandasExec, FlatMapGroupsInPandasExec, MapInPandasExec, PythonMapInArrowExec} import org.apache.spark.sql.types.LongType import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite From 4f40ce49b23515093046fc297a0b9ea27b34aa31 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Mon, 16 Mar 2026 13:46:13 +0200 Subject: [PATCH 13/15] disabled ui on unittests --- .../org/apache/spark/dataflint/DataFlintPythonExecSpec.scala | 1 + .../org/apache/spark/dataflint/DataFlintWindowExecSpec.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala index a1e616f..7f3e5f9 100644 --- a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala +++ b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintPythonExecSpec.scala @@ -19,6 +19,7 @@ class DataFlintPythonExecSpec extends AnyFunSuite with Matchers with BeforeAndAf .master("local[1]") .appName("DataFlintPythonExecSpec") .config(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, "true") + .config("spark.ui.enabled", "false") .withExtensions(new DataFlintInstrumentationExtension) .getOrCreate() } diff --git a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala index d865df9..ded8182 100644 --- a/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala +++ b/spark-plugin/pluginspark3/src/test/scala/org/apache/spark/dataflint/DataFlintWindowExecSpec.scala @@ -39,7 +39,7 @@ class DataFlintWindowExecSpec extends AnyFunSuite with Matchers with BeforeAndAf // .config("spark.sql.extensions", "org.apache.spark.dataflint.DataFlintInstrumentationExtension") .config(DataflintSparkUICommonLoader.INSTRUMENT_SPARK_ENABLED, "true") .config(DataflintSparkUICommonLoader.INSTRUMENT_WINDOW_ENABLED, "true") - .config("spark.ui.enabled", "true") + .config("spark.ui.enabled", "false") .config("spark.sql.adaptive.enabled", "true") .config("spark.sql.adaptive.coalescePartitions.enabled", "true") // .config("spark.sql.adaptive.advisoryPartitionSizeInBytes", "64MB") From 8894332702a55fbe1bdbb18a60889a417ecbb441 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Mon, 16 Mar 2026 14:23:10 +0200 Subject: [PATCH 14/15] fixed unittests spark by adding javaOptions --- spark-plugin/build.sbt | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/spark-plugin/build.sbt b/spark-plugin/build.sbt index 19884fc..979059e 100644 --- a/spark-plugin/build.sbt +++ b/spark-plugin/build.sbt @@ -98,6 +98,15 @@ lazy val pluginspark3 = (project in file("pluginspark3")) // Include source and resources from plugin directory for tests Test / unmanagedSourceDirectories += (plugin / Compile / sourceDirectory).value / "scala", + // Fork JVM for tests so javaOptions are applied; required for Spark on Java 9+ + Test / fork := true, + Test / javaOptions ++= Seq( + "--add-opens=java.base/java.lang=ALL-UNNAMED", + "--add-opens=java.base/java.nio=ALL-UNNAMED", + "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", + "--add-opens=java.base/java.util=ALL-UNNAMED", + "--add-opens=java.base/java.io=ALL-UNNAMED", + ), ) lazy val pluginspark4 = (project in file("pluginspark4")) From edff3decaf4e4143cf0494bc31e185cca3f60854 Mon Sep 17 00:00:00 2001 From: Avi Minsky Date: Mon, 16 Mar 2026 14:32:36 +0200 Subject: [PATCH 15/15] removed parallel testing --- spark-plugin/build.sbt | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/spark-plugin/build.sbt b/spark-plugin/build.sbt index 979059e..7457d5c 100644 --- a/spark-plugin/build.sbt +++ b/spark-plugin/build.sbt @@ -100,13 +100,20 @@ lazy val pluginspark3 = (project in file("pluginspark3")) // Fork JVM for tests so javaOptions are applied; required for Spark on Java 9+ Test / fork := true, - Test / javaOptions ++= Seq( - "--add-opens=java.base/java.lang=ALL-UNNAMED", - "--add-opens=java.base/java.nio=ALL-UNNAMED", - "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", - "--add-opens=java.base/java.util=ALL-UNNAMED", - "--add-opens=java.base/java.io=ALL-UNNAMED", - ), + // Run test suites sequentially — parallel suites share the SparkSession via getOrCreate() + // and one suite stopping the session causes NPEs in concurrently-running suites + Test / parallelExecution := false, + Test / javaOptions ++= { + // --add-opens is not supported on Java 8 (spec version starts with "1.") + if (sys.props("java.specification.version").startsWith("1.")) Seq.empty + else Seq( + "--add-opens=java.base/java.lang=ALL-UNNAMED", + "--add-opens=java.base/java.nio=ALL-UNNAMED", + "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", + "--add-opens=java.base/java.util=ALL-UNNAMED", + "--add-opens=java.base/java.io=ALL-UNNAMED", + ) + }, ) lazy val pluginspark4 = (project in file("pluginspark4"))