You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by HeartSaVioR <gi...@git.apache.org> on 2018/06/23 02:51:21 UTC

[GitHub] spark pull request #21617: [SPARK-24634][SS] Add a new metric regarding numb...

GitHub user HeartSaVioR opened a pull request:

    https://github.com/apache/spark/pull/21617

    [SPARK-24634][SS] Add a new metric regarding number of rows later than watermark

    ## What changes were proposed in this pull request?
    
    This adds a new metric to count the number of rows arrived later than watermark. 
    
    The metric will be exposed to two places: 
    1. streaming query listener -`numLateInputRows` in `stateOperators`
    2. SQL tab in UI - `number of rows which are later than watermark` in state operator exec
    
    Please refer https://issues.apache.org/jira/browse/SPARK-24634 to see rationalization of the issue.
    
    ## How was this patch tested?
    
    Modified existing UTs.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/HeartSaVioR/spark SPARK-24634

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/21617.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #21617
    
----
commit ff1b89553acc7ea3a19b586457dd295255047377
Author: Jungtaek Lim <ka...@...>
Date:   2018-06-23T02:34:16Z

    SPARK-24634 Add a new metric regarding number of rows later than watermark
    
    * This adds a new metric to count the number of rows arrived later than watermark

----


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    **[Test build #92276 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92276/testReport)** for PR 21617 at commit [`ff1b895`](https://github.com/apache/spark/commit/ff1b89553acc7ea3a19b586457dd295255047377).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    **[Test build #92241 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92241/testReport)** for PR 21617 at commit [`ff1b895`](https://github.com/apache/spark/commit/ff1b89553acc7ea3a19b586457dd295255047377).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Retest this please.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    **[Test build #92239 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92239/testReport)** for PR 21617 at commit [`ff1b895`](https://github.com/apache/spark/commit/ff1b89553acc7ea3a19b586457dd295255047377).


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Abandoning the patch. While I think the JIRA issue is still valid, looks like we should address watermark issue to have correct number of late events. Thanks for reviewing @jose-torres @arunmahadevan .


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    **[Test build #92239 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92239/testReport)** for PR 21617 at commit [`ff1b895`](https://github.com/apache/spark/commit/ff1b89553acc7ea3a19b586457dd295255047377).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    @jose-torres 
    Yes, you're right. They would be the rows which applies other transformation and filtering, not origin input rows. I just haven't find proper alternative word than "input row" since in point of state operator's view, they're input rows.
    
    Btw, as I described in the JIRA, my final goal is pushing late events to side-output (as Beam and Flink represented) but being stuck with couple of concerns (Please correct me anytime if I'm missing here):
    
    1. Which events to push?
    
    Query can have couple of transformations before reaching stateful operator and being filtered out due to watermark. This is not ideal and I guess that's you said as "aren't necessarily the input rows".  
    
    Ideally we would be better to provide origin input rows, rather than transformed one, but then we should put major restriction on watermark: `Filter with watermark` should be applied in data reader (or having a filter just after data reader), which means input rows itself should have timestamp field. 
    
    We can't apply transformation(s) to populate/manipulate timestamp field, and timestamp field **must not** be modified during transformations. For example, Flink provides timestamp assigner to extract timestamp value from input stream, and reserved field name `rowtime` is used for timestamp field.
    
    2. Does the nature of RDD support multiple outputs?
    
    I have been struggling on this, but as far as my understanding is correct, RDD itself doesn't support multiple outputs, as the nature of RDD. For me, this looks like major difference between pull model vs push model, cause in push model which other streaming frameworks use, defining another output stream is really straightforward, just like adding remote listener, whereas I'm not sure how it can be clearly defined in pull model. I also googled about multiple outputs on RDD (as someone could have struggled before) but no luck.
    
    The alternative approaches I can imagine are kinds of workarounds: RPC, listener bus, callback function. Nothing can define another stream within current DAG, and I'm also not sure that we can create DataFrame based on the data and let end users compose another query. 
    
    It would be really helpful if you can think about better alternatives and share.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by jose-torres <gi...@git.apache.org>.
Github user jose-torres commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Well, "clear" is relative. Since we're trying to provide functionality in the Dataframe API, it's perfectly alright for the RDD graph to end up looking a bit weird. It seems feasible to do something like:
    
    * Have a stream reader RDD write side output to some special shuffle partition (set of partitions?) which the main query knows not to read.
    * Have a stream writer RDD with two heterogeneous sets of partitions: one to write the main query to the sink, and another to apply the specified action to the side output.
    
    I agree that watermarks should be applied immediately after the data reader - other streaming systems generally require this, and Spark does not seem to be getting any benefits from having a more general watermark concept. I haven't had time to push for this change, but I think it's known that the current Spark watermark model is flawed - I'd support fixing it for sure.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92241/
    Test FAILed.


---

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


[GitHub] spark pull request #21617: [SPARK-24634][SS] Add a new metric regarding numb...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21617#discussion_r197986093
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala ---
    @@ -48,12 +49,13 @@ class StateOperatorProgress private[sql](
       def prettyJson: String = pretty(render(jsonValue))
     
       private[sql] def copy(newNumRowsUpdated: Long): StateOperatorProgress =
    -    new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes)
    +    new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes, numLateInputRows)
     
       private[sql] def jsonValue: JValue = {
         ("numRowsTotal" -> JInt(numRowsTotal)) ~
         ("numRowsUpdated" -> JInt(numRowsUpdated)) ~
    -    ("memoryUsedBytes" -> JInt(memoryUsedBytes))
    +    ("memoryUsedBytes" -> JInt(memoryUsedBytes)) ~
    +    ("numLateInputRows" -> JInt(numLateInputRows))
    --- End diff --
    
    @arunmahadevan Ah yes got it. If we would want to have accurate number we need to filter out late events from the first time anyway. I guess we may need to defer addressing this until we change the behavior.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92276/
    Test PASSed.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by jose-torres <gi...@git.apache.org>.
Github user jose-torres commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    LGTM, but note that the rows being counted here are the rows persisted into the state store, which aren't necessarily the input rows. So the side-channel described in the JIRA would be orthogonal to this.


---

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


[GitHub] spark pull request #21617: [SPARK-24634][SS] Add a new metric regarding numb...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21617#discussion_r197984227
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala ---
    @@ -48,12 +49,13 @@ class StateOperatorProgress private[sql](
       def prettyJson: String = pretty(render(jsonValue))
     
       private[sql] def copy(newNumRowsUpdated: Long): StateOperatorProgress =
    -    new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes)
    +    new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes, numLateInputRows)
     
       private[sql] def jsonValue: JValue = {
         ("numRowsTotal" -> JInt(numRowsTotal)) ~
         ("numRowsUpdated" -> JInt(numRowsUpdated)) ~
    -    ("memoryUsedBytes" -> JInt(memoryUsedBytes))
    +    ("memoryUsedBytes" -> JInt(memoryUsedBytes)) ~
    +    ("numLateInputRows" -> JInt(numLateInputRows))
    --- End diff --
    
    What I meant was, if the input to the state operator is the result of the aggregate, then we would not be counting the actual input rows to the group by. There would be max one row per key, so would give the impression that there are not as many late events but in reality it may be more.
    
     If this is not the case then I am fine.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    **[Test build #92276 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92276/testReport)** for PR 21617 at commit [`ff1b895`](https://github.com/apache/spark/commit/ff1b89553acc7ea3a19b586457dd295255047377).


---

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


[GitHub] spark pull request #21617: [SPARK-24634][SS] Add a new metric regarding numb...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR closed the pull request at:

    https://github.com/apache/spark/pull/21617


---

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


[GitHub] spark pull request #21617: [SPARK-24634][SS] Add a new metric regarding numb...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21617#discussion_r197981651
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala ---
    @@ -48,12 +49,13 @@ class StateOperatorProgress private[sql](
       def prettyJson: String = pretty(render(jsonValue))
     
       private[sql] def copy(newNumRowsUpdated: Long): StateOperatorProgress =
    -    new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes)
    +    new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes, numLateInputRows)
     
       private[sql] def jsonValue: JValue = {
         ("numRowsTotal" -> JInt(numRowsTotal)) ~
         ("numRowsUpdated" -> JInt(numRowsUpdated)) ~
    -    ("memoryUsedBytes" -> JInt(memoryUsedBytes))
    +    ("memoryUsedBytes" -> JInt(memoryUsedBytes)) ~
    +    ("numLateInputRows" -> JInt(numLateInputRows))
    --- End diff --
    
    @arunmahadevan 
    
    > Here you are measuring the number of "keys" filtered out of the state store since they have crossed the late threshold correct ?
    
    No, it is based on "input" rows which are filtered out due to watermark threshold. Note that the meaning of "input" is relative, cause it doesn't represent for input rows in overall query, but represents for input rows in state operator.
    
    > Its better if we could rather expose the actual number of events that were late.
    
    I guess the comment is based on missing thing, but I would think that it would be correct that we filtered out late events from the first phase of query (not from state operator) so that we can get correct count of late events. For now filters affect the count.


---

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


[GitHub] spark pull request #21617: [SPARK-24634][SS] Add a new metric regarding numb...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21617#discussion_r197980605
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala ---
    @@ -48,12 +49,13 @@ class StateOperatorProgress private[sql](
       def prettyJson: String = pretty(render(jsonValue))
     
       private[sql] def copy(newNumRowsUpdated: Long): StateOperatorProgress =
    -    new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes)
    +    new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes, numLateInputRows)
     
       private[sql] def jsonValue: JValue = {
         ("numRowsTotal" -> JInt(numRowsTotal)) ~
         ("numRowsUpdated" -> JInt(numRowsUpdated)) ~
    -    ("memoryUsedBytes" -> JInt(memoryUsedBytes))
    +    ("memoryUsedBytes" -> JInt(memoryUsedBytes)) ~
    +    ("numLateInputRows" -> JInt(numLateInputRows))
    --- End diff --
    
    Here you are measuring the number of "keys" filtered out of the state store since they have crossed the late threshold correct ? It may be better to rename this metrics here and at other places to "number of evicted rows". Its better if we could rather expose the actual number of events that were late.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    add to whitelist


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    cc. @tdas @zsxwing @jose-torres @jerryshao @arunmahadevan @HyukjinKwon 


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92239/
    Test PASSed.


---

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


[GitHub] spark issue #21617: [SPARK-24634][SS] Add a new metric regarding number of r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21617
  
    **[Test build #92241 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92241/testReport)** for PR 21617 at commit [`ff1b895`](https://github.com/apache/spark/commit/ff1b89553acc7ea3a19b586457dd295255047377).


---

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