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

    https://github.com/apache/spark/pull/7913#discussion_r36148595
  
    --- Diff: 
streaming/src/test/scala/org/apache/spark/streaming/scheduler/RateControllerSuite.scala
 ---
    @@ -32,72 +29,66 @@ class RateControllerSuite extends TestSuiteBase {
     
       override def useManualClock: Boolean = false
     
    -  test("rate controller publishes updates") {
    -    val ssc = new StreamingContext(conf, batchDuration)
    -    withStreamingContext(ssc) { ssc =>
    -      val dstream = new RateLimitInputDStream(ssc)
    -      dstream.register()
    -      ssc.start()
    +  override def batchDuration = Milliseconds(50)
     
    -      eventually(timeout(10.seconds)) {
    -        assert(dstream.publishCalls > 0)
    -      }
    -    }
    -  }
    -
    -  test("publish rates reach receivers") {
    +  test("RateController - rate controller publishes updates after batches 
complete") {
         val ssc = new StreamingContext(conf, batchDuration)
         withStreamingContext(ssc) { ssc =>
    -      val dstream = new RateLimitInputDStream(ssc) {
    -        override val rateController =
    -          Some(new ReceiverRateController(id, new 
ConstantEstimator(200.0)))
    -      }
    +      val dstream = new RateTestInputDStream(ssc)
           dstream.register()
    -      SingletonTestRateReceiver.reset()
           ssc.start()
     
           eventually(timeout(10.seconds)) {
    -        assert(dstream.getCurrentRateLimit === Some(200))
    +        assert(dstream.publishedRates > 0)
           }
         }
       }
     
    -  test("multiple publish rates reach receivers") {
    +  test("ReceiverRateController - published rates reach receivers") {
         val ssc = new StreamingContext(conf, batchDuration)
         withStreamingContext(ssc) { ssc =>
    -      val rates = Seq(100L, 200L, 300L)
    +      val estimator = new ConstantEstimator(100)
     
    -      val dstream = new RateLimitInputDStream(ssc) {
    +      val dstream = new RateTestInputDStream(ssc) {
             override val rateController =
    -          Some(new ReceiverRateController(id, new 
ConstantEstimator(rates.map(_.toDouble): _*)))
    +          Some(new ReceiverRateController(id, estimator))
           }
    -      SingletonTestRateReceiver.reset()
           dstream.register()
     
           val observedRates = mutable.HashSet.empty[Long]
           ssc.start()
     
    -      eventually(timeout(20.seconds)) {
    -        dstream.getCurrentRateLimit.foreach(observedRates += _)
    -        // Long.MaxValue (essentially, no rate limit) is the initial rate 
limit for any Receiver
    -        observedRates should contain theSameElementsAs (rates :+ 
Long.MaxValue)
    +      // Wait for receiver to start
    +      eventually(timeout(5.seconds)) {
    +        RateTestReceiver.getActive().nonEmpty
    +      }
    +
    +      // Update rate in the estimator and verify whether the rate was 
published to the receiver
    +      def updateRateAndVerify(rate: Long): Unit = {
    --- End diff --
    
    @dragos Note this test. This makes it more deterministic and less flaky.


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to