Saisai Shao created SPARK-13669: ----------------------------------- Summary: Job will always fail in the external shuffle service unavailable situation Key: SPARK-13669 URL: https://issues.apache.org/jira/browse/SPARK-13669 Project: Spark Issue Type: Bug Components: Spark Core, YARN Reporter: Saisai Shao
Currently we are running into an issue with Yarn work preserving enabled + external shuffle service. In the work preserving enabled scenario, the failure of NM will not lead to the exit of executors, so executors can still accept and run the tasks. The problem here is when NM is failed, external shuffle service is actually inaccessible, so reduce tasks will always complain about the “Fetch failure”, and the failure of reduce stage will make the parent stage (map stage) rerun. The tricky thing here is Spark scheduler is not aware of the unavailability of external shuffle service, and will reschedule the map tasks on the executor where NM is failed, and again reduce stage will be failed with “Fetch failure”, and after 4 retries, the job is failed. So here the actual problem is Spark’s scheduler is not aware of the unavailability of external shuffle service, and will still assign the tasks on to that nodes. The fix is to avoid assigning tasks on to that nodes. Currently in the Spark, one related configuration is “spark.scheduler.executorTaskBlacklistTime”, but I don’t think it will be worked in this scenario. This configuration is used to avoid same reattempt task to run on the same executor. Also ways like MapReduce’s blacklist mechanism may not handle this scenario, since all the reduce tasks will be failed, so counting the failure tasks will equally mark all the executors as “bad” one. -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org