diff --git a/common/utils-java/src/main/java/org/apache/spark/internal/LogKeys.java b/common/utils-java/src/main/java/org/apache/spark/internal/LogKeys.java index d8238912aec63..8b2cefc1fe5ac 100644 --- a/common/utils-java/src/main/java/org/apache/spark/internal/LogKeys.java +++ b/common/utils-java/src/main/java/org/apache/spark/internal/LogKeys.java @@ -331,6 +331,10 @@ public enum LogKeys implements LogKey { LABEL_COLUMN, LARGEST_CLUSTER_INDEX, LAST_ACCESS_TIME, + LAST_ATTEMPT_ACC_INVALIDATE, + LAST_ATTEMPT_ACC_SYSTEM_METRIC, + LAST_ATTEMPT_ACC_UNEXPECTED_REASON, + LAST_ATTEMPT_ACC_USER_METRIC, LAST_COMMITTED_CHECKPOINT_ID, LAST_COMMIT_BASED_CHECKPOINT_ID, LAST_SCAN_TIME, diff --git a/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala b/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala index 810bdabebb38a..3fac57dbe5dda 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala @@ -95,6 +95,11 @@ class LogEntry(messageWithContext: => MessageWithContext) { def message: String = cachedMessageWithContext.message def context: java.util.Map[String, String] = cachedMessageWithContext.context + + def +(other: LogEntry): LogEntry = { + val combined = cachedMessageWithContext + other.cachedMessageWithContext + new LogEntry(combined) + } } /** diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fad9bb522ad92..0262144490ce8 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -3152,6 +3152,8 @@ object SparkContext extends Logging { private[spark] val RDD_SCOPE_KEY = "spark.rdd.scope" private[spark] val RDD_SCOPE_NO_OVERRIDE_KEY = "spark.rdd.scope.noOverride" private[spark] val SQL_EXECUTION_ID_KEY = "spark.sql.execution.id" + private[spark] val DATASET_QUERY_EXECUTION_ID_KEY = + "spark.sql.dataset.queryExecution.id" /** * Executor id for the driver. In earlier versions of Spark, this was ``, but this was diff --git a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala index 98b80317db982..8ecb14be1dfb8 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala @@ -39,6 +39,13 @@ private[spark] object Tests { .booleanConf .createOptional + val INJECT_SHUFFLE_FETCH_FAILURES = + ConfigBuilder("spark.testing.injectShuffleFetchFailures") + .doc("Injecting fetch failures for shuffle stages by providing an invalid BlockManager " + + "location for the first stage attempt. Testing only flag!") + .booleanConf + .createWithDefault(false) + val TEST_NO_STAGE_RETRY = ConfigBuilder("spark.test.noStageRetry") .version("1.2.0") .booleanConf diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala index 49c259999a471..675c44153cd4d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala @@ -130,6 +130,22 @@ private[spark] object RDDOperationScope extends Logging { name: String, allowNesting: Boolean, ignoreParent: Boolean)(body: => T): T = { + withScope(sc, name, allowNesting, ignoreParent, + nextScopeId().toString)(body) + } + + /** + * Execute the given body such that all RDDs created in this body + * will have the same scope, with an explicit scope ID. + * + * Note: Return statements are NOT allowed in body. + */ + private[spark] def withScope[T]( + sc: SparkContext, + name: String, + allowNesting: Boolean, + ignoreParent: Boolean, + rddScopeId: String)(body: => T): T = { // Save the old scope to restore it later val scopeKey = SparkContext.RDD_SCOPE_KEY val noOverrideKey = SparkContext.RDD_SCOPE_NO_OVERRIDE_KEY @@ -139,10 +155,12 @@ private[spark] object RDDOperationScope extends Logging { try { if (ignoreParent) { // Ignore all parent settings and scopes and start afresh with our own root scope - sc.setLocalProperty(scopeKey, new RDDOperationScope(name).toJson) + sc.setLocalProperty(scopeKey, + new RDDOperationScope(name, None, rddScopeId).toJson) } else if (sc.getLocalProperty(noOverrideKey) == null) { // Otherwise, set the scope only if the higher level caller allows us to do so - sc.setLocalProperty(scopeKey, new RDDOperationScope(name, oldScope).toJson) + sc.setLocalProperty(scopeKey, + new RDDOperationScope(name, oldScope, rddScopeId).toJson) } // Optionally disallow the child body to override our scope if (!allowNesting) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5fbd160bc683b..f3958bfddec95 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1858,6 +1858,11 @@ private[spark] class DAGScheduler( throw SparkCoreErrors.accessNonExistentAccumulatorError(id) } acc.merge(updates.asInstanceOf[AccumulatorV2[Any, Any]]) + if (acc.isInstanceOf[LastAttemptAccumulator[_, _, _]]) { + acc.asInstanceOf[LastAttemptAccumulator[_, _, _]].mergeLastAttempt( + updates, stage.rdd, event.taskInfo, + task.stageId, task.stageAttemptId, task.localProperties) + } // To avoid UI cruft, ignore cases where value wasn't updated if (acc.name.isDefined && !updates.isZero) { stage.latestInfo.accumulables(id) = acc.toInfo(None, Some(acc.value)) @@ -2333,6 +2338,19 @@ private[spark] class DAGScheduler( // The epoch of the task is acceptable (i.e., the task was launched after the most // recent failure we're aware of for the executor), so mark the task's output as // available. + // For testing purposes, inject fetch failures controlled from the driver-side by + // supplying an invalid location. + if (Utils.isTesting && + sc.conf.get(config.Tests.INJECT_SHUFFLE_FETCH_FAILURES) && + task.stageAttemptId == 0) { + val currentLocation = status.location + val invalidLocation = BlockManagerId( + execId = BlockManagerId.INVALID_EXECUTOR_ID, + host = currentLocation.host, + port = currentLocation.port, + topologyInfo = currentLocation.topologyInfo) + status.updateLocation(invalidLocation) + } val isChecksumMismatched = mapOutputTracker.registerMapOutput( shuffleStage.shuffleDep.shuffleId, smt.partitionId, status) if (isChecksumMismatched) { diff --git a/core/src/main/scala/org/apache/spark/util/LastAttemptAccumulator.scala b/core/src/main/scala/org/apache/spark/util/LastAttemptAccumulator.scala new file mode 100644 index 0000000000000..2c115b65d1e56 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/LastAttemptAccumulator.scala @@ -0,0 +1,867 @@ +/* + * 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.util + +import scala.math.Ordering.Implicits._ +import scala.reflect.ClassTag +import scala.util.control.NonFatal + +import org.apache.spark.SparkContext +import org.apache.spark.internal.{LogEntry, Logging, LogKey, LogKeys} +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.TaskInfo + +/* + * Last Attempt Accumulators are Accumulators that track the value of a metric aggregated across + * the "last execution" that produced the values. "Last execution" can be defined as: + * - For RDDs: the last execution of a given RDD partition, in the latest Stage and Stage attempt + * that recomputed it. + * - Across RDDs: lastAttemptValueForRDDId, lastAttemptValueForRDDIds, lastAttemptValueForAllRDDs, + * lastAttemptValueForHighestRDDId let specify that only values from specific RDDs should be + * aggregated. + * - For Spark SQL Execution: In SQLLastAttemptAccumulator, lastAttemptValueForDataset, + * lastAttemptValueForQueryExecution let specify that only values from the last SQL execution of + * a specific Dataset (or QueryExecution) should be aggregated. + * + * In specific situations the last attempt value cannot be computed. This is both because of known + * specific user actions (e.g. mixing driver updates with task updates), and because the + * accumulator performs (and logs) various internal sanity checks and bails out if it detects an + * unexpected situation. Therefore, all the lastAttempt methods return an Option[OUT], where None + * means that it has bailed out. + * + * Updates to the accumulator from completed Tasks are merged in mergeLastAttempt, called from + * DAGScheduler.updateAccumulators, called from DAGScheduler.handleTaskCompletion in the single + * threaded DAGScheduler event loop. Therefore, we don't need to worry about concurrency control + * when updating the accumulator values. However, reading of the last attempt value can potentially + * be done concurrently, so we use synchronization. When there is normally no contention, JVM + * synchronization should be very low overhead. + * + * In order to be able to provide last attempt value, we need to keep track of partial metric + * values, so that after a partial re-attempt the partial value can be updated, and then + * re-aggregated. + * There are various sources of re-attempts that we have to track: + * + * 1. Spark Core. + * ============== + * - Updates from failed tasks are filtered in Task.collectAccumulatorUpdates before they are + * even passed back to the driver. We don't need to worry about them here. + * - We should not get results from two successful attempts of a Task in the same Stage attempt. + * TaskSetManager.handleSuccessfulTask ensures that. + * - Therefore we only need to track Stage retries. The Last Attempt Metric will aggregate the + * metric value of a given RDD partition from the last attempt of the Stage with the highest + * stageId. + * Normally recomputation creates a new stageAttemptId in the same Stage, but there can also + * be multiple new Stages due to: + * - In AQE, a materialized QueryStage is submitted as a new Stage, which would normally get + * skipped, as it is already materialized. However, if results of that stage have been lost, + * the recomputation will happen in that Stage. + * - If the same Dataset with the same QueryExecution and same executedPlan is reused for + * another execution (e.g. again calling collect()). All map stages should be materialized, + * so like with AQE, they should be skipped, unless the results have been lost. Then, + * recomputation will happen in that Stage. The result stage computing the action will be + * fully re-executed. + * - Due to the async nature of cancellation, there can be tasks from previous attempts that + * arrive later than the last attempt. Therefore, we need to track and compare stageId and + * stageAttemptId of every computed RDD partition, in order to discard latecomers. + * + * 2. Spark SQL. + * ============= + * LastAttemptAccumulator offers simple tracking of the last SQL execution, by assuming that + * the last execution will be in the scope of an RDD with the highest id, and using + * [[lastAttemptValueForHighestRDDId]]. See SQLLastAttemptAccumulator for more possibilities + * of tracking SQL execution. + * + * Simple last SQL execution tracking + * ---------------------------------- + * Whenever an AQE replan happens, or a repeated execution is submitted, there will be a new + * RDD created for that execution. If AQE creates a new plan, it always uses it and cancels + * the previous one. So, aggregating the metric updates from the RDD with the highest id + * should correspond to the last execution and the latest AQE plan. + * This has some limitations, e.g. doesn't work if the same metric is used in multiple places + * in the query plan, and we want all occurrences to be aggregated together. + * It also wouldn't work if a SparkPlan splits its execution into multiple RDDs. This for example + * happens in BroadcastNestedLoopJoinExec with matchedStreamRows and notMatchedBroadcastRows. + * One can use this simple last attempt tracking by using lastAttemptLastRDDValue. + * + * 3. Driver only updates. + * ======================= + * Sometimes the metric is manipulated directly from the driver, not from within a Task. + * It can be either explicit by user code, or implicit by Catalyst Optimizer, for example + * ConvertToLocalRelation rule, folding a piece of the plan by evaluating it manually on the + * driver. + * When this happens, LastAttemptAccumulator has no information to reason about what was the + * last execution. If the only metric updates are coming from the driver, it assumes that these are + * the "last attempt". If there are both updates from executors and from the driver, it bails out. + * + * Implementation + * ============== + * To track the last attempts, we track a map of metric values per RDD id: + * - Map[RddId, LastAttemptRDDVals[PARTIAL]] + * + * In LastAttemptRDDVals we track an Array of per RDD partition partial merge values, together with + * the stageId and stageAttemptId and taskAttemptNumber to record task execution. + * We also track the RDD id, RDDScope id and last SQL execution id updating that RDD. + * + * Normally to merge partial values, two full Accumulators are used. However, accumulator classes + * that support Last Attempt have to implement partialMerge which merges PARTIAL type. + * This is used to have more compact representation, as PARTIAL can be e.g. a primitive type as + * opposed to a full AccumulatorV2 object instance. + */ + + +private class LastAttemptRDDVals[@specialized T]( + val rddId: Int, + val rddScopeId: Option[String], + // Arrays of partial metric values, and the corresponding stage, stage attempt and task attempt, + // with index representing RDD partition id. + // Metric updates to a given RDD partition can come from different stageAttempts if a retry + // happens while a Job with the Stage is running (a downstream Stage within a Job detects + // missing blocks and triggers recompute), or from different Stages, if a retry happens later + // (a new Job is submitted that depends on data from the RDD, if it finds it's missing it will + // recompute it in a new Stage). + // If a missing output is detected in a Stage while the stage is still running (e.g. executor + // is lost or decommissioned while the stage is running, and loses the output of some already + // finished tasks), a new Task with new taskAttemptNumber will be started for that Task. + // There may be multiple Tasks with different taskAttemptNumbers running in parallel due to + // speculation, but DAGScheduler guarantees that only one of them will reach metrics reporting, + // so it doesn't have to be dealt with here. + // + // There may be partitions that are either not computed at all (for example, due to early stop + // in take/limit), or AQE task coalescing may be visible as an update of the partition id of + // the first partition of the coalesced range. AQE guarantees that if these are retried, they + // will be coalesced in the same ranges, so update the same values. + // Not computed partitions should have EMPTY_ID in all the Int Arrays. + // + // Arrays of primitive types are more memory efficient than an array of objects due to + // references, object headers and paddings overheads. + // The `@specialized` annotation should make scala specialize it to use primitive array instead + // of boxed objects. + val partitionPartialVals: Array[T], + val stageIds: Array[Int], + val stageAttemptIds: Array[Int], + val taskAttemptNumbers: Array[Int]) + { + + // In a case of repeated execution of the same QueryExecution and reuse of the SparkPlan + // (for example multiple `collect()` on the same Dataset), a new RDD may be executed in the same + // RDDOperationScope for the new execution. Hence, we can have multiple RDDs with the same + // RDDOperationScope, coming from different SQL executions and we should only count the last one. + // However, it may also be an old RDD that is reused in the new execution, but needs to be + // partially recomputed because part of it is missing. In that case, the last attempt value needs + // to still be aggregated over the whole RDD, because the whole RDD is used in the new execution. + // Note that this only applies per RDDOperationScope/SparkPlan, because other plans in the same + // new execution may have reused their RDD in whole, and hence have the last SQL executionId + // come from an earlier execution. + // Note: This doesn't work in case a user concurrently executed multiple actions on the same + // Dataset, resulting in multiple concurrent executions trying to compute the same RDD. This + // however should not happen in practice and would likely produce other unexpected effects. + var lastSqlExecutionId: Option[Long] = None + + def numPartitions: Int = stageIds.length + + def isEmptyAt(partitionId: Int): Boolean = { + if (stageIds(partitionId) == LastAttemptRDDVals.EMPTY_ID) { + assert(stageAttemptIds(partitionId) == LastAttemptRDDVals.EMPTY_ID) + assert(taskAttemptNumbers(partitionId) == LastAttemptRDDVals.EMPTY_ID) + true + } else { + false + } + } + + def update(partialValue: AccumulatorPartialVal[T]): Unit = { + partitionPartialVals(partialValue.rddPartitionId) = partialValue.partialMergeVal + stageIds(partialValue.rddPartitionId) = partialValue.stageId + stageAttemptIds(partialValue.rddPartitionId) = partialValue.stageAttemptId + taskAttemptNumbers(partialValue.rddPartitionId) = partialValue.taskAttemptNumber + lastSqlExecutionId = partialValue.sqlExecutionId + } + + def partialValueAt(partId: Int): AccumulatorPartialVal[T] = { + AccumulatorPartialVal( + partialMergeVal = partitionPartialVals(partId), + rddId = rddId, + rddPartitionId = partId, + rddNumPartitions = stageIds.length, + rddScopeId = rddScopeId, + stageId = stageIds(partId), + stageAttemptId = stageAttemptIds(partId), + taskAttemptNumber = taskAttemptNumbers(partId), + sqlExecutionId = lastSqlExecutionId) + } + + override def toString: String = { + s"""LastAttemptVal( + | rddId=$rddId, + | rddScopeId=$rddScopeId, + | lastSqlExecutionId=$lastSqlExecutionId, + | partitionPartialVals=${partitionPartialVals.mkString("[", ",", "]")}, + | stageIds=${stageIds.mkString("[", ",", "]")}, + | stageAttemptIds=${stageAttemptIds.mkString("[", ",", "]")}, + | taskAttemptNumbers=${taskAttemptNumbers.mkString("[", ",", "]")} + |)""".stripMargin + } +} + +private object LastAttemptRDDVals { + // EMPTY_ID in stageId means that the partition was not computed. + val EMPTY_ID: Int = -1 + + def apply[@specialized T]( + rddId: Int, + rddScopeId: Option[String], + numPartitions: Int)(implicit ct: ClassTag[T]): LastAttemptRDDVals[T] = { + new LastAttemptRDDVals[T]( + rddId, + rddScopeId, + new Array[T](numPartitions), + Array.fill(numPartitions)(LastAttemptRDDVals.EMPTY_ID), + Array.fill(numPartitions)(LastAttemptRDDVals.EMPTY_ID), + Array.fill(numPartitions)(LastAttemptRDDVals.EMPTY_ID)) + } + + def createFromFirstUpdate[@specialized T]( + update: AccumulatorPartialVal[T])(implicit ct: ClassTag[T]): LastAttemptRDDVals[T] = { + val newVal = LastAttemptRDDVals[T]( + rddId = update.rddId, + rddScopeId = update.rddScopeId, + update.rddNumPartitions) + newVal.update(update) + newVal + } +} + +private class LastAttemptMap[K, V] { + // Map used to keep metric updates, keyed by RDD id or RDD scope id, backed by a List. + // In the majority of cases (when there are no stage retries and no AQE replanning + // cancelling already running stages), there will be only one key, so a list backed map + // should have less overhead. + // + // Accumulators are modified only from DAGScheduler.updateAccumulators -> mergeLastAttempt, + // which is running from a single thread (scheduling loop), so no concurrency control is needed + // for updates. Read accesses to an immutable list should use a consistent state without extra + // synchronization. + + @volatile private var map: List[(K, V)] = Nil + + def contains(key: K): Boolean = map.exists(_._1 == key) + + def get(key: K): Option[V] = map.collectFirst { case (k, v) if k == key => v } + + def put(key: K, value: V): Unit = synchronized { + map = (key, value) :: map.filterNot(_._1 == key) + } + + def keys: Iterable[K] = map.map(_._1) + def values: Iterable[V] = map.map(_._2) + def isEmpty: Boolean = map.isEmpty + def nonEmpty: Boolean = map.nonEmpty + def clear(): Unit = synchronized { map = Nil } + + override def toString: String = map + .map(elem => s"${elem._1} -> ${elem._2}").mkString("LastAttemptMap {\n", ",\n", "\n}") +} + +private case class AccumulatorPartialVal[PARTIAL]( + partialMergeVal: PARTIAL, + rddId: Int, + rddPartitionId: Int, + rddNumPartitions: Int, + rddScopeId: Option[String], + stageId: Int, + stageAttemptId: Int, + taskAttemptNumber: Int, + sqlExecutionId: Option[Long] +) { + override def toString: String = { + s"""AccumulatorPartialVal( + | partialMergeVal=$partialMergeVal, + | rddId=$rddId, + | rddPartitionId=$rddPartitionId, + | rddNumPartitions=$rddNumPartitions, + | rddScopeId=$rddScopeId, + | stageId=$stageId, + | stageAttemptId=$stageAttemptId, + | taskAttemptNumber=$taskAttemptNumber, + | sqlExecutionId=$sqlExecutionId + |)""".stripMargin + } + + /** Tuple of stage id, stage attempt id and taskAttemptNumber, defining the order of attempts. */ + val attempt: (Int, Int, Int) = (stageId, stageAttemptId, taskAttemptNumber) +} + +/** + * A trait that can be mixed into a subclass of [[AccumulatorV2]] to track the "logical" + * value of the "last attempt" of the execution using the accumulator - aggregated from the last + * attempts of any Task that calculated some RDD partitions and used this accumulator, and + * discarding any values coming from earlier attempts that have been recomputed. + * If the accumulator is used by multiple RDDs, the last attempt value is tracked separately for + * each, and can be retrieved for each or all of them separately, see lastAttemptValueForX methods. + * If the accumulator is used directly on the Spark Driver using [[AccumulatorV2.add]], + * that value is considered the last attempt value. + * If the accumulator was both used in Tasks and updated directly on the driver, it can't determine + * what should be considered the last attempt, and lastAttemptValueForX methods will return None. + * + * Contract for driver-only updates: + * A driver-side value (set via [[AccumulatorV2.add]] on the driver, outside any Task) is only + * returned by methods that do not narrow by RDD, namely [[lastAttemptValueForAllRDDs]] and + * [[lastAttemptValueForHighestRDDId]]. Methods that narrow to specific RDDs or RDD scopes + * ([[lastAttemptValueForRDDId]], [[lastAttemptValueForRDDIds]], [[lastAttemptValueForRDDScopes]]) + * return the zero value when a driver-only value is present, because a driver-side update cannot + * be attributed to any particular RDD or scope. + * + * [[LastAttemptAccumulator]] is not reset by the [[AccumulatorV2.reset]] method implementation, + * and its state is not copied by the [[AccumulatorV2.copy]] method implementation, and it should + * not be serialized to the Executors. The internal state should only be initialized by the + * [[initializeLastAttemptAccumulator]] method on the "main" instance of the accumulator, that was + * created and registered with [[AccumulatorContext]] with AccumulatorV2.register. All the + * interfaces of [[LastAttemptAccumulator]]: [[mergeLastAttempt]] (used only by DAGScheduler) and + * lastAttemptValueForX, [[logAccumulatorState]] (used by the using code) should only be invoked on + * that instance, on the Spark Driver. + * + * The [[LastAttemptAccumulator]] is not thread-safe. [[mergeLastAttempt]] should only be used by + * DAGScheduler, by the scheduler thread. Retrieving the value using lastAttemptValueForXXX while + * it is concurrently updated (execution is running) can produce some inconsistencies, but should + * not crash. + * If an RDD using the [[LastAttemptAccumulator]] is used concurrently by multiple actions that + * all try to recompute it, it may produce unexpected results and the semantics of what is "last + * attempt" becomes ambiguous. This should not be done in practice, and will likely result in more + * unexpected behaviours in Spark. + * + * Implementations must implement [[partialMergeVal]] and [[partialMerge]] methods operating on + * PARTIAL type. In regular [[AccumulatorV2]] implementations, the [[AccumulatorV2]] object + * itself holds the intermediate value of the accumulator, and [[AccumulatorV2.merge]] method is + * used to merge these objects together. [[LastAttemptAccumulator]] needs to keep track of partial + * values of every partition of every RDD that used the accumulator, and holding a full + * [[AccumulatorV2]] object for each would have a high overhead. Therefore, an implementation should + * be able to return PARTIAL value from [[partialMergeVal]] that represents an intermediate + * mergeable value, and a [[partialMerge]] method that can merge that value into the accumulator. + * Implementations must also implement an [[isMergeable]] method that checks if the other + * [[AccumulatorV2]] is of a compatible type to be merged with this using [[partialMergeVal]]. In + * regular [[AccumulatorV2]] implementations, this check is normally done inside the + * [[AccumulatorV2.merge]] method, which is not used here. + * + * If an implementation is used to keep user data in the accumulator, it should override + * [[accumulatorStoresUserData]] to return true, to ensure correct structured logging annotation. + * Otherwise it should override it to false. + */ +trait LastAttemptAccumulator[IN, OUT, PARTIAL] extends Logging { + this: AccumulatorV2[IN, OUT] => + + // For every RDD that participated in the computation of this accumulator, keep the partial + // value of the accumulator for the latest stage and stage attempt that computed it. + // Keyed by rdd.id. + // Only kept and accessed on the driver, in the instance of the LastAttemptAccumulator that was + // created and registered with AccumulatorContext with AccumulatorV2.register(). + // Should not be copied / reset by the implementation of copy() / reset() functions. + // Transient: only needed on the driver and doesn't need to be serialized. + @transient + private var lastAttemptRddsMap: LastAttemptMap[Int, LastAttemptRDDVals[PARTIAL]] = _ + + // ClassTag for PARTIAL, captured at initialization time. + @transient private var partialClassTag: ClassTag[PARTIAL] = _ + + // Metric value set directly on the driver, not from within a task. + // Only kept and accessed on the driver, in the instance of the LastAttemptAccumulator that was + // created and registered with AccumulatorContext with AccumulatorV2.register(). + // Should not be copied / reset by the implementation of copy() / reset() functions. + // Transient: only needed on the driver and doesn't need to be serialized. + @transient + private var lastAttemptDirectDriverValue: Option[OUT] = _ + + // Flipped to true if unexpected metrics updates are received and we can no longer reason + // about the last attempt. + // Should not be copied / reset by the implementation of copy() / reset() functions. + // Transient: only needed on the driver and doesn't need to be serialized. + @transient + protected var lastAttemptAccumulatorInvalid: Boolean = false + + // Indicates that the LastAttemptAccumulator has been initialized. + // It is initialized in assertValid(). + // Should not be copied / reset by the implementation of copy() / reset() functions. + // Transient: only needed on the driver and doesn't need to be serialized. + @transient + protected var lastAttemptAccumulatorInitialized: Boolean = false + + /** Reset the state of the last attempt accumulator, discarding all the past attempts, and + * making it valid again if it was invalidated. */ + def resetLastAttemptAccumulator(): Unit = try { + lastAttemptRddsMap.clear() + lastAttemptDirectDriverValue = None + lastAttemptAccumulatorInvalid = false + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in resetLastAttemptAccumulator", + exception = Some(e)) + } + + def initializeLastAttemptAccumulator()(implicit ct: ClassTag[PARTIAL]): Unit = try { + assert(isAtDriverSide) + assert(!lastAttemptAccumulatorInitialized) + assert(!lastAttemptAccumulatorInvalid) + assert(lastAttemptRddsMap == null) + assert(lastAttemptDirectDriverValue == null) + partialClassTag = ct + lastAttemptRddsMap = new LastAttemptMap[Int, LastAttemptRDDVals[PARTIAL]] + lastAttemptDirectDriverValue = None + lastAttemptAccumulatorInitialized = true + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in initializeLastAttemptAccumulator", + exception = Some(e)) + } + + private def accumulatorId: Long = { + // This can throw if this is a copy/serialized accumulator, + // not the instance registered with AccumulatorContext. + // Catch it so we can safely use it for logging in unexpected situations. + try { + this.id + } catch { + case NonFatal(e) => + logWarning(log"Unexpected exception in getting accumulator id", e) + -1L // needs to be a long for LogKeys.ACCUMULATOR_ID + } + } + + /** Log entry to log debug information about the internal state of the accumulator. */ + def logAccumulatorState: LogEntry = try { + log"""LastAttemptAccumulator id=${MDC(LogKeys.ACCUMULATOR_ID, accumulatorId)}: + |Invalidated: ${MDC(LogKeys.LAST_ATTEMPT_ACC_INVALIDATE, lastAttemptAccumulatorInvalid)}. + |Direct driver value: ${MDC(logKeyAccumulatorState, lastAttemptDirectDriverValue)}. + |Value: ${MDC(logKeyAccumulatorState, value)}. + |lastAttemptRddsMap: + |${MDC(logKeyAccumulatorState, lastAttemptRddsMap)}.""" + .stripMargin + } catch { + case NonFatal(e) => + logWarning(log"Unexpected exception in logAccumulatorState", e) + log"" + } + + private def logAccumulatorUpdate( + newAccumPartialValue: Option[AccumulatorPartialVal[PARTIAL]] = None, + oldAccumPartialValue: Option[AccumulatorPartialVal[PARTIAL]] = None): LogEntry = try { + log"""Old partial RDD value: ${MDC(logKeyAccumulatorState, oldAccumPartialValue)}. + |New partial RDD value: ${MDC(logKeyAccumulatorState, newAccumPartialValue)}.""" + .stripMargin + } catch { + case NonFatal(e) => + logWarning(log"Unexpected exception in logAccumulatorUpdate", e) + log"" + } + + private def unexpectedLastAttemptMetricUpdate( + invalidate: Boolean, + reason: String, + exception: Option[Throwable] = None, + newAccumPartialValue: Option[AccumulatorPartialVal[PARTIAL]] = None, + oldAccumPartialValue: Option[AccumulatorPartialVal[PARTIAL]] = None): Unit = { + val logEntry = + log"""Unexpected last attempt tracking for accumulator ${ + MDC(LogKeys.ACCUMULATOR_ID, accumulatorId)}. + |Invalidate: ${MDC(LogKeys.LAST_ATTEMPT_ACC_INVALIDATE, invalidate)}. + |Reason: ${MDC(LogKeys.LAST_ATTEMPT_ACC_UNEXPECTED_REASON, reason)}. + |""".stripMargin + + log"State:\n" + logAccumulatorState + + log"Update:\n" + logAccumulatorUpdate(newAccumPartialValue, oldAccumPartialValue) + exception match { + case Some(e) => logWarning(logEntry, e) + case None => logWarning(logEntry) + } + if (invalidate) { + lastAttemptAccumulatorInvalid = true + } + if (Utils.isTesting && lastAttemptAccumulatorInitialized && exception.isDefined) { + // If this is a test, rethrow the exception. + // (Rethrow only if lastAttemptAccumulatorInitialized. In some tests, we check for proper + // graceful handling of unexpected exceptions in accumulators that are not properly + // initialized, so we don't want to throw there.) + throw exception.get + } + } + + protected def unexpectedLastAttemptMetricOperation( + invalidate: Boolean, + reason: String, + exception: Option[Throwable] = None): Unit = { + // subclasses don't have visibility of private class AccumulatorPartialVal. + unexpectedLastAttemptMetricUpdate( + invalidate = invalidate, + reason = reason, + exception = exception, + newAccumPartialValue = None, + oldAccumPartialValue = None) + } + + /** Set of assertions that should always hold for a valid [[LastAttemptAccumulator]]. */ + protected def assertValid(): Unit = { + assert(lastAttemptAccumulatorInitialized) + assert(!lastAttemptAccumulatorInvalid) + assert(isAtDriverSide) + assert(metadata != null) + assert(!metadata.countFailedValues) + assert(lastAttemptDirectDriverValue.isEmpty || lastAttemptRddsMap.isEmpty) + } + + /** + * Accumulator subclasses where metric values can contain user data (for example, maximum of + * processed values, observable metrics) as opposed to system measurements (for example, count + * of processed rows) should return true to ensure correct structured logging annotation. + */ + protected def accumulatorStoresUserData: Boolean + + protected def logKeyAccumulatorState: LogKey = { + if (accumulatorStoresUserData) { + LogKeys.LAST_ATTEMPT_ACC_USER_METRIC + } else { + LogKeys.LAST_ATTEMPT_ACC_SYSTEM_METRIC + } + } + + /** Return intermediate value of PARTIAL type that can be merged together by partialMerge. */ + protected def partialMergeVal: PARTIAL + + /** Merge together partial values of PARTIAL type returned by partialMergeVal. */ + protected def partialMerge(otherVal: PARTIAL): Unit + + /** Check if the other accumulator is mergeable with this one. */ + protected def isMergeable(other: AccumulatorV2[_, _]): Boolean + + /** + * Check if the value is set on the driver side, not from within a task. + * This must be called from `add` and `set` methods of any AccumulatorV2 subclass supporting + * last attempt metrics to set what the `value` of the metric is after the operation. + */ + protected def setValueIfOnDriverSide(value: OUT): Unit = try { + if (isAtDriverSide && lastAttemptAccumulatorInitialized && !lastAttemptAccumulatorInvalid) { + // Direct update on the driver, not from within a task. + // This gives little information about the source of the update, so we can't reason about + // "last attempt" if it's mixed with non-driver updates. + lastAttemptDirectDriverValue = Some(value) + if (lastAttemptRddsMap.nonEmpty) { + unexpectedLastAttemptMetricUpdate( + invalidate = true, + reason = "Incoming direct driver value while task updates exist") + } + } + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in setValueIfOnDriverSide", + exception = Some(e)) + } + + /** + * It needs Task and Stage information to reason about the last attempt. + * + * Called from a single thread in DAGScheduler, no synchronization needed. + * Should be used only on the Spark Driver, on the instance of [[LastAttemptAccumulator]] that + * was created and registered in [[AccumulatorContext]] by AccumulatorV2.register. + */ + private[spark] def mergeLastAttempt( + other: AccumulatorV2[_, _], + rdd: RDD[_], + taskInfo: TaskInfo, + stageId: Int, + stageAttemptId: Int, + localProperties: java.util.Properties): Unit = try { + implicit val ct: ClassTag[PARTIAL] = partialClassTag + if (lastAttemptAccumulatorInvalid) return + // Skip zero-value updates. They contribute nothing to the aggregate and can come + // from stages where the accumulator was present in the task closure but never incremented. + if (other.isZero) return + assertValid() + + if (!isMergeable(other)) { + // This should never happen. + unexpectedLastAttemptMetricUpdate( + invalidate = true, + "Merging accumulators of different types") + return + } + + if (!other.isInstanceOf[LastAttemptAccumulator[_, _, _]]) { + // This should never happen. + unexpectedLastAttemptMetricUpdate( + invalidate = true, + "Merging with accumulator which is not SLAM") + return + } + val lastAttemptOther = other + .asInstanceOf[LastAttemptAccumulator[IN, OUT, PARTIAL]] + + val update = AccumulatorPartialVal( + partialMergeVal = lastAttemptOther.partialMergeVal, + rddId = rdd.id, + rddPartitionId = taskInfo.partitionId, + rddNumPartitions = rdd.getNumPartitions, + rddScopeId = rdd.scope.map(_.id), + stageId = stageId, + stageAttemptId = stageAttemptId, + taskAttemptNumber = taskInfo.attemptNumber, + sqlExecutionId = + Option(localProperties.getProperty(SparkContext.SQL_EXECUTION_ID_KEY)).map(_.toLong)) + + if (lastAttemptDirectDriverValue.nonEmpty) { + unexpectedLastAttemptMetricUpdate(invalidate = true, + "Incoming task updates while direct driver value exists", + newAccumPartialValue = Some(update)) + return + } + + lastAttemptRddsMap.get(update.rddId) match { + case Some(oldRDDValue) => // This RDD was already seen. + val oldValue = oldRDDValue.partialValueAt(update.rddPartitionId) + + logTrace(log"mergeLastAttempt existing RDD update:\n" + + log"${MDC(logKeyAccumulatorState, oldRDDValue)}\n" + + logAccumulatorUpdate( + newAccumPartialValue = Some(update), oldAccumPartialValue = Some(oldValue))) + + // Check basic consistency + if (oldValue.rddNumPartitions != update.rddNumPartitions) { + unexpectedLastAttemptMetricUpdate( + invalidate = true, + reason = "RDD with changing number of partitions", + newAccumPartialValue = Some(update), + oldAccumPartialValue = Some(oldValue)) + return + } + if (oldValue.rddScopeId != update.rddScopeId) { + unexpectedLastAttemptMetricUpdate( + invalidate = true, + reason = "RDD with changing RDDOperationScope", + newAccumPartialValue = Some(update), + oldAccumPartialValue = Some(oldValue)) + return + } + + if (oldRDDValue.isEmptyAt(update.rddPartitionId)) { + // No previous attempt for this RDD partition. + oldRDDValue.update(update) + } else { + if (update.attempt > oldValue.attempt) { + // New last attempt for this RDD partition. + oldRDDValue.update(update) + } else if (update.attempt == oldValue.attempt) { + // Same attempt, should not happen. + unexpectedLastAttemptMetricUpdate( + invalidate = true, + reason = "Same stage, stageAttemptId and taskAttemptNumber reported multiple times", + newAccumPartialValue = Some(update), + oldAccumPartialValue = Some(oldValue)) + } + // else: Older attempt reported after newer attempt. Not fatal, discard it. + } + + case None => // First time we see this RDD. + logTrace(log"mergeLastAttempt new RDD update:\n" + logAccumulatorUpdate( + newAccumPartialValue = Some(update), oldAccumPartialValue = None)) + val newVal = LastAttemptRDDVals.createFromFirstUpdate(update) + lastAttemptRddsMap.put(update.rddId, newVal) + } + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricUpdate( + invalidate = true, + reason = "Unexpected exception in mergeLastAttempt", + exception = Some(e)) + } + + /** Accumulates last attempt values from given RDD into an acc. */ + private def lastAttemptValueAggregateInternal(rddId: Int, acc: this.type) = { + // Note: even if the given RDD is not present, we can't tell if it executed but just never + // updated this accumulator, so we still report the zero value back. + for { + lastAttemptVal <- lastAttemptRddsMap.get(rddId) + partitionId <- lastAttemptVal.partitionPartialVals.indices + } { + // Some partitions may not be computed. + // May be because of operations like take. + // May be because of AQE coalescing executing tasks covering multiple partitions. + if (!lastAttemptVal.isEmptyAt(partitionId)) { + acc.partialMerge(lastAttemptVal.partitionPartialVals(partitionId)) + } + } + } + + /** + * Returns the last attempt value of this accumulator, aggregated from a set of RDDs. + * + * Should be used only on the Spark Driver, on the instance of [[LastAttemptAccumulator]] that + * was created and registered in [[AccumulatorContext]] by AccumulatorV2.register. + * + * @return None if the last attempt value cannot be established, Some(value) otherwise. + */ + def lastAttemptValueForRDDIds(rddIds: Seq[Int]): Option[OUT] = try { + if (lastAttemptAccumulatorInvalid) return None + assertValid() + if (lastAttemptDirectDriverValue.isDefined) { + // return zero value if there is no RDD execution recorded. + return Some(copyAndReset().asInstanceOf[this.type].value) + } + + val acc = copyAndReset().asInstanceOf[this.type] + rddIds.distinct.foreach(lastAttemptValueAggregateInternal(_, acc)) + Some(acc.value) + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in lastAttemptValueForRDDs", + exception = Some(e)) + None + } + + /** + * Returns the last attempt value of this accumulator, aggregated from a specific RDD. + * + * Should be used only on the Spark Driver, on the instance of [[LastAttemptAccumulator]] that + * was created and registered in [[AccumulatorContext]] by AccumulatorV2.register. + * + * @return None if the last attempt value cannot be established, Some(value) otherwise. + */ + def lastAttemptValueForRDDId(rddId: Int): Option[OUT] = try { + lastAttemptValueForRDDIds(Seq(rddId)) + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in lastAttemptValueForRDD", + exception = Some(e)) + None + } + + /** + * Returns the last attempt value of this accumulator, aggregated from all RDDs that ever + * returned any values for it. + * + * If the metric was used directly on the driver, and was not used in any RDD execution, + * the driver value will be used instead. + * + * Should be used only on the Spark Driver, on the instance of [[LastAttemptAccumulator]] that + * was created and registered in [[AccumulatorContext]] by AccumulatorV2.register. + * + * @return None if the last attempt value cannot be established, Some(value) otherwise. + */ + def lastAttemptValueForAllRDDs(): Option[OUT] = try { + if (lastAttemptAccumulatorInvalid) return None + assertValid() + if (lastAttemptDirectDriverValue.isDefined) return lastAttemptDirectDriverValue + lastAttemptValueForRDDIds(lastAttemptRddsMap.keys.toSeq) + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in lastAttemptValueForAllRDDs", + exception = Some(e)) + None + } + + /** + * Returns the last attempt value of this accumulator, aggregated from the RDD with the highest + * id that ever returned any values for it. + * + * If the metric was used directly on the driver, and was not used in any RDD execution, + * the driver value will be used instead. + * + * Should be used only on the Spark Driver, on the instance of [[LastAttemptAccumulator]] that + * was created and registered in [[AccumulatorContext]] by AccumulatorV2.register. + * + * @return None if the last attempt value cannot be established, Some(value) otherwise. + */ + def lastAttemptValueForHighestRDDId(): Option[OUT] = try { + if (lastAttemptAccumulatorInvalid) return None + assertValid() + if (lastAttemptDirectDriverValue.isDefined) return lastAttemptDirectDriverValue + + if (lastAttemptRddsMap.nonEmpty) { + lastAttemptValueForRDDId(lastAttemptRddsMap.keys.max) + } else { + // return zero value if there is no RDD execution recorded. + Some(copyAndReset().asInstanceOf[this.type].value) + } + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in lastAttemptValueForHighestRDDId", + exception = Some(e)) + None + } + + /** + * Returns the last attempt value of this accumulator, aggregated from RDDs with given scope ids. + * + * Should be used only on the Spark Driver, on the instance of [[LastAttemptAccumulator]] that + * was created and registered in [[AccumulatorContext]] by AccumulatorV2.register. + * + * @return None if the last attempt value cannot be established, Some(value) otherwise. + */ + def lastAttemptValueForRDDScopes(rddScopeIds: Seq[String]): Option[OUT] = try { + if (lastAttemptAccumulatorInvalid) return None + assertValid() + if (lastAttemptDirectDriverValue.isDefined) { + // Return zero value if there is no RDD execution recorded. + return Some(copyAndReset().asInstanceOf[this.type].value) + } + val scopesLookup = rddScopeIds.toSet + val matchingRDDs = lastAttemptRddsMap.values.filter { rddVal => + rddVal.rddScopeId.exists(scopesLookup.contains) + }.toSeq + // When multiple RDDs share the same scope (e.g. repeated Dataset.collect() calls create + // new wrapper RDDs in the same scope, or BroadcastNestedLoopJoin executing the probe side + // twice), only aggregate the latest one per scope, identified by the highest RDD id. + // RDD ids are globally monotonic, so the highest id is the latest. + val rddIds = matchingRDDs.groupBy(_.rddScopeId).values.map(_.maxBy(_.rddId).rddId).toSeq + lastAttemptValueForRDDIds(rddIds) + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in lastAttemptValueForRDDScopes", + exception = Some(e)) + None + } + + /** Visible for testing. */ + def getDirectDriverValue: Option[OUT] = { + lastAttemptDirectDriverValue + } + + /** Visible for testing */ + def getHighestRDDId: Option[Int] = { + if (lastAttemptRddsMap.nonEmpty) Some(lastAttemptRddsMap.keys.max) else None + } + + /** Visible for testing */ + def getNumRDDs: Int = { + lastAttemptRddsMap.keys.size + } + + /** Visible for testing */ + def getValid: Boolean = { + !lastAttemptAccumulatorInvalid + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 866a535c6d951..e387817b4a428 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1699,7 +1699,10 @@ object Unidoc { "-tag", "todo:X", "-tag", "groupname:X", "-tag", "inheritdoc", - "--ignore-source-errors", "-notree" + "--ignore-source-errors", "-notree", + "-Xmaxerrs", "999999", + "-Xmaxwarns", "999999", + "-Xdoclint:reference" ) }, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala index f2827faf59435..9c310bf3d9fb1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala @@ -164,7 +164,7 @@ object InMemoryTableWithV2Filter { /** * Evaluates a single V2 predicate by resolving column values through the - * given function. Supports =, <=>, IS_NULL, IS_NOT_NULL, and ALWAYS_TRUE. + * given function. Supports `=`, `<=>`, `IS_NULL`, `IS_NOT_NULL`, and `ALWAYS_TRUE`. */ def evalPredicate( pred: Predicate, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala index 5bef4e35ba57e..91d51163b319e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala @@ -2259,9 +2259,11 @@ class Dataset[T] private[sql]( */ private def withAction[U](name: String, qe: QueryExecution)(action: SparkPlan => U) = { SQLExecution.withNewExecutionId(qe, Some(name)) { - QueryExecution.withInternalError(s"""The "$name" action failed.""") { - qe.executedPlan.resetMetrics() - action(qe.executedPlan) + qe.withQueryExecutionId(sparkSession) { + QueryExecution.withInternalError(s"""The "$name" action failed.""") { + qe.executedPlan.resetMetrics() + action(qe.executedPlan) + } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index f08b561d6ef9a..c0ab906de4841 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.Path -import org.apache.spark.SparkException +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.EXTENDED_EXPLAIN_GENERATOR import org.apache.spark.rdd.RDD @@ -327,7 +327,30 @@ class QueryExecution( protected def executePhase[T](phase: String)(block: => T): T = sparkSession.withActive { QueryExecution.withInternalError(s"The Spark SQL phase $phase failed with an internal error.") { - tracker.measurePhase(phase)(block) + withQueryExecutionId(sparkSession) { + tracker.measurePhase(phase)(block) + } + } + } + + /** + * Set the query execution id in thread-local properties while + * executing the block. This is used by + * [[org.apache.spark.sql.execution.metric.SQLLastAttemptAccumulator]] to associate + * driver-side metric updates with a specific QueryExecution. + */ + private[sql] def withQueryExecutionId[T]( + session: SparkSession)(block: => T): T = { + val sc = session.sparkContext + val oldId = sc.getLocalProperty( + SparkContext.DATASET_QUERY_EXECUTION_ID_KEY) + sc.setLocalProperty( + SparkContext.DATASET_QUERY_EXECUTION_ID_KEY, id.toString) + try { + block + } finally { + sc.setLocalProperty( + SparkContext.DATASET_QUERY_EXECUTION_ID_KEY, oldId) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 7f94cc77f3454..cf2d0218d0fdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -249,12 +249,21 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ doExecuteWrite(writeFilesSpec) } + /** + * A deterministic scope ID for RDDs created by this SparkPlan, + * used by LastAttemptAccumulator to track which RDD belongs + * to which SparkPlan node. + */ + private[spark] def rddScopeId: String = + "spark_plan_" + id.toString + /** * Executes a query after preparing the query and adding query plan information to created RDDs * for visualization. */ protected final def executeQuery[T](query: => T): T = { - RDDOperationScope.withScope(sparkContext, nodeName, false, true) { + RDDOperationScope.withScope( + sparkContext, nodeName, false, true, rddScopeId) { prepare() waitForSubqueries() query @@ -375,6 +384,11 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ */ private def getByteArrayRdd( n: Int = -1, takeFromEnd: Boolean = false): RDD[(Long, ChunkedByteBuffer)] = { + // Wrap in the plan's RDD scope so that the wrapper RDD created by mapPartitionsInternal + // inherits this plan's deterministic scope ID rather than getting an anonymous auto-generated + // one. + val rdd = RDDOperationScope.withScope( + sparkContext, nodeName, false, true, rddScopeId) { execute().mapPartitionsInternal { iter => var count = 0 val buffer = new Array[Byte](4 << 10) // 4K @@ -409,8 +423,10 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ out.writeInt(-1) out.flush() out.close() - Iterator((count, cbbos.toChunkedByteBuffer)) + Iterator((count.toLong, cbbos.toChunkedByteBuffer)) + } } + rdd } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQETestHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQETestHelper.scala new file mode 100644 index 0000000000000..7e78ee41900ce --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQETestHelper.scala @@ -0,0 +1,79 @@ +/* + * 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.adaptive + +import scala.collection.mutable + +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.util.{AccumulatorContext, Utils} + +/** Testing only helpers for AQE. */ +object AQETestHelper { + // See [withForcedCancellation]. + @volatile private var metricIdsForForcedCancellation: Set[Long] = Set.empty + + /** + * Set `triggerMetrics` to induce a forced cancellation into the execution when any of the + * metrics is non-empty. + * In this case the results will be discarded and the stage re-run, causing the metrics to be + * incremented again. + */ + def withForcedCancellation[T](triggerMetrics: SQLMetric*)(thunk: => T): T = { + metricIdsForForcedCancellation = triggerMetrics.map(_.id).toSet + val res = try { + thunk + } finally { + metricIdsForForcedCancellation = Set.empty + forcedCancellationTriggeredForPlans.clear() + } + res + } + + /* + * Track for which plans we have already triggered the forced replanning so we only do it once. + */ + private val forcedCancellationTriggeredForPlans = mutable.HashSet.empty[Int] + + /** Return `true` if forced cancellation mechanism is enabled. */ + def isForcedCancellationEnabled: Boolean = + Utils.isTesting && metricIdsForForcedCancellation.nonEmpty + + /** Return `true` if forced cancellation has already been triggered for `plan`. */ + private def wasForcedCancellationTriggeredForPlan(plan: SparkPlan): Boolean = synchronized { + forcedCancellationTriggeredForPlans.contains(plan.id) + } + + /** Mark that force cancellation was successfully triggered for `plan`. */ + def markForcedCancellationTriggeredForPlan(plan: SparkPlan): Unit = synchronized { + assert(!forcedCancellationTriggeredForPlans.contains(plan.id), + "A plan was forced to cancel a second time.") + forcedCancellationTriggeredForPlans += plan.id + } + + /** Return `true` if we should try to force cancellation for `plan` at this point. */ + def shouldForceCancellation(plan: SparkPlan): Boolean = { + // Trigger the forced cancellation only if we are in testing + Utils.isTesting && + // ...and if we haven't triggered it yet + !wasForcedCancellationTriggeredForPlan(plan) && + // ...and if any of the trigger metrics > 0. + metricIdsForForcedCancellation.exists { id => + AccumulatorContext.get(id).map(!_.isZero).getOrElse(false) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 4840016bf745d..112ee82314c4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -340,6 +340,8 @@ case class AdaptiveSparkPlanExec( if (errors.nonEmpty) { cleanUpAndThrowException(errors.toSeq, None) } + val testTriggerForceCancellation = AQETestHelper.shouldForceCancellation(this) + if (!currentPhysicalPlan.isInstanceOf[ResultQueryStageExec]) { // Try re-optimizing and re-planning. Adopt the new plan if its cost is equal to or less // than that of the current plan; otherwise keep the current physical plan together with @@ -352,14 +354,21 @@ case class AdaptiveSparkPlanExec( // the current physical plan. Once a new plan is adopted and both logical and physical // plans are updated, we can clear the query stage list because at this point the two // plans are semantically and physically in sync again. - val logicalPlan = replaceWithQueryStagesInLogicalPlan(currentLogicalPlan, stagesToReplace) + var logicalPlan = replaceWithQueryStagesInLogicalPlan(currentLogicalPlan, stagesToReplace) + if (testTriggerForceCancellation) { + // Force unwrap all LogicalQueryStage so they get replanned. + logicalPlan = logicalPlan.transformDown { + case LogicalQueryStage(logical, _) => logical + } + } val afterReOptimize = reOptimize(logicalPlan) if (afterReOptimize.isDefined) { val (newPhysicalPlan, newLogicalPlan) = afterReOptimize.get val origCost = costEvaluator.evaluateCost(currentPhysicalPlan) val newCost = costEvaluator.evaluateCost(newPhysicalPlan) if (newCost < origCost || - (newCost == origCost && currentPhysicalPlan != newPhysicalPlan)) { + (newCost == origCost && currentPhysicalPlan != newPhysicalPlan) || + testTriggerForceCancellation) { lazy val plans = sideBySide( currentPhysicalPlan.treeString, newPhysicalPlan.treeString).mkString("\n") logOnLevel(log"Plan changed:\n${MDC(QUERY_PLAN, plans)}") @@ -369,6 +378,9 @@ case class AdaptiveSparkPlanExec( stagesToReplace = Seq.empty[QueryStageExec] } } + if (testTriggerForceCancellation) { + AQETestHelper.markForcedCancellationTriggeredForPlan(this) + } } // Now that some stages have finished, we can try creating new stages. result = createQueryStages(fun, currentPhysicalPlan, firstRun = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala index 2556edee8d02f..eea664b29fd52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala @@ -94,6 +94,21 @@ trait AdaptiveSparkPlanHelper { collect(p) { case plan if allChildren(plan).isEmpty => plan } } + /** + * Returns true if the condition specified by `f` is satisfied by any node in this tree. + */ + def exists(p: SparkPlan)(f: SparkPlan => Boolean): Boolean = { + find(p)(f).isDefined + } + + /** + * Like [[exists]], but also considers plan nodes inside subqueries. + */ + def existsWithSubqueries( + p: SparkPlan)(f: SparkPlan => Boolean): Boolean = { + exists(p)(f) || subqueriesAll(p).exists(exists(_)(f)) + } + /** * Finds and returns the first [[SparkPlan]] of the tree for which the given partial function * is defined (pre-order), and applies the partial function to it. @@ -138,3 +153,5 @@ trait AdaptiveSparkPlanHelper { case other => other } } + +private[sql] object AdaptiveSparkPlanHelper extends AdaptiveSparkPlanHelper diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 7dcbf3779b93d..7f757c651c560 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -241,17 +241,22 @@ case class ShuffleExchangeExec( */ @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 + // Wrap in the exchange's RDD scope so that any wrapper RDDs created during shuffle dependency + // preparation (e.g. by prepareShuffleDependency's mapPartitionsInternal calls) get this + // exchange's scope ID. + RDDOperationScope.withScope(sparkContext, nodeName, false, true, rddScopeId) { + 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 + } } protected override def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptAccumulator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptAccumulator.scala new file mode 100644 index 0000000000000..114d3974bb0ee --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptAccumulator.scala @@ -0,0 +1,435 @@ +/* + * 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.metric + +import scala.collection.mutable +import scala.reflect.ClassTag +import scala.util.control.NonFatal + +import org.apache.spark.SparkContext +import org.apache.spark.internal.{LogEntry, Logging} +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.execution.{BaseSubqueryExec, QueryExecution, SparkPlan, SubqueryAdaptiveBroadcastExec, SubqueryBroadcastExec, SubqueryExec, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, BroadcastExchangeLike, ReusedExchangeExec, ShuffleExchangeExec, ShuffleExchangeLike} +import org.apache.spark.util.{AccumulatorV2, LastAttemptAccumulator} + +/* + * SQLLastAttemptAccumulator is a LastAttemptAccumulator that allows tracking the last attempt + * updates that happened in the scope of execution of a plan created by a specific Dataset's + * QueryExecution. + * + * Tracking RDDs belonging to a Dataset execution. + * ----------------------------------------------- + * Dataset executes executedPlan from its QueryExecution. Each SparkPlan node in the + * executedPlan saves the RDD with its execution (executeRDD or executeColumnarRDD). However, + * the root RDD of a Spark Stage that actually gets submitted and executed is not necessarily + * that RDD. It may be an ephemeral RDD created on the fly when submitting the job, e.g.: + * - for result stages, there may be additional transformations to format the results, like + * apply an Encoder (e.g. turn InternalRows into Rows) + * or transformations for Arrow + * - for map stages, there may be some additional transformations to prepare the shuffle + * data in correct format. + * - operations like dataframe caching may wrap the plan results to format and write to cache. + * We therefore cannot track the metrics updates just by RDD id. However, each SparkPlan also + * creates an RDDOperationScope, and wraps the execution it submits by that scope. + * The completed Tasks should have RDDOperationScope of the SparkPlan that submitted the + * Stage. We need to extract the RDDOperationScopes from Dataset.queryExecution.executedPlan + * to track last attempt metric updates coming from that execution. + * + * Additionally, it is possible that the same queryExecution.executedPlan is reused. For + * example, when collect() is called multiple times on the same Dataset. + * - Part of the execution (e.g. the shuffles) should then be reused. Accumulator should still + * keep their partial values associated to its RDDOperationScope, and return it for this + * new attempt. + * - Some of the execution (e.g. the result stage) may be recomputed. Since the SparkPlan will + * be the same, RDDOperationScope will be the same, and this should become a newer execution + * of the same RDD, which should replace the previous one. + * + * AQE plan changes + * ---------------- + * AQE re-optimizes LogicalPlan and creates new SparkPlan. If the new plan doesn't contain + * some of the QueryStages from the previous plan, they can be cancelled while they already + * started running and accumulated some metric results. + * If the metric is part of SparkPlan.metrics, then the newly created plan will have new + * metrics and the old metrics would have been discarded; so nothing needs to be tracked here. + * But if the metric is coming from outside, it can be reused by the new SparkPlan. + * A new plan will have a new RDD and a new RDDOperationScope, so by tracking these for the + * final AQE plan, only values from the final plan and execution should be aggregated. + * + * It can also happen that the new AQE plan reuses SparkPlan instances from the old plan, + * see CancelShuffleStageInBroadcastJoin. However, in that case, the old plan will be put + * under some new plan in newly submitted Stages. Since we only truly track the plans that + * submit Stages, these should be different and enough to disambiguate. + * + * Driver only updates + * ------------------- + * The metric can be updated directly on the driver side, during the execution of catalyst + * optimizer. One example is [[ConvertToLocalRelation]] optimization rule, which constant folds + * pieces of the plan. + * Execution in this scope is tagged with [[QueryExecution.id]] using + * [[SparkContext.DATASET_QUERY_EXECUTION_ID_KEY]] property, and this metric is tracking + * the metric value separately for each QueryExecution. + * Like with LastAttemptAccumulator, the metric will bail out if it's updated both from the driver + * and from executor Tasks. + * + * Cached / Checkpointed plans + * --------------------------- + * If the metric was used inside a cached (df.cache, df.persist) or checkpointed (df.checkpoint, + * df.localCheckpoint) plan, which is then turned into an RDDScanExec or InMemoryTableScanExec + * in the Dataset's executedPlan, [[lastAttemptValueForDataset]] and + * [[lastAttemptValueForQueryExecution]] are declared undefined behavior. In this case, + * [[lastAttemptValueForHighestRDDId()]] should be used instead, which returns the value from + * the execution in which the plan was cached/checkpointed. + * + * The main issue is if the metric is in the top stage of the cached plan. When that plan is + * executed in some Dataset (as lazy execution), the metric will be executed in the scope of the + * stage that contains the InMemoryTableScanExec / RDDScanExec, which will be some parent of that + * plan, and not plan of the cached plan. So if the cached plan is then used in another Dataset, + * that Dataset will not have information about that parent. + * There could be some hacks done to fix it by recording in the InMemoryRelation the scopes in + * which it was materialized. There are also other issues, like that checkpoint throws away the + * plan, so it would also have to record the RDD scopes used during checkpointing. This gets + * further complicated if recomputations are involved, and are done in yet another scope. + * It was declared undefined behavior instead of pursuing this. + */ + +/** + * A trait that can be mixed into a subclass of [[AccumulatorV2]] to track the "logical" + * value of the "last attempt" of the execution using the accumulator. + * In addition to what [[LastAttemptAccumulator]] does, it allows tracking the last attempt + * executed in the scope of a Dataset's QueryExecution, via + * [[lastAttemptValueForDataset]] and [[lastAttemptValueForQueryExecution]] methods. + */ +trait SQLLastAttemptAccumulator[IN, OUT, PARTIAL, DRIVER_ACC] + extends LastAttemptAccumulator[IN, OUT, PARTIAL] { + this: AccumulatorV2[IN, OUT] => + + /** Create a fresh accumulator to hold driver-side values for one QueryExecution. */ + protected def newDriverQueryExecutionAcc(): DRIVER_ACC + /** Add a value to a driver-side per-QueryExecution accumulator. */ + protected def addToDriverAcc(acc: DRIVER_ACC, value: IN): Unit + /** Set the value of a driver-side per-QueryExecution accumulator. */ + protected def setDriverAcc(acc: DRIVER_ACC, value: OUT): Unit + /** Read the value of a driver-side per-QueryExecution accumulator. */ + protected def driverAccValue(acc: DRIVER_ACC): OUT + + @transient + private var lastAttemptDirectDriverQueryExecutionValues: mutable.Map[String, DRIVER_ACC] = _ + + override def initializeLastAttemptAccumulator()(implicit ct: ClassTag[PARTIAL]): Unit = try { + super.initializeLastAttemptAccumulator()(ct) + lastAttemptDirectDriverQueryExecutionValues = new mutable.HashMap[String, DRIVER_ACC]() + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in initializeLastAttemptAccumulator", + exception = Some(e)) + } + + override def resetLastAttemptAccumulator(): Unit = try { + super.resetLastAttemptAccumulator() + lastAttemptDirectDriverQueryExecutionValues = new mutable.HashMap[String, DRIVER_ACC]() + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in resetLastAttemptAccumulator", + exception = Some(e)) + } + + override protected def assertValid() = { + super.assertValid() + assert(lastAttemptDirectDriverQueryExecutionValues != null) + } + + protected def getOrCreateDirectDriverQueryExecutionValue(queryExecutionId: String): DRIVER_ACC = { + lastAttemptDirectDriverQueryExecutionValues.synchronized { + if (!lastAttemptDirectDriverQueryExecutionValues.contains(queryExecutionId)) { + lastAttemptDirectDriverQueryExecutionValues.put( + queryExecutionId, newDriverQueryExecutionAcc()) + } + lastAttemptDirectDriverQueryExecutionValues(queryExecutionId) + } + } + + protected def getActiveDatasetQueryExecutionId: Option[String] = { + SparkContext + .getActive + .flatMap(sc => Option(sc.getLocalProperty(SparkContext.DATASET_QUERY_EXECUTION_ID_KEY))) + } + + /** + * Check if the value is added on the driver side, not from within a task. + * If it is set in the scope of a Dataset's QueryExecution, associate it with that scope. + * This must be called from `add` methods of any AccumulatorV2 subclass supporting + * SQL last attempt metrics to set what the `value` of the metric is after the operation. + * This should be called there after [[setValueIfOnDriverSide]]. + */ + protected def addQueryExecutionValueIfOnDriverSide(value: IN): Unit = try { + // Note: setValueIfOnDriverSide will already make it invalid if there are also RDD updates. + if (isAtDriverSide && lastAttemptAccumulatorInitialized && !lastAttemptAccumulatorInvalid) { + // Direct update on the driver, not from within a task. + getActiveDatasetQueryExecutionId match { + case Some(qeId) => + addToDriverAcc(getOrCreateDirectDriverQueryExecutionValue(qeId), value) + case None => // pass + } + } + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in addQueryExecutionValueIfOnDriverSide", + exception = Some(e)) + } + + /** + * Like [[addQueryExecutionValueIfOnDriverSide]], but for set operations. + */ + protected def setQueryExecutionValueIfOnDriverSide(value: OUT): Unit = try { + if (isAtDriverSide && lastAttemptAccumulatorInitialized && !lastAttemptAccumulatorInvalid) { + getActiveDatasetQueryExecutionId match { + case Some(qeId) => + setDriverAcc(getOrCreateDirectDriverQueryExecutionValue(qeId), value) + case None => // pass + } + } + } catch { + case NonFatal(e) => + unexpectedLastAttemptMetricOperation( + invalidate = true, + reason = "Unexpected exception in setQueryExecutionValueIfOnDriverSide", + exception = Some(e)) + } + + override def logAccumulatorState: LogEntry = try { + val driverQEVals = Option(lastAttemptDirectDriverQueryExecutionValues) + .map(_.map { case (key, acc) => s"$key -> ${driverAccValue(acc)}" }.mkString("\n")) + .getOrElse("") + super.logAccumulatorState + + log""" + |Direct driver QE values: + |${MDC(logKeyAccumulatorState, driverQEVals)} + """.stripMargin + } catch { + case NonFatal(e) => + logWarning(log"Unexpected exception in logAccumulatorState", e) + log"" + } + + /** + * Returns the last attempt value of this accumulator, aggregated from the last execution of this + * QueryExecution. + * + * @note The output of this method is undefined if this metric was used inside a part of the plan + * which was either checkpointed (e.g. df.localCheckpoint(), df.checkpoint()) or cached + * (e.g. df.cache(), df.persist()). + * [[lastAttemptValueForHighestRDDId()]] should be used instead, which returns the + * value from the execution in which the plan was cached/checkpointed. + * + * @return None if the last attempt value cannot be established, Some(value) otherwise. + */ + def lastAttemptValueForQueryExecution(qe: QueryExecution): Option[OUT] = { + if (lastAttemptAccumulatorInvalid) return None + assertValid() + // If there was a driver set value defined in the scope of this QueryExecution, return that. + lastAttemptDirectDriverQueryExecutionValues.get(qe.id.toString) match { + case Some(acc) => return Some(driverAccValue(acc)) + case None => // pass + } + // Otherwise, gather the RDD scopes from the plan and find metric updates from these scopes. + val scopes = SQLLastAttemptAccumulator.extractStageRDDScopes(qe.executedPlan) + scopes match { + case Left(bailOutReason) => + unexpectedLastAttemptMetricOperation( + invalidate = false, + reason = s"Unable to extract RDD scopes from query execution plan: $bailOutReason") + None + case Right(scopes) => + lastAttemptValueForRDDScopes(scopes) + } + } + + /** + * Returns the last attempt value of this accumulator, aggregated from the last execution of this + * Dataset. + * + * @note The output of this method is undefined if this metric was used inside a part of the plan + * which was either checkpointed (e.g. df.localCheckpoint(), df.checkpoint()) or cached + * (e.g. df.cache(), df.persist()). + * [[lastAttemptValueForHighestRDDId()]] should be used instead, which returns the + * value from the execution in which the plan was cached/checkpointed. + * + * @return None if the last attempt value cannot be established, Some(value) otherwise. + */ + def lastAttemptValueForDataset(ds: Dataset[_]): Option[OUT] = { + lastAttemptValueForQueryExecution(ds.queryExecution) + } + + /** Visible for testing. */ + def getDirectDriverQueryExecutionValue(qeId: String): Option[OUT] = { + lastAttemptDirectDriverQueryExecutionValues.get(qeId).map(driverAccValue) + } +} + +object SQLLastAttemptAccumulator extends Logging { + + private[metric] def extractStageRDDScopes(sparkPlan: SparkPlan): Either[String, Seq[String]] = { + var bailOutReason: Option[String] = None + + // recurse, setting the bailOutReason on failure, or returning the list of scopes on success. + def recurse(sparkPlan: SparkPlan): Seq[String] = { + if (bailOutReason.isDefined) { + Nil + } else { + extractStageRDDScopes(sparkPlan) match { + case Left(reason) => + bailOutReason = Some(reason) + Nil + case Right(scopes) => scopes + } + } + } + + def scopeIds(sparkPlan: SparkPlan): Seq[String] = { + AdaptiveSparkPlanHelper.stripAQEPlan(sparkPlan) match { + case w: WholeStageCodegenExec => + // WholeStageCodegenExec can fallback and execute the child plan without codegen instead, + // we don't know when this happens, so we need to account for both cases. + // It will never be both at the same time as this is a compilation time decision, so + // returning both won't result in duplicates. + Seq(w.rddScopeId, w.child.rddScopeId) + case p => Seq(p.rddScopeId) + } + } + + // The root of the plan is submitted as a result stage. + val resultStageScopes = scopeIds(sparkPlan) + + val stagesScopes = AdaptiveSparkPlanHelper.flatMap(sparkPlan) { + case _ if bailOutReason.isDefined => Nil + + // broadcast exchange stage submitting nodes + case bl: BroadcastExchangeLike => bl match { + case b: BroadcastExchangeExec => + // The job is submitted in scope of child of the broadcast exchange. + // ``` + // val rs = child.executeCollectResult() + // ``` + // <- executeCollectResult() is called on the child, and child executes it in its scope. + scopeIds(b.child) + case p => + // Bail out if future unknown implementation is encountered. + bailOutReason = Some(s"Unsupported BroadcastExchangeLike: ${p.getClass.getName}") + Nil + } + + // shuffle exchange stage submitting nodes + case sl: ShuffleExchangeLike => sl match { + // All shuffle exchange implementations create the ShuffledRowRDD / ShuffledBlockRDD + // with its own scope, and it will be executed in that scope. + case s: ShuffleExchangeExec => scopeIds(s) + case p => + // Bail out if future unknown implementation is encountered. + bailOutReason = Some(s"Unsupported ShuffleExchangeLike: ${p.getClass.getName}") + Nil + } + + // reused exchange + case r: ReusedExchangeExec => + // Reused exchange is going to reuse stuff executed in the scope of its child, + // i.e. the exchange it reuses. + recurse(r.child) + + case sl: BaseSubqueryExec => sl match { + case s: SubqueryExec => + // ``` + // val rows: Array[InternalRow] = if (maxNumRows.isDefined) { + // child.executeTake(maxNumRows.get) + // } else { + // child.executeCollect() + // } + // ``` + // will launch stages in scope of child. + scopeIds(s.child) + case _: SubqueryBroadcastExec => + // Used by DPP filter only, not part of main flow of query execution. + Nil + case _: SubqueryAdaptiveBroadcastExec => + // Used by DPP filter only. + Nil + case p => + // Bail out if future unknown implementation is encountered. + bailOutReason = Some(s"Unsupported BaseSubqueryExec: ${p.getClass.getName}") + Nil + } + + /* Useful comments for posterity. + // cached table node + case _: InMemoryTableScanLike => + // Do nothing for cached tables. There are many border cases where it wouldn't work. + // Some notes for posterity: + // For [[InMemoryTableScanExec]], we could recursed into the cachedPlan, but: + // - if the metric is in the top stage of that plan, then it would be executed in the scope + // of the stage of whatever execution that InMemoryTableScanExec is part of when the + // plan is cached. [[InMemoryTableScanExec]] is not a stage submitting node by itself, and + // by itself it doesn't have visibility into the parent that submits the stage that + // materializes the cache. If the current executedPlan is not the one that materializes, + // then the metric would return 0 instead of the value from the cached execution. If the + // current executedPlan is the one that materializes the cache, then it would be the + // correct value. + // - if the metric is in a map stage of the cachedPlan, then it would be correctly + // annotated with the scope of that stage, and it would work correctly. + // + // Since it's hard to achieve a consistent behavior here, we just do not support it. + Nil + + // RDD node + case _: RDDScanExec => + // Similar as with cached tables, do nothing with RDDs. + // This could be a plan coming from an execution of df.checkpoint(). + // Since checkpointing cuts the references to the original plan, there is no way to descend + // into it to check attribution. + // We could try to make checkpoint collect and store the scopes of the original execution, + // but even then it would face similar inconsistencies as described above for cached plans. + // - if the metric is in the top stage of that plan, then if it was executed in the scope + // of this execution, it would be attributed to the scope of the parent stage that is + // consuming the checkpointed RDD, not to any scope of the original plan. + // - if the metric is in a map stage of the plan that was checkpointed, it requires that + // checkpoint would track these stages and scopes. + // + // Since it's hard to achieve a consistent behavior here, we just do not support it. + Nil + */ + + case _ => Nil // only extract from nodes that submit stages + } + + // also collect the plan scopes of all subqueries, which are executed "on the side". + val subqueriesScopes = AdaptiveSparkPlanHelper.flatMap(sparkPlan) { p => + p.subqueries.flatMap(recurse) + } + + if (bailOutReason.isDefined) { + Left(bailOutReason.get) + } else { + Right(resultStageScopes ++ stagesScopes ++ subqueriesScopes) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetric.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetric.scala new file mode 100644 index 0000000000000..33326fb8e5bc4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetric.scala @@ -0,0 +1,88 @@ +/* + * 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.metric + +import org.apache.spark.SparkContext +import org.apache.spark.util.AccumulatorV2 + +class SQLLastAttemptMetric( + metricType: String, + initValue: Long = 0L) + extends SQLMetric(metricType, initValue) + with SQLLastAttemptAccumulator[Long, Long, Long, SQLMetric] { + + override protected def partialMergeVal: Long = _value + + override protected def partialMerge(value: Long): Unit = { + // For SQLLastAttemptMetric, this is just add to the underlying SQLMetric. + super.add(value) + } + + override protected def isMergeable(other: AccumulatorV2[_, _]): Boolean = other match { + case o: SQLLastAttemptMetric => o.metricType == metricType + case _ => false + } + + // SQLLastAttemptMetric is used internally to aggregate system metrics (counters) such as + // number of rows processed, and it should not store user data. + protected def accumulatorStoresUserData: Boolean = false + + override protected def newDriverQueryExecutionAcc(): SQLMetric = + new SQLMetric(metricType, initValue) + override protected def addToDriverAcc(acc: SQLMetric, value: Long): Unit = acc.add(value) + override protected def setDriverAcc(acc: SQLMetric, value: Long): Unit = acc.set(value) + override protected def driverAccValue(acc: SQLMetric): Long = acc.value + + override def copy(): SQLLastAttemptMetric = { + val newAcc = new SQLLastAttemptMetric(metricType, initValue) + newAcc._value = _value + newAcc + } + + override def add(v: Long): Unit = { + super.add(v) + if (v >= 0) { + // set value of SQLMetric after the add. + setValueIfOnDriverSide(value) + addQueryExecutionValueIfOnDriverSide(v) + } + } + + override def set(v: Long): Unit = { + super.set(v) + if (v >= 0) { + // set value of SQLMetric after the set. + setValueIfOnDriverSide(value) + setQueryExecutionValueIfOnDriverSide(value) + } + } + +} + +object SQLLastAttemptMetrics { + /** + * Create a metric to report the value aggregated from the last attempt of each task. These + * would be the values for the tasks that actually contributed to the final output of the + * execution. + */ + def createMetric(sc: SparkContext, name: String): SQLLastAttemptMetric = { + val acc = new SQLLastAttemptMetric(SQLMetrics.SUM_METRIC) + acc.register(sc, name = SQLMetrics.metricsCache.get(name), countFailedValues = false) + acc.initializeLastAttemptAccumulator() + acc + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 13f4d7926bea8..0523df282cda5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -45,7 +45,7 @@ class SQLMetric( // for SPARK-11013. assert(initValue <= 0) // _value will always be either initValue or non-negative. - private var _value = initValue + private[metric] var _value = initValue override def copy(): SQLMetric = { val newAcc = new SQLMetric(metricType, initValue) @@ -110,7 +110,7 @@ class SQLMetric( } object SQLMetrics { - private val SUM_METRIC = "sum" + private[metric] val SUM_METRIC = "sum" private val SIZE_METRIC = "size" private val TIMING_METRIC = "timing" private val NS_TIMING_METRIC = "nsTiming" @@ -120,7 +120,7 @@ object SQLMetrics { val cachedSQLAccumIdentifier = Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER) - private val metricsCache: LoadingCache[String, Option[String]] = + private[metric] val metricsCache: LoadingCache[String, Option[String]] = CacheBuilder.newBuilder().maximumSize(10000) .build(new CacheLoader[String, Option[String]] { override def load(name: String): Option[String] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 036ec943127d8..291aa7cab7256 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -300,6 +300,28 @@ trait QueryTestBase super.withSQLConf(pairs: _*)(f) } + /** + * Temporarily sets SparkContext configuration values for testing. + * This is for configs that must be set on the SparkContext (not + * SQLConf), such as testing flags. + */ + protected def withSparkContextConf[T]( + pairs: (String, String)*)(f: => T): T = { + val sc = spark.sparkContext + val oldValues = pairs.map { case (k, _) => + k -> sc.conf.getOption(k) + } + try { + pairs.foreach { case (k, v) => sc.conf.set(k, v) } + f + } finally { + oldValues.foreach { + case (k, Some(v)) => sc.conf.set(k, v) + case (k, None) => sc.conf.remove(k) + } + } + } + /** * Drops functions after calling `f`. A function is represented by (functionName, isTemporary). */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/MetricsFailureInjectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/MetricsFailureInjectionSuite.scala new file mode 100644 index 0000000000000..847a12f4f305c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/MetricsFailureInjectionSuite.scala @@ -0,0 +1,364 @@ +/* + * 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.metric + +import scala.util.Random + +import org.apache.spark.internal.config +import org.apache.spark.sql.{Column, Dataset} +import org.apache.spark.sql.execution.adaptive.{AQETestHelper, DisableAdaptiveExecutionSuite} +import org.apache.spark.sql.functions.udf +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +class MetricsFailureInjectionSuite + extends SharedSparkSession + with SQLMetricsTestUtils + // Need to control AQE per-test to ensure expected plan shapes. + with DisableAdaptiveExecutionSuite { + + import testImplicits._ + + override def beforeAll(): Unit = { + super.beforeAll() + // Disable re-use, since it interferes with the forced replanning. + spark.conf.set(SQLConf.EXCHANGE_REUSE_ENABLED, false) + } + + def setUpTestTable(tableName: String): Unit = { + val rand = new Random(1) + val randomPrefix = rand.nextString(30) + spark.range(300).map { id => + (id, (id % 5).toInt, randomPrefix + (id % 111)) + }.toDF("id", "low_cardinality_col", "large_col") + .write.format("parquet").saveAsTable(tableName) + val numRecords = spark.read.table(tableName).count() + assert(numRecords === 300) + } + + for { + useAQE <- BOOLEAN_DOMAIN + } test(s"Two stage metrics AQE cancellation injection - useAQE=$useAQE") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> useAQE.toString) { + val stage1Metric = SQLMetrics.createMetric(spark.sparkContext, "stage 1 counter") + val stage2Metric = SQLMetrics.createMetric(spark.sparkContext, "stage 2 counter") + val stage1SLAMetric = + SQLLastAttemptMetrics.createMetric(spark.sparkContext, "stage 1 SLAM") + val stage2SLAMetric = + SQLLastAttemptMetrics.createMetric(spark.sparkContext, "stage 2 SLAM") + + def runQueryWithMetrics( + triggerMetrics: SQLMetric*)( + postRunChecks: Dataset[_] => Unit): Unit = { + assert(stage1Metric.value === 0) + assert(stage2Metric.value === 0) + withTable("test_table") { + setUpTestTable("test_table") + AQETestHelper.withForcedCancellation(triggerMetrics: _*) { + val stage1MetricsExpr = incrementMetrics(Seq(stage1Metric, stage1SLAMetric)) + val stage1 = spark.read.table("test_table").filter(Column(stage1MetricsExpr)) + val stage2MetricsExpr = incrementMetrics(Seq(stage2Metric, stage2SLAMetric)) + val stage2 = + stage1.groupBy("low_cardinality_col").count().filter(Column(stage2MetricsExpr)) + val finalDf = stage2.as[(Int, Long)] + val result = finalDf.collect() + + assert(result.toMap === (0 until 5).map(v => (v, 300 / 5)).toMap) + postRunChecks(finalDf) + stage1Metric.reset() + stage2Metric.reset() + } + } + } + + // SLAM values don't change with retries, so we can reuse the same assertions for all cases. + def assertSLAM(finalDf: Dataset[_]): Unit = { + assert(stage1SLAMetric.lastAttemptValueForHighestRDDId() === Some(300)) + assert(stage2SLAMetric.lastAttemptValueForHighestRDDId() === Some(5)) + + assert(stage1SLAMetric.lastAttemptValueForDataset(finalDf) === Some(300)) + assert(stage2SLAMetric.lastAttemptValueForDataset(finalDf) === Some(5)) + } + + // Case 1: No forced replanning. + runQueryWithMetrics() { finalDf => + assert(stage1Metric.value === 300) + assert(stage2Metric.value === 5) + + assertSLAM(finalDf) + } + + // Case 2: Replan on stage1Metric. + runQueryWithMetrics(stage1Metric) { finalDf => + if (useAQE) { + assert(stage1Metric.value > 300) + } else { + assert(stage1Metric.value === 300) + } + assert(stage2Metric.value === 5) + + assertSLAM(finalDf) + } + + // Case 3: Replan on stage2Metric (will be ignored, because this is a result stage). + runQueryWithMetrics(stage2Metric) { finalDf => + assert(stage1Metric.value === 300) + assert(stage2Metric.value === 5) + + assertSLAM(finalDf) + } + + // Case 4: Replan on both metrics (only first will actually trigger). + runQueryWithMetrics(stage1Metric, stage2Metric) { finalDf => + if (useAQE) { + assert(stage1Metric.value > 300) + } else { + assert(stage1Metric.value === 300) + } + assert(stage2Metric.value === 5) + + assertSLAM(finalDf) + } + } + } + + for { + useAQE <- BOOLEAN_DOMAIN + } test(s"Three stage metrics AQE cancellation injection - useAQE=$useAQE") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> useAQE.toString) { + val stage1Metric = SQLMetrics.createMetric(spark.sparkContext, "stage 1 counter") + val stage2Metric = SQLMetrics.createMetric(spark.sparkContext, "stage 2 counter") + val stage3Metric = SQLMetrics.createMetric(spark.sparkContext, "stage 3 counter") + val stage1SLAMetric = + SQLLastAttemptMetrics.createMetric(spark.sparkContext, "stage 1 SLAM") + val stage2SLAMetric = + SQLLastAttemptMetrics.createMetric(spark.sparkContext, "stage 2 SLAM") + val stage3SLAMetric = + SQLLastAttemptMetrics.createMetric(spark.sparkContext, "stage 3 SLAM") + + def runQueryWithMetrics( + triggerMetrics: SQLMetric*)(postRunChecks: Dataset[_] => Unit): Unit = { + assert(stage1Metric.value === 0) + assert(stage2Metric.value === 0) + withTable("primary_table", "secondary_table") { + // Use the same layout for both. Makes the query a non-obvious self-join essentially. + setUpTestTable("primary_table") + setUpTestTable("secondary_table") + AQETestHelper.withForcedCancellation(triggerMetrics: _*) { + val stage1MetricsExpr = incrementMetrics(Seq(stage1Metric, stage1SLAMetric)) + val stage1 = spark.read.table("primary_table") + .filter(Column(stage1MetricsExpr)) + val stage2MetricsExpr = incrementMetrics(Seq(stage2Metric, stage2SLAMetric)) + val stage2 = stage1.join( + spark.read.table("secondary_table"), + usingColumn = "id", + joinType = "fullOuter") + .filter(Column(stage2MetricsExpr)) + val stage3MetricsExpr = incrementMetrics(Seq(stage3Metric, stage3SLAMetric)) + val stage3 = stage2 + .groupBy("primary_table.low_cardinality_col") + .count() + .filter(Column(stage3MetricsExpr)) + val finalDf = stage3.as[(Int, Long)] + val result = finalDf.collect() + assert(result.toMap === (0 until 5).map(v => (v, 300 / 5)).toMap) + postRunChecks(finalDf) + stage1Metric.reset() + stage2Metric.reset() + stage3Metric.reset() + } + } + } + + // SLAM values don't change with retries, so we can reuse the same assertions for all cases. + def assertSLAM(finalDf: Dataset[_]): Unit = { + assert(stage1SLAMetric.lastAttemptValueForHighestRDDId() === Some(300)) + assert(stage2SLAMetric.lastAttemptValueForHighestRDDId() === Some(300)) + assert(stage3SLAMetric.lastAttemptValueForHighestRDDId() === Some(5)) + + assert(stage1SLAMetric.lastAttemptValueForDataset(finalDf) === Some(300)) + assert(stage2SLAMetric.lastAttemptValueForDataset(finalDf) === Some(300)) + assert(stage3SLAMetric.lastAttemptValueForDataset(finalDf) === Some(5)) + } + + // Case 1: No forced replanning. + runQueryWithMetrics() { finalDf => + assert(stage1Metric.value === 300) + assert(stage2Metric.value === 300) + assert(stage3Metric.value === 5) + + assertSLAM(finalDf) + } + + // Case 2: Replan on stage1Metric. + runQueryWithMetrics(stage1Metric) { finalDf => + if (useAQE) { + assert(stage1Metric.value > 300) + } else { + assert(stage1Metric.value === 300) + } + assert(stage2Metric.value === 300) + assert(stage3Metric.value === 5) + + assertSLAM(finalDf) + } + + // Case 3: Replan on stage2Metric (will also re-run the first stage). + runQueryWithMetrics(stage2Metric) { finalDf => + if (useAQE) { + assert(stage1Metric.value > 300) + assert(stage2Metric.value > 300) + } else { + assert(stage1Metric.value === 300) + assert(stage2Metric.value === 300) + } + assert(stage3Metric.value === 5) + + assertSLAM(finalDf) + } + + // Case 4: Replan on all metrics (only first will actually trigger). + runQueryWithMetrics(stage1Metric, stage2Metric, stage3Metric) { finalDf => + if (useAQE) { + assert(stage1Metric.value > 300) + } else { + assert(stage1Metric.value === 300) + } + assert(stage2Metric.value === 300) + assert(stage3Metric.value === 5) + + assertSLAM(finalDf) + } + } + } + + for { + injectFailure <- BOOLEAN_DOMAIN + } test(s"Two stage metrics block failure injection - injectFailure=$injectFailure") { + val stage1Metric = SQLMetrics.createMetric(spark.sparkContext, "stage 1 counter") + val stage2Metric = SQLMetrics.createMetric(spark.sparkContext, "stage 2 counter") + val stage1SLAMetric = + SQLLastAttemptMetrics.createMetric(spark.sparkContext, "stage 1 SLAM") + val stage2SLAMetric = + SQLLastAttemptMetrics.createMetric(spark.sparkContext, "stage 2 SLAM") + + def runQueryWithMetrics( + triggerMetrics: SQLMetric*)(postRunChecks: Dataset[_] => Unit): Unit = { + assert(stage1Metric.value === 0) + assert(stage2Metric.value === 0) + withTable("test_table") { + setUpTestTable("test_table") + withSparkContextConf( + config.Tests.INJECT_SHUFFLE_FETCH_FAILURES.key -> injectFailure.toString) { + val stage1MetricsExpr = incrementMetrics(Seq(stage1Metric, stage1SLAMetric)) + val stage1 = spark.read.table("test_table").filter(Column(stage1MetricsExpr)) + val stage2MetricsExpr = incrementMetrics(Seq(stage2Metric, stage2SLAMetric)) + val stage2 = + stage1.groupBy("low_cardinality_col").count().filter(Column(stage2MetricsExpr)) + val finalDf = stage2.as[(Int, Long)] + val result = finalDf.collect() + assert(result.toMap === (0 until 5).map(v => (v, 300 / 5)).toMap) + postRunChecks(finalDf) + stage1Metric.reset() + stage2Metric.reset() + } + } + } + + runQueryWithMetrics() { finalDf => + if (injectFailure) { + assert(stage1Metric.value > 300) + } else { + assert(stage1Metric.value === 300) + } + // Stage2 doesn't have a downstream shuffle stage we can fail. + assert(stage2Metric.value === 5) + + assert(stage1SLAMetric.lastAttemptValueForHighestRDDId() === Some(300)) + assert(stage2SLAMetric.lastAttemptValueForHighestRDDId() === Some(5)) + + assert(stage1SLAMetric.lastAttemptValueForDataset(finalDf) === Some(300)) + assert(stage2SLAMetric.lastAttemptValueForDataset(finalDf) === Some(5)) + } + } + + for { + injectFailure <- BOOLEAN_DOMAIN + } test(s"Non-deterministic stage block failure injection - injectFailure=$injectFailure") { + val stage1Metric = SQLMetrics.createMetric(spark.sparkContext, "stage 1 counter") + val stage2Metric = SQLMetrics.createMetric(spark.sparkContext, "stage 2 counter") + val stage1SLAMetric = + SQLLastAttemptMetrics.createMetric(spark.sparkContext, "stage 1 SLAM") + val stage2SLAMetric = + SQLLastAttemptMetrics.createMetric(spark.sparkContext, "stage 2 SLAM") + + def runQueryWithMetrics( + triggerMetrics: SQLMetric*)(postRunChecks: Dataset[_] => Unit): Unit = { + assert(stage1Metric.value === 0) + assert(stage2Metric.value === 0) + withTable("test_table") { + setUpTestTable("test_table") + withSparkContextConf( + config.Tests.INJECT_SHUFFLE_FETCH_FAILURES.key -> injectFailure.toString) { + val stage1MetricsExpr = incrementMetrics(Seq(stage1Metric, stage1SLAMetric)) + val udfRand = + udf { + () => { + new Random().nextDouble() + } + }.asNondeterministic().apply().expr + val stage1 = spark.read.table("test_table") + .withColumn("non_deterministic_col", Column(udfRand)) + .filter(Column(stage1MetricsExpr)) + val stage2MetricsExpr = incrementMetrics(Seq(stage2Metric, stage2SLAMetric)) + val stage2 = stage1 + .groupBy("low_cardinality_col") + .avg("non_deterministic_col") + .filter(Column(stage2MetricsExpr)) + // Add an extra stage with a single task to avoid flaky failures. If a ResultTask + // returns non-deterministic results to the client, it forces the query to abort + // instead of retrying the input stages. + val finalDf = stage2.repartition(1).as[(Int, Double)] + val result = finalDf.collect() + // Don't compare the second value, since it's random. + assert(result.map(_._1).toSet === (0 until 5).toSet) + postRunChecks(finalDf) + stage1Metric.reset() + stage2Metric.reset() + } + } + } + + runQueryWithMetrics() { finalDf => + if (injectFailure) { + assert(stage1Metric.value > 300) + } else { + assert(stage1Metric.value === 300) + } + // Stage2 doesn't have a downstream shuffle stage we can fail. + assert(stage2Metric.value === 5) + + assert(stage1SLAMetric.lastAttemptValueForHighestRDDId() === Some(300)) + assert(stage2SLAMetric.lastAttemptValueForHighestRDDId() === Some(5)) + + assert(stage1SLAMetric.lastAttemptValueForDataset(finalDf) === Some(300)) + assert(stage2SLAMetric.lastAttemptValueForDataset(finalDf) === Some(5)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetricIntegrationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetricIntegrationSuite.scala new file mode 100644 index 0000000000000..2e7af075a3e74 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetricIntegrationSuite.scala @@ -0,0 +1,705 @@ +/* + * 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.metric + +import org.apache.spark.internal.config +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.optimizer.BuildRight +import org.apache.spark.sql.catalyst.plans.RightOuter +import org.apache.spark.sql.execution.{CoalescedPartitionSpec, CoalescedShuffleRead, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +/** Tests [[SQLLastAttemptMetric]] used by [[RDD]]s and [[Dataset]]s */ +class SQLLastAttemptMetricIntegrationSuite + extends SharedSparkSession + with SQLMetricsTestUtils { + import testImplicits._ + + protected def withRetries = false + + test("single stage rdd updates with shared slam") { + val slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "test SLAM") + val rdd1 = spark.sparkContext.parallelize(1 to 10, 2).map { x => + slam.add(1) + x + } + + rdd1.count() + assert(withRetries || slam.value === 10) + assert(slam.lastAttemptValueForAllRDDs() === Some(10)) + assert(slam.lastAttemptValueForRDDId(rdd1.id) === Some(10)) + assert(slam.lastAttemptValueForRDDIds(Seq(rdd1.id, rdd1.id)) === Some(10)) + assert(slam.lastAttemptValueForRDDIds(Seq(rdd1.id + 1, rdd1.id + 2)) === Some(0)) + assert(slam.lastAttemptValueForRDDIds(Seq(rdd1.id, rdd1.id + 10, rdd1.id)) === Some(10)) + assert(slam.lastAttemptValueForHighestRDDId() === Some(10)) + + val rdd2 = spark.sparkContext.parallelize(1 to 50, 3).map { x => + slam.add(3) + x + } + rdd2.count() + assert(withRetries || slam.value === 160) // +150 + assert(slam.lastAttemptValueForRDDId(rdd1.id) === Some(10)) // value for first rdd unaffected + assert(slam.lastAttemptValueForRDDId(rdd2.id) === Some(150)) // value for second rdd recorded + assert(slam.lastAttemptValueForAllRDDs() === Some(160)) // value for all rdds summed + assert(slam.getNumRDDs === 2) + assert(slam.lastAttemptValueForHighestRDDId() === Some(150)) // highest RDD id updated. + + // Re-executing rdd1 + rdd1.count() + assert(withRetries || slam.value === 170) // +10 + // Re-execution doesn't produce duplicate last attempt values + assert(slam.lastAttemptValueForRDDId(rdd1.id) === Some(10)) + assert(slam.lastAttemptValueForAllRDDs() === Some(160)) + assert(slam.getNumRDDs === 2) + // Highest RDD id tracks highest rdd.id, not the last RDD to be executed + assert(slam.lastAttemptValueForHighestRDDId() === Some(150)) + + // New RDD on top of rdd1, but in a single stage. + val rdd3 = rdd1.map { x => + slam.add(2) + x + } + rdd3.count() + assert(withRetries || slam.value === 200) // +30 + assert(slam.lastAttemptValueForRDDId(rdd1.id) === Some(10)) // stays the same + // The increment from rdd1 and rdd3 are in the same stage, so they are recorded together. + assert(slam.getNumRDDs === 3) + assert(slam.lastAttemptValueForRDDId(rdd3.id) === Some(30)) + assert(slam.getHighestRDDId === Some(rdd3.id)) + assert(slam.lastAttemptValueForHighestRDDId() === Some(30)) + + // Setting a value directly from the driver makes slam bail out, because it can't reason + // about what is the "last attempt" on driver vs. coming from RDD executions. + slam.set(42) + assert(!slam.getValid) + // Information stays available for logging and debugging. + assert(slam.getNumRDDs === 3) + assert(slam.getHighestRDDId === Some(rdd3.id)) + + logInfo(slam.logAccumulatorState) + } + + test("multi stage rdd updates") { + val slam1 = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "SLAM1") + val slam2 = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "SLAM2") + + val rdd1 = spark.sparkContext.parallelize(1 to 10, 2).map { x => + slam1.add(1) + x + } + val repartition = rdd1.repartition(10) + val rdd2 = repartition.map { x => + slam2.add(1) + x + } + rdd2.collect() + assert(withRetries || slam1.value === 10) + assert(withRetries || slam2.value === 10) + assert(slam1.lastAttemptValueForAllRDDs() === Some(10)) + assert(slam2.lastAttemptValueForAllRDDs() === Some(10)) + assert(slam1.lastAttemptValueForHighestRDDId() === Some(10)) + assert(slam2.lastAttemptValueForHighestRDDId() === Some(10)) + // It is executed in a Stage submitted by the repartition. + assert(slam1.lastAttemptValueForRDDId(rdd1.id) === Some(0)) + assert(slam1.lastAttemptValueForRDDId(repartition.id) === Some(0)) // Surprise, nope. + // Repartition creates a number of MapPartitionsRDDs, CoalescedRDDs, ShuffledRDDs... + // The actual stage that submits the map stage is somewhere internal. + assert(slam1.getHighestRDDId.isDefined) + val mapStageRddId = slam1.getHighestRDDId.get + assert(slam1.lastAttemptValueForRDDId(mapStageRddId) === Some(10)) + + // Test passing multiple ids. + assert(slam1.lastAttemptValueForRDDIds(Seq(rdd1.id, repartition.id)) === Some(0)) + assert(slam1.lastAttemptValueForRDDIds( + Seq(rdd1.id, mapStageRddId, repartition.id)) === Some(10)) + assert(slam1.lastAttemptValueForRDDIds(Seq(rdd1.id, rdd2.id)) === Some(0)) + assert(slam1.lastAttemptValueForRDDIds(Seq(-10)) === Some(0)) + + rdd2.collect() + // Repartition stage is reused, but result stage is re-executed. + assert(withRetries || slam1.value === 10) // no change + assert(withRetries || slam2.value === 20) // +10 + // Last attempt value is not duplicated, since result stage is an action on the same RDD. + assert(slam1.lastAttemptValueForAllRDDs() === Some(10)) + assert(slam2.lastAttemptValueForAllRDDs() === Some(10)) + + rdd1.collect() + assert(withRetries || slam1.value === 20) // +10 + // The first time around it was executed in the repartition RDD stage. + // This time around it is executed from action of rdd1. + assert(slam1.getNumRDDs === 2) + assert(slam1.lastAttemptValueForAllRDDs() === Some(20)) + assert(slam1.lastAttemptValueForRDDId(rdd1.id) === Some(10)) // new + assert(slam1.lastAttemptValueForRDDId(mapStageRddId) === Some(10)) // old + // Highest RDD id stays the same. + assert(slam1.getHighestRDDId === Some(mapStageRddId)) + assert(slam1.lastAttemptValueForHighestRDDId() === Some(10)) + + rdd1.collect() + assert(withRetries || slam1.value === 30) // +10 + // Still the same + assert(slam1.lastAttemptValueForAllRDDs() === Some(20)) + assert(slam1.lastAttemptValueForHighestRDDId() === Some(10)) + + rdd2.collect() + // Repartition stage is reused (again), but result stage is re-executed (again). + assert(withRetries || slam1.value === 30) // no change + assert(withRetries || slam2.value === 30) // +10 + // Last attempt value is not duplicated, since result stage is an action on the same RDD. + assert(slam1.lastAttemptValueForAllRDDs() === Some(20)) + assert(slam2.lastAttemptValueForAllRDDs() === Some(10)) + + // Executed in different RDDs, but never duplicated. + assert(slam1.lastAttemptValueForRDDId(rdd1.id) === Some(10)) + assert(slam1.lastAttemptValueForRDDId(mapStageRddId) === Some(10)) + assert(slam1.lastAttemptValueForRDDId(repartition.id) === Some(0)) + assert(slam1.lastAttemptValueForRDDId(rdd2.id) === Some(0)) + assert(slam2.lastAttemptValueForRDDId(rdd1.id) === Some(0)) + assert(slam2.lastAttemptValueForRDDId(mapStageRddId) === Some(0)) + assert(slam1.lastAttemptValueForRDDId(repartition.id) === Some(0)) + assert(slam2.lastAttemptValueForRDDId(rdd2.id) === Some(10)) + + val newRepartition = rdd1.repartition(10) + val newRdd2 = newRepartition.map { x => + slam2.add(1) + x + } + newRdd2.collect() + assert(withRetries || slam1.value === 40) // +10 + assert(withRetries || slam2.value === 40) // +10 + // SLAM metrics get re-executed in the new RDDs + // rdd1 is reused, but the shuffle is new, and that is what submits the map stage. + assert(slam1.getNumRDDs === 3) + assert(slam1.lastAttemptValueForAllRDDs() === Some(30)) // +10 + assert(slam2.getNumRDDs === 2) + assert(slam2.lastAttemptValueForAllRDDs() === Some(20)) // +10 + // Values are recorded for the new highest RDD id. + assert(slam1.getHighestRDDId.isDefined) + val newMapStageId = slam1.getHighestRDDId.get + assert(newMapStageId > mapStageRddId) + assert(slam2.getHighestRDDId === Some(newRdd2.id)) + assert(slam1.lastAttemptValueForRDDId(newMapStageId) === Some(10)) + assert(slam2.lastAttemptValueForRDDId(newRdd2.id) === Some(10)) + assert(slam1.lastAttemptValueForHighestRDDId() === Some(10)) + assert(slam2.lastAttemptValueForHighestRDDId() === Some(10)) + + logInfo(slam1.logAccumulatorState) + logInfo(slam2.logAccumulatorState) + } + + test("rdd take") { + val slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "test SLAM") + val rdd = spark.sparkContext.parallelize(1 to 100, 100).map { x => + slam.add(1) + x + } + withSparkContextConf( + // make it fixed to not be affected by potential changes of default. + config.RDD_LIMIT_INITIAL_NUM_PARTITIONS.key -> "1", + config.RDD_LIMIT_SCALE_UP_FACTOR.key -> "4" + ) { + rdd.take(1) // execute 1 partition + assert(withRetries || slam.value === 1) + assert(slam.lastAttemptValueForAllRDDs() === Some(1)) + + // take(2) scales up from 1 partition; the exact number of partitions scanned + // depends on the scale-up algorithm. + val valueBefore = slam.value + rdd.take(2) + val slamAfterTake2 = slam.lastAttemptValueForAllRDDs() + assert(slamAfterTake2.isDefined) + assert(slamAfterTake2.get >= 2) // at least 2 partitions + assert(slamAfterTake2.get < 100) // but not all partitions. + + // take(100) should execute all 100 partitions + rdd.take(100) + assert(slam.lastAttemptValueForAllRDDs() === Some(100)) + assert(slam.getNumRDDs === 1) + } + + logInfo(slam.logAccumulatorState) + } + + test("rdd coalesce") { + val slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "test SLAM") + val rdd1 = spark.sparkContext.parallelize(1 to 100, 100).map { x => + slam.add(1) + x + } + val rdd2 = rdd1.coalesce(20) + // Test that coalescing that changes partition count doesn't break anything. + rdd2.collect() + assert(slam.lastAttemptValueForRDDId(rdd2.id) === Some(100)) + rdd1.collect() + assert(slam.lastAttemptValueForRDDId(rdd1.id) === Some(100)) + + logInfo(slam.logAccumulatorState) + } + + test("dataset updates") { + val slam1 = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "SLAM1") + val df1 = spark.range(10).filter(Column(incrementMetric(slam1))) + + df1.collect() + assert(withRetries || slam1.value === 10) + assert(slam1.getHighestRDDId.isDefined) + val df1HighestId = slam1.getHighestRDDId.get + val df1ExecutedPlanRddId = df1.queryExecution.executedPlan.execute().id + assert(slam1.lastAttemptValueForHighestRDDId() === Some(10)) + assert(slam1.lastAttemptValueForRDDId(df1HighestId) === Some(10)) + // Values retrieved from the Dataset are the same as from the RDD. + assert(slam1.lastAttemptValueForDataset(df1) === Some(10)) + assert(slam1.lastAttemptValueForQueryExecution(df1.queryExecution) === Some(10)) + + df1.collect() + assert(withRetries || slam1.value === 20) // +10 + // The same executedPlan RDD is reused, but getByteArrayRdd creates a new wrapper. + assert(df1.queryExecution.executedPlan.execute().id === df1ExecutedPlanRddId) + assert(slam1.lastAttemptValueForHighestRDDId() === Some(10)) + // Both wrapper RDDs are summed in allRDDs. + assert(slam1.lastAttemptValueForAllRDDs() === Some(20)) + assert(slam1.lastAttemptValueForDataset(df1) === Some(10)) + + val df2 = df1.filter("id < 5").filter(Column(incrementMetric(slam1))) + df2.collect() + assert(withRetries || slam1.value === 35) // +15 + assert(slam1.getHighestRDDId.isDefined) + // Both incrementMetric expressions are within the same Stage, so they record together. + assert(slam1.lastAttemptValueForHighestRDDId() === Some(15)) + // allRDDs includes wrapper RDDs from repeated df1.collect() calls. + assert(slam1.lastAttemptValueForAllRDDs() === Some(35)) + // New Dataset records only new value. + assert(slam1.lastAttemptValueForDataset(df2) === Some(15)) + // Value df1 is still remembered. + assert(slam1.lastAttemptValueForDataset(df1) === Some(10)) + + val slam2 = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "SLAM2") + val df3 = df1.repartition(1).filter(Column(incrementMetric(slam2))) + df3.collect() + assert(withRetries || slam1.value === 45) // +10 + assert(withRetries || slam2.value === 10) // new + // df3 creates a new plan, and the plan / RDD from df1 is not reused. + assert(slam1.getHighestRDDId.isDefined) + val slam1HighestId = slam1.getHighestRDDId.get + assert(slam1HighestId != df1HighestId) // new plan, new RDD id + assert(slam1.lastAttemptValueForHighestRDDId() === Some(10)) // from new execution + assert(slam1.lastAttemptValueForRDDId(df1HighestId) === Some(10)) // from first exec of df1 + // allRDDs includes wrapper RDDs from repeated collects. + assert(slam1.lastAttemptValueForAllRDDs() === Some(45)) + assert(slam2.lastAttemptValueForAllRDDs() === Some(10)) + // slam1 and slam2 are both executed in df3 + assert(slam1.lastAttemptValueForDataset(df3) === Some(10)) + assert(slam2.lastAttemptValueForDataset(df3) === Some(10)) + // slam2 is not executed in df1 and df2. + assert(slam2.lastAttemptValueForDataset(df1) === Some(0)) + assert(slam2.lastAttemptValueForDataset(df2) === Some(0)) + // slam1 value from df1 and df2 are still remembered. + assert(slam1.lastAttemptValueForDataset(df1) === Some(10)) + assert(slam1.lastAttemptValueForDataset(df2) === Some(15)) + + // Plans and RDDs get reused (result stage is re-executed; shuffle stage is purely reused). + df3.collect() + // No change in dataset values. + assert(slam1.lastAttemptValueForDataset(df3) === Some(10)) + assert(slam2.lastAttemptValueForDataset(df3) === Some(10)) + assert(slam1.lastAttemptValueForDataset(df1) === Some(10)) + assert(slam1.lastAttemptValueForDataset(df2) === Some(15)) + + logInfo(slam1.logAccumulatorState) + logInfo(slam2.logAccumulatorState) + } + + test("dataset limit") { + val slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "SLAM1") + // 10 partitions of 10 elements each + val df = spark.range(0, 1000, 1, 10).filter(Column(incrementMetric(slam))) + var expectedMetricValue = 0 + var expectedSLAMValue = 0 + + // Note: this is sensitive to the internal implementation of LimitExec. + + df.take(5) + // One partition executed, local limit pushed into partition. + expectedMetricValue = 5 + expectedSLAMValue = 5 + assert(withRetries || slam.value === expectedMetricValue) + assert(slam.lastAttemptValueForHighestRDDId() === Some(expectedSLAMValue)) + // take(5) actually inline creates a new Dataset, with new executed plan + assert(slam.lastAttemptValueForDataset(df) === Some(0)) + + df.take(50) + // One partition executed, local limit pushed into partition. + expectedMetricValue += 50 + expectedSLAMValue = 50 + assert(withRetries || slam.value === expectedMetricValue) + // New SQL plan creates new RDDs, so this is seen as new execution. + assert(slam.lastAttemptValueForHighestRDDId() === Some(expectedSLAMValue)) + assert(slam.getNumRDDs === 2) + assert(slam.lastAttemptValueForAllRDDs() === Some(expectedMetricValue)) + // take(50) executes a different inline Dataset and plan. + assert(slam.lastAttemptValueForDataset(df) === Some(0)) + + df.take(220) + // Three partitions executed. + expectedMetricValue += 300 + expectedSLAMValue = 300 + assert(withRetries || slam.value === expectedMetricValue) + assert(slam.lastAttemptValueForHighestRDDId() === Some(expectedSLAMValue)) + assert(slam.getNumRDDs === 3) + assert(slam.lastAttemptValueForAllRDDs() === Some(expectedMetricValue)) + // take(220) executes a different inline Dataset and plan. + assert(slam.lastAttemptValueForDataset(df) === Some(0)) + + df.take(320) + // Five partitions executed. + expectedMetricValue += 500 + expectedSLAMValue = 500 + assert(withRetries || slam.value === expectedMetricValue) + assert(slam.lastAttemptValueForHighestRDDId() === Some(expectedSLAMValue)) + assert(slam.getNumRDDs === 4) + assert(slam.lastAttemptValueForAllRDDs() === Some(expectedMetricValue)) + // take(320) executes a different inline Dataset and plan. + assert(slam.lastAttemptValueForDataset(df) === Some(0)) + + df.take(1) + // One partition scanned, local limit pushed into partition. + expectedMetricValue += 1 + expectedSLAMValue = 1 + assert(withRetries || slam.value === expectedMetricValue) + // New RDD, so the value from new execution is back to 1. + assert(slam.lastAttemptValueForHighestRDDId() === Some(expectedSLAMValue)) + assert(slam.getNumRDDs === 5) + assert(slam.lastAttemptValueForAllRDDs() === Some(expectedMetricValue)) + // take(1) executes a different inline Dataset and plan. + assert(slam.lastAttemptValueForDataset(df) === Some(0)) + + logInfo(slam.logAccumulatorState) + } + + test("driver set value") { + val slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "test SLAM") + slam.set(10) + + // Regular metric value + assert(withRetries || slam.value === 10) + + assert(slam.getDirectDriverValue === Some(10)) + // "Driver update" is returned under "highest" and "all" RDDs + assert(slam.lastAttemptValueForHighestRDDId() === Some(10)) + assert(slam.lastAttemptValueForAllRDDs() === Some(10)) + assert(slam.getNumRDDs === 0) + // When specific RDDs are requested, driver value is not returned. + assert(slam.lastAttemptValueForRDDId(42) === Some(0)) + assert(slam.lastAttemptValueForRDDIds(Seq(7, 42)) === Some(0)) + + // Incrementing works + slam.add(5) + assert(withRetries || slam.value === 15) + assert(slam.lastAttemptValueForHighestRDDId() === Some(15)) + assert(slam.lastAttemptValueForAllRDDs() === Some(15)) + assert(slam.getDirectDriverValue === Some(15)) + + // Negative increments are ignored by SQLMetric + slam.add(-3) + assert(withRetries || slam.value === 15) + assert(slam.lastAttemptValueForHighestRDDId() === Some(15)) + assert(slam.lastAttemptValueForAllRDDs() === Some(15)) + assert(slam.getDirectDriverValue === Some(15)) + + // Reset does not reset SLAM. + slam.reset() + assert(withRetries || slam.value === 0) + assert(slam.lastAttemptValueForHighestRDDId() === Some(15)) + assert(slam.lastAttemptValueForAllRDDs() === Some(15)) + assert(slam.getDirectDriverValue === Some(15)) + + // Setting it back... + slam.set(20) + assert(withRetries || slam.value === 20) + assert(slam.lastAttemptValueForHighestRDDId() === Some(20)) + assert(slam.lastAttemptValueForAllRDDs() === Some(20)) + assert(slam.getDirectDriverValue === Some(20)) + assert(slam.getNumRDDs === 0) + + val df = spark.range(10).filter(Column(incrementMetric(slam))) + // SLAM was not executed in this Dataset, the driver value set manually + // before should not be returned. + assert(slam.lastAttemptValueForDataset(df) === Some(0)) + assert(slam.getDirectDriverQueryExecutionValue(df.queryExecution.id.toString) === None) + df.collect() + assert(withRetries || slam.value === 30) + // SLAM bails out when it sees both driver and executor values + assert(slam.lastAttemptValueForHighestRDDId() === None) + assert(slam.lastAttemptValueForAllRDDs() === None) + assert(slam.lastAttemptValueForRDDId(42) === None) + assert(slam.lastAttemptValueForRDDIds(Seq(7, 42)) === None) + assert(slam.lastAttemptValueForDataset(df) === None) + assert(!slam.getValid) + assert(slam.getNumRDDs === 0) // invalidated before RDD got recorded + assert(slam.getDirectDriverValue === Some(20)) + // Invalidated before QueryExecution value was recorded. + assert(slam.getDirectDriverQueryExecutionValue(df.queryExecution.id.toString) === None) + + slam.reset() + slam.set(10) + assert(withRetries || slam.value === 10) + // SLAM stays bailed out. + assert(slam.lastAttemptValueForHighestRDDId() === None) + assert(slam.lastAttemptValueForAllRDDs() === None) + assert(slam.lastAttemptValueForRDDId(42) === None) + assert(slam.lastAttemptValueForRDDIds(Seq(7, 42)) === None) + assert(slam.lastAttemptValueForDataset(df) === None) + assert(!slam.getValid) + // SLAM info doesn't get updated anymore when invalid, but stays around for debugging purposes. + assert(slam.getNumRDDs === 0) + assert(slam.getDirectDriverValue === Some(20)) + assert(slam.getDirectDriverQueryExecutionValue(df.queryExecution.id.toString) === None) + + logInfo(slam.logAccumulatorState) + + // resetLastAttemptAccumulator resets it and makes it valid to be used again. + slam.resetLastAttemptAccumulator() + assert(slam.getValid) + slam.set(42) + assert(slam.lastAttemptValueForHighestRDDId() === Some(42)) + assert(slam.getDirectDriverValue === Some(42)) + assert(slam.getNumRDDs === 0) + assert(slam.lastAttemptValueForDataset(df) === Some(0)) + } + + test("ConvertToLocalRelation direct driver execution") { + // Normally ConvertToLocalRelation is disabled in tests. + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "") { + val slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "test SLAM") + val df = Seq(1, 2, 3).toDF("a").filter(Column(incrementMetric(slam))) + + // SLAM is executed on the driver in the Optimized by ConvertToLocalRelation + df.collect() + assert(slam.lastAttemptValueForAllRDDs() === Some(3)) + assert(slam.lastAttemptValueForHighestRDDId() === Some(3)) + assert(slam.getDirectDriverValue === Some(3)) + // SLAM recognizes it was executed on the driver + // in the scope of the QueryExecution of this Dataset. + assert(slam.lastAttemptValueForDataset(df) === Some(3)) + assert(slam.getDirectDriverQueryExecutionValue(df.queryExecution.id.toString) === Some(3)) + + // Second action does not re-execute Optimizer. + df.collect() + assert(slam.lastAttemptValueForAllRDDs() === Some(3)) + assert(slam.lastAttemptValueForHighestRDDId() === Some(3)) + assert(slam.getDirectDriverValue === Some(3)) + assert(slam.lastAttemptValueForDataset(df) === Some(3)) + + // Limitation: When a new Dataset is built and Optimizer reexecutes ConvertToLocalRelation, + // SLAM RDD retrieval cannot reason about re-execution on the driver, + // leading to duplicated metrics. + val df2 = df.withColumn("foo", Column(Literal("foo"))) + df2.collect() + assert(slam.lastAttemptValueForAllRDDs() === Some(6)) + assert(slam.lastAttemptValueForHighestRDDId() === Some(6)) + assert(slam.getDirectDriverValue === Some(6)) + // But it recognizes that it is done in a new QueryExecution and is able to distinguish that + // without duplicates. + assert(slam.lastAttemptValueForDataset(df) === Some(3)) + assert(slam.lastAttemptValueForDataset(df2) === Some(3)) + assert(slam.getDirectDriverQueryExecutionValue(df.queryExecution.id.toString) === Some(3)) + assert(slam.getDirectDriverQueryExecutionValue(df2.queryExecution.id.toString) === Some(3)) + + // No RDD executions were recorded. + assert(slam.getNumRDDs === 0) + + logInfo(slam.logAccumulatorState) + } + } + + test("ConvertToLocalRelation manual optimizer triggering") { + // Normally ConvertToLocalRelation is disabled in tests. + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "") { + val slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "test SLAM") + val df = Seq(1, 2, 3).toDF("a").filter(Column(incrementMetric(slam))) + // Trigger the optimizer manually, which will trigger ConvertToLocalRelation + df.queryExecution.assertOptimized() + + // SLAM recognizes it was executed on the driver + // in the scope of the QueryExecution of this Dataset. + assert(slam.lastAttemptValueForDataset(df) === Some(3)) + assert(slam.getDirectDriverQueryExecutionValue(df.queryExecution.id.toString) === Some(3)) + + // Repeated actions do not re-execute Optimizer. + df.collect() + assert(slam.lastAttemptValueForDataset(df) === Some(3)) + df.collect() + assert(slam.lastAttemptValueForDataset(df) === Some(3)) + + logInfo(slam.logAccumulatorState) + } + } + + test("ConvertToLocalRelation in explain") { + // Normally ConvertToLocalRelation is disabled in tests. + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "") { + val slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "test SLAM") + val df = Seq(1, 2, 3).toDF("a").filter(Column(incrementMetric(slam))) + + // EXPLAIN triggers the optimizer and triggered ConvertToLocalRelation to execute + df.explain(true) + assert(withRetries || slam.value === 3) + assert(slam.lastAttemptValueForAllRDDs() === Some(3)) + assert(slam.lastAttemptValueForHighestRDDId() === Some(3)) + assert(slam.getDirectDriverValue === Some(3)) + assert(slam.getDirectDriverQueryExecutionValue(df.queryExecution.id.toString) === Some(3)) + assert(slam.lastAttemptValueForDataset(df) === Some(3)) + // Retriggering EXPLAIN does not cause duplicates + df.explain(true) + assert(withRetries || slam.value === 3) + assert(slam.lastAttemptValueForAllRDDs() === Some(3)) + assert(slam.lastAttemptValueForHighestRDDId() === Some(3)) + assert(slam.getDirectDriverValue === Some(3)) + assert(slam.lastAttemptValueForDataset(df) === Some(3)) + assert(slam.getDirectDriverQueryExecutionValue(df.queryExecution.id.toString) === Some(3)) + + // Execution does not re-execute Optimizer and does not duplicate metric. + df.collect() + assert(withRetries || slam.value === 3) + assert(slam.lastAttemptValueForAllRDDs() === Some(3)) + assert(slam.lastAttemptValueForHighestRDDId() === Some(3)) + assert(slam.getDirectDriverValue === Some(3)) + assert(slam.lastAttemptValueForDataset(df) === Some(3)) + assert(slam.getDirectDriverQueryExecutionValue(df.queryExecution.id.toString) === Some(3)) + + // No RDD executions were recorded. + assert(slam.getNumRDDs === 0) + + logInfo(slam.logAccumulatorState) + } + } + + test("BroadcastNestedLoopJoin outer executes probe side twice") { + val slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "test SLAM") + val build = + spark.range(5).selectExpr("id as b").hint("broadcast") + val probe = spark.range(100).selectExpr("id as p").filter(Column(incrementMetric(slam))) + val df = probe.join(build, usingColumns = Seq(), joinType = "rightouter") + df.collect() + assert(AdaptiveSparkPlanHelper.exists(df.queryExecution.executedPlan) { + case BroadcastNestedLoopJoinExec(_, _, BuildRight, RightOuter, None) => true + case _ => false + }) + // When build side is outer, probe side gets executed twice by BNLJ: + // once for matches, and once to mark unmatched build rows. + // This is a non-determinism correctness issue, and the two executions + // should not be double-counted in the last attempt value. + assert(slam.getNumRDDs === 2) + assert(slam.lastAttemptValueForAllRDDs() === Some(200)) + // The two executions are different RDDs, but only one of them is highest id. + assert(slam.lastAttemptValueForHighestRDDId() === Some(100)) + // Dataset dedups per scope and returns only the latest RDD's value. + assert(slam.lastAttemptValueForDataset(df) === Some(100)) + } + + test("SLAM with AQE CoalesceShufflePartitions") { + // Adapted from tests in CoalesceShufflePartitionsSuite + + val stage1Slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "SLAM2") + val stage2Slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "SLAM2") + val stage1MetricExpr = Column(incrementMetric(stage1Slam)) + val stage2MetricExpr = Column(incrementMetric(stage2Slam)) + + // Dataframe with a SLAM before and after a shuffle. + val df = spark.range(0, 1000, 1, numPartitions = 10) + .selectExpr("id % 20 as key", "id as value") + .filter(stage1MetricExpr) + .groupBy("key").count() + .filter(stage2MetricExpr) + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "2000") { + df.collect() + } + + // Verify the AQE coalescing happened and coalesced the shuffle into 3 partitions. + // (based on ADVISORY_PARTITION_SIZE_IN_BYTES config) + val finalPlan = AdaptiveSparkPlanHelper.stripAQEPlan(df.queryExecution.executedPlan) + val shuffleReads = finalPlan.collect { + case r @ CoalescedShuffleRead() => r + } + assert(shuffleReads.nonEmpty) + shuffleReads.foreach { read => + // check there is actual coalescing of partitions happening + assert(read.isCoalescedRead) + assert(read.partitionSpecs.exists { + case p: CoalescedPartitionSpec if p.startReducerIndex < p.endReducerIndex - 1 => true + case _ => false + }) + } + + // Verify SLAM metrics. + assert(stage1Slam.lastAttemptValueForHighestRDDId() === Some(1000)) + assert(stage2Slam.lastAttemptValueForHighestRDDId() === Some(20)) + assert(stage1Slam.lastAttemptValueForDataset(df) === Some(1000)) + assert(stage2Slam.lastAttemptValueForDataset(df) === Some(20)) + } + + test("WholeStageCodegenExec fallback to non-codegen") { + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "1" // force fallback due to too large method + ) { + // This test is to verify that SLAM works correctly when WholeStageCodegenExec falls back + // to non-codegen execution. + val slam = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "test SLAM") + val df = spark + .range(10) + .filter(Column(incrementMetric(slam))) + // these two operators will be turned into a WholeStageCodegen, + .selectExpr("id + 1 as foo", "id + 2 as bar") + .filter("foo < bar") + df.collect() + assert(slam.lastAttemptValueForDataset(df) === Some(10)) + // Metric is attributed to the child of the WSCG node. + val wscg = df.queryExecution.executedPlan.collectFirst { + case w: WholeStageCodegenExec => w + } + assert(wscg.isDefined) + assert(slam.getHighestRDDId.isDefined) + } + } +} + +class SQLLastAttemptMetricIntegrationSuiteWithStageRetries + extends SQLLastAttemptMetricIntegrationSuite { + override protected def withRetries = true + + override protected def test( + testName: String, + testTags: org.scalatest.Tag*) + (testFun: => Any) + (implicit pos: org.scalactic.source.Position): Unit = { + super.test(testName, testTags : _*) { + withSparkContextConf(config.Tests.INJECT_SHUFFLE_FETCH_FAILURES.key -> "true") { + // Stage retries should not affect SLAM metrics. + testFun + } + }(pos) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetricPlanShapesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetricPlanShapesSuite.scala new file mode 100644 index 0000000000000..ea8d9568f7e4b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetricPlanShapesSuite.scala @@ -0,0 +1,490 @@ +/* + * 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.metric + +import scala.reflect.ClassTag +import scala.util.Random + +import org.scalatest.Tag + +import org.apache.spark.internal.config +import org.apache.spark.sql.execution.{CollectLimitExec, RDDScanExec, SparkPlan} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, AQETestHelper, DisableAdaptiveExecutionSuite} +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.exchange._ +import org.apache.spark.sql.functions.udf +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +class SQLLastAttemptMetricPlanShapesSuite + extends SharedSparkSession + with SQLMetricsTestUtils + // Need to control AQE per-test to ensure expected plan shapes. + with DisableAdaptiveExecutionSuite { + + import testImplicits._ + + import SQLLastAttemptMetricPlanShapesSuite._ + + // Avoid initialising this before the Spark Context is initialised. + protected var testSLAMetric: SQLLastAttemptMetric = _ + + protected def setUpTestTable(): Unit = { + val rand = new Random(1) + val randomPrefix = rand.nextString(30) + spark + .range(NUM_RECORDS) + .map { id => + (id, (id % LOW_CARDINALITY).toInt, randomPrefix + (id % LARGE_CARDINALITY)) + }.toDF("id", "low_cardinality_col", "large_col") + .write.format("parquet").saveAsTable(TABLE_NAME) + val numRecords = spark.read.table(TABLE_NAME).count() + assert(numRecords === 300) + } + + override protected def beforeAll(): Unit = { + super.beforeAll() + setUpTestTable() + testSLAMetric = SQLLastAttemptMetrics.createMetric(spark.sparkContext, "test SLAM") + // Move this into a local field so the closure doesn't hang on to the whole `this` + // reference as well. + val metric = testSLAMetric + val incrementMetric = () => { metric += 1; true } + val incrementMetricUdf = udf(incrementMetric).asNondeterministic() + spark.udf.register("increment_metric", incrementMetricUdf) + } + + override protected def afterAll(): Unit = { + spark.sql(s"DROP TABLE IF EXISTS $TABLE_NAME") + super.afterAll() + } + + override protected def beforeEach(): Unit = { + super.beforeEach() + // note: reset() does not influence lastAttemptValue, but influences regular value + testSLAMetric.reset() + } + + object MetricValue { + type Check = Option[Long] => Unit + + // Having the asserts in these helpers instead of in testPhysicalPlanShape + // produces better error messages. + def exactly(expectedValue: Long): Check = actualValue => + assert(actualValue === Some(expectedValue)) + + def atLeast(minimumValue: Long): Check = { actualValue => + assert(actualValue.isDefined) + assert(actualValue.get >= minimumValue) + } + } + + object PhysicalPlan { + type Check = SparkPlan => Unit + + val ANY: Check = _ => () // Ignore. + + def contains[T <: SparkPlan: ClassTag](implicit cls: ClassTag[T]): Check = { plan => + val existsSomeNodeOfTypeT = + AdaptiveSparkPlanHelper.existsWithSubqueries(plan)(_.getClass == cls.runtimeClass) + assert( + existsSomeNodeOfTypeT, + s"Expected a node ${cls.runtimeClass.getSimpleName}. Actual Plan:\n${plan.treeString}") + } + + def exists(pf: PartialFunction[SparkPlan, Boolean]): Check = { plan => + val existsMatchingNode = + AdaptiveSparkPlanHelper.existsWithSubqueries(plan)(pf.lift(_).getOrElse(false)) + assert( + existsMatchingNode, + s"Unexpected plan (check match function). Actual Plan:\n${plan.treeString}") + } + + def isAQE: Boolean = SQLConf.get.getConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED) + + def hasStageRetries: Boolean = spark.sparkContext.conf + .getOption(config.Tests.INJECT_SHUFFLE_FETCH_FAILURES.key).contains("true") + + def hasAQEReplans: Boolean = AQETestHelper.isForcedCancellationEnabled + } + + protected def testPhysicalPlanShape( + label: String, + setup: () => Unit = () => (), + extraSQLConfs: Map[String, String] = Map.empty, + sqlQuery: String, + executedPlanCheck: PhysicalPlan.Check, + metricValueCheck: MetricValue.Check + )(testTags: Tag*): Unit = { + for { + useAQE <- BOOLEAN_DOMAIN + stageRetries <- BOOLEAN_DOMAIN + aqeReplans <- if (useAQE) BOOLEAN_DOMAIN else Seq(false) + } test(s"$label - " + + s"useAQE=$useAQE, stageRetries=$stageRetries, aqeReplans=$aqeReplans", + testTags: _*) { + + // There is some special handling for df.cache() / df.persist() / df.localCheckpoint() tests. + val cachedPlanTest = label.startsWith("cache - ") + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> useAQE.toString) { + setup() + withSQLConf(extraSQLConfs.toSeq: _*) { + val aqeRetryMetrics = if (aqeReplans) Seq(testSLAMetric) else Seq.empty + AQETestHelper.withForcedCancellation(aqeRetryMetrics: _*) { + withSparkContextConf( + config.Tests.INJECT_SHUFFLE_FETCH_FAILURES.key -> stageRetries.toString) { + val resultDf = spark.sql(sqlQuery) + val _ = resultDf.collect() + + // normal value of the metrics shall not work with retries or replans + if (!stageRetries && !aqeReplans) { + metricValueCheck(Some(testSLAMetric.value)) + } + // test LastRDDValue + metricValueCheck(testSLAMetric.lastAttemptValueForHighestRDDId()) + // test Dataset value + if (!cachedPlanTest) { + // SLAM.lastAttemptValueForDataset is undefined when SLAM is inside + // cached or checkpointed plan. + metricValueCheck(testSLAMetric.lastAttemptValueForDataset(resultDf)) + } + // test expected plan shape + val executedPlan = resultDf.queryExecution.executedPlan + executedPlanCheck(executedPlan) + val rddIdExec = testSLAMetric.getHighestRDDId + + // Repeated execution should not affect SLAM metric value + resultDf.collect() + // test LastRDDValue again + metricValueCheck(testSLAMetric.lastAttemptValueForHighestRDDId()) + // test Dataset value again + if (!cachedPlanTest) { + // SLAM.lastAttemptValueForDataset is undefined when SLAM is inside + // cached or checkpointed plan. + metricValueCheck(testSLAMetric.lastAttemptValueForDataset(resultDf)) + } + + // count() transformation creates a new Dataset. + // It should not affect the SLAM metric value of the first Dataset. + resultDf.count() + // test Dataset value again + if (!cachedPlanTest) { + // SLAM.lastAttemptValueForDataset is undefined when SLAM is inside + // cached or checkpointed plan. + metricValueCheck(testSLAMetric.lastAttemptValueForDataset(resultDf)) + } + // This should have created a new plan and executed new RDDs, + // unless it's a test of cached plan. + val rddIdExecCount = testSLAMetric.getHighestRDDId + if (cachedPlanTest) { + assert(rddIdExecCount === rddIdExec) + } else { + // count() creates a new plan with new RDDs. + assert(rddIdExecCount.get > rddIdExec.get) + } + } + } + } + } + } + } + + protected def testPlanShape( + label: String, + sqlQuery: String, + // Assert on the result of the test metric. + metricValueCheck: MetricValue.Check, + testTags: Tag* + ): Unit = { + testPhysicalPlanShape( + label = label, + sqlQuery = sqlQuery, + executedPlanCheck = PhysicalPlan.ANY, + metricValueCheck = metricValueCheck + )(testTags: _*) + } + + testPlanShape( + label = "simple plan", + sqlQuery = s"SELECT * FROM $TABLE_NAME WHERE increment_metric()", + metricValueCheck = MetricValue.exactly(NUM_RECORDS) + ) + + /* ******************** + * Various Subquery Plans + * ********************** */ + testPlanShape( + label = "subquery - IN", + sqlQuery = + s"""SELECT * + | FROM $TABLE_NAME + | WHERE id IN ( + | SELECT low_cardinality_col + | FROM $TABLE_NAME + | WHERE increment_metric())""".stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS) + ) + + testPlanShape( + label = "subquery - IN - aggregation", + sqlQuery = + s"""SELECT * + | FROM $TABLE_NAME + | WHERE id IN ( + | SELECT DISTINCT(low_cardinality_col) + | FROM $TABLE_NAME + | WHERE increment_metric())""".stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS) + ) + + testPlanShape( + label = "subquery - IN - TVF", + sqlQuery = + s"""SELECT * + | FROM $TABLE_NAME + | WHERE id IN ( + | SELECT * + | FROM range(5) + | WHERE increment_metric())""".stripMargin, + metricValueCheck = MetricValue.exactly(5) + ) + + testPlanShape( + label = "subquery - IN - explode", + sqlQuery = + s"""SELECT * + | FROM $TABLE_NAME + | WHERE id IN ( + | SELECT explode(array(low_cardinality_col, low_cardinality_col + 1)) + | FROM $TABLE_NAME + | WHERE increment_metric())""".stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS) + ) + + testPlanShape( + label = "subquery - IN - lateral view explode", + sqlQuery = + s"""SELECT * + | FROM $TABLE_NAME + | WHERE id IN ( + | SELECT new_column + | FROM $TABLE_NAME LATERAL VIEW + | explode(array(low_cardinality_col, low_cardinality_col + 1)) AS new_column + | WHERE increment_metric())""".stripMargin, + metricValueCheck = MetricValue.exactly(2 * NUM_RECORDS) + ) + + testPlanShape( + label = "subquery - scalar", + sqlQuery = + s"""SELECT * + | FROM $TABLE_NAME + | WHERE id == ( + | SELECT MAX(low_cardinality_col) + | FROM $TABLE_NAME + | WHERE increment_metric())""".stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS) + ) + + testPhysicalPlanShape( + label = "subquery - EXISTS", + sqlQuery = + s"""SELECT * + | FROM $TABLE_NAME + | WHERE EXISTS ( + | SELECT low_cardinality_col + | FROM $TABLE_NAME + | WHERE increment_metric())""".stripMargin, + // This turns into a LIMIT query. + metricValueCheck = MetricValue.atLeast(1), + executedPlanCheck = PhysicalPlan.contains[CollectLimitExec] + )() + + testPhysicalPlanShape( + label = "subquery - EXISTS (correlated)", + sqlQuery = + s"""SELECT * + | FROM $TABLE_NAME outer_table + | WHERE EXISTS ( + | SELECT low_cardinality_col + | FROM $TABLE_NAME inner_table + | WHERE increment_metric() + | AND inner_table.low_cardinality_col == outer_table.low_cardinality_col) + | """.stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS), + executedPlanCheck = PhysicalPlan.exists { + case _: BroadcastExchangeExec => true + case _: ShuffleExchangeExec => true + case _: ReusedExchangeExec => true + } + )() + + /* ***************************** + * Plans with different Exchanges + * ****************************** */ + + /* + * To cover: + * - ShuffleExchangeLike + * - ShuffleExchangeExec: covered by exchange - Shuffle + * - ReusedExchangeExec: covered by exchange - ReusedExchangeExec + * - BroadcastExchangeLike: + * - BroadcastExchangeExec: covered above by subquery - EXISTS (correlated)) + * - InMemoryTableScanLike (InMemoryTableScanExec): covered by exchange - InMemoryTableScanExec + */ + + testPhysicalPlanShape( + label = "exchange - Shuffle", + sqlQuery = + s"""SELECT * + | FROM $TABLE_NAME orig + | FULL OUTER JOIN ( + | SELECT * + | FROM $TABLE_NAME + | WHERE increment_metric() + | ) with_metric USING (id)""".stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS), + executedPlanCheck = PhysicalPlan.exists { + case _: ShuffleExchangeExec => true + // After forced AQE replans it may use ReusedExchange. + case _: ReusedExchangeExec if PhysicalPlan.hasAQEReplans => true + } + )() + + testPhysicalPlanShape( + label = "exchange - ReusedExchangeExec", + sqlQuery = + s"""WITH subquery_with_metric AS ( + | SELECT * + | FROM $TABLE_NAME + | WHERE increment_metric() + | ) + |SELECT * + | FROM subquery_with_metric a JOIN subquery_with_metric b USING (id)""".stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS), + executedPlanCheck = PhysicalPlan.contains[ReusedExchangeExec] + )() + + for (eager <- Seq("true", "false", "manual")) { + // SLAM metric in the top stage of cached query. + testPhysicalPlanShape( + label = s"cache - InMemoryTableScanExec - result stage - eager=$eager", + setup = () => { + spark.sql(s""" + |CREATE OR REPLACE TEMP VIEW table_with_metric AS ( + | SELECT low_cardinality_col + | FROM $TABLE_NAME + | WHERE increment_metric() + |)""".stripMargin) + if (eager == "true") { + spark.sql("CACHE TABLE table_with_metric") + } else { // false or manual + spark.sql("CACHE LAZY TABLE table_with_metric") + } + if (eager == "manual") { + spark.sql("select count(*) from table_with_metric").collect() + } + }, + sqlQuery = + s"""SELECT * + | FROM $TABLE_NAME + | WHERE id IN (SELECT * FROM table_with_metric)""".stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS), + executedPlanCheck = PhysicalPlan.contains[InMemoryTableScanExec] + )() + + // SLAM metric in the map stage of cached query. + testPhysicalPlanShape( + label = s"cache - InMemoryTableScanExec - map stage - eager=$eager", + setup = () => { + spark.sql(s""" + |CREATE OR REPLACE TEMP VIEW table_with_metric AS ( + | SELECT id, SUM(low_cardinality_col) + | FROM $TABLE_NAME + | WHERE increment_metric() + | GROUP BY id + |)""".stripMargin) + if (eager == "true") { + spark.sql("CACHE TABLE table_with_metric") + } else { // false or manual + spark.sql("CACHE LAZY TABLE table_with_metric") + } + if (eager == "manual") { + spark.sql("select count(*) from table_with_metric").collect() + } + }, + sqlQuery = + s"""SELECT * + | FROM table_with_metric""".stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS), + executedPlanCheck = PhysicalPlan.contains[InMemoryTableScanExec] + )() + + testPhysicalPlanShape( + label = s"cache - localCheckpoint - result stage - eager=$eager", + setup = () => { + val df = spark.sql(s""" + |SELECT low_cardinality_col + |FROM $TABLE_NAME + |WHERE increment_metric()""".stripMargin) + val cpEager = if (eager == "true") true else false + val cpDf = df.localCheckpoint(eager = cpEager) + if (eager == "manual") { + cpDf.count() + } + cpDf.createOrReplaceTempView("cp_table_with_metric") + }, + sqlQuery = + s"""SELECT * + | FROM cp_table_with_metric""".stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS), + executedPlanCheck = PhysicalPlan.contains[RDDScanExec] + )() + + testPhysicalPlanShape( + label = s"cache - localCheckpoint - map stage - eager=$eager", + setup = () => { + val df = spark.sql(s""" + |SELECT id, SUM(low_cardinality_col) + |FROM $TABLE_NAME + |WHERE increment_metric() + |GROUP BY id""".stripMargin) + val cpEager = if (eager == "true") true else false + val cpDf = df.localCheckpoint(eager = cpEager) + if (eager == "manual") { + cpDf.count() + } + cpDf.createOrReplaceTempView("cp_table_with_metric") + }, + sqlQuery = + s"""SELECT * + | FROM cp_table_with_metric""".stripMargin, + metricValueCheck = MetricValue.exactly(NUM_RECORDS), + executedPlanCheck = PhysicalPlan.contains[RDDScanExec] + )() + } +} + +object SQLLastAttemptMetricPlanShapesSuite { + val NUM_RECORDS: Long = 300 + val LOW_CARDINALITY: Int = 5 + val LARGE_CARDINALITY: Int = 111 + + val TABLE_NAME: String = "test_table" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetricUnitSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetricUnitSuite.scala new file mode 100644 index 0000000000000..f3a696f81450e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLLastAttemptMetricUnitSuite.scala @@ -0,0 +1,188 @@ +/* + * 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.metric + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream} +import java.util.Properties + +import org.mockito.Mockito.when +import org.scalatestplus.mockito.MockitoSugar.mock + +import org.apache.spark.{SharedSparkContext, SparkFunSuite} +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.TaskInfo + +/** Tests internals of [[SQLLastAttemptMetric]]. */ +class SQLLastAttemptMetricUnitSuite extends SparkFunSuite with SharedSparkContext { + + // scalastyle:off classforname + private val sqlLastAttemptMetricClass = Class + .forName("org.apache.spark.sql.execution.metric.SQLLastAttemptMetric") + // scalastyle:on classforname + + private val lastAttemptInitializedField = + sqlLastAttemptMetricClass.getDeclaredField("lastAttemptAccumulatorInitialized") + + private val lastAttemptRddsMapField = + sqlLastAttemptMetricClass.getDeclaredField( + "org$apache$spark$util$LastAttemptAccumulator$$lastAttemptRddsMap") + + private val directDriverValueField = + sqlLastAttemptMetricClass.getDeclaredField( + "org$apache$spark$util$LastAttemptAccumulator$$lastAttemptDirectDriverValue") + + private val partialMergeValMethod = sqlLastAttemptMetricClass.getMethod("partialMergeVal") + + private val mockRdd = mock[RDD[_]] + private val mockTaskInfo = mock[TaskInfo] + private val mockProperties = new Properties + + // Set mock attempt for mock Task, TaskInfo and RDD + // that can be used with mergeLastAttempt. + // stageId and stageAttemptId are passed directly to mergeLastAttempt. + def setMockAttempt(rddId: Int, partitionId: Int): Unit = { + // reset to mock defaults + when(mockTaskInfo.attemptNumber).thenReturn(0) + when(mockRdd.scope).thenReturn(None) + when(mockRdd.getNumPartitions).thenReturn(5) + + when(mockRdd.id).thenReturn(rddId) + when(mockTaskInfo.partitionId).thenReturn(partitionId) + } + + override def beforeAll(): Unit = { + super.beforeAll() + lastAttemptInitializedField.setAccessible(true) + lastAttemptRddsMapField.setAccessible(true) + directDriverValueField.setAccessible(true) + partialMergeValMethod.setAccessible(true) + } + + override def afterAll(): Unit = { + lastAttemptInitializedField.setAccessible(false) + lastAttemptRddsMapField.setAccessible(false) + directDriverValueField.setAccessible(false) + partialMergeValMethod.setAccessible(false) + super.afterAll() + } + + test("serialization and deserialization") { + val slam = SQLLastAttemptMetrics.createMetric(sc, "test SLAM") + + assert(lastAttemptInitializedField.getBoolean(slam) === true) + assert(lastAttemptRddsMapField.get(slam) != null) + assert(directDriverValueField.get(slam) != null) + + // Serialize slam to ObjectOutputStream and deserialize it back. + val obs1 = new ByteArrayOutputStream() + val oos1 = new ObjectOutputStream(obs1) + oos1.writeObject(slam) + oos1.close() + val ois1 = new ObjectInputStream(new ByteArrayInputStream(obs1.toByteArray)) + val deser = ois1.readObject().asInstanceOf[SQLLastAttemptMetric] + + // serialized version should not be initialized + assert(lastAttemptInitializedField.getBoolean(deser) === false) + assert(lastAttemptRddsMapField.get(deser) == null) + assert(directDriverValueField.get(deser) == null) + + deser.set(42) + deser.add(7) + assert(deser.value === 49) + // these functions shouldn't be used on the deserialized metric, + // but assertions should be caught and None should be returned. + assert(deser.lastAttemptValueForHighestRDDId() === None) + assert(deser.lastAttemptValueForRDDId(1) === None) + assert(deser.lastAttemptValueForRDDIds(Seq(1, 2, 3)) === None) + assert(deser.lastAttemptValueForAllRDDs() === None) + // mergeLastAttempt shouldn't be used on the deserialized metric, + // but it should catch error and not fail. + deser.mergeLastAttempt(slam, null, null, 0, 0, null) + + // Serialize and deserialize again. + val obs2 = new ByteArrayOutputStream() + val oos2 = new ObjectOutputStream(obs2) + oos2.writeObject(deser) + oos2.close() + val ois2 = new ObjectInputStream(new ByteArrayInputStream(obs2.toByteArray)) + val reser = ois2.readObject().asInstanceOf[SQLLastAttemptMetric] + // Check that the value is brought back and can be used as partialMergeVal. + assert(reser.value === 49L) + assert(partialMergeValMethod.invoke(reser) === 49L) + } + + test("copy and mergeLastAttempt") { + val slam = SQLLastAttemptMetrics.createMetric(sc, "test SLAM") + + assert(lastAttemptInitializedField.getBoolean(slam) == true) + assert(lastAttemptRddsMapField.get(slam) != null) + assert(directDriverValueField.get(slam) != null) + + // copy should not initialize SLAM data. + val acc = slam.copy() + assert(lastAttemptInitializedField.getBoolean(acc) == false) + assert(lastAttemptRddsMapField.get(acc) == null) + assert(directDriverValueField.get(acc) == null) + // these functions shouldn't be used on the copy, + // but assertions should be caught and None should be returned. + assert(acc.lastAttemptValueForHighestRDDId() === None) + assert(acc.lastAttemptValueForRDDId(1) === None) + assert(acc.lastAttemptValueForRDDIds(Seq(1, 2, 3)) === None) + assert(acc.lastAttemptValueForAllRDDs() === None) + // mergeLastAttempt shouldn't be used on the copy, + // but it should catch error and not fail. + acc.mergeLastAttempt(slam, null, null, 0, 0, null) + + // Let's play with merging acc into slam. + setMockAttempt(rddId = 1, partitionId = 0) + acc.set(10) + slam.mergeLastAttempt(acc, mockRdd, mockTaskInfo, 10, 10, mockProperties) + assert(slam.lastAttemptValueForRDDId(1) === Some(10)) + + setMockAttempt(rddId = 1, partitionId = 1) + acc.set(10) // new partition id + slam.mergeLastAttempt(acc, mockRdd, mockTaskInfo, 10, 10, mockProperties) + assert(slam.lastAttemptValueForRDDId(1) === Some(20)) // 10 + 10, aggregated new partition id + + setMockAttempt(rddId = 1, partitionId = 1) + acc.set(7) // same partition id, older attempt. + slam.mergeLastAttempt(acc, mockRdd, mockTaskInfo, 10, 9, mockProperties) + assert(slam.lastAttemptValueForRDDId(1) === Some(20)) // no change + + setMockAttempt(rddId = 1, partitionId = 1) + acc.set(7) // same partition id, older stage. + slam.mergeLastAttempt(acc, mockRdd, mockTaskInfo, 9, 11, mockProperties) + assert(slam.lastAttemptValueForRDDId(1) === Some(20)) // no change + + setMockAttempt(rddId = 1, partitionId = 1) + acc.set(7) // same partition id, newer attempt. + slam.mergeLastAttempt(acc, mockRdd, mockTaskInfo, 10, 11, mockProperties) + assert(slam.lastAttemptValueForRDDId(1) === Some(17)) // 10 replaced with 7 + + setMockAttempt(rddId = 1, partitionId = 1) + acc.set(8) // same partition id, newer stage. + slam.mergeLastAttempt(acc, mockRdd, mockTaskInfo, 11, 1, mockProperties) + assert(slam.lastAttemptValueForRDDId(1) === Some(18)) // 7 replaced with 8 + + setMockAttempt(rddId = 2, partitionId = 2) + acc.set(42) // new RDD + slam.mergeLastAttempt(acc, mockRdd, mockTaskInfo, 1, 1, mockProperties) + assert(slam.lastAttemptValueForRDDId(1) === Some(18)) // no change for rddId=1 + assert(slam.lastAttemptValueForRDDId(2) === Some(42)) // new RDD added + assert(slam.lastAttemptValueForAllRDDs() === Some(60)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala index e8902ed6fb1a1..483d2a72637d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala @@ -25,14 +25,38 @@ import org.apache.spark.TestUtils import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{DataFrame, QueryTest} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.{SparkPlan, SparkPlanInfo} import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SQLAppStatusStore} +import org.apache.spark.sql.functions.udf import org.apache.spark.sql.internal.SQLConf.WHOLESTAGE_CODEGEN_ENABLED trait SQLMetricsTestUtils extends QueryTest { import testImplicits._ + protected val BOOLEAN_DOMAIN: Seq[Boolean] = Seq(true, false) + + /** + * @return An `Expression` that increments a SQL metric and + * evaluates to true. Can be used in a filter. + */ + protected def incrementMetric( + metric: SQLMetric): Expression = { + udf { () => + { metric += 1; true } + }.asNondeterministic().apply().expr + } + + /** @return An `Expression` to increment multiple SQL metrics */ + protected def incrementMetrics(metrics: Seq[SQLMetric]): Expression = { + metrics.map(incrementMetric(_)).fold( + org.apache.spark.sql.catalyst.expressions.Literal(true): Expression) { + (acc, incrMetric) => + org.apache.spark.sql.catalyst.expressions.And(acc, incrMetric) + } + } + protected def currentExecutionIds(): Set[Long] = { spark.sparkContext.listenerBus.waitUntilEmpty(10000) statusStore.executionsList().map(_.executionId).toSet diff --git a/sql/hive/src/test/resources/conf/binding-policy-exceptions/configs-without-binding-policy-exceptions b/sql/hive/src/test/resources/conf/binding-policy-exceptions/configs-without-binding-policy-exceptions index 4563e81d14064..2aa6cb885ca31 100644 --- a/sql/hive/src/test/resources/conf/binding-policy-exceptions/configs-without-binding-policy-exceptions +++ b/sql/hive/src/test/resources/conf/binding-policy-exceptions/configs-without-binding-policy-exceptions @@ -1173,6 +1173,7 @@ spark.taskMetrics.trackUpdatedBlockStatuses spark.test.noStageRetry spark.testing spark.testing.dynamicAllocation.schedule.enabled +spark.testing.injectShuffleFetchFailures spark.testing.memory spark.testing.nCoresPerExecutor spark.testing.nExecutorsPerHost