spark-reviews mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From pwendell <...@git.apache.org>
Subject [GitHub] spark pull request: [SPARK-1331] Added graceful shutdown to Spark ...
Date Fri, 04 Apr 2014 03:56:49 GMT
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/247#discussion_r11282396
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
---
    @@ -124,84 +143,109 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable
with Logging
           receivingThread
     
           // Call user-defined onStart()
    +      logInfo("Calling onStart")
           onStart()
    +
    +      // Wait until interrupt is called on this thread
    +      while(true) Thread.sleep(100000)
         } catch {
           case ie: InterruptedException =>
    -        logInfo("Receiving thread interrupted")
    -        //println("Receiving thread interrupted")
    +        logInfo("Receiving thread has been interrupted, receiver "  + streamId + " stopped")
           case e: Exception =>
    -        stopOnError(e)
    +        logError("Error receiving data in receiver " + streamId, e)
    +        exceptions += e
    +    }
    +
    +    // Call user-defined onStop()
    +    logInfo("Calling onStop")
    +    try {
    +      onStop()
    +    } catch {
    +      case  e: Exception =>
    +        logError("Error stopping receiver " + streamId, e)
    +        exceptions += e
    +    }
    +
    +    val message = if (exceptions.isEmpty) {
    +      null
    +    } else if (exceptions.size == 1) {
    +      val e = exceptions.head
    +      "Exception in receiver " + streamId + ": " + e.getMessage + "\n" + e.getStackTraceString
    +    } else {
    +      "Multiple exceptions in receiver " + streamId + "(" + exceptions.size + "):\n"
    +        exceptions.zipWithIndex.map {
    +          case (e, i) => "Exception " + i + ": " + e.getMessage + "\n" + e.getStackTraceString
    +        }.mkString("\n")
         }
    +    logInfo("Deregistering receiver " + streamId)
    +    val future = trackerActor.ask(DeregisterReceiver(streamId, message))(askTimeout)
    +    Await.result(future, askTimeout)
    +    logInfo("Deregistered receiver " + streamId)
    +    env.actorSystem.stop(actor)
    +    logInfo("Stopped receiver " + streamId)
       }
     
       /**
    -   * Stops the receiver. First it interrupts the main receiving thread,
    +   * Stop the receiver. First it interrupts the main receiving thread,
        * that is, the thread that called receiver.start(). Then it calls the user-defined
    --- End diff --
    
    It says "Then it calls the user-defined onStop method" but in this change you remove that
call.


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

Mime
View raw message