-
Notifications
You must be signed in to change notification settings - Fork 28k
/
TransformWithStateExec.scala
529 lines (477 loc) · 21.2 KB
/
TransformWithStateExec.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
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
/*
* 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.execution.streaming
import java.util.UUID
import java.util.concurrent.TimeUnit.NANOSECONDS
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeRow}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.physical.Distribution
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper
import org.apache.spark.sql.execution.streaming.state._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.{OutputMode, StatefulProcessor, StatefulProcessorWithInitialState, TimeoutMode}
import org.apache.spark.sql.types._
import org.apache.spark.util.{CompletionIterator, SerializableConfiguration, Utils}
/**
* Physical operator for executing `TransformWithState`
*
* @param keyDeserializer used to extract the key object for each group.
* @param valueDeserializer used to extract the items in the iterator from an input row.
* @param groupingAttributes used to group the data
* @param dataAttributes used to read the data
* @param statefulProcessor processor methods called on underlying data
* @param timeoutMode defines the timeout mode
* @param outputMode defines the output mode for the statefulProcessor
* @param keyEncoder expression encoder for the key type
* @param outputObjAttr Defines the output object
* @param batchTimestampMs processing timestamp of the current batch.
* @param eventTimeWatermarkForLateEvents event time watermark for filtering late events
* @param eventTimeWatermarkForEviction event time watermark for state eviction
* @param isStreaming defines whether the query is streaming or batch
* @param child the physical plan for the underlying data
*/
case class TransformWithStateExec(
keyDeserializer: Expression,
valueDeserializer: Expression,
groupingAttributes: Seq[Attribute],
dataAttributes: Seq[Attribute],
statefulProcessor: StatefulProcessor[Any, Any, Any],
timeoutMode: TimeoutMode,
outputMode: OutputMode,
keyEncoder: ExpressionEncoder[Any],
outputObjAttr: Attribute,
stateInfo: Option[StatefulOperatorStateInfo],
batchTimestampMs: Option[Long],
eventTimeWatermarkForLateEvents: Option[Long],
eventTimeWatermarkForEviction: Option[Long],
child: SparkPlan,
isStreaming: Boolean = true,
hasInitialState: Boolean = false,
initialStateGroupingAttrs: Seq[Attribute],
initialStateDataAttrs: Seq[Attribute],
initialStateDeserializer: Expression,
initialState: SparkPlan)
extends BinaryExecNode with StateStoreWriter with WatermarkSupport with ObjectProducerExec {
override def shortName: String = "transformWithStateExec"
override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = {
timeoutMode match {
// TODO: check if we can return true only if actual timers are registered
case ProcessingTime =>
true
case EventTime =>
eventTimeWatermarkForEviction.isDefined &&
newInputWatermark > eventTimeWatermarkForEviction.get
case _ =>
false
}
}
override def left: SparkPlan = child
override def right: SparkPlan = initialState
override protected def withNewChildrenInternal(
newLeft: SparkPlan, newRight: SparkPlan): TransformWithStateExec =
copy(child = newLeft, initialState = newRight)
override def keyExpressions: Seq[Attribute] = groupingAttributes
protected val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
protected val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
/**
* Distribute by grouping attributes - We need the underlying data and the initial state data
* to have the same grouping so that the data are co-located on the same task.
*/
override def requiredChildDistribution: Seq[Distribution] = {
StatefulOperatorPartitioning.getCompatibleDistribution(
groupingAttributes, getStateInfo, conf) ::
StatefulOperatorPartitioning.getCompatibleDistribution(
initialStateGroupingAttrs, getStateInfo, conf) ::
Nil
}
/**
* We need the initial state to also use the ordering as the data so that we can co-locate the
* keys from the underlying data and the initial state.
*/
override def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq(
groupingAttributes.map(SortOrder(_, Ascending)),
initialStateGroupingAttrs.map(SortOrder(_, Ascending)))
private def handleInputRows(keyRow: UnsafeRow, valueRowIter: Iterator[InternalRow]):
Iterator[InternalRow] = {
val getKeyObj =
ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes)
val getValueObj =
ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes)
val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjectType)
val keyObj = getKeyObj(keyRow) // convert key to objects
ImplicitGroupingKeyTracker.setImplicitKey(keyObj)
val valueObjIter = valueRowIter.map(getValueObj.apply)
val mappedIterator = statefulProcessor.handleInputRows(
keyObj,
valueObjIter,
new TimerValuesImpl(batchTimestampMs, eventTimeWatermarkForEviction),
new ExpiredTimerInfoImpl(isValid = false)).map { obj =>
getOutputRow(obj)
}
ImplicitGroupingKeyTracker.removeImplicitKey()
mappedIterator
}
private def processInitialStateRows(
keyRow: UnsafeRow,
initStateIter: Iterator[InternalRow]): Unit = {
val getKeyObj =
ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes)
val getInitStateValueObj =
ObjectOperator.deserializeRowToObject(initialStateDeserializer, initialStateDataAttrs)
val keyObj = getKeyObj(keyRow) // convert key to objects
ImplicitGroupingKeyTracker.setImplicitKey(keyObj)
val initStateObjIter = initStateIter.map(getInitStateValueObj.apply)
var seenInitStateOnKey = false
initStateObjIter.foreach { initState =>
// cannot re-initialize state on the same grouping key during initial state handling
if (seenInitStateOnKey) {
throw StateStoreErrors.cannotReInitializeStateOnKey(keyObj.toString)
}
seenInitStateOnKey = true
statefulProcessor
.asInstanceOf[StatefulProcessorWithInitialState[Any, Any, Any, Any]]
.handleInitialState(keyObj, initState)
}
ImplicitGroupingKeyTracker.removeImplicitKey()
}
private def processNewData(dataIter: Iterator[InternalRow]): Iterator[InternalRow] = {
val groupedIter = GroupedIterator(dataIter, groupingAttributes, child.output)
groupedIter.flatMap { case (keyRow, valueRowIter) =>
val keyUnsafeRow = keyRow.asInstanceOf[UnsafeRow]
handleInputRows(keyUnsafeRow, valueRowIter)
}
}
private def handleTimerRows(
keyObj: Any,
expiryTimestampMs: Long,
processorHandle: StatefulProcessorHandleImpl): Iterator[InternalRow] = {
val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjectType)
ImplicitGroupingKeyTracker.setImplicitKey(keyObj)
val mappedIterator = statefulProcessor.handleInputRows(
keyObj,
Iterator.empty,
new TimerValuesImpl(batchTimestampMs, eventTimeWatermarkForEviction),
new ExpiredTimerInfoImpl(isValid = true, Some(expiryTimestampMs))).map { obj =>
getOutputRow(obj)
}
processorHandle.deleteTimer(expiryTimestampMs)
ImplicitGroupingKeyTracker.removeImplicitKey()
mappedIterator
}
private def processTimers(
timeoutMode: TimeoutMode,
processorHandle: StatefulProcessorHandleImpl): Iterator[InternalRow] = {
timeoutMode match {
case ProcessingTime =>
assert(batchTimestampMs.isDefined)
val batchTimestamp = batchTimestampMs.get
val procTimeIter = processorHandle.getExpiredTimers()
procTimeIter.flatMap { case (keyObj, expiryTimestampMs) =>
if (expiryTimestampMs < batchTimestamp) {
handleTimerRows(keyObj, expiryTimestampMs, processorHandle)
} else {
Iterator.empty
}
}
case EventTime =>
assert(eventTimeWatermarkForEviction.isDefined)
val watermark = eventTimeWatermarkForEviction.get
val eventTimeIter = processorHandle.getExpiredTimers()
eventTimeIter.flatMap { case (keyObj, expiryTimestampMs) =>
if (expiryTimestampMs < watermark) {
handleTimerRows(keyObj, expiryTimestampMs, processorHandle)
} else {
Iterator.empty
}
}
case _ => Iterator.empty
}
}
private def processDataWithPartition(
iter: Iterator[InternalRow],
store: StateStore,
processorHandle: StatefulProcessorHandleImpl):
CompletionIterator[InternalRow, Iterator[InternalRow]] = {
val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
val commitTimeMs = longMetric("commitTimeMs")
val timeoutLatencyMs = longMetric("allRemovalsTimeMs")
val currentTimeNs = System.nanoTime
val updatesStartTimeNs = currentTimeNs
var timeoutProcessingStartTimeNs = currentTimeNs
// If timeout is based on event time, then filter late data based on watermark
val filteredIter = watermarkPredicateForDataForLateEvents match {
case Some(predicate) =>
applyRemovingRowsOlderThanWatermark(iter, predicate)
case _ =>
iter
}
val newDataProcessorIter =
CompletionIterator[InternalRow, Iterator[InternalRow]](
processNewData(filteredIter), {
// Once the input is processed, mark the start time for timeout processing to measure
// it separately from the overall processing time.
timeoutProcessingStartTimeNs = System.nanoTime
processorHandle.setHandleState(StatefulProcessorHandleState.DATA_PROCESSED)
})
// Late-bind the timeout processing iterator so it is created *after* the input is
// processed (the input iterator is exhausted) and the state updates are written into the
// state store. Otherwise the iterator may not see the updates (e.g. with RocksDB state store).
val timeoutProcessorIter = new Iterator[InternalRow] {
private lazy val itr = getIterator()
override def hasNext = itr.hasNext
override def next() = itr.next()
private def getIterator(): Iterator[InternalRow] =
CompletionIterator[InternalRow, Iterator[InternalRow]](
processTimers(timeoutMode, processorHandle), {
// Note: `timeoutLatencyMs` also includes the time the parent operator took for
// processing output returned through iterator.
timeoutLatencyMs += NANOSECONDS.toMillis(System.nanoTime - timeoutProcessingStartTimeNs)
processorHandle.setHandleState(StatefulProcessorHandleState.TIMER_PROCESSED)
})
}
val outputIterator = newDataProcessorIter ++ timeoutProcessorIter
// Return an iterator of all the rows generated by all the keys, such that when fully
// consumed, all the state updates will be committed by the state store
CompletionIterator[InternalRow, Iterator[InternalRow]](outputIterator, {
// Note: Due to the iterator lazy execution, this metric also captures the time taken
// by the upstream (consumer) operators in addition to the processing in this operator.
allUpdatesTimeMs += NANOSECONDS.toMillis(System.nanoTime - updatesStartTimeNs)
commitTimeMs += timeTakenMs {
if (isStreaming) {
store.commit()
} else {
store.abort()
}
}
setStoreMetrics(store)
setOperatorMetrics()
statefulProcessor.close()
statefulProcessor.setHandle(null)
processorHandle.setHandleState(StatefulProcessorHandleState.CLOSED)
})
}
override protected def doExecute(): RDD[InternalRow] = {
metrics // force lazy init at driver
timeoutMode match {
case ProcessingTime =>
if (batchTimestampMs.isEmpty) {
StateStoreErrors.missingTimeoutValues(timeoutMode.toString)
}
case EventTime =>
if (eventTimeWatermarkForEviction.isEmpty) {
StateStoreErrors.missingTimeoutValues(timeoutMode.toString)
}
case _ =>
}
if (hasInitialState) {
val storeConf = new StateStoreConf(session.sqlContext.sessionState.conf)
val hadoopConfBroadcast =
new SerializableConfiguration(session.sqlContext.sessionState.newHadoopConf())
child.execute().stateStoreAwareZipPartitions(
initialState.execute(),
getStateInfo,
storeNames = Seq(),
session.sqlContext.streams.stateStoreCoordinator) {
// The state store aware zip partitions will provide us with two iterators,
// child data iterator and the initial state iterator per partition.
case (partitionId, childDataIterator, initStateIterator) =>
if (isStreaming) {
val stateStoreId = StateStoreId(stateInfo.get.checkpointLocation,
stateInfo.get.operatorId, partitionId)
val storeProviderId = StateStoreProviderId(stateStoreId, stateInfo.get.queryRunId)
val store = StateStore.get(
storeProviderId = storeProviderId,
keySchema = schemaForKeyRow,
valueSchema = schemaForValueRow,
numColsPrefixKey = 0,
version = stateInfo.get.storeVersion,
useColumnFamilies = true,
storeConf = storeConf,
hadoopConf = hadoopConfBroadcast.value
)
processDataWithInitialState(store, childDataIterator, initStateIterator)
} else {
val providerId = {
val tempDirPath = Utils.createTempDir().getAbsolutePath
new StateStoreProviderId(
StateStoreId(tempDirPath, 0, partitionId), getStateInfo.queryRunId)
}
val sqlConf = new SQLConf()
sqlConf.setConfString(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
classOf[RocksDBStateStoreProvider].getName)
// Create StateStoreProvider for this partition
val stateStoreProvider = StateStoreProvider.createAndInit(
providerId,
schemaForKeyRow,
schemaForValueRow,
numColsPrefixKey = 0,
useColumnFamilies = true,
storeConf = new StateStoreConf(sqlConf),
hadoopConf = hadoopConfBroadcast.value,
useMultipleValuesPerKey = true)
val store = stateStoreProvider.getStore(0)
processDataWithInitialState(store, childDataIterator, initStateIterator)
}
}
} else {
if (isStreaming) {
child.execute().mapPartitionsWithStateStore[InternalRow](
getStateInfo,
schemaForKeyRow,
schemaForValueRow,
numColsPrefixKey = 0,
session.sqlContext.sessionState,
Some(session.sqlContext.streams.stateStoreCoordinator),
useColumnFamilies = true
) {
case (store: StateStore, singleIterator: Iterator[InternalRow]) =>
processData(store, singleIterator)
}
} else {
// If the query is running in batch mode, we need to create a new StateStore and instantiate
// a temp directory on the executors in mapPartitionsWithIndex.
val broadcastedHadoopConf =
new SerializableConfiguration(session.sessionState.newHadoopConf())
child.execute().mapPartitionsWithIndex[InternalRow](
(i, iter) => {
val providerId = {
val tempDirPath = Utils.createTempDir().getAbsolutePath
new StateStoreProviderId(
StateStoreId(tempDirPath, 0, i), getStateInfo.queryRunId)
}
val sqlConf = new SQLConf()
sqlConf.setConfString(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
classOf[RocksDBStateStoreProvider].getName)
val storeConf = new StateStoreConf(sqlConf)
// Create StateStoreProvider for this partition
val stateStoreProvider = StateStoreProvider.createAndInit(
providerId,
schemaForKeyRow,
schemaForValueRow,
numColsPrefixKey = 0,
useColumnFamilies = true,
storeConf = storeConf,
hadoopConf = broadcastedHadoopConf.value,
useMultipleValuesPerKey = true)
val store = stateStoreProvider.getStore(0)
val outputIterator = processData(store, iter)
CompletionIterator[InternalRow, Iterator[InternalRow]](outputIterator.iterator, {
stateStoreProvider.close()
statefulProcessor.close()
})
}
)
}
}
}
/**
* Process the data in the partition using the state store and the stateful processor.
* @param store The state store to use
* @param singleIterator The iterator of rows to process
* @return An iterator of rows that are the result of processing the input rows
*/
private def processData(store: StateStore, singleIterator: Iterator[InternalRow]):
CompletionIterator[InternalRow, Iterator[InternalRow]] = {
val processorHandle = new StatefulProcessorHandleImpl(
store, getStateInfo.queryRunId, keyEncoder, timeoutMode, isStreaming)
assert(processorHandle.getHandleState == StatefulProcessorHandleState.CREATED)
statefulProcessor.setHandle(processorHandle)
statefulProcessor.init(outputMode, timeoutMode)
processorHandle.setHandleState(StatefulProcessorHandleState.INITIALIZED)
processDataWithPartition(singleIterator, store, processorHandle)
}
private def processDataWithInitialState(
store: StateStore,
childDataIterator: Iterator[InternalRow],
initStateIterator: Iterator[InternalRow]):
CompletionIterator[InternalRow, Iterator[InternalRow]] = {
val processorHandle = new StatefulProcessorHandleImpl(store, getStateInfo.queryRunId,
keyEncoder, timeoutMode, isStreaming)
assert(processorHandle.getHandleState == StatefulProcessorHandleState.CREATED)
statefulProcessor.setHandle(processorHandle)
statefulProcessor.init(outputMode, timeoutMode)
processorHandle.setHandleState(StatefulProcessorHandleState.INITIALIZED)
// Check if is first batch
// Only process initial states for first batch
if (processorHandle.getQueryInfo().getBatchId == 0) {
// If the user provided initial state, we need to have the initial state and the
// data in the same partition so that we can still have just one commit at the end.
val groupedInitialStateIter = GroupedIterator(initStateIterator,
initialStateGroupingAttrs, initialState.output)
groupedInitialStateIter.foreach {
case (keyRow, valueRowIter) =>
processInitialStateRows(keyRow.asInstanceOf[UnsafeRow], valueRowIter)
}
}
processDataWithPartition(childDataIterator, store, processorHandle)
}
}
// scalastyle:off
object TransformWithStateExec {
// Plan logical transformWithState for batch queries
def generateSparkPlanForBatchQueries(
keyDeserializer: Expression,
valueDeserializer: Expression,
groupingAttributes: Seq[Attribute],
dataAttributes: Seq[Attribute],
statefulProcessor: StatefulProcessor[Any, Any, Any],
timeoutMode: TimeoutMode,
outputMode: OutputMode,
keyEncoder: ExpressionEncoder[Any],
outputObjAttr: Attribute,
child: SparkPlan,
hasInitialState: Boolean = false,
initialStateGroupingAttrs: Seq[Attribute],
initialStateDataAttrs: Seq[Attribute],
initialStateDeserializer: Expression,
initialState: SparkPlan): SparkPlan = {
val shufflePartitions = child.session.sessionState.conf.numShufflePartitions
val statefulOperatorStateInfo = StatefulOperatorStateInfo(
checkpointLocation = "", // empty checkpointLocation will be populated in doExecute
queryRunId = UUID.randomUUID(),
operatorId = 0,
storeVersion = 0,
numPartitions = shufflePartitions
)
new TransformWithStateExec(
keyDeserializer,
valueDeserializer,
groupingAttributes,
dataAttributes,
statefulProcessor,
timeoutMode,
outputMode,
keyEncoder,
outputObjAttr,
Some(statefulOperatorStateInfo),
Some(System.currentTimeMillis),
None,
None,
child,
isStreaming = false,
hasInitialState,
initialStateGroupingAttrs,
initialStateDataAttrs,
initialStateDeserializer,
initialState)
}
}
// scalastyle:on