aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala
blob: 3d69c8a18711bdf7fbf598ebe54557e4e92c8ce8 (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
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
/*
 * 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.sql.streaming

import scala.concurrent.Future
import scala.util.Random
import scala.util.control.NonFatal

import org.scalatest.BeforeAndAfter
import org.scalatest.concurrent.Eventually._
import org.scalatest.concurrent.PatienceConfiguration.Timeout
import org.scalatest.time.Span
import org.scalatest.time.SpanSugar._

import org.apache.spark.SparkException
import org.apache.spark.sql.{ContinuousQuery, Dataset, StreamTest}
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.util.Utils

class ContinuousQueryManagerSuite extends StreamTest with SharedSQLContext with BeforeAndAfter {

  import AwaitTerminationTester._
  import testImplicits._

  override val streamingTimeout = 20.seconds

  before {
    assert(sqlContext.streams.active.isEmpty)
    sqlContext.streams.resetTerminated()
  }

  after {
    assert(sqlContext.streams.active.isEmpty)
    sqlContext.streams.resetTerminated()
  }

  testQuietly("listing") {
    val (m1, ds1) = makeDataset
    val (m2, ds2) = makeDataset
    val (m3, ds3) = makeDataset

    withQueriesOn(ds1, ds2, ds3) { queries =>
      require(queries.size === 3)
      assert(sqlContext.streams.active.toSet === queries.toSet)
      val (q1, q2, q3) = (queries(0), queries(1), queries(2))

      assert(sqlContext.streams.get(q1.name).eq(q1))
      assert(sqlContext.streams.get(q2.name).eq(q2))
      assert(sqlContext.streams.get(q3.name).eq(q3))
      intercept[IllegalArgumentException] {
        sqlContext.streams.get("non-existent-name")
      }

      q1.stop()

      assert(sqlContext.streams.active.toSet === Set(q2, q3))
      val ex1 = withClue("no error while getting non-active query") {
        intercept[IllegalArgumentException] {
          sqlContext.streams.get(q1.name)
        }
      }
      assert(ex1.getMessage.contains(q1.name), "error does not contain name of query to be fetched")
      assert(sqlContext.streams.get(q2.name).eq(q2))

      m2.addData(0)   // q2 should terminate with error

      eventually(Timeout(streamingTimeout)) {
        require(!q2.isActive)
        require(q2.exception.isDefined)
      }
      withClue("no error while getting non-active query") {
        intercept[IllegalArgumentException] {
          sqlContext.streams.get(q2.name).eq(q2)
        }
      }

      assert(sqlContext.streams.active.toSet === Set(q3))
    }
  }

  testQuietly("awaitAnyTermination without timeout and resetTerminated") {
    val datasets = Seq.fill(5)(makeDataset._2)
    withQueriesOn(datasets: _*) { queries =>
      require(queries.size === datasets.size)
      assert(sqlContext.streams.active.toSet === queries.toSet)

      // awaitAnyTermination should be blocking
      testAwaitAnyTermination(ExpectBlocked)

      // Stop a query asynchronously and see if it is reported through awaitAnyTermination
      val q1 = stopRandomQueryAsync(stopAfter = 100 milliseconds, withError = false)
      testAwaitAnyTermination(ExpectNotBlocked)
      require(!q1.isActive) // should be inactive by the time the prev awaitAnyTerm returned

      // All subsequent calls to awaitAnyTermination should be non-blocking
      testAwaitAnyTermination(ExpectNotBlocked)

      // Resetting termination should make awaitAnyTermination() blocking again
      sqlContext.streams.resetTerminated()
      testAwaitAnyTermination(ExpectBlocked)

      // Terminate a query asynchronously with exception and see awaitAnyTermination throws
      // the exception
      val q2 = stopRandomQueryAsync(100 milliseconds, withError = true)
      testAwaitAnyTermination(ExpectException[SparkException])
      require(!q2.isActive) // should be inactive by the time the prev awaitAnyTerm returned

      // All subsequent calls to awaitAnyTermination should throw the exception
      testAwaitAnyTermination(ExpectException[SparkException])

      // Resetting termination should make awaitAnyTermination() blocking again
      sqlContext.streams.resetTerminated()
      testAwaitAnyTermination(ExpectBlocked)

      // Terminate multiple queries, one with failure and see whether awaitAnyTermination throws
      // the exception
      val q3 = stopRandomQueryAsync(10 milliseconds, withError = false)
      testAwaitAnyTermination(ExpectNotBlocked)
      require(!q3.isActive)
      val q4 = stopRandomQueryAsync(10 milliseconds, withError = true)
      eventually(Timeout(streamingTimeout)) { require(!q4.isActive) }
      // After q4 terminates with exception, awaitAnyTerm should start throwing exception
      testAwaitAnyTermination(ExpectException[SparkException])
    }
  }

  testQuietly("awaitAnyTermination with timeout and resetTerminated") {
    val datasets = Seq.fill(6)(makeDataset._2)
    withQueriesOn(datasets: _*) { queries =>
      require(queries.size === datasets.size)
      assert(sqlContext.streams.active.toSet === queries.toSet)

      // awaitAnyTermination should be blocking or non-blocking depending on timeout values
      testAwaitAnyTermination(
        ExpectBlocked,
        awaitTimeout = 4 seconds,
        expectedReturnedValue = false,
        testBehaviorFor = 2 seconds)

      testAwaitAnyTermination(
        ExpectNotBlocked,
        awaitTimeout = 50 milliseconds,
        expectedReturnedValue = false,
        testBehaviorFor = 1 second)

      // Stop a query asynchronously within timeout and awaitAnyTerm should be unblocked
      val q1 = stopRandomQueryAsync(stopAfter = 100 milliseconds, withError = false)
      testAwaitAnyTermination(
        ExpectNotBlocked,
        awaitTimeout = 2 seconds,
        expectedReturnedValue = true,
        testBehaviorFor = 4 seconds)
      require(!q1.isActive) // should be inactive by the time the prev awaitAnyTerm returned

      // All subsequent calls to awaitAnyTermination should be non-blocking even if timeout is high
      testAwaitAnyTermination(
        ExpectNotBlocked, awaitTimeout = 4 seconds, expectedReturnedValue = true)

      // Resetting termination should make awaitAnyTermination() blocking again
      sqlContext.streams.resetTerminated()
      testAwaitAnyTermination(
        ExpectBlocked,
        awaitTimeout = 4 seconds,
        expectedReturnedValue = false,
        testBehaviorFor = 1 second)

      // Terminate a query asynchronously with exception within timeout, awaitAnyTermination should
      // throws the exception
      val q2 = stopRandomQueryAsync(100 milliseconds, withError = true)
      testAwaitAnyTermination(
        ExpectException[SparkException],
        awaitTimeout = 4 seconds,
        testBehaviorFor = 6 seconds)
      require(!q2.isActive) // should be inactive by the time the prev awaitAnyTerm returned

      // All subsequent calls to awaitAnyTermination should throw the exception
      testAwaitAnyTermination(
        ExpectException[SparkException],
        awaitTimeout = 2 seconds,
        testBehaviorFor = 4 seconds)

      // Terminate a query asynchronously outside the timeout, awaitAnyTerm should be blocked
      sqlContext.streams.resetTerminated()
      val q3 = stopRandomQueryAsync(2 seconds, withError = true)
      testAwaitAnyTermination(
        ExpectNotBlocked,
        awaitTimeout = 100 milliseconds,
        expectedReturnedValue = false,
        testBehaviorFor = 4 seconds)

      // After that query is stopped, awaitAnyTerm should throw exception
      eventually(Timeout(streamingTimeout)) { require(!q3.isActive) } // wait for query to stop
      testAwaitAnyTermination(
        ExpectException[SparkException],
        awaitTimeout = 100 milliseconds,
        testBehaviorFor = 4 seconds)


      // Terminate multiple queries, one with failure and see whether awaitAnyTermination throws
      // the exception
      sqlContext.streams.resetTerminated()

      val q4 = stopRandomQueryAsync(10 milliseconds, withError = false)
      testAwaitAnyTermination(
        ExpectNotBlocked, awaitTimeout = 2 seconds, expectedReturnedValue = true)
      require(!q4.isActive)
      val q5 = stopRandomQueryAsync(10 milliseconds, withError = true)
      eventually(Timeout(streamingTimeout)) { require(!q5.isActive) }
      // After q5 terminates with exception, awaitAnyTerm should start throwing exception
      testAwaitAnyTermination(ExpectException[SparkException], awaitTimeout = 2 seconds)
    }
  }


  /** Run a body of code by defining a query each on multiple datasets */
  private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[ContinuousQuery] => Unit): Unit = {
    failAfter(streamingTimeout) {
      val queries = withClue("Error starting queries") {
        datasets.map { ds =>
          @volatile var query: StreamExecution = null
          try {
            val df = ds.toDF
            val metadataRoot =
              Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath
            query = sqlContext
              .streams
              .startQuery(
                StreamExecution.nextName,
                metadataRoot,
                df,
                new MemorySink(df.schema))
              .asInstanceOf[StreamExecution]
          } catch {
            case NonFatal(e) =>
              if (query != null) query.stop()
              throw e
          }
          query
        }
      }
      try {
        body(queries)
      } finally {
        queries.foreach(_.stop())
      }
    }
  }

  /** Test the behavior of awaitAnyTermination */
  private def testAwaitAnyTermination(
      expectedBehavior: ExpectedBehavior,
      expectedReturnedValue: Boolean = false,
      awaitTimeout: Span = null,
      testBehaviorFor: Span = 4 seconds
    ): Unit = {

    def awaitTermFunc(): Unit = {
      if (awaitTimeout != null && awaitTimeout.toMillis > 0) {
        val returnedValue = sqlContext.streams.awaitAnyTermination(awaitTimeout.toMillis)
        assert(returnedValue === expectedReturnedValue, "Returned value does not match expected")
      } else {
        sqlContext.streams.awaitAnyTermination()
      }
    }

    AwaitTerminationTester.test(expectedBehavior, awaitTermFunc, testBehaviorFor)
  }

  /** Stop a random active query either with `stop()` or with an error */
  private def stopRandomQueryAsync(stopAfter: Span, withError: Boolean): ContinuousQuery = {

    import scala.concurrent.ExecutionContext.Implicits.global

    val activeQueries = sqlContext.streams.active
    val queryToStop = activeQueries(Random.nextInt(activeQueries.length))
    Future {
      Thread.sleep(stopAfter.toMillis)
      if (withError) {
        logDebug(s"Terminating query ${queryToStop.name} with error")
        queryToStop.asInstanceOf[StreamExecution].logicalPlan.collect {
          case StreamingExecutionRelation(source, _) =>
            source.asInstanceOf[MemoryStream[Int]].addData(0)
        }
      } else {
        logDebug(s"Stopping query ${queryToStop.name}")
        queryToStop.stop()
      }
    }
    queryToStop
  }

  private def makeDataset: (MemoryStream[Int], Dataset[Int]) = {
    val inputData = MemoryStream[Int]
    val mapped = inputData.toDS.map(6 / _)
    (inputData, mapped)
  }
}