spark-reviews mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From kayousterhout <...@git.apache.org>
Subject [GitHub] spark pull request #15644: [SPARK-18117][CORE] Add test for TaskSetBlacklist
Date Thu, 17 Nov 2016 19:33:41 GMT
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/15644#discussion_r88528705
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
---
    @@ -282,6 +317,188 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext
with B
         assert(!failedTaskSet)
       }
     
    +  test("scheduled tasks obey task and stage blacklists") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist()
    +    (0 to 2).foreach {stageId =>
    +      val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId
= 0)
    +      taskScheduler.submitTasks(taskSet)
    +    }
    +
    +    // Setup our mock blacklist:
    +    // * stage 0 is blacklisted on node "host1"
    +    // * stage 1 is blacklisted on executor "executor3"
    +    // * stage 0, partition 0 is blacklisted on executor 0
    +    // (mocked methods default to returning false, ie. no blacklisting)
    +    when(stageToMockTaskSetBlacklist(0).isNodeBlacklistedForTaskSet("host1")).thenReturn(true)
    +    when(stageToMockTaskSetBlacklist(1).isExecutorBlacklistedForTaskSet("executor3"))
    +      .thenReturn(true)
    +    when(stageToMockTaskSetBlacklist(0).isExecutorBlacklistedForTask("executor0", 0))
    +      .thenReturn(true)
    +
    +    val offers = IndexedSeq(
    +      new WorkerOffer("executor0", "host0", 1),
    +      new WorkerOffer("executor1", "host1", 1),
    +      new WorkerOffer("executor2", "host1", 1),
    +      new WorkerOffer("executor3", "host2", 10)
    +    )
    +    val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten
    +    // We should schedule all tasks.
    +    assert(firstTaskAttempts.size === 6)
    +    // Whenever we schedule a task, we must consult the node and executor blacklist.
 (The test
    +    // doesn't check exactly what checks are made because the offers get shuffled.)
    +    (0 to 2).foreach { stageId =>
    +      verify(stageToMockTaskSetBlacklist(stageId), atLeast(1))
    +        .isNodeBlacklistedForTaskSet(anyString())
    +      verify(stageToMockTaskSetBlacklist(stageId), atLeast(1))
    +        .isExecutorBlacklistedForTaskSet(anyString())
    +    }
    +
    +    def tasksForStage(stageId: Int): Seq[TaskDescription] = {
    +      firstTaskAttempts.filter{_.name.contains(s"stage $stageId")}
    +    }
    +    tasksForStage(0).foreach { task =>
    +      // executors 1 & 2 blacklisted for node
    +      // executor 0 blacklisted just for partition 0
    +      if (task.index == 0) {
    +        assert(task.executorId === "executor3")
    +      } else {
    +        assert(Set("executor0", "executor3").contains(task.executorId))
    +      }
    +    }
    +    tasksForStage(1).foreach { task =>
    +      // executor 3 blacklisted
    +      assert("executor3" != task.executorId)
    +    }
    +    // no restrictions on stage 2
    +
    +    // Finally, just make sure that we can still complete tasks as usual with blacklisting
    +    // in effect.  Finish each of the tasksets -- taskset 0 & 1 complete successfully,
taskset 2
    +    // fails.
    +    (0 to 2).foreach { stageId =>
    +      val tasks = tasksForStage(stageId)
    +      val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get
    +      val valueSer = SparkEnv.get.serializer.newInstance()
    +      if (stageId == 2) {
    +        // Just need to make one task fail 4 times.
    +        var task = tasks(0)
    +        val taskIndex = task.index
    +        (0 until 4).foreach { attempt =>
    +          assert(task.attemptNumber === attempt)
    +          tsm.handleFailedTask(task.taskId, TaskState.FAILED, TaskResultLost)
    +          val nextAttempts =
    +            taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("executor4", "host4",
1))).flatten
    +          if (attempt < 3) {
    +            assert(nextAttempts.size === 1)
    +            task = nextAttempts(0)
    +            assert(task.index === taskIndex)
    +          } else {
    +            assert(nextAttempts.size === 0)
    +          }
    +        }
    +        // End the other task of the taskset, doesn't matter whether it succeeds or fails.
    +        val otherTask = tasks(1)
    +        val result = new DirectTaskResult[Int](valueSer.serialize(otherTask.taskId),
Seq())
    +        tsm.handleSuccessfulTask(otherTask.taskId, result)
    +      } else {
    +        tasks.foreach { task =>
    +          val result = new DirectTaskResult[Int](valueSer.serialize(task.taskId), Seq())
    +          tsm.handleSuccessfulTask(task.taskId, result)
    +        }
    +      }
    +      assert(tsm.isZombie)
    +    }
    +  }
    +
    +  /**
    +   * Helper for performance tests.  Takes the explicitly blacklisted nodes and executors;
verifies
    +   * that the blacklists are used efficiently to ensure scheduling is not O(numPendingTasks).
    +   */
    +  private def blacklistPerformanceCheck(
    +      nodeBlacklist: Seq[String],
    +      execBlacklist: Seq[String]): Unit = {
    +    // When an executor or node is blacklisted, we want to make sure that we don't try
scheduling
    +    // each pending task, one by one, to discover they are all blacklisted.  This is
important for
    +    // performance -- if we did check each task one-by-one, then responding to a resource
offer
    +    // (which is usually O(1)-ish) would become O(numPendingTasks), which would slow
down
    +    // scheduler throughput and slow down scheduling even on healthy executors.
    +    // Here, we check a proxy for the runtime -- we make sure the scheduling is short-circuited
    +    // at the node or executor blacklist, so we never check the per-task blacklist. 
We also make
    +    // sure we don't check the node & executor blacklist for the entire taskset O(numPendingTasks)
    +    // times.
    +
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist()
    +    // we schedule 500 tasks so we can clearly distinguish anything that is O(numPendingTasks)
    +    val taskSet = FakeTask.createTaskSet(numTasks = 500, stageId = 0, stageAttemptId
= 0)
    +    taskScheduler.submitTasks(taskSet)
    +
    +    val offers = IndexedSeq(
    +      new WorkerOffer("executor1", "host1", 1),
    +      new WorkerOffer("executor2", "host1", 1),
    +      new WorkerOffer("executor3", "host2", 1)
    +    )
    +    // We should check the node & exec blacklists, but only O(numOffers), not O(numPendingTasks)
    +    // times.  Its O(numOffers), not exactly numOffers, because we offer resources multiple
    +    // times.  An upper bound on the worst case is -- we check it once for every core
at each
    +    // locality level.  (We could tighten the bound a bit but that should be a good enough
check.)
    +    val numCoresOnAllOffers = offers.map(_.cores).sum
    +    val numLocalityLevels = TaskLocality.values.size
    +    val maxBlacklistChecks = numCoresOnAllOffers * numLocalityLevels
    +
    +    // Setup the blacklist, and get back a list of the executors & nodes that have
any blacklisting
    +    // (even implicit blacklisting).
    +
    +    nodeBlacklist.foreach { node =>
    +      when(stageToMockTaskSetBlacklist(0).isNodeBlacklistedForTaskSet(node)).thenReturn(true)
    +    }
    +    execBlacklist.foreach { exec =>
    +      when(stageToMockTaskSetBlacklist(0).isExecutorBlacklistedForTaskSet(exec)).thenReturn(true)
    +    }
    +    val nodesForBlacklistedExecutors = offers.filter { offer =>
    +      execBlacklist.contains(offer.executorId)
    +    }.map(_.host).toSet.toSeq
    +    val nodesToCheck = nodeBlacklist ++ nodesForBlacklistedExecutors
    +    val execsForBlacklistedNodes = offers.filter { offer =>
    +      nodeBlacklist.contains(offer.host)
    +    }.map(_.executorId).toSeq
    +    val executorsToCheck = execBlacklist ++ execsForBlacklistedNodes
    --- End diff --
    
    Also I know in an earlier comment you said you wanted some successful tasks here too.
 I'm in favor of eliminating that because all of these tests have become super integration-test-y
rather than unit-test-y, and I think having smaller unit tests is helpful for debugging future
issues (and you do have some other more integration-y tests for the blacklist).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Mime
View raw message