/
StreamingQueryManager.scala
449 lines (412 loc) · 17.7 KB
/
StreamingQueryManager.scala
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
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
/*
* 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.{ConcurrentModificationException, UUID}
import java.util.concurrent.{TimeoutException, TimeUnit}
import javax.annotation.concurrent.GuardedBy
import scala.collection.JavaConverters._
import scala.collection.mutable
import org.apache.hadoop.fs.Path
import org.apache.spark.SparkException
import org.apache.spark.annotation.Evolving
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession}
import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker
import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table}
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution
import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS
import org.apache.spark.util.{Clock, SystemClock, Utils}
/**
* A class to manage all the [[StreamingQuery]] active in a `SparkSession`.
*
* @since 2.0.0
*/
@Evolving
class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Logging {
private[sql] val stateStoreCoordinator =
StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env)
private val listenerBus = new StreamingQueryListenerBus(sparkSession.sparkContext.listenerBus)
@GuardedBy("activeQueriesSharedLock")
private val activeQueries = new mutable.HashMap[UUID, StreamingQuery]
// A global lock to keep track of active streaming queries across Spark sessions
private val activeQueriesSharedLock = sparkSession.sharedState.activeQueriesLock
private val awaitTerminationLock = new Object
/**
* Track the last terminated query and remember the last failure since the creation of the
* context, or since `resetTerminated()` was called. There are three possible values:
*
* - null: no query has been been terminated.
* - None: some queries have been terminated and no one has failed.
* - Some(StreamingQueryException): Some queries have been terminated and at least one query has
* failed. The exception is the exception of the last failed query.
*/
@GuardedBy("awaitTerminationLock")
private var lastTerminatedQueryException: Option[StreamingQueryException] = null
try {
sparkSession.sparkContext.conf.get(STREAMING_QUERY_LISTENERS).foreach { classNames =>
Utils.loadExtensions(classOf[StreamingQueryListener], classNames,
sparkSession.sparkContext.conf).foreach(listener => {
addListener(listener)
logInfo(s"Registered listener ${listener.getClass.getName}")
})
}
sparkSession.sharedState.streamingQueryStatusListener.foreach { listener =>
addListener(listener)
}
} catch {
case e: Exception =>
throw new SparkException("Exception when registering StreamingQueryListener", e)
}
/**
* Returns a list of active queries associated with this SQLContext
*
* @since 2.0.0
*/
def active: Array[StreamingQuery] = activeQueriesSharedLock.synchronized {
activeQueries.values.toArray
}
/**
* Returns the query if there is an active query with the given id, or null.
*
* @since 2.1.0
*/
def get(id: UUID): StreamingQuery = activeQueriesSharedLock.synchronized {
activeQueries.get(id).orNull
}
/**
* Returns the query if there is an active query with the given id, or null.
*
* @since 2.1.0
*/
def get(id: String): StreamingQuery = get(UUID.fromString(id))
/**
* Wait until any of the queries on the associated SQLContext has terminated since the
* creation of the context, or since `resetTerminated()` was called. If any query was terminated
* with an exception, then the exception will be thrown.
*
* If a query has terminated, then subsequent calls to `awaitAnyTermination()` will either
* return immediately (if the query was terminated by `query.stop()`),
* or throw the exception immediately (if the query was terminated with exception). Use
* `resetTerminated()` to clear past terminations and wait for new terminations.
*
* In the case where multiple queries have terminated since `resetTermination()` was called,
* if any query has terminated with exception, then `awaitAnyTermination()` will
* throw any of the exception. For correctly documenting exceptions across multiple queries,
* users need to stop all of them after any of them terminates with exception, and then check the
* `query.exception()` for each query.
*
* @throws StreamingQueryException if any query has terminated with an exception
*
* @since 2.0.0
*/
@throws[StreamingQueryException]
def awaitAnyTermination(): Unit = {
awaitTerminationLock.synchronized {
while (lastTerminatedQueryException == null) {
awaitTerminationLock.wait(10)
}
if (lastTerminatedQueryException != null && lastTerminatedQueryException.nonEmpty) {
throw lastTerminatedQueryException.get
}
}
}
/**
* Wait until any of the queries on the associated SQLContext has terminated since the
* creation of the context, or since `resetTerminated()` was called. Returns whether any query
* has terminated or not (multiple may have terminated). If any query has terminated with an
* exception, then the exception will be thrown.
*
* If a query has terminated, then subsequent calls to `awaitAnyTermination()` will either
* return `true` immediately (if the query was terminated by `query.stop()`),
* or throw the exception immediately (if the query was terminated with exception). Use
* `resetTerminated()` to clear past terminations and wait for new terminations.
*
* In the case where multiple queries have terminated since `resetTermination()` was called,
* if any query has terminated with exception, then `awaitAnyTermination()` will
* throw any of the exception. For correctly documenting exceptions across multiple queries,
* users need to stop all of them after any of them terminates with exception, and then check the
* `query.exception()` for each query.
*
* @throws StreamingQueryException if any query has terminated with an exception
*
* @since 2.0.0
*/
@throws[StreamingQueryException]
def awaitAnyTermination(timeoutMs: Long): Boolean = {
val startTime = System.nanoTime()
def isTimedout = {
System.nanoTime() - startTime >= TimeUnit.MILLISECONDS.toNanos(timeoutMs)
}
awaitTerminationLock.synchronized {
while (!isTimedout && lastTerminatedQueryException == null) {
awaitTerminationLock.wait(10)
}
if (lastTerminatedQueryException != null && lastTerminatedQueryException.nonEmpty) {
throw lastTerminatedQueryException.get
}
lastTerminatedQueryException != null
}
}
/**
* Forget about past terminated queries so that `awaitAnyTermination()` can be used again to
* wait for new terminations.
*
* @since 2.0.0
*/
def resetTerminated(): Unit = {
awaitTerminationLock.synchronized {
lastTerminatedQueryException = null
}
}
/**
* Register a [[StreamingQueryListener]] to receive up-calls for life cycle events of
* [[StreamingQuery]].
*
* @since 2.0.0
*/
def addListener(listener: StreamingQueryListener): Unit = {
listenerBus.addListener(listener)
}
/**
* Deregister a [[StreamingQueryListener]].
*
* @since 2.0.0
*/
def removeListener(listener: StreamingQueryListener): Unit = {
listenerBus.removeListener(listener)
}
/**
* List all [[StreamingQueryListener]]s attached to this [[StreamingQueryManager]].
*
* @since 3.0.0
*/
def listListeners(): Array[StreamingQueryListener] = {
listenerBus.listeners.asScala.toArray
}
/** Post a listener event */
private[sql] def postListenerEvent(event: StreamingQueryListener.Event): Unit = {
listenerBus.post(event)
}
private def createQuery(
userSpecifiedName: Option[String],
userSpecifiedCheckpointLocation: Option[String],
df: DataFrame,
extraOptions: Map[String, String],
sink: Table,
outputMode: OutputMode,
useTempCheckpointLocation: Boolean,
recoverFromCheckpointLocation: Boolean,
trigger: Trigger,
triggerClock: Clock): StreamingQueryWrapper = {
var deleteCheckpointOnStop = false
val checkpointLocation = userSpecifiedCheckpointLocation.map { userSpecified =>
new Path(userSpecified).toString
}.orElse {
df.sparkSession.sessionState.conf.checkpointLocation.map { location =>
new Path(location, userSpecifiedName.getOrElse(UUID.randomUUID().toString)).toString
}
}.getOrElse {
if (useTempCheckpointLocation) {
deleteCheckpointOnStop = true
val tempDir = Utils.createTempDir(namePrefix = s"temporary").getCanonicalPath
logWarning("Temporary checkpoint location created which is deleted normally when" +
s" the query didn't fail: $tempDir. If it's required to delete it under any" +
s" circumstances, please set ${SQLConf.FORCE_DELETE_TEMP_CHECKPOINT_LOCATION.key} to" +
s" true. Important to know deleting temp checkpoint folder is best effort.")
tempDir
} else {
throw new AnalysisException(
"checkpointLocation must be specified either " +
"""through option("checkpointLocation", ...) or """ +
s"""SparkSession.conf.set("${SQLConf.CHECKPOINT_LOCATION.key}", ...)""")
}
}
// If offsets have already been created, we trying to resume a query.
if (!recoverFromCheckpointLocation) {
val checkpointPath = new Path(checkpointLocation, "offsets")
val fs = checkpointPath.getFileSystem(df.sparkSession.sessionState.newHadoopConf())
if (fs.exists(checkpointPath)) {
throw new AnalysisException(
s"This query does not support recovering from checkpoint location. " +
s"Delete $checkpointPath to start over.")
}
}
val analyzedPlan = df.queryExecution.analyzed
df.queryExecution.assertAnalyzed()
val operationCheckEnabled = sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled
if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) {
logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} " +
"is not supported in streaming DataFrames/Datasets and will be disabled.")
}
(sink, trigger) match {
case (table: SupportsWrite, trigger: ContinuousTrigger) =>
if (operationCheckEnabled) {
UnsupportedOperationChecker.checkForContinuous(analyzedPlan, outputMode)
}
new StreamingQueryWrapper(new ContinuousExecution(
sparkSession,
userSpecifiedName.orNull,
checkpointLocation,
analyzedPlan,
table,
trigger,
triggerClock,
outputMode,
extraOptions,
deleteCheckpointOnStop))
case _ =>
if (operationCheckEnabled) {
UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode)
}
new StreamingQueryWrapper(new MicroBatchExecution(
sparkSession,
userSpecifiedName.orNull,
checkpointLocation,
analyzedPlan,
sink,
trigger,
triggerClock,
outputMode,
extraOptions,
deleteCheckpointOnStop))
}
}
/**
* Start a [[StreamingQuery]].
*
* @param userSpecifiedName Query name optionally specified by the user.
* @param userSpecifiedCheckpointLocation Checkpoint location optionally specified by the user.
* @param df Streaming DataFrame.
* @param sink Sink to write the streaming outputs.
* @param outputMode Output mode for the sink.
* @param useTempCheckpointLocation Whether to use a temporary checkpoint location when the user
* has not specified one. If false, then error will be thrown.
* @param recoverFromCheckpointLocation Whether to recover query from the checkpoint location.
* If false and the checkpoint location exists, then error
* will be thrown.
* @param trigger [[Trigger]] for the query.
* @param triggerClock [[Clock]] to use for the triggering.
*/
@throws[TimeoutException]
private[sql] def startQuery(
userSpecifiedName: Option[String],
userSpecifiedCheckpointLocation: Option[String],
df: DataFrame,
extraOptions: Map[String, String],
sink: Table,
outputMode: OutputMode,
useTempCheckpointLocation: Boolean = false,
recoverFromCheckpointLocation: Boolean = true,
trigger: Trigger = Trigger.ProcessingTime(0),
triggerClock: Clock = new SystemClock()): StreamingQuery = {
val query = createQuery(
userSpecifiedName,
userSpecifiedCheckpointLocation,
df,
extraOptions,
sink,
outputMode,
useTempCheckpointLocation,
recoverFromCheckpointLocation,
trigger,
triggerClock)
// The following code block checks if a stream with the same name or id is running. Then it
// returns an Option of an already active stream to stop outside of the lock
// to avoid a deadlock.
val activeRunOpt = activeQueriesSharedLock.synchronized {
// Make sure no other query with same name is active
userSpecifiedName.foreach { name =>
if (activeQueries.values.exists(_.name == name)) {
throw new IllegalArgumentException(s"Cannot start query with name $name as a query " +
s"with that name is already active in this SparkSession")
}
}
// Make sure no other query with same id is active across all sessions
val activeOption = Option(sparkSession.sharedState.activeStreamingQueries.get(query.id))
.orElse(activeQueries.get(query.id)) // shouldn't be needed but paranoia ...
val shouldStopActiveRun =
sparkSession.sessionState.conf.getConf(SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART)
if (activeOption.isDefined) {
if (shouldStopActiveRun) {
val oldQuery = activeOption.get
logWarning(s"Stopping existing streaming query [id=${query.id}, " +
s"runId=${oldQuery.runId}], as a new run is being started.")
Some(oldQuery)
} else {
throw new IllegalStateException(
s"Cannot start query with id ${query.id} as another query with same id is " +
s"already active. Perhaps you are attempting to restart a query from checkpoint " +
s"that is already active. You may stop the old query by setting the SQL " +
"configuration: " +
s"""spark.conf.set("${SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART.key}", true) """ +
"and retry.")
}
} else {
// nothing to stop so, no-op
None
}
}
// stop() will clear the queryId from activeStreamingQueries as well as activeQueries
activeRunOpt.foreach(_.stop())
activeQueriesSharedLock.synchronized {
// We still can have a race condition when two concurrent instances try to start the same
// stream, while a third one was already active and stopped above. In this case, we throw a
// ConcurrentModificationException.
val oldActiveQuery = sparkSession.sharedState.activeStreamingQueries.put(
query.id, query.streamingQuery) // we need to put the StreamExecution, not the wrapper
if (oldActiveQuery != null) {
throw new ConcurrentModificationException(
"Another instance of this query was just started by a concurrent session.")
}
activeQueries.put(query.id, query)
}
try {
// When starting a query, it will call `StreamingQueryListener.onQueryStarted` synchronously.
// As it's provided by the user and can run arbitrary codes, we must not hold any lock here.
// Otherwise, it's easy to cause dead-lock, or block too long if the user codes take a long
// time to finish.
query.streamingQuery.start()
} catch {
case e: Throwable =>
unregisterTerminatedStream(query)
throw e
}
query
}
/** Notify (by the StreamingQuery) that the query has been terminated */
private[sql] def notifyQueryTermination(terminatedQuery: StreamingQuery): Unit = {
unregisterTerminatedStream(terminatedQuery)
awaitTerminationLock.synchronized {
if (lastTerminatedQueryException == null || terminatedQuery.exception.nonEmpty) {
lastTerminatedQueryException = terminatedQuery.exception
}
awaitTerminationLock.notifyAll()
}
stateStoreCoordinator.deactivateInstances(terminatedQuery.runId)
}
private def unregisterTerminatedStream(terminatedQuery: StreamingQuery): Unit = {
activeQueriesSharedLock.synchronized {
// remove from shared state only if the streaming execution also matches
sparkSession.sharedState.activeStreamingQueries.remove(
terminatedQuery.id, terminatedQuery)
activeQueries -= terminatedQuery.id
}
}
}