spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Fei Shao (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (SPARK-21206) the window slice of Dstream is wrong
Date Sat, 01 Jul 2017 09:34:00 GMT

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

Fei Shao edited comment on SPARK-21206 at 7/1/17 9:33 AM:
----------------------------------------------------------

[~srowen]

This issue is about the xpressions about oldRDDs and newRDDs.
Here is the case I want to express:

!screenshot-2.png!


I think the code is:
=============
    // Get the RDDs of the reduced values in "old time steps"
    val oldRDDs =
      if (windowDuration < slideDuration ) {
        reducedStream.slice(previousWindow.beginTime, previousWindow.endTime)
      }
      else {
        reducedStream.slice(previousWindow.beginTime,
          currentWindow.beginTime - parent.slideDuration)
      }

    logDebug("# old RDDs = " + oldRDDs.size)

    // Get the RDDs of the reduced values in "new time steps"
    val newRDDs =
      if (windowDuration < slideDuration ) {
        reducedStream.slice(currentWindow.beginTime, currentWindow.endTime)
      }
      else {
        reducedStream.slice(previousWindow.endTime + parent.slideDuration, currentWindow.endTime)
      }

    logDebug("# new RDDs = " + newRDDs.size)
==============================

I sent an email named "the function of countByValueAndWindow and foreachRDD in DStream, would
you like help me understand it please?".
It is related to this issues.



was (Author: robin shao):
[~srowen]

Here is the case I want to express:

!screenshot-2.png!

I sent an email named "the function of countByValueAndWindow and foreachRDD in DStream, would
you like help me understand it please?".
It is related to this issues.


> the window slice of Dstream is wrong
> ------------------------------------
>
>                 Key: SPARK-21206
>                 URL: https://issues.apache.org/jira/browse/SPARK-21206
>             Project: Spark
>          Issue Type: Bug
>          Components: DStreams
>    Affects Versions: 2.1.0
>            Reporter: Fei Shao
>         Attachments: screenshot-1.png, screenshot-2.png
>
>
> the code is :
>     val conf = new SparkConf().setAppName("testDstream").setMaster("local[4]")
>     val ssc = new StreamingContext(conf, Seconds(1))
>     ssc.checkpoint( "path")
>     val lines = ssc.socketTextStream("IP", PORT)
>     lines.countByValueAndWindow( Seconds(2), Seconds(8)).foreachRDD( s => {
>       println( "RDD ID IS : " + s.id)
>       s.foreach( e => println("data is " + e._1 + " :" + e._2))
>       println()
>     })
> The result is wrong. 
> I checked the log, it showed:
> 17/06/25 17:31:26 DEBUG ReducedWindowedDStream: Time 1498383086000 ms is valid
> 17/06/25 17:31:26 DEBUG ReducedWindowedDStream: Window time = 2000 ms
> 17/06/25 17:31:26 DEBUG ReducedWindowedDStream: Slide time = 8000 ms
> 17/06/25 17:31:26 DEBUG ReducedWindowedDStream: Zero time = 1498383078000 ms
> 17/06/25 17:31:26 DEBUG ReducedWindowedDStream: Current window = [1498383085000 ms, 1498383086000
ms]
> 17/06/25 17:31:26 DEBUG ReducedWindowedDStream: Previous window = [1498383077000 ms,
1498383078000 ms]
> 17/06/25 17:31:26 INFO ShuffledDStream: Slicing from 1498383077000 ms to 1498383084000
ms (aligned to 1498383077000 ms and 1498383084000 ms)
> 17/06/25 17:31:26 INFO ShuffledDStream: Time 1498383078000 ms is invalid as zeroTime
is 1498383078000 ms , slideDuration is 1000 ms and difference is 0 ms
> 17/06/25 17:31:26 DEBUG ShuffledDStream: Time 1498383079000 ms is valid
> 17/06/25 17:31:26 DEBUG MappedDStream: Time 1498383079000 ms is valid
> the slice time is wrong.
> [BTW]: Team members,
> If it was a bug, please don't fix it.I try to fix it myself.Thanks:)



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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


Mime
View raw message