Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/15249#discussion_r81402071
  
    --- Diff: 
core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala 
---
    @@ -51,37 +54,67 @@ class BlacklistIntegrationSuite extends 
SchedulerIntegrationSuite[MultiExecutorM
         assertDataStructuresEmpty(noFailure = false)
       }
     
    -  // even with the blacklist turned on, if maxTaskFailures is not more 
than the number
    -  // of executors on the bad node, then locality preferences will lead to 
us cycling through
    -  // the executors on the bad node, and still failing the job
    +  // even with the blacklist turned on, bad configs can lead to job 
failure.  To survive one
    +  // bad node, you need to make sure that
    +  // maxTaskFailures > min(spark.blacklist.task.maxTaskAttemptsPerNode, 
nExecutorsPerHost)
       testScheduler(
         "With blacklist on, job will still fail if there are too many bad 
executors on bad host",
         extraConfs = Seq(
    -      // set this to something much longer than the test duration so that 
executors don't get
    -      // removed from the blacklist during the test
    -      ("spark.scheduler.executorTaskBlacklistTime", "10000000")
    +      config.BLACKLIST_ENABLED.key -> "true",
    +      config.MAX_TASK_ATTEMPTS_PER_NODE.key -> "5",
    +      config.MAX_TASK_FAILURES.key -> "4",
    +      "spark.testing.nHosts" -> "2",
    +      "spark.testing.nExecutorsPerHost" -> "5",
    +      "spark.testing.nCoresPerExecutor" -> "10",
    +      // Blacklisting will normally immediately complain that this config 
is invalid -- the point
    +      // of this test is to expose that the configuration is unsafe, so 
skip the validation.
    +      "spark.blacklist.testing.skipValidation" -> "true"
         )
       ) {
    -    val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost)
    +    // to reliably reproduce the failure, we have to use 1 task.  That 
way, we ensure this
    +    // 1 task gets rotated through enough bad executors on the host to 
fail the taskSet,
    +    // before we have a bunch of different tasks fail in the executors so 
we blacklist them.
    +    // But the point here is -- we never try scheduling tasks on the good 
host-1, since we
    +    // hit too many failures trying our preferred host-0.
    +    val rdd = new MockRDDWithLocalityPrefs(sc, 1, Nil, badHost)
         withBackend(badHostBackend _) {
    -      val jobFuture = submit(rdd, (0 until 10).toArray)
    +      val jobFuture = submit(rdd, (0 until 1).toArray)
           awaitJobTermination(jobFuture, duration)
         }
         assertDataStructuresEmpty(noFailure = false)
       }
     
    -  // Here we run with the blacklist on, and maxTaskFailures high enough 
that we'll eventually
    -  // schedule on a good node and succeed the job
    +  testScheduler(
    +    "With default settings, job can succeed despite multiple bad executors 
on node",
    +    extraConfs = Seq(
    +      config.BLACKLIST_ENABLED.key -> "true",
    +      config.MAX_TASK_FAILURES.key -> "4",
    +      "spark.testing.nHosts" -> "2",
    +      "spark.testing.nExecutorsPerHost" -> "5",
    +      "spark.testing.nCoresPerExecutor" -> "10"
    +    )
    +  ) {
    +    // to reliably reproduce the failure, we have to use 1 task.  That 
way, we ensure this
    --- End diff --
    
    this comment is confusing because the job is supposed to succeed right?


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to