aboutsummaryrefslogtreecommitdiff
path: root/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
blob: f7f3346f81db58607903a92527816f55ae48a8fe (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
/*
 * 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

import org.scalatest.{FunSuite, BeforeAndAfter}
import org.scalatest.exceptions.TestFailedDueToTimeoutException
import org.scalatest.concurrent.Timeouts
import org.scalatest.time.SpanSugar._
import org.apache.spark.{SparkException, SparkConf, SparkContext}
import org.apache.spark.util.{Utils, MetadataCleaner}
import org.apache.spark.streaming.dstream.DStream

class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts {

  val master = "local[2]"
  val appName = this.getClass.getSimpleName
  val batchDuration = Seconds(1)
  val sparkHome = "someDir"
  val envPair = "key" -> "value"
  val ttl = StreamingContext.DEFAULT_CLEANER_TTL + 100

  var sc: SparkContext = null
  var ssc: StreamingContext = null

  before {
    System.clearProperty("spark.cleaner.ttl")
  }

  after {
    if (ssc != null) {
      ssc.stop()
      ssc = null
    }
    if (sc != null) {
      sc.stop()
      sc = null
    }
  }

  test("from no conf constructor") {
    ssc = new StreamingContext(master, appName, batchDuration)
    assert(ssc.sparkContext.conf.get("spark.master") === master)
    assert(ssc.sparkContext.conf.get("spark.app.name") === appName)
    assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) ===
      StreamingContext.DEFAULT_CLEANER_TTL)
  }

  test("from no conf + spark home") {
    ssc = new StreamingContext(master, appName, batchDuration, sparkHome, Nil)
    assert(ssc.conf.get("spark.home") === sparkHome)
    assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) ===
      StreamingContext.DEFAULT_CLEANER_TTL)
  }

  test("from no conf + spark home + env") {
    ssc = new StreamingContext(master, appName, batchDuration,
      sparkHome, Nil, Map(envPair))
    assert(ssc.conf.getExecutorEnv.exists(_ == envPair))
    assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) ===
      StreamingContext.DEFAULT_CLEANER_TTL)
  }

  test("from conf without ttl set") {
    val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName)
    ssc = new StreamingContext(myConf, batchDuration)
    assert(MetadataCleaner.getDelaySeconds(ssc.conf) ===
      StreamingContext.DEFAULT_CLEANER_TTL)
  }

  test("from conf with ttl set") {
    val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName)
    myConf.set("spark.cleaner.ttl", ttl.toString)
    ssc = new StreamingContext(myConf, batchDuration)
    assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === ttl)
  }

  test("from existing SparkContext without ttl set") {
    sc = new SparkContext(master, appName)
    val exception = intercept[SparkException] {
      ssc = new StreamingContext(sc, batchDuration)
    }
    assert(exception.getMessage.contains("ttl"))
  }

  test("from existing SparkContext with ttl set") {
    val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName)
    myConf.set("spark.cleaner.ttl", ttl.toString)
    ssc = new StreamingContext(myConf, batchDuration)
    assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === ttl)
  }

  test("from checkpoint") {
    val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName)
    myConf.set("spark.cleaner.ttl", ttl.toString)
    val ssc1 = new StreamingContext(myConf, batchDuration)
    val cp = new Checkpoint(ssc1, Time(1000))
    assert(MetadataCleaner.getDelaySeconds(cp.sparkConf) === ttl)
    ssc1.stop()
    val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp))
    assert(MetadataCleaner.getDelaySeconds(newCp.sparkConf) === ttl)
    ssc = new StreamingContext(null, cp, null)
    assert(MetadataCleaner.getDelaySeconds(ssc.conf) === ttl)
  }

  test("start multiple times") {
    ssc = new StreamingContext(master, appName, batchDuration)
    addInputStream(ssc).register

    ssc.start()
    intercept[SparkException] {
      ssc.start()
    }
  }

  test("stop multiple times") {
    ssc = new StreamingContext(master, appName, batchDuration)
    addInputStream(ssc).register
    ssc.start()
    ssc.stop()
    ssc.stop()
    ssc = null
  }

  test("stop only streaming context") {
    ssc = new StreamingContext(master, appName, batchDuration)
    sc = ssc.sparkContext
    addInputStream(ssc).register
    ssc.start()
    ssc.stop(false)
    ssc = null
    assert(sc.makeRDD(1 to 100).collect().size === 100)
    ssc = new StreamingContext(sc, batchDuration)
  }

  test("awaitTermination") {
    ssc = new StreamingContext(master, appName, batchDuration)
    val inputStream = addInputStream(ssc)
    inputStream.map(x => x).register

    // test whether start() blocks indefinitely or not
    failAfter(2000 millis) {
      ssc.start()
    }

    // test whether waitForStop() exits after give amount of time
    failAfter(1000 millis) {
      ssc.awaitTermination(500)
    }

    // test whether waitForStop() does not exit if not time is given
    val exception = intercept[Exception] {
      failAfter(1000 millis) {
        ssc.awaitTermination()
        throw new Exception("Did not wait for stop")
      }
    }
    assert(exception.isInstanceOf[TestFailedDueToTimeoutException], "Did not wait for stop")

    // test whether wait exits if context is stopped
    failAfter(10000 millis) { // 10 seconds because spark takes a long time to shutdown
      new Thread() {
        override def run {
          Thread.sleep(500)
          ssc.stop()
        }
      }.start()
      ssc.awaitTermination()
    }
  }

  test("awaitTermination with error in task") {
    ssc = new StreamingContext(master, appName, batchDuration)
    val inputStream = addInputStream(ssc)
    inputStream.map(x => { throw new TestException("error in map task"); x})
               .foreachRDD(_.count)

    val exception = intercept[Exception] {
      ssc.start()
      ssc.awaitTermination(5000)
    }
    assert(exception.getMessage.contains("map task"), "Expected exception not thrown")
  }

  test("awaitTermination with error in job generation") {
    ssc = new StreamingContext(master, appName, batchDuration)
    val inputStream = addInputStream(ssc)

    inputStream.transform(rdd => { throw new TestException("error in transform"); rdd }).register
    val exception = intercept[TestException] {
      ssc.start()
      ssc.awaitTermination(5000)
    }
    assert(exception.getMessage.contains("transform"), "Expected exception not thrown")
  }

  def addInputStream(s: StreamingContext): DStream[Int] = {
    val input = (1 to 100).map(i => (1 to i))
    val inputStream = new TestInputStream(s, input, 1)
    s.registerInputStream(inputStream)
    inputStream
  }
}

class TestException(msg: String) extends Exception(msg)