[ https://issues.apache.org/jira/browse/SPARK-20286?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15972231#comment-15972231 ]
Umesh Chaudhary commented on SPARK-20286: ----------------------------------------- Yep, +1 to the UI changes. However, I tested the behaviour by following the steps mentioned by you and this seems to be fixed in 2.1.0. {code} spark2-shell --conf spark.dynamicAllocation.executorIdleTimeout=7s --conf spark.dynamicAllocation.cachedExecutorIdleTimeout=20s --conf spark.dynamicAllocation.minExecutors=2 scala> sc.setLogLevel("INFO") scala> val rdd=sc.textFile("/tmp/config.txt") .. 17/04/17 23:46:58 INFO spark.SparkContext: Created broadcast 0 from textFile at <console>:24 rdd: org.apache.spark.rdd.RDD[String] = /tmp/config.txt MapPartitionsRDD[1] at textFile at <console>:24 .. .. scala> rdd.collect 17/04/17 23:47:10 INFO mapred.FileInputFormat: Total input paths to process : 1 17/04/17 23:47:10 INFO spark.SparkContext: Starting job: collect at <console>:27 .. .. 17/04/17 23:47:13 INFO scheduler.DAGScheduler: Job 0 finished: collect at <console>:27, took 3.457215 s .. scala> 17/04/17 23:47:20 INFO spark.ExecutorAllocationManager: Request to remove executorIds: 1 {code} As expected executors remove request was received above after 7 secs. Then I tested behaviour for persist: {code} scala> rdd.persist res2: rdd.type = /tmp/config.txt MapPartitionsRDD[1] at textFile at <console>:24 scala> rdd.count 17/04/17 23:47:45 INFO spark.SparkContext: Starting job: count at <console>:27 .. 17/04/17 23:47:45 INFO scheduler.DAGScheduler: Job 1 finished: count at <console>:27, took 0.293053 s .. scala> 17/04/17 23:48:05 INFO spark.ExecutorAllocationManager: Request to remove executorIds: 1 {code} In this case also request for removing the executor was received after 20 secs. Afterwords I tested unpersist: {code} scala> rdd.unpersist(true) 17/04/17 23:50:22 INFO rdd.MapPartitionsRDD: Removing RDD 1 from persistence list 17/04/17 23:50:22 INFO storage.BlockManager: Removing RDD 1 res7: rdd.type = /tmp/config.txt MapPartitionsRDD[1] at textFile at <console>:24 scala> rdd.count 17/04/17 23:50:31 INFO spark.SparkContext: Starting job: count at <console>:27 .. 17/04/17 23:50:31 INFO scheduler.DAGScheduler: Job 3 finished: count at <console>:27, took 0.219764 s res8: Long = 100 .. scala> 17/04/17 23:50:38 INFO spark.ExecutorAllocationManager: Request to remove executorIds: 1 {code} This time remove request was received after 7 secs. > dynamicAllocation.executorIdleTimeout is ignored after unpersist > ---------------------------------------------------------------- > > Key: SPARK-20286 > URL: https://issues.apache.org/jira/browse/SPARK-20286 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 2.0.1 > Reporter: Miguel Pérez > > With dynamic allocation enabled, it seems that executors with cached data > which are unpersisted are still being killed using the > {{dynamicAllocation.cachedExecutorIdleTimeout}} configuration, instead of > {{dynamicAllocation.executorIdleTimeout}}. Assuming the default configuration > ({{dynamicAllocation.cachedExecutorIdleTimeout = Infinity}}), an executor > with unpersisted data won't be released until the job ends. > *How to reproduce* > - Set different values for {{dynamicAllocation.executorIdleTimeout}} and > {{dynamicAllocation.cachedExecutorIdleTimeout}} > - Load a file into a RDD and persist it > - Execute an action on the RDD (like a count) so some executors are activated. > - When the action has finished, unpersist the RDD > - The application UI removes correctly the persisted data from the *Storage* > tab, but if you look in the *Executors* tab, you will find that the executors > remain *active* until ({{dynamicAllocation.cachedExecutorIdleTimeout}} is > reached. -- This message was sent by Atlassian JIRA (v6.3.15#6346) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org