-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-25449][CORE] Heartbeat shouldn't include accumulators for zero metrics #22473
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from 2 commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
3e0d953
Don't send zero accumulators for metrics in heartbeat
mukulmurthy 3cf88a4
add tests
mukulmurthy fc98256
Refactor heartbeat configs and gate dropping zero metrics behind a co…
mukulmurthy 6bb91b0
update tests
mukulmurthy 7d0c39b
refactor code per Ryan's review
mukulmurthy db14bd1
minor refactor
mukulmurthy e22ac78
use PrivateMethodTester
mukulmurthy 9cbe14c
Use TestMemoryManager for test to not have to mock object field
mukulmurthy f6fa337
Use getTimeAsSeconds
mukulmurthy File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,9 +21,10 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} | |
| import java.lang.Thread.UncaughtExceptionHandler | ||
| import java.nio.ByteBuffer | ||
| import java.util.Properties | ||
| import java.util.concurrent.{CountDownLatch, TimeUnit} | ||
| import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, TimeUnit} | ||
| import java.util.concurrent.atomic.AtomicBoolean | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.collection.mutable.Map | ||
| import scala.concurrent.duration._ | ||
| import scala.language.postfixOps | ||
|
|
@@ -39,14 +40,14 @@ import org.scalatest.mockito.MockitoSugar | |
| import org.apache.spark._ | ||
| import org.apache.spark.TaskState.TaskState | ||
| import org.apache.spark.memory.MemoryManager | ||
| import org.apache.spark.metrics.MetricsSystem | ||
| import org.apache.spark.metrics.{JVMHeapMemory, JVMOffHeapMemory, MetricsSystem} | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.rpc.RpcEnv | ||
| import org.apache.spark.scheduler.{FakeTask, ResultTask, TaskDescription} | ||
| import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcTimeout} | ||
| import org.apache.spark.scheduler.{FakeTask, ResultTask, Task, TaskDescription} | ||
| import org.apache.spark.serializer.{JavaSerializer, SerializerManager} | ||
| import org.apache.spark.shuffle.FetchFailedException | ||
| import org.apache.spark.storage.BlockManagerId | ||
| import org.apache.spark.util.UninterruptibleThread | ||
| import org.apache.spark.storage.{BlockManager, BlockManagerId} | ||
| import org.apache.spark.util.{LongAccumulator, UninterruptibleThread, Utils} | ||
|
|
||
| class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar with Eventually { | ||
|
|
||
|
|
@@ -252,18 +253,105 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug | |
| } | ||
| } | ||
|
|
||
| test("Heartbeat should drop zero metrics") { | ||
| withHeartbeatExecutor((executor, heartbeats) => { | ||
| // When no tasks are running, there should be no accumulators sent in heartbeat | ||
| invokeReportHeartbeat(executor) | ||
| assert(heartbeats.length == 1) | ||
| assert(heartbeats(0).accumUpdates.length == 0, | ||
| "No updates should be sent when no tasks are running") | ||
|
|
||
| // When we start a task with a nonzero accumulator, that should end up in the heartbeat | ||
| val metrics = new TaskMetrics() | ||
| val nonZeroAccumulator = new LongAccumulator() | ||
| nonZeroAccumulator.add(1) | ||
| metrics.registerAccumulator(nonZeroAccumulator) | ||
|
|
||
| val executorClass = classOf[Executor] | ||
| val tasksMap = { | ||
| val field = | ||
| executorClass.getDeclaredField("org$apache$spark$executor$Executor$$runningTasks") | ||
| field.setAccessible(true) | ||
| field.get(executor).asInstanceOf[ConcurrentHashMap[Long, executor.TaskRunner]] | ||
| } | ||
| val mockTaskRunner = mock[executor.TaskRunner] | ||
| val mockTask = mock[Task[Any]] | ||
| when(mockTask.metrics).thenReturn(metrics) | ||
| when(mockTaskRunner.taskId).thenReturn(6) | ||
| when(mockTaskRunner.task).thenReturn(mockTask) | ||
| when(mockTaskRunner.startGCTime).thenReturn(1) | ||
| tasksMap.put(6, mockTaskRunner) | ||
|
|
||
| invokeReportHeartbeat(executor) | ||
| assert(heartbeats.length == 2) | ||
| val updates = heartbeats(1).accumUpdates | ||
| assert(updates.length == 1 && updates(0)._1 == 6, | ||
| "Heartbeat should only send update for the one task running") | ||
| val accumsSent = updates(0)._2.length | ||
| assert(accumsSent > 0, "The nonzero accumulator we added should be sent") | ||
| assert(accumsSent == metrics.accumulators().count(!_.isZero), | ||
| "The number of accumulators sent should match the number of nonzero accumulators") | ||
| }) | ||
| } | ||
|
|
||
| private def withHeartbeatExecutor(f: (Executor, ArrayBuffer[Heartbeat]) => Unit): Unit = { | ||
| val conf = new SparkConf | ||
| val serializer = new JavaSerializer(conf) | ||
| val env = createMockEnv(conf, serializer) | ||
| val executor = | ||
| new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true) | ||
| val executorClass = classOf[Executor] | ||
|
|
||
| // Set ExecutorMetricType.values to be a minimal set to avoid get null exceptions | ||
| val metricClass = | ||
| Utils.classForName(classOf[org.apache.spark.metrics.ExecutorMetricType].getName() + "$") | ||
| val metricTypeValues = metricClass.getDeclaredField("values") | ||
| metricTypeValues.setAccessible(true) | ||
| metricTypeValues.set( | ||
| org.apache.spark.metrics.ExecutorMetricType, | ||
| IndexedSeq(JVMHeapMemory, JVMOffHeapMemory)) | ||
|
|
||
| // Save all heartbeats sent into an ArrayBuffer for verification | ||
| val heartbeats = ArrayBuffer[Heartbeat]() | ||
| val mockReceiver = mock[RpcEndpointRef] | ||
| when(mockReceiver.askSync(any[Heartbeat], any[RpcTimeout])(any)) | ||
| .thenAnswer(new Answer[HeartbeatResponse] { | ||
| override def answer(invocation: InvocationOnMock): HeartbeatResponse = { | ||
| val args = invocation.getArguments() | ||
| val mock = invocation.getMock | ||
| heartbeats += args(0).asInstanceOf[Heartbeat] | ||
| HeartbeatResponse(false) | ||
| } | ||
| }) | ||
| val receiverRef = executorClass.getDeclaredField("heartbeatReceiverRef") | ||
| receiverRef.setAccessible(true) | ||
| receiverRef.set(executor, mockReceiver) | ||
|
|
||
| f(executor, heartbeats) | ||
| } | ||
|
|
||
| private def invokeReportHeartbeat(executor: Executor): Unit = { | ||
|
||
| val method = classOf[Executor] | ||
| .getDeclaredMethod("org$apache$spark$executor$Executor$$reportHeartBeat") | ||
| method.setAccessible(true) | ||
| method.invoke(executor) | ||
| } | ||
|
|
||
| private def createMockEnv(conf: SparkConf, serializer: JavaSerializer): SparkEnv = { | ||
| val mockEnv = mock[SparkEnv] | ||
| val mockRpcEnv = mock[RpcEnv] | ||
| val mockMetricsSystem = mock[MetricsSystem] | ||
| val mockMemoryManager = mock[MemoryManager] | ||
| val mockBlockManager = mock[BlockManager] | ||
| when(mockEnv.conf).thenReturn(conf) | ||
| when(mockEnv.serializer).thenReturn(serializer) | ||
| when(mockEnv.serializerManager).thenReturn(mock[SerializerManager]) | ||
| when(mockEnv.rpcEnv).thenReturn(mockRpcEnv) | ||
| when(mockEnv.metricsSystem).thenReturn(mockMetricsSystem) | ||
| when(mockEnv.memoryManager).thenReturn(mockMemoryManager) | ||
| when(mockEnv.closureSerializer).thenReturn(serializer) | ||
| when(mockBlockManager.blockManagerId).thenReturn(BlockManagerId("1", "hostA", 1234)) | ||
| when(mockEnv.blockManager).thenReturn(mockBlockManager) | ||
| SparkEnv.set(mockEnv) | ||
| mockEnv | ||
| } | ||
|
|
||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you add a flag for this behavior change?