Skip to content

Commit

Permalink
[VL] Do not skip updating children's metrics while visiting an operat…
Browse files Browse the repository at this point in the history
…or with NoopMetricsUpdater
  • Loading branch information
zhztheplayer committed May 31, 2024
1 parent 0d665f7 commit 0a98a1a
Show file tree
Hide file tree
Showing 6 changed files with 50 additions and 27 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ class HashAggregateMetricsUpdater(val metrics: Map[String, SQLMetric])
}
}
} catch {
case e: Throwable =>
case e: Exception =>
logError(s"Updating native metrics failed due to ${e.getCause}.")
throw e
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ class HashJoinMetricsUpdater(val metrics: Map[String, SQLMetric])
}
}
} catch {
case e: Throwable =>
case e: Exception =>
logError(s"Updating native metrics failed due to ${e.getCause}.")
throw e
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ object MetricsUtil extends Logging {
aggParamsMap)
}
} catch {
case e: Throwable =>
case e: Exception =>
logWarning(s"Updating native metrics failed due to ${e.getCause}.")
()
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,8 @@ package org.apache.gluten.execution
import org.apache.gluten.GlutenConfig
import org.apache.gluten.sql.shims.SparkShimLoader

import org.apache.spark.sql.execution.CommandResultExec
import org.apache.spark.SparkConf
import org.apache.spark.sql.execution.{CommandResultExec, InputIteratorTransformer}
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
import org.apache.spark.sql.internal.SQLConf

Expand Down Expand Up @@ -52,6 +53,11 @@ class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa
super.afterAll()
}

override protected def sparkConf: SparkConf = {
super.sparkConf
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
}

test("test sort merge join metrics") {
withSQLConf(
GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false",
Expand Down Expand Up @@ -164,4 +170,18 @@ class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa
}
}
}

test("Metrics of top-n's children") {
runQueryAndCompare("SELECT c1, c2 FROM metrics_t1 order by c2 limit 5") {
df =>
val iit = find(df.queryExecution.executedPlan) {
case _: InputIteratorTransformer => true
case _ => false
}
assert(iit.isDefined)
val metrics = iit.get.metrics
assert(metrics("numOutputRows").value == 5)
assert(metrics("outputVectors").value == 1)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
*/
package org.apache.gluten.metrics

import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.utils.OASPackageBridge.InputMetricsWrapper

/**
Expand All @@ -26,16 +25,19 @@ import org.apache.spark.sql.utils.OASPackageBridge.InputMetricsWrapper
* TODO: place it to some other where since it's used not only by whole stage facilities
*/
trait MetricsUpdater extends Serializable {

def metrics: Map[String, SQLMetric]

def updateInputMetrics(inputMetrics: InputMetricsWrapper): Unit = {}

def updateNativeMetrics(operatorMetrics: IOperatorMetrics): Unit = {}
}

object NoopMetricsUpdater extends MetricsUpdater {}

final case class MetricsUpdaterTree(updater: MetricsUpdater, children: Seq[MetricsUpdaterTree])

object NoopMetricsUpdater extends MetricsUpdater {
override def metrics: Map[String, SQLMetric] = Map.empty
object MetricsUpdaterTree {
object Terminate extends MetricsUpdater {
override def updateInputMetrics(inputMetrics: InputMetricsWrapper): Unit =
throw new UnsupportedOperationException()
override def updateNativeMetrics(operatorMetrics: IOperatorMetrics): Unit =
throw new UnsupportedOperationException()
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ object MetricsUtil extends Logging {
case t: TransformSupport =>
MetricsUpdaterTree(t.metricsUpdater(), t.children.map(treeifyMetricsUpdaters))
case _ =>
MetricsUpdaterTree(NoopMetricsUpdater, Seq())
MetricsUpdaterTree(MetricsUpdaterTree.Terminate, Seq())
}
}

Expand Down Expand Up @@ -180,6 +180,8 @@ object MetricsUtil extends Logging {
)
}

// FIXME: Metrics updating code is too magical to maintain. Tree-walking algorithm should be made
// more declarative than by counting down some kind of counters that don't have fixed definition.
/**
* @return
* operator index and metrics index
Expand All @@ -192,6 +194,9 @@ object MetricsUtil extends Logging {
metricsIdx: Int,
joinParamsMap: JMap[JLong, JoinParams],
aggParamsMap: JMap[JLong, AggregationParams]): (JLong, Int) = {
if (mutNode.updater == MetricsUpdaterTree.Terminate) {
return (operatorIdx, metricsIdx)
}
val operatorMetrics = new JArrayList[OperatorMetrics]()
var curMetricsIdx = metricsIdx
relMap
Expand Down Expand Up @@ -245,18 +250,16 @@ object MetricsUtil extends Logging {

mutNode.children.foreach {
child =>
if (child.updater != NoopMetricsUpdater) {
val result = updateTransformerMetricsInternal(
child,
relMap,
newOperatorIdx,
metrics,
newMetricsIdx,
joinParamsMap,
aggParamsMap)
newOperatorIdx = result._1
newMetricsIdx = result._2
}
val result = updateTransformerMetricsInternal(
child,
relMap,
newOperatorIdx,
metrics,
newMetricsIdx,
joinParamsMap,
aggParamsMap)
newOperatorIdx = result._1
newMetricsIdx = result._2
}

(newOperatorIdx, newMetricsIdx)
Expand Down Expand Up @@ -292,8 +295,6 @@ object MetricsUtil extends Logging {
val numNativeMetrics = metrics.inputRows.length
if (numNativeMetrics == 0) {
()
} else if (mutNode.updater == NoopMetricsUpdater) {
()
} else {
updateTransformerMetricsInternal(
mutNode,
Expand All @@ -305,7 +306,7 @@ object MetricsUtil extends Logging {
aggParamsMap)
}
} catch {
case e: Throwable =>
case e: Exception =>
logWarning(s"Updating native metrics failed due to ${e.getCause}.")
()
}
Expand Down

0 comments on commit 0a98a1a

Please sign in to comment.