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

    https://github.com/apache/spark/pull/3868#discussion_r23705218
  
    --- Diff: 
streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
 ---
    @@ -68,36 +69,60 @@ class JobScheduler(val ssc: StreamingContext) extends 
Logging {
       }
     
       def stop(processAllReceivedData: Boolean): Unit = synchronized {
    -    if (eventActor == null) return // scheduler has already been stopped
    -    logDebug("Stopping JobScheduler")
    +    val shutdownExecutor = Executors.newFixedThreadPool(1)
    +    implicit val context = 
ExecutionContext.fromExecutorService(shutdownExecutor)
     
    -    // First, stop receiving
    -    receiverTracker.stop()
    +    val shutdown = Future {
    +      if (eventActor == null) return // scheduler has already been stopped
    +      logDebug("Stopping JobScheduler")
     
    -    // Second, stop generating jobs. If it has to process all received 
data,
    -    // then this will wait for all the processing through JobScheduler to 
be over.
    -    jobGenerator.stop(processAllReceivedData)
    +      // First, stop receiving
    +      receiverTracker.stop(processAllReceivedData)
     
    -    // Stop the executor for receiving new jobs
    -    logDebug("Stopping job executor")
    -    jobExecutor.shutdown()
    +      // Second, stop generating jobs. If it has to process all received 
data,
    +      // then this will wait for all the processing through JobScheduler 
to be over.
    +      jobGenerator.stop(processAllReceivedData)
     
    -    // Wait for the queued jobs to complete if indicated
    +      // Stop the executor for receiving new jobs
    +      logDebug("Stopping job executor")
    +      jobExecutor.shutdown()
    +
    +      // Wait for the queued jobs to complete if indicated
    +      val terminated = if (processAllReceivedData) {
    +        jobExecutor.awaitTermination(1, TimeUnit.HOURS) // just a very 
large period of time
    +      } else {
    +        jobExecutor.awaitTermination(2, TimeUnit.SECONDS)
    +      }
    +      if (!terminated) {
    +        jobExecutor.shutdownNow()
    +      }
    +      logDebug("Stopped job executor")
    +
    +      // Stop everything else
    +      listenerBus.stop()
    +      ssc.env.actorSystem.stop(eventActor)
    +      eventActor = null
    +    }
    +
    +    shutdownExecutor.shutdown()
    +
    +    // Wait for the JobScheduler shutdown sequence to finish
         val terminated = if (processAllReceivedData) {
    -      jobExecutor.awaitTermination(1, TimeUnit.HOURS)  // just a very 
large period of time
    +      val gracefulTimeout = ssc.conf.getLong(
    +        "spark.streaming.gracefulStopTimeout",
    +        100 * ssc.graph.batchDuration.milliseconds
    --- End diff --
    
    Naah, I thought more about it. This is more or less fine. The default 
should be multiple of the batch duration (so that it scales), but when the app 
sets it up explicitly, it should be something that the app developer more 
intuitively understands. Though I recommend reducing it to 10 * batch duration. 


---
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