aboutsummaryrefslogtreecommitdiff
path: root/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala
blob: df122ac090c3e2b3ac902b3027a4e707adafaf92 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.streaming.scheduler

import scala.collection.mutable.ArrayBuffer

import org.scalatest.concurrent.Eventually._
import org.scalatest.time.SpanSugar._

import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart, TaskLocality}
import org.apache.spark.scheduler.TaskLocality.TaskLocality
import org.apache.spark.storage.{StorageLevel, StreamBlockId}
import org.apache.spark.streaming._
import org.apache.spark.streaming.dstream.{ConstantInputDStream, ReceiverInputDStream}
import org.apache.spark.streaming.receiver._

/** Testsuite for receiver scheduling */
class ReceiverTrackerSuite extends TestSuiteBase {

  test("send rate update to receivers") {
    withStreamingContext(new StreamingContext(conf, Milliseconds(100))) { ssc =>
      val newRateLimit = 100L
      val inputDStream = new RateTestInputDStream(ssc)
      val tracker = new ReceiverTracker(ssc)
      tracker.start()
      try {
        // we wait until the Receiver has registered with the tracker,
        // otherwise our rate update is lost
        eventually(timeout(5 seconds)) {
          assert(RateTestReceiver.getActive().nonEmpty)
        }


        // Verify that the rate of the block generator in the receiver get updated
        val activeReceiver = RateTestReceiver.getActive().get
        tracker.sendRateUpdate(inputDStream.id, newRateLimit)
        eventually(timeout(5 seconds)) {
          assert(activeReceiver.getDefaultBlockGeneratorRateLimit() === newRateLimit,
            "default block generator did not receive rate update")
          assert(activeReceiver.getCustomBlockGeneratorRateLimit() === newRateLimit,
            "other block generator did not receive rate update")
        }
      } finally {
        tracker.stop(false)
      }
    }
  }

  test("should restart receiver after stopping it") {
    withStreamingContext(new StreamingContext(conf, Milliseconds(100))) { ssc =>
      @volatile var startTimes = 0
      ssc.addStreamingListener(new StreamingListener {
        override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted): Unit = {
          startTimes += 1
        }
      })
      val input = ssc.receiverStream(new StoppableReceiver)
      val output = new TestOutputStream(input)
      output.register()
      ssc.start()
      StoppableReceiver.shouldStop = true
      eventually(timeout(10 seconds), interval(10 millis)) {
        // The receiver is stopped once, so if it's restarted, it should be started twice.
        assert(startTimes === 2)
      }
    }
  }

  test("SPARK-11063: TaskSetManager should use Receiver RDD's preferredLocations") {
    // Use ManualClock to prevent from starting batches so that we can make sure the only task is
    // for starting the Receiver
    val _conf = conf.clone.set("spark.streaming.clock", "org.apache.spark.util.ManualClock")
    withStreamingContext(new StreamingContext(_conf, Milliseconds(100))) { ssc =>
      @volatile var receiverTaskLocality: TaskLocality = null
      ssc.sparkContext.addSparkListener(new SparkListener {
        override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = {
          receiverTaskLocality = taskStart.taskInfo.taskLocality
        }
      })
      val input = ssc.receiverStream(new TestReceiver)
      val output = new TestOutputStream(input)
      output.register()
      ssc.start()
      eventually(timeout(10 seconds), interval(10 millis)) {
        // If preferredLocations is set correctly, receiverTaskLocality should be PROCESS_LOCAL
        assert(receiverTaskLocality === TaskLocality.PROCESS_LOCAL)
      }
    }
  }

  test("get allocated executors") {
    // Test get allocated executors when 1 receiver is registered
    withStreamingContext(new StreamingContext(conf, Milliseconds(100))) { ssc =>
      val input = ssc.receiverStream(new TestReceiver)
      val output = new TestOutputStream(input)
      output.register()
      ssc.start()
      assert(ssc.scheduler.receiverTracker.allocatedExecutors().size === 1)
    }

    // Test get allocated executors when there's no receiver registered
    withStreamingContext(new StreamingContext(conf, Milliseconds(100))) { ssc =>
      val rdd = ssc.sc.parallelize(1 to 10)
      val input = new ConstantInputDStream(ssc, rdd)
      val output = new TestOutputStream(input)
      output.register()
      ssc.start()
      assert(ssc.scheduler.receiverTracker.allocatedExecutors() === Map.empty)
    }
  }
}

/** An input DStream with for testing rate controlling */
private[streaming] class RateTestInputDStream(_ssc: StreamingContext)
  extends ReceiverInputDStream[Int](_ssc) {

  override def getReceiver(): Receiver[Int] = new RateTestReceiver(id)

  @volatile
  var publishedRates = 0

  override val rateController: Option[RateController] = {
    Some(new RateController(id, new ConstantEstimator(100)) {
      override def publish(rate: Long): Unit = {
        publishedRates += 1
      }
    })
  }
}

/** A receiver implementation for testing rate controlling */
private[streaming] class RateTestReceiver(receiverId: Int, host: Option[String] = None)
  extends Receiver[Int](StorageLevel.MEMORY_ONLY) {

  private lazy val customBlockGenerator = supervisor.createBlockGenerator(
    new BlockGeneratorListener {
      override def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]): Unit = {}
      override def onError(message: String, throwable: Throwable): Unit = {}
      override def onGenerateBlock(blockId: StreamBlockId): Unit = {}
      override def onAddData(data: Any, metadata: Any): Unit = {}
    }
  )

  setReceiverId(receiverId)

  override def onStart(): Unit = {
    customBlockGenerator
    RateTestReceiver.registerReceiver(this)
  }

  override def onStop(): Unit = {
    RateTestReceiver.deregisterReceiver()
  }

  override def preferredLocation: Option[String] = host

  def getDefaultBlockGeneratorRateLimit(): Long = {
    supervisor.getCurrentRateLimit
  }

  def getCustomBlockGeneratorRateLimit(): Long = {
    customBlockGenerator.getCurrentLimit
  }
}

/**
 * A helper object to RateTestReceiver that give access to the currently active RateTestReceiver
 * instance.
 */
private[streaming] object RateTestReceiver {
  @volatile private var activeReceiver: RateTestReceiver = null

  def registerReceiver(receiver: RateTestReceiver): Unit = {
    activeReceiver = receiver
  }

  def deregisterReceiver(): Unit = {
    activeReceiver = null
  }

  def getActive(): Option[RateTestReceiver] = Option(activeReceiver)
}

/**
 * A custom receiver that could be stopped via StoppableReceiver.shouldStop
 */
class StoppableReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) {

  var receivingThreadOption: Option[Thread] = None

  def onStart() {
    val thread = new Thread() {
      override def run() {
        while (!StoppableReceiver.shouldStop) {
          Thread.sleep(10)
        }
        StoppableReceiver.this.stop("stop")
      }
    }
    thread.start()
  }

  def onStop() {
    StoppableReceiver.shouldStop = true
    receivingThreadOption.foreach(_.join())
    // Reset it so as to restart it
    StoppableReceiver.shouldStop = false
  }
}

object StoppableReceiver {
  @volatile var shouldStop = false
}