@@ -34,65 +34,70 @@ class RateControllerSuite extends TestSuiteBase {
34
34
35
35
test(" rate controller publishes updates" ) {
36
36
val ssc = new StreamingContext (conf, batchDuration)
37
- val dstream = new MockRateLimitDStream (ssc, Seq (Seq (1 )), 1 )
38
- val output = new TestOutputStreamWithPartitions (dstream)
39
- output.register()
40
- runStreams(ssc, 1 , 1 )
41
-
42
- eventually(timeout(2 .seconds)) {
43
- assert(dstream.publishCalls === 1 )
37
+ withStreamingContext(ssc) { ssc =>
38
+ val dstream = new MockRateLimitDStream (ssc, Seq (Seq (1 )), 1 )
39
+ val output = new TestOutputStreamWithPartitions (dstream)
40
+ output.register()
41
+ runStreams(ssc, 1 , 1 )
42
+
43
+ eventually(timeout(2 .seconds)) {
44
+ assert(dstream.publishCalls === 1 )
45
+ }
44
46
}
45
47
}
46
48
47
49
test(" receiver rate controller updates reach receivers" ) {
48
50
val ssc = new StreamingContext (conf, batchDuration)
51
+ withStreamingContext(ssc) { ssc =>
52
+ val dstream = new RateLimitInputDStream (ssc) {
53
+ override val rateController =
54
+ Some (new ReceiverRateController (id, new ConstantEstimator (200.0 )))
55
+ }
56
+ SingletonDummyReceiver .reset()
49
57
50
- val dstream = new RateLimitInputDStream (ssc) {
51
- override val rateController =
52
- Some (new ReceiverRateController (id, new ConstantEstimator (200.0 )))
53
- }
54
- SingletonDummyReceiver .reset()
55
-
56
- val output = new TestOutputStreamWithPartitions (dstream)
57
- output.register()
58
- runStreams(ssc, 2 , 2 )
58
+ val output = new TestOutputStreamWithPartitions (dstream)
59
+ output.register()
60
+ runStreams(ssc, 2 , 2 )
59
61
60
- eventually(timeout(5 .seconds)) {
61
- assert(dstream.getCurrentRateLimit === Some (200 ))
62
+ eventually(timeout(5 .seconds)) {
63
+ assert(dstream.getCurrentRateLimit === Some (200 ))
64
+ }
62
65
}
63
66
}
64
67
65
68
test(" multiple rate controller updates reach receivers" ) {
66
69
val ssc = new StreamingContext (conf, batchDuration)
67
- val rates = Seq (100L , 200L , 300L )
70
+ withStreamingContext(ssc) { ssc =>
71
+ val rates = Seq (100L , 200L , 300L )
68
72
69
- val dstream = new RateLimitInputDStream (ssc) {
70
- override val rateController =
71
- Some (new ReceiverRateController (id, new ConstantEstimator (rates.map(_.toDouble): _* )))
72
- }
73
- SingletonDummyReceiver .reset()
74
-
75
- val output = new TestOutputStreamWithPartitions (dstream)
76
- output.register()
77
-
78
- val observedRates = mutable.HashSet .empty[Long ]
79
-
80
- @ volatile var done = false
81
- runInBackground {
82
- while (! done) {
83
- try {
84
- dstream.getCurrentRateLimit.foreach(observedRates += _)
85
- } catch {
86
- case NonFatal (_) => () // don't stop if the executor wasn't installed yet
73
+ val dstream = new RateLimitInputDStream (ssc) {
74
+ override val rateController =
75
+ Some (new ReceiverRateController (id, new ConstantEstimator (rates.map(_.toDouble): _* )))
76
+ }
77
+ SingletonDummyReceiver .reset()
78
+
79
+ val output = new TestOutputStreamWithPartitions (dstream)
80
+ output.register()
81
+
82
+ val observedRates = mutable.HashSet .empty[Long ]
83
+
84
+ @ volatile var done = false
85
+ runInBackground {
86
+ while (! done) {
87
+ try {
88
+ dstream.getCurrentRateLimit.foreach(observedRates += _)
89
+ } catch {
90
+ case NonFatal (_) => () // don't stop if the executor wasn't installed yet
91
+ }
92
+ Thread .sleep(20 )
87
93
}
88
- Thread .sleep(20 )
89
94
}
90
- }
91
- runStreams(ssc, 4 , 4 )
92
- done = true
95
+ runStreams(ssc, 4 , 4 )
96
+ done = true
93
97
94
- // Long.MaxValue (essentially, no rate limit) is the initial rate limit for any Receiver
95
- observedRates should contain theSameElementsAs (rates :+ Long .MaxValue )
98
+ // Long.MaxValue (essentially, no rate limit) is the initial rate limit for any Receiver
99
+ observedRates should contain theSameElementsAs (rates :+ Long .MaxValue )
100
+ }
96
101
}
97
102
98
103
private def runInBackground (f : => Unit ): Unit = {
0 commit comments