aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala
blob: 831543a47420ae2b642b2c9803ff739b9c4bb24d (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
/*
 * 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 java.util.concurrent.ConcurrentLinkedQueue

import org.scalatest.BeforeAndAfter
import org.scalatest.PrivateMethodTester._
import org.scalatest.concurrent.AsyncAssertions.Waiter
import org.scalatest.concurrent.Eventually._
import org.scalatest.concurrent.PatienceConfiguration.Timeout
import org.scalatest.time.SpanSugar._

import org.apache.spark.SparkException
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.util.JsonProtocol


class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter {

  import testImplicits._
  import StreamingQueryListener._

  after {
    spark.streams.active.foreach(_.stop())
    assert(spark.streams.active.isEmpty)
    assert(addedListeners.isEmpty)
    // Make sure we don't leak any events to the next test
    spark.sparkContext.listenerBus.waitUntilEmpty(10000)
  }

  test("single listener") {
    val listener = new QueryStatusCollector
    val input = MemoryStream[Int]
    withListenerAdded(listener) {
      testStream(input.toDS)(
        StartStream(),
        AssertOnQuery("Incorrect query status in onQueryStarted") { query =>
          val status = listener.startStatus
          assert(status != null)
          assert(status.name === query.name)
          assert(status.id === query.id)
          assert(status.sourceStatuses.size === 1)
          assert(status.sourceStatuses(0).description.contains("Memory"))

          // The source and sink offsets must be None as this must be called before the
          // batches have started
          assert(status.sourceStatuses(0).offsetDesc === None)
          assert(status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString)

          // No progress events or termination events
          assert(listener.progressStatuses.isEmpty)
          assert(listener.terminationStatus === null)
          true
        },
        AddDataMemory(input, Seq(1, 2, 3)),
        CheckAnswer(1, 2, 3),
        AssertOnQuery("Incorrect query status in onQueryProgress") { query =>
          eventually(Timeout(streamingTimeout)) {

            // There should be only on progress event as batch has been processed
            assert(listener.progressStatuses.size === 1)
            val status = listener.progressStatuses.peek()
            assert(status != null)
            assert(status.name === query.name)
            assert(status.id === query.id)
            assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString))
            assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString)

            // No termination events
            assert(listener.terminationStatus === null)
          }
          true
        },
        StopStream,
        AssertOnQuery("Incorrect query status in onQueryTerminated") { query =>
          eventually(Timeout(streamingTimeout)) {
            val status = listener.terminationStatus
            assert(status != null)
            assert(status.name === query.name)
            assert(status.id === query.id)
            assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString))
            assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString)
            assert(listener.terminationException === None)
          }
          listener.checkAsyncErrors()
          true
        }
      )
    }
  }

  test("adding and removing listener") {
    def isListenerActive(listener: QueryStatusCollector): Boolean = {
      listener.reset()
      testStream(MemoryStream[Int].toDS)(
        StartStream(),
        StopStream
      )
      listener.startStatus != null
    }

    try {
      val listener1 = new QueryStatusCollector
      val listener2 = new QueryStatusCollector

      spark.streams.addListener(listener1)
      assert(isListenerActive(listener1) === true)
      assert(isListenerActive(listener2) === false)
      spark.streams.addListener(listener2)
      assert(isListenerActive(listener1) === true)
      assert(isListenerActive(listener2) === true)
      spark.streams.removeListener(listener1)
      assert(isListenerActive(listener1) === false)
      assert(isListenerActive(listener2) === true)
    } finally {
      addedListeners.foreach(spark.streams.removeListener)
    }
  }

  test("event ordering") {
    val listener = new QueryStatusCollector
    withListenerAdded(listener) {
      for (i <- 1 to 100) {
        listener.reset()
        require(listener.startStatus === null)
        testStream(MemoryStream[Int].toDS)(
          StartStream(),
          Assert(listener.startStatus !== null, "onQueryStarted not called before query returned"),
          StopStream,
          Assert { listener.checkAsyncErrors() }
        )
      }
    }
  }

  testQuietly("exception should be reported in QueryTerminated") {
    val listener = new QueryStatusCollector
    withListenerAdded(listener) {
      val input = MemoryStream[Int]
      testStream(input.toDS.map(_ / 0))(
        StartStream(),
        AddData(input, 1),
        ExpectFailure[SparkException](),
        Assert {
          spark.sparkContext.listenerBus.waitUntilEmpty(10000)
          assert(listener.terminationStatus !== null)
          assert(listener.terminationException.isDefined)
          // Make sure that the exception message reported through listener
          // contains the actual exception and relevant stack trace
          assert(!listener.terminationException.get.contains("StreamingQueryException"))
          assert(listener.terminationException.get.contains("java.lang.ArithmeticException"))
          assert(listener.terminationException.get.contains("StreamingQueryListenerSuite"))
        }
      )
    }
  }

  test("QueryStarted serialization") {
    val queryStartedInfo = new StreamingQueryInfo(
      "name",
      1,
      Seq(new SourceStatus("source1", None), new SourceStatus("source2", None)),
      new SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString))
    val queryStarted = new StreamingQueryListener.QueryStarted(queryStartedInfo)
    val json = JsonProtocol.sparkEventToJson(queryStarted)
    val newQueryStarted = JsonProtocol.sparkEventFromJson(json)
      .asInstanceOf[StreamingQueryListener.QueryStarted]
    assertStreamingQueryInfoEquals(queryStarted.queryInfo, newQueryStarted.queryInfo)
  }

  test("QueryProgress serialization") {
    val queryProcessInfo = new StreamingQueryInfo(
      "name",
      1,
      Seq(
        new SourceStatus("source1", Some(LongOffset(0).toString)),
        new SourceStatus("source2", Some(LongOffset(1).toString))),
      new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString))
    val queryProcess = new StreamingQueryListener.QueryProgress(queryProcessInfo)
    val json = JsonProtocol.sparkEventToJson(queryProcess)
    val newQueryProcess = JsonProtocol.sparkEventFromJson(json)
      .asInstanceOf[StreamingQueryListener.QueryProgress]
    assertStreamingQueryInfoEquals(queryProcess.queryInfo, newQueryProcess.queryInfo)
  }

  test("QueryTerminated serialization") {
    val queryTerminatedInfo = new StreamingQueryInfo(
      "name",
      1,
      Seq(
        new SourceStatus("source1", Some(LongOffset(0).toString)),
        new SourceStatus("source2", Some(LongOffset(1).toString))),
      new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString))
    val exception = new RuntimeException("exception")
    val queryQueryTerminated = new StreamingQueryListener.QueryTerminated(
      queryTerminatedInfo,
      Some(exception.getMessage))
    val json =
      JsonProtocol.sparkEventToJson(queryQueryTerminated)
    val newQueryTerminated = JsonProtocol.sparkEventFromJson(json)
      .asInstanceOf[StreamingQueryListener.QueryTerminated]
    assertStreamingQueryInfoEquals(queryQueryTerminated.queryInfo, newQueryTerminated.queryInfo)
    assert(queryQueryTerminated.exception === newQueryTerminated.exception)
  }

  private def assertStreamingQueryInfoEquals(
      expected: StreamingQueryInfo,
      actual: StreamingQueryInfo): Unit = {
    assert(expected.name === actual.name)
    assert(expected.sourceStatuses.size === actual.sourceStatuses.size)
    expected.sourceStatuses.zip(actual.sourceStatuses).foreach {
      case (expectedSource, actualSource) =>
        assertSourceStatus(expectedSource, actualSource)
    }
    assertSinkStatus(expected.sinkStatus, actual.sinkStatus)
  }

  private def assertSourceStatus(expected: SourceStatus, actual: SourceStatus): Unit = {
    assert(expected.description === actual.description)
    assert(expected.offsetDesc === actual.offsetDesc)
  }

  private def assertSinkStatus(expected: SinkStatus, actual: SinkStatus): Unit = {
    assert(expected.description === actual.description)
    assert(expected.offsetDesc === actual.offsetDesc)
  }

  private def withListenerAdded(listener: StreamingQueryListener)(body: => Unit): Unit = {
    try {
      failAfter(1 minute) {
        spark.streams.addListener(listener)
        body
      }
    } finally {
      spark.streams.removeListener(listener)
    }
  }

  private def addedListeners(): Array[StreamingQueryListener] = {
    val listenerBusMethod =
      PrivateMethod[StreamingQueryListenerBus]('listenerBus)
    val listenerBus = spark.streams invokePrivate listenerBusMethod()
    listenerBus.listeners.toArray.map(_.asInstanceOf[StreamingQueryListener])
  }

  class QueryStatusCollector extends StreamingQueryListener {
    // to catch errors in the async listener events
    @volatile private var asyncTestWaiter = new Waiter

    @volatile var startStatus: StreamingQueryInfo = null
    @volatile var terminationStatus: StreamingQueryInfo = null
    @volatile var terminationException: Option[String] = null

    val progressStatuses = new ConcurrentLinkedQueue[StreamingQueryInfo]

    def reset(): Unit = {
      startStatus = null
      terminationStatus = null
      progressStatuses.clear()
      asyncTestWaiter = new Waiter
    }

    def checkAsyncErrors(): Unit = {
      asyncTestWaiter.await(timeout(streamingTimeout))
    }


    override def onQueryStarted(queryStarted: QueryStarted): Unit = {
      asyncTestWaiter {
        startStatus = queryStarted.queryInfo
      }
    }

    override def onQueryProgress(queryProgress: QueryProgress): Unit = {
      asyncTestWaiter {
        assert(startStatus != null, "onQueryProgress called before onQueryStarted")
        progressStatuses.add(queryProgress.queryInfo)
      }
    }

    override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = {
      asyncTestWaiter {
        assert(startStatus != null, "onQueryTerminated called before onQueryStarted")
        terminationStatus = queryTerminated.queryInfo
        terminationException = queryTerminated.exception
      }
      asyncTestWaiter.dismiss()
    }
  }
}