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

    https://github.com/apache/spark/pull/16108#discussion_r90565023
  
    --- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala
 ---
    @@ -191,6 +192,50 @@ class StreamingQueryListenerSuite extends StreamTest 
with BeforeAndAfter {
         assert(queryQueryTerminated.exception === newQueryTerminated.exception)
       }
     
    +  test("only one progress event per interval when no data") {
    +    // This test will start a query but not push any data, and then check 
if we push too many events
    +    withSQLConf(SQLConf.STREAMING_NO_DATA_EVENT_INTERVAL.key -> "100ms") {
    +      @volatile var progressEventCount = 0
    +
    +      val listener = new StreamingQueryListener {
    +        override def onQueryStarted(event: QueryStartedEvent): Unit = {}
    +
    +        override def onQueryProgress(event: QueryProgressEvent): Unit = {
    +          progressEventCount += 1
    +        }
    +
    +        override def onQueryTerminated(event: QueryTerminatedEvent): Unit 
= {}
    +      }
    +      spark.streams.addListener(listener)
    +      try {
    +        val clock = new StreamManualClock()
    +        val actions = mutable.ArrayBuffer[StreamAction]()
    +        actions += StartStream(trigger = ProcessingTime(10), triggerClock 
= clock)
    +        actions += AssertOnQuery { _ =>
    +          // It should report at least one progress
    +          eventually(timeout(streamingTimeout)) {
    +            assert(progressEventCount > 0)
    +          }
    +          true
    +        }
    +        for (_ <- 1 to 100) {
    +          actions += AdvanceManualClock(10)
    +          actions += AssertOnQuery { _ =>
    +            // Sleep so that if the config `noDataEventInterval` doesn't 
work, it has enough time
    +            // to report too many events.
    +            Thread.sleep(10)
    +            true
    +          }
    +        }
    +        testStream(MemoryStream[Int].toDS)(actions: _*)
    +        // 11 is the max value of the possible numbers of events.
    --- End diff --
    
    I think this test looks a little complicated. Also you are assuming the 
fact the when there is no data, it will sleep for 10 ms real time. why not 
something like this. 
    ```
    val input = new MemoryStream() {
      @volatile var numTriggers = 0
      override def getOffset(): Offset = { numTriggers +=1 ; super.getOffset() 
} 
    }
    ...
    // add listener with progressEventCount and flag isTerminationEventPosted
    ...
    testStream() {   // on real clock
       AddData(...)
       AssertOnQuery {  q => 
          eventually (timeout(streamingTimeout)) {
              assert(input.numTriggers > 100)   // at least 100 triggers have 
occurred
          }
       }, 
       StopStream, 
       AssertOnQuery { q => 
         eventually(timeout(streamingTimeout) {
           assert(isTerminationEventPosted)   // event queue has drained
           assert(numProgressEvent >=1 && numProgressEvent <= 100)  // no more 
than 10 events. 
        } 
       }
    }
    
    ```


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to