You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Shixiong Zhu (JIRA)" <ji...@apache.org> on 2015/09/18 15:00:05 UTC

[jira] [Comment Edited] (SPARK-10692) Failed batches are never reported through the StreamingListener interface

    [ https://issues.apache.org/jira/browse/SPARK-10692?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14852736#comment-14852736 ] 

Shixiong Zhu edited comment on SPARK-10692 at 9/18/15 12:59 PM:
----------------------------------------------------------------

I noticed it when I was developing the Streaming UI. Actually, I think this may not be a bug.

{{reportError}} will make {{ssc.awaitTermination()}} throw an Exception. Then the user application usually will call `ssc.stop()`. To prevent the application from exiting, the user has to write something like
{code}
    while(true) {
      try {
        ssc.awaitTermination()
      } catch {
        case e: Exception => 
          // do something
      }
    }
{code}
However, since {{awaitTermination}} implies StreamingContext is terminated, I don't think the user will write such codes. Since StreamingContext cannot recovery from such exception (we cannot skip this failure batch because it means data lost), the only way is exiting.

Therefore I suppose this issue only affects the users that use `StreamingListener`. It's not an issue for the UI because StreamingContext will exit at once along with the Streaming UI.


was (Author: zsxwing):
I noticed it when I was developing the Streaming UI. Actually, I think this may not a bug.

{{reportError}} will make {{ssc.awaitTermination()}} throw an Exception. Then the user application usually will call `ssc.stop()`. To prevent the application from exiting, the user has to write something like
{code}
    while(true) {
      try {
        ssc.awaitTermination()
      } catch {
        case e: Exception => 
          // do something
      }
    }
{code}
However, since {{awaitTermination}} implies StreamingContext is terminated, I don't think the user will write such codes. Since StreamingContext cannot recovery from such exception (we cannot skip this failure batch because it means data lost), the only way is exiting.

Therefore I suppose this issue only affects the users that use `StreamingListener`. It's not an issue for the UI because StreamingContext will exit at once along with the Streaming UI.

> Failed batches are never reported through the StreamingListener interface
> -------------------------------------------------------------------------
>
>                 Key: SPARK-10692
>                 URL: https://issues.apache.org/jira/browse/SPARK-10692
>             Project: Spark
>          Issue Type: Bug
>          Components: Streaming
>    Affects Versions: 1.3.1, 1.4.1, 1.5.0
>            Reporter: Tathagata Das
>            Assignee: Shixiong Zhu
>            Priority: Blocker
>
> If an output operation fails, then corresponding batch is never marked as completed, as the data structure are not updated properly.
> https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala#L183



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org