-
Notifications
You must be signed in to change notification settings - Fork 28k
/
FlatMapGroupsWithStateSuite.scala
1307 lines (1152 loc) · 51.6 KB
/
FlatMapGroupsWithStateSuite.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
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.io.File
import java.sql.Date
import org.apache.commons.io.FileUtils
import org.scalatest.BeforeAndAfterAll
import org.scalatest.exceptions.TestFailedException
import org.apache.spark.SparkException
import org.apache.spark.api.java.function.FlatMapGroupsWithStateFunction
import org.apache.spark.sql.Encoder
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.catalyst.plans.logical.FlatMapGroupsWithState
import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
import org.apache.spark.sql.execution.RDDScanExec
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.state.{FlatMapGroupsWithStateExecHelper, MemoryStateStore, StateStore, StateStoreId, StateStoreMetrics, UnsafeRowPair}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.util.StreamManualClock
import org.apache.spark.sql.types.{DataType, IntegerType}
import org.apache.spark.util.Utils
/** Class to check custom state types */
case class RunningCount(count: Long)
case class Result(key: Long, count: Int)
class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest
with BeforeAndAfterAll {
import testImplicits._
import GroupStateImpl._
import GroupStateTimeout._
import FlatMapGroupsWithStateSuite._
override def afterAll(): Unit = {
super.afterAll()
StateStore.stop()
}
test("GroupState - get, exists, update, remove") {
var state: GroupStateImpl[String] = null
def testState(
expectedData: Option[String],
shouldBeUpdated: Boolean = false,
shouldBeRemoved: Boolean = false): Unit = {
if (expectedData.isDefined) {
assert(state.exists)
assert(state.get === expectedData.get)
} else {
assert(!state.exists)
intercept[NoSuchElementException] {
state.get
}
}
assert(state.getOption === expectedData)
assert(state.hasUpdated === shouldBeUpdated)
assert(state.hasRemoved === shouldBeRemoved)
}
// === Tests for state in streaming queries ===
// Updating empty state
state = GroupStateImpl.createForStreaming(
None, 1, 1, NoTimeout, hasTimedOut = false, watermarkPresent = false)
testState(None)
state.update("")
testState(Some(""), shouldBeUpdated = true)
// Updating exiting state
state = GroupStateImpl.createForStreaming(
Some("2"), 1, 1, NoTimeout, hasTimedOut = false, watermarkPresent = false)
testState(Some("2"))
state.update("3")
testState(Some("3"), shouldBeUpdated = true)
// Removing state
state.remove()
testState(None, shouldBeRemoved = true, shouldBeUpdated = false)
state.remove() // should be still callable
state.update("4")
testState(Some("4"), shouldBeRemoved = false, shouldBeUpdated = true)
// Updating by null throw exception
intercept[IllegalArgumentException] {
state.update(null)
}
}
test("GroupState - setTimeout - with NoTimeout") {
for (initValue <- Seq(None, Some(5))) {
val states = Seq(
GroupStateImpl.createForStreaming(
initValue, 1000, 1000, NoTimeout, hasTimedOut = false, watermarkPresent = false),
GroupStateImpl.createForBatch(NoTimeout, watermarkPresent = false)
)
for (state <- states) {
// for streaming queries
testTimeoutDurationNotAllowed[UnsupportedOperationException](state)
testTimeoutTimestampNotAllowed[UnsupportedOperationException](state)
// for batch queries
testTimeoutDurationNotAllowed[UnsupportedOperationException](state)
testTimeoutTimestampNotAllowed[UnsupportedOperationException](state)
}
}
}
test("GroupState - setTimeout - with ProcessingTimeTimeout") {
// for streaming queries
var state: GroupStateImpl[Int] = GroupStateImpl.createForStreaming(
None, 1000, 1000, ProcessingTimeTimeout, hasTimedOut = false, watermarkPresent = false)
assert(state.getTimeoutTimestamp === NO_TIMESTAMP)
state.setTimeoutDuration(500)
assert(state.getTimeoutTimestamp === 1500) // can be set without initializing state
testTimeoutTimestampNotAllowed[UnsupportedOperationException](state)
state.update(5)
assert(state.getTimeoutTimestamp === 1500) // does not change
state.setTimeoutDuration(1000)
assert(state.getTimeoutTimestamp === 2000)
state.setTimeoutDuration("2 second")
assert(state.getTimeoutTimestamp === 3000)
testTimeoutTimestampNotAllowed[UnsupportedOperationException](state)
state.remove()
assert(state.getTimeoutTimestamp === 3000) // does not change
state.setTimeoutDuration(500) // can still be set
assert(state.getTimeoutTimestamp === 1500)
testTimeoutTimestampNotAllowed[UnsupportedOperationException](state)
// for batch queries
state = GroupStateImpl.createForBatch(
ProcessingTimeTimeout, watermarkPresent = false).asInstanceOf[GroupStateImpl[Int]]
assert(state.getTimeoutTimestamp === NO_TIMESTAMP)
state.setTimeoutDuration(500)
testTimeoutTimestampNotAllowed[UnsupportedOperationException](state)
state.update(5)
state.setTimeoutDuration(1000)
state.setTimeoutDuration("2 second")
testTimeoutTimestampNotAllowed[UnsupportedOperationException](state)
state.remove()
state.setTimeoutDuration(500)
testTimeoutTimestampNotAllowed[UnsupportedOperationException](state)
}
test("GroupState - setTimeout - with EventTimeTimeout") {
var state: GroupStateImpl[Int] = GroupStateImpl.createForStreaming(
None, 1000, 1000, EventTimeTimeout, false, watermarkPresent = true)
assert(state.getTimeoutTimestamp === NO_TIMESTAMP)
testTimeoutDurationNotAllowed[UnsupportedOperationException](state)
state.setTimeoutTimestamp(5000)
assert(state.getTimeoutTimestamp === 5000) // can be set without initializing state
state.update(5)
assert(state.getTimeoutTimestamp === 5000) // does not change
state.setTimeoutTimestamp(10000)
assert(state.getTimeoutTimestamp === 10000)
state.setTimeoutTimestamp(new Date(20000))
assert(state.getTimeoutTimestamp === 20000)
testTimeoutDurationNotAllowed[UnsupportedOperationException](state)
state.remove()
assert(state.getTimeoutTimestamp === 20000)
state.setTimeoutTimestamp(5000)
assert(state.getTimeoutTimestamp === 5000) // can be set after removing state
testTimeoutDurationNotAllowed[UnsupportedOperationException](state)
// for batch queries
state = GroupStateImpl.createForBatch(EventTimeTimeout, watermarkPresent = false)
.asInstanceOf[GroupStateImpl[Int]]
assert(state.getTimeoutTimestamp === NO_TIMESTAMP)
testTimeoutDurationNotAllowed[UnsupportedOperationException](state)
state.setTimeoutTimestamp(5000)
state.update(5)
state.setTimeoutTimestamp(10000)
state.setTimeoutTimestamp(new Date(20000))
testTimeoutDurationNotAllowed[UnsupportedOperationException](state)
state.remove()
state.setTimeoutTimestamp(5000)
testTimeoutDurationNotAllowed[UnsupportedOperationException](state)
}
test("GroupState - illegal params to setTimeout") {
var state: GroupStateImpl[Int] = null
// Test setTimeout****() with illegal values
def testIllegalTimeout(body: => Unit): Unit = {
intercept[IllegalArgumentException] {
body
}
assert(state.getTimeoutTimestamp === NO_TIMESTAMP)
}
state = GroupStateImpl.createForStreaming(
Some(5), 1000, 1000, ProcessingTimeTimeout, hasTimedOut = false, watermarkPresent = false)
testIllegalTimeout {
state.setTimeoutDuration(-1000)
}
testIllegalTimeout {
state.setTimeoutDuration(0)
}
testIllegalTimeout {
state.setTimeoutDuration("-2 second")
}
testIllegalTimeout {
state.setTimeoutDuration("-1 month")
}
testIllegalTimeout {
state.setTimeoutDuration("1 month -1 day")
}
state = GroupStateImpl.createForStreaming(
Some(5), 1000, 1000, EventTimeTimeout, hasTimedOut = false, watermarkPresent = false)
testIllegalTimeout {
state.setTimeoutTimestamp(-10000)
}
testIllegalTimeout {
state.setTimeoutTimestamp(10000, "-3 second")
}
testIllegalTimeout {
state.setTimeoutTimestamp(10000, "-1 month")
}
testIllegalTimeout {
state.setTimeoutTimestamp(10000, "1 month -1 day")
}
testIllegalTimeout {
state.setTimeoutTimestamp(new Date(-10000))
}
testIllegalTimeout {
state.setTimeoutTimestamp(new Date(-10000), "-3 second")
}
testIllegalTimeout {
state.setTimeoutTimestamp(new Date(-10000), "-1 month")
}
testIllegalTimeout {
state.setTimeoutTimestamp(new Date(-10000), "1 month -1 day")
}
}
test("GroupState - hasTimedOut") {
for (timeoutConf <- Seq(NoTimeout, ProcessingTimeTimeout, EventTimeTimeout)) {
// for streaming queries
for (initState <- Seq(None, Some(5))) {
val state1 = GroupStateImpl.createForStreaming(
initState, 1000, 1000, timeoutConf, hasTimedOut = false, watermarkPresent = false)
assert(state1.hasTimedOut === false)
val state2 = GroupStateImpl.createForStreaming(
initState, 1000, 1000, timeoutConf, hasTimedOut = true, watermarkPresent = false)
assert(state2.hasTimedOut === true)
}
// for batch queries
assert(
GroupStateImpl.createForBatch(timeoutConf, watermarkPresent = false).hasTimedOut === false)
}
}
test("GroupState - getCurrentWatermarkMs") {
def streamingState(timeoutConf: GroupStateTimeout, watermark: Option[Long]): GroupState[Int] = {
GroupStateImpl.createForStreaming(
None, 1000, watermark.getOrElse(-1), timeoutConf,
hasTimedOut = false, watermark.nonEmpty)
}
def batchState(timeoutConf: GroupStateTimeout, watermarkPresent: Boolean): GroupState[Any] = {
GroupStateImpl.createForBatch(timeoutConf, watermarkPresent)
}
def assertWrongTimeoutError(test: => Unit): Unit = {
val e = intercept[UnsupportedOperationException] { test }
assert(e.getMessage.contains(
"Cannot get event time watermark timestamp without setting watermark"))
}
for (timeoutConf <- Seq(NoTimeout, EventTimeTimeout, ProcessingTimeTimeout)) {
// Tests for getCurrentWatermarkMs in streaming queries
assertWrongTimeoutError { streamingState(timeoutConf, None).getCurrentWatermarkMs() }
assert(streamingState(timeoutConf, Some(1000)).getCurrentWatermarkMs() === 1000)
assert(streamingState(timeoutConf, Some(2000)).getCurrentWatermarkMs() === 2000)
// Tests for getCurrentWatermarkMs in batch queries
assertWrongTimeoutError {
batchState(timeoutConf, watermarkPresent = false).getCurrentWatermarkMs()
}
assert(batchState(timeoutConf, watermarkPresent = true).getCurrentWatermarkMs() === -1)
}
}
test("GroupState - getCurrentProcessingTimeMs") {
def streamingState(
timeoutConf: GroupStateTimeout,
procTime: Long,
watermarkPresent: Boolean): GroupState[Int] = {
GroupStateImpl.createForStreaming(
None, procTime, -1, timeoutConf, hasTimedOut = false, watermarkPresent = false)
}
def batchState(timeoutConf: GroupStateTimeout, watermarkPresent: Boolean): GroupState[Any] = {
GroupStateImpl.createForBatch(timeoutConf, watermarkPresent)
}
for (timeoutConf <- Seq(NoTimeout, EventTimeTimeout, ProcessingTimeTimeout)) {
for (watermarkPresent <- Seq(false, true)) {
// Tests for getCurrentProcessingTimeMs in streaming queries
assert(streamingState(timeoutConf, NO_TIMESTAMP, watermarkPresent)
.getCurrentProcessingTimeMs() === -1)
assert(streamingState(timeoutConf, 1000, watermarkPresent)
.getCurrentProcessingTimeMs() === 1000)
assert(streamingState(timeoutConf, 2000, watermarkPresent)
.getCurrentProcessingTimeMs() === 2000)
// Tests for getCurrentProcessingTimeMs in batch queries
val currentTime = System.currentTimeMillis()
assert(batchState(timeoutConf, watermarkPresent).getCurrentProcessingTimeMs >= currentTime)
}
}
}
test("GroupState - primitive type") {
var intState = GroupStateImpl.createForStreaming[Int](
None, 1000, 1000, NoTimeout, hasTimedOut = false, watermarkPresent = false)
intercept[NoSuchElementException] {
intState.get
}
assert(intState.getOption === None)
intState = GroupStateImpl.createForStreaming[Int](
Some(10), 1000, 1000, NoTimeout, hasTimedOut = false, watermarkPresent = false)
assert(intState.get == 10)
intState.update(0)
assert(intState.get == 0)
intState.remove()
intercept[NoSuchElementException] {
intState.get
}
}
// Values used for testing InputProcessor
val currentBatchTimestamp = 1000
val currentBatchWatermark = 1000
val beforeTimeoutThreshold = 999
val afterTimeoutThreshold = 1001
// Tests for InputProcessor.processNewData() when timeout = NoTimeout
for (priorState <- Seq(None, Some(0))) {
val priorStateStr = if (priorState.nonEmpty) "prior state set" else "no prior state"
val testName = s"NoTimeout - $priorStateStr - "
testStateUpdateWithData(
testName + "no update",
stateUpdates = state => {
assert(state.getCurrentProcessingTimeMs() === currentBatchTimestamp)
intercept[Exception] { state.getCurrentWatermarkMs() } // watermark not specified
/* no updates */
},
timeoutConf = GroupStateTimeout.NoTimeout,
priorState = priorState,
expectedState = priorState) // should not change
testStateUpdateWithData(
testName + "state updated",
stateUpdates = state => { state.update(5) },
timeoutConf = GroupStateTimeout.NoTimeout,
priorState = priorState,
expectedState = Some(5)) // should change
testStateUpdateWithData(
testName + "state removed",
stateUpdates = state => { state.remove() },
timeoutConf = GroupStateTimeout.NoTimeout,
priorState = priorState,
expectedState = None) // should be removed
}
// Tests for InputProcessor.processTimedOutState() when timeout != NoTimeout
for (priorState <- Seq(None, Some(0))) {
for (priorTimeoutTimestamp <- Seq(NO_TIMESTAMP, 1000)) {
var testName = ""
if (priorState.nonEmpty) {
testName += "prior state set, "
if (priorTimeoutTimestamp == 1000) {
testName += "prior timeout set"
} else {
testName += "no prior timeout"
}
} else {
testName += "no prior state"
}
for (timeoutConf <- Seq(ProcessingTimeTimeout, EventTimeTimeout)) {
testStateUpdateWithData(
s"$timeoutConf - $testName - no update",
stateUpdates = state => {
assert(state.getCurrentProcessingTimeMs() === currentBatchTimestamp)
intercept[Exception] { state.getCurrentWatermarkMs() } // watermark not specified
/* no updates */
},
timeoutConf = timeoutConf,
priorState = priorState,
priorTimeoutTimestamp = priorTimeoutTimestamp,
expectedState = priorState, // state should not change
expectedTimeoutTimestamp = NO_TIMESTAMP) // timestamp should be reset
testStateUpdateWithData(
s"$timeoutConf - $testName - state updated",
stateUpdates = state => { state.update(5) },
timeoutConf = timeoutConf,
priorState = priorState,
priorTimeoutTimestamp = priorTimeoutTimestamp,
expectedState = Some(5), // state should change
expectedTimeoutTimestamp = NO_TIMESTAMP) // timestamp should be reset
testStateUpdateWithData(
s"$timeoutConf - $testName - state removed",
stateUpdates = state => { state.remove() },
timeoutConf = timeoutConf,
priorState = priorState,
priorTimeoutTimestamp = priorTimeoutTimestamp,
expectedState = None) // state should be removed
}
// Tests with ProcessingTimeTimeout
if (priorState == None) {
testStateUpdateWithData(
s"ProcessingTimeTimeout - $testName - timeout updated without initializing state",
stateUpdates = state => { state.setTimeoutDuration(5000) },
timeoutConf = ProcessingTimeTimeout,
priorState = None,
priorTimeoutTimestamp = priorTimeoutTimestamp,
expectedState = None,
expectedTimeoutTimestamp = currentBatchTimestamp + 5000)
}
testStateUpdateWithData(
s"ProcessingTimeTimeout - $testName - state and timeout duration updated",
stateUpdates =
(state: GroupState[Int]) => { state.update(5); state.setTimeoutDuration(5000) },
timeoutConf = ProcessingTimeTimeout,
priorState = priorState,
priorTimeoutTimestamp = priorTimeoutTimestamp,
expectedState = Some(5), // state should change
expectedTimeoutTimestamp = currentBatchTimestamp + 5000) // timestamp should change
testStateUpdateWithData(
s"ProcessingTimeTimeout - $testName - timeout updated after state removed",
stateUpdates = state => { state.remove(); state.setTimeoutDuration(5000) },
timeoutConf = ProcessingTimeTimeout,
priorState = priorState,
priorTimeoutTimestamp = priorTimeoutTimestamp,
expectedState = None,
expectedTimeoutTimestamp = currentBatchTimestamp + 5000)
// Tests with EventTimeTimeout
if (priorState == None) {
testStateUpdateWithData(
s"EventTimeTimeout - $testName - setting timeout without init state not allowed",
stateUpdates = state => {
state.setTimeoutTimestamp(10000)
},
timeoutConf = EventTimeTimeout,
priorState = None,
priorTimeoutTimestamp = priorTimeoutTimestamp,
expectedState = None,
expectedTimeoutTimestamp = 10000)
}
testStateUpdateWithData(
s"EventTimeTimeout - $testName - state and timeout timestamp updated",
stateUpdates =
(state: GroupState[Int]) => { state.update(5); state.setTimeoutTimestamp(5000) },
timeoutConf = EventTimeTimeout,
priorState = priorState,
priorTimeoutTimestamp = priorTimeoutTimestamp,
expectedState = Some(5), // state should change
expectedTimeoutTimestamp = 5000) // timestamp should change
testStateUpdateWithData(
s"EventTimeTimeout - $testName - timeout timestamp updated to before watermark",
stateUpdates =
(state: GroupState[Int]) => {
state.update(5)
intercept[IllegalArgumentException] {
state.setTimeoutTimestamp(currentBatchWatermark - 1) // try to set to < watermark
}
},
timeoutConf = EventTimeTimeout,
priorState = priorState,
priorTimeoutTimestamp = priorTimeoutTimestamp,
expectedState = Some(5), // state should change
expectedTimeoutTimestamp = NO_TIMESTAMP) // timestamp should not update
testStateUpdateWithData(
s"EventTimeTimeout - $testName - setting timeout with state removal not allowed",
stateUpdates = state => {
state.remove(); state.setTimeoutTimestamp(10000)
},
timeoutConf = EventTimeTimeout,
priorState = priorState,
priorTimeoutTimestamp = priorTimeoutTimestamp,
expectedState = None,
expectedTimeoutTimestamp = 10000)
}
}
// Tests for InputProcessor.processTimedOutState()
val preTimeoutState = Some(5)
for (timeoutConf <- Seq(ProcessingTimeTimeout, EventTimeTimeout)) {
testStateUpdateWithTimeout(
s"$timeoutConf - should not timeout",
stateUpdates = state => { assert(false, "function called without timeout") },
timeoutConf = timeoutConf,
priorTimeoutTimestamp = afterTimeoutThreshold,
expectedState = preTimeoutState, // state should not change
expectedTimeoutTimestamp = afterTimeoutThreshold) // timestamp should not change
testStateUpdateWithTimeout(
s"$timeoutConf - should timeout - no update/remove",
stateUpdates = state => {
assert(state.getCurrentProcessingTimeMs() === currentBatchTimestamp)
intercept[Exception] { state.getCurrentWatermarkMs() } // watermark not specified
/* no updates */
},
timeoutConf = timeoutConf,
priorTimeoutTimestamp = beforeTimeoutThreshold,
expectedState = preTimeoutState, // state should not change
expectedTimeoutTimestamp = NO_TIMESTAMP) // timestamp should be reset
testStateUpdateWithTimeout(
s"$timeoutConf - should timeout - update state",
stateUpdates = state => { state.update(5) },
timeoutConf = timeoutConf,
priorTimeoutTimestamp = beforeTimeoutThreshold,
expectedState = Some(5), // state should change
expectedTimeoutTimestamp = NO_TIMESTAMP) // timestamp should be reset
testStateUpdateWithTimeout(
s"$timeoutConf - should timeout - remove state",
stateUpdates = state => { state.remove() },
timeoutConf = timeoutConf,
priorTimeoutTimestamp = beforeTimeoutThreshold,
expectedState = None, // state should be removed
expectedTimeoutTimestamp = NO_TIMESTAMP)
}
testStateUpdateWithTimeout(
"ProcessingTimeTimeout - should timeout - timeout duration updated",
stateUpdates = state => { state.setTimeoutDuration(2000) },
timeoutConf = ProcessingTimeTimeout,
priorTimeoutTimestamp = beforeTimeoutThreshold,
expectedState = preTimeoutState, // state should not change
expectedTimeoutTimestamp = currentBatchTimestamp + 2000) // timestamp should change
testStateUpdateWithTimeout(
"ProcessingTimeTimeout - should timeout - timeout duration and state updated",
stateUpdates = state => { state.update(5); state.setTimeoutDuration(2000) },
timeoutConf = ProcessingTimeTimeout,
priorTimeoutTimestamp = beforeTimeoutThreshold,
expectedState = Some(5), // state should change
expectedTimeoutTimestamp = currentBatchTimestamp + 2000) // timestamp should change
testStateUpdateWithTimeout(
"EventTimeTimeout - should timeout - timeout timestamp updated",
stateUpdates = state => { state.setTimeoutTimestamp(5000) },
timeoutConf = EventTimeTimeout,
priorTimeoutTimestamp = beforeTimeoutThreshold,
expectedState = preTimeoutState, // state should not change
expectedTimeoutTimestamp = 5000) // timestamp should change
testStateUpdateWithTimeout(
"EventTimeTimeout - should timeout - timeout and state updated",
stateUpdates = state => { state.update(5); state.setTimeoutTimestamp(5000) },
timeoutConf = EventTimeTimeout,
priorTimeoutTimestamp = beforeTimeoutThreshold,
expectedState = Some(5), // state should change
expectedTimeoutTimestamp = 5000) // timestamp should change
testWithAllStateVersions("flatMapGroupsWithState - streaming") {
// Function to maintain running count up to 2, and then remove the count
// Returns the data and the count if state is defined, otherwise does not return anything
val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => {
assertCanGetProcessingTime { state.getCurrentProcessingTimeMs() >= 0 }
assertCannotGetWatermark { state.getCurrentWatermarkMs() }
val count = state.getOption.map(_.count).getOrElse(0L) + values.size
if (count == 3) {
state.remove()
Iterator.empty
} else {
state.update(RunningCount(count))
Iterator((key, count.toString))
}
}
val inputData = MemoryStream[String]
val result =
inputData.toDS()
.groupByKey(x => x)
.flatMapGroupsWithState(Update, GroupStateTimeout.NoTimeout)(stateFunc)
testStream(result, Update)(
AddData(inputData, "a"),
CheckNewAnswer(("a", "1")),
assertNumStateRows(total = 1, updated = 1),
AddData(inputData, "a", "b"),
CheckNewAnswer(("a", "2"), ("b", "1")),
assertNumStateRows(total = 2, updated = 2),
StopStream,
StartStream(),
AddData(inputData, "a", "b"), // should remove state for "a" and not return anything for a
CheckNewAnswer(("b", "2")),
assertNumStateRows(total = 1, updated = 2),
StopStream,
StartStream(),
AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 and
CheckNewAnswer(("a", "1"), ("c", "1")),
assertNumStateRows(total = 3, updated = 2)
)
}
test("flatMapGroupsWithState - streaming + func returns iterator that updates state lazily") {
// Function to maintain running count up to 2, and then remove the count
// Returns the data and the count if state is defined, otherwise does not return anything
// Additionally, it updates state lazily as the returned iterator get consumed
val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => {
values.flatMap { _ =>
val count = state.getOption.map(_.count).getOrElse(0L) + 1
if (count == 3) {
state.remove()
None
} else {
state.update(RunningCount(count))
Some((key, count.toString))
}
}
}
val inputData = MemoryStream[String]
val result =
inputData.toDS()
.groupByKey(x => x)
.flatMapGroupsWithState(Update, GroupStateTimeout.NoTimeout)(stateFunc)
testStream(result, Update)(
AddData(inputData, "a", "a", "b"),
CheckNewAnswer(("a", "1"), ("a", "2"), ("b", "1")),
StopStream,
StartStream(),
AddData(inputData, "a", "b"), // should remove state for "a" and not return anything for a
CheckNewAnswer(("b", "2")),
StopStream,
StartStream(),
AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 and
CheckNewAnswer(("a", "1"), ("c", "1"))
)
}
testWithAllStateVersions("flatMapGroupsWithState - streaming + aggregation") {
// Function to maintain running count up to 2, and then remove the count
// Returns the data and the count (-1 if count reached beyond 2 and state was just removed)
val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => {
val count = state.getOption.map(_.count).getOrElse(0L) + values.size
if (count == 3) {
state.remove()
Iterator(key -> "-1")
} else {
state.update(RunningCount(count))
Iterator(key -> count.toString)
}
}
val inputData = MemoryStream[String]
val result =
inputData.toDS()
.groupByKey(x => x)
.flatMapGroupsWithState(Append, GroupStateTimeout.NoTimeout)(stateFunc)
.groupByKey(_._1)
.count()
testStream(result, Complete)(
AddData(inputData, "a"),
CheckNewAnswer(("a", 1)),
AddData(inputData, "a", "b"),
// mapGroups generates ("a", "2"), ("b", "1"); so increases counts of a and b by 1
CheckNewAnswer(("a", 2), ("b", 1)),
StopStream,
StartStream(),
AddData(inputData, "a", "b"),
// mapGroups should remove state for "a" and generate ("a", "-1"), ("b", "2") ;
// so increment a and b by 1
CheckNewAnswer(("a", 3), ("b", 2)),
StopStream,
StartStream(),
AddData(inputData, "a", "c"),
// mapGroups should recreate state for "a" and generate ("a", "1"), ("c", "1") ;
// so increment a and c by 1
CheckNewAnswer(("a", 4), ("b", 2), ("c", 1))
)
}
test("flatMapGroupsWithState - batch") {
// Function that returns running count only if its even, otherwise does not return
val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => {
assertCanGetProcessingTime { state.getCurrentProcessingTimeMs() > 0 }
assertCannotGetWatermark { state.getCurrentWatermarkMs() }
if (state.exists) throw new IllegalArgumentException("state.exists should be false")
Iterator((key, values.size))
}
val df = Seq("a", "a", "b").toDS
.groupByKey(x => x)
.flatMapGroupsWithState(Update, GroupStateTimeout.NoTimeout)(stateFunc).toDF
checkAnswer(df, Seq(("a", 2), ("b", 1)).toDF)
}
testWithAllStateVersions("flatMapGroupsWithState - streaming with processing time timeout") {
// Function to maintain the count as state and set the proc. time timeout delay of 10 seconds.
// It returns the count if changed, or -1 if the state was removed by timeout.
val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => {
assertCanGetProcessingTime { state.getCurrentProcessingTimeMs() >= 0 }
assertCannotGetWatermark { state.getCurrentWatermarkMs() }
if (state.hasTimedOut) {
state.remove()
Iterator((key, "-1"))
} else {
val count = state.getOption.map(_.count).getOrElse(0L) + values.size
state.update(RunningCount(count))
state.setTimeoutDuration("10 seconds")
Iterator((key, count.toString))
}
}
val clock = new StreamManualClock
val inputData = MemoryStream[String]
val result =
inputData.toDS()
.groupByKey(x => x)
.flatMapGroupsWithState(Update, ProcessingTimeTimeout)(stateFunc)
testStream(result, Update)(
StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
AddData(inputData, "a"),
AdvanceManualClock(1 * 1000),
CheckNewAnswer(("a", "1")),
assertNumStateRows(total = 1, updated = 1),
AddData(inputData, "b"),
AdvanceManualClock(1 * 1000),
CheckNewAnswer(("b", "1")),
assertNumStateRows(total = 2, updated = 1),
AddData(inputData, "b"),
AdvanceManualClock(10 * 1000),
CheckNewAnswer(("a", "-1"), ("b", "2")),
assertNumStateRows(total = 1, updated = 2),
StopStream,
StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
AddData(inputData, "c"),
AdvanceManualClock(11 * 1000),
CheckNewAnswer(("b", "-1"), ("c", "1")),
assertNumStateRows(total = 1, updated = 2),
AdvanceManualClock(12 * 1000),
AssertOnQuery { _ => clock.getTimeMillis() == 35000 },
Execute { q =>
failAfter(streamingTimeout) {
while (q.lastProgress.timestamp != "1970-01-01T00:00:35.000Z") {
Thread.sleep(1)
}
}
},
CheckNewAnswer(("c", "-1")),
assertNumStateRows(total = 0, updated = 0)
)
}
testWithAllStateVersions("flatMapGroupsWithState - streaming w/ event time timeout + watermark") {
// Function to maintain the max event time as state and set the timeout timestamp based on the
// current max event time seen. It returns the max event time in the state, or -1 if the state
// was removed by timeout.
val stateFunc = (key: String, values: Iterator[(String, Long)], state: GroupState[Long]) => {
assertCanGetProcessingTime { state.getCurrentProcessingTimeMs() >= 0 }
assertCanGetWatermark { state.getCurrentWatermarkMs() >= -1 }
val timeoutDelaySec = 5
if (state.hasTimedOut) {
state.remove()
Iterator((key, -1))
} else {
val valuesSeq = values.toSeq
val maxEventTimeSec = math.max(valuesSeq.map(_._2).max, state.getOption.getOrElse(0L))
val timeoutTimestampSec = maxEventTimeSec + timeoutDelaySec
state.update(maxEventTimeSec)
state.setTimeoutTimestamp(timeoutTimestampSec * 1000)
Iterator((key, maxEventTimeSec.toInt))
}
}
val inputData = MemoryStream[(String, Int)]
val result =
inputData.toDS
.select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime"))
.withWatermark("eventTime", "10 seconds")
.as[(String, Long)]
.groupByKey(_._1)
.flatMapGroupsWithState(Update, EventTimeTimeout)(stateFunc)
testStream(result, Update)(
StartStream(),
AddData(inputData, ("a", 11), ("a", 13), ("a", 15)),
// Max event time = 15. Timeout timestamp for "a" = 15 + 5 = 20. Watermark = 15 - 10 = 5.
CheckNewAnswer(("a", 15)), // Output = max event time of a
AddData(inputData, ("a", 4)), // Add data older than watermark for "a"
CheckNewAnswer(), // No output as data should get filtered by watermark
AddData(inputData, ("a", 10)), // Add data newer than watermark for "a"
CheckNewAnswer(("a", 15)), // Max event time is still the same
// Timeout timestamp for "a" is still 20 as max event time for "a" is still 15.
// Watermark is still 5 as max event time for all data is still 15.
AddData(inputData, ("b", 31)), // Add data newer than watermark for "b", not "a"
// Watermark = 31 - 10 = 21, so "a" should be timed out as timeout timestamp for "a" is 20.
CheckNewAnswer(("a", -1), ("b", 31)) // State for "a" should timeout and emit -1
)
}
test("flatMapGroupsWithState - uses state format version 2 by default") {
val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => {
val count = state.getOption.map(_.count).getOrElse(0L) + values.size
state.update(RunningCount(count))
Iterator((key, count.toString))
}
val inputData = MemoryStream[String]
val result = inputData.toDS()
.groupByKey(x => x)
.flatMapGroupsWithState(Update, GroupStateTimeout.NoTimeout)(stateFunc)
testStream(result, Update)(
AddData(inputData, "a"),
CheckNewAnswer(("a", "1")),
Execute { query =>
// Verify state format = 2
val f = query.lastExecution.executedPlan.collect { case f: FlatMapGroupsWithStateExec => f }
assert(f.size == 1)
assert(f.head.stateFormatVersion == 2)
}
)
}
test("flatMapGroupsWithState - recovery from checkpoint uses state format version 1") {
// Function to maintain the max event time as state and set the timeout timestamp based on the
// current max event time seen. It returns the max event time in the state, or -1 if the state
// was removed by timeout.
val stateFunc = (key: String, values: Iterator[(String, Long)], state: GroupState[Long]) => {
assertCanGetProcessingTime { state.getCurrentProcessingTimeMs() >= 0 }
assertCanGetWatermark { state.getCurrentWatermarkMs() >= -1 }
val timeoutDelaySec = 5
if (state.hasTimedOut) {
state.remove()
Iterator((key, -1))
} else {
val valuesSeq = values.toSeq
val maxEventTimeSec = math.max(valuesSeq.map(_._2).max, state.getOption.getOrElse(0L))
val timeoutTimestampSec = maxEventTimeSec + timeoutDelaySec
state.update(maxEventTimeSec)
state.setTimeoutTimestamp(timeoutTimestampSec * 1000)
Iterator((key, maxEventTimeSec.toInt))
}
}
val inputData = MemoryStream[(String, Int)]
val result =
inputData.toDS
.select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime"))
.withWatermark("eventTime", "10 seconds")
.as[(String, Long)]
.groupByKey(_._1)
.flatMapGroupsWithState(Update, EventTimeTimeout)(stateFunc)
val resourceUri = this.getClass.getResource(
"/structured-streaming/checkpoint-version-2.3.1-flatMapGroupsWithState-state-format-1/").toURI
val checkpointDir = Utils.createTempDir().getCanonicalFile
// Copy the checkpoint to a temp dir to prevent changes to the original.
// Not doing this will lead to the test passing on the first run, but fail subsequent runs.
FileUtils.copyDirectory(new File(resourceUri), checkpointDir)
inputData.addData(("a", 11), ("a", 13), ("a", 15))
inputData.addData(("a", 4))
testStream(result, Update)(
StartStream(
checkpointLocation = checkpointDir.getAbsolutePath,
additionalConfs = Map(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> "2")),
/*
Note: The checkpoint was generated using the following input in Spark version 2.3.1
AddData(inputData, ("a", 11), ("a", 13), ("a", 15)),
// Max event time = 15. Timeout timestamp for "a" = 15 + 5 = 20. Watermark = 15 - 10 = 5.
CheckNewAnswer(("a", 15)), // Output = max event time of a
AddData(inputData, ("a", 4)), // Add data older than watermark for "a"
CheckNewAnswer(), // No output as data should get filtered by watermark
*/
AddData(inputData, ("a", 10)), // Add data newer than watermark for "a"
CheckNewAnswer(("a", 15)), // Max event time is still the same
// Timeout timestamp for "a" is still 20 as max event time for "a" is still 15.
// Watermark is still 5 as max event time for all data is still 15.
Execute { query =>
// Verify state format = 1
val f = query.lastExecution.executedPlan.collect { case f: FlatMapGroupsWithStateExec => f }
assert(f.size == 1)
assert(f.head.stateFormatVersion == 1)
},
AddData(inputData, ("b", 31)), // Add data newer than watermark for "b", not "a"
// Watermark = 31 - 10 = 21, so "a" should be timed out as timeout timestamp for "a" is 20.
CheckNewAnswer(("a", -1), ("b", 31)) // State for "a" should timeout and emit -1
)
}
test("mapGroupsWithState - streaming") {
// Function to maintain running count up to 2, and then remove the count
// Returns the data and the count (-1 if count reached beyond 2 and state was just removed)
val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => {
assertCanGetProcessingTime { state.getCurrentProcessingTimeMs() >= 0 }
assertCannotGetWatermark { state.getCurrentWatermarkMs() }
val count = state.getOption.map(_.count).getOrElse(0L) + values.size
if (count == 3) {
state.remove()
(key, "-1")
} else {
state.update(RunningCount(count))
(key, count.toString)
}
}
val inputData = MemoryStream[String]
val result =
inputData.toDS()
.groupByKey(x => x)
.mapGroupsWithState(stateFunc) // Types = State: MyState, Out: (Str, Str)
testStream(result, Update)(
AddData(inputData, "a"),
CheckNewAnswer(("a", "1")),
assertNumStateRows(total = 1, updated = 1),
AddData(inputData, "a", "b"),
CheckNewAnswer(("a", "2"), ("b", "1")),
assertNumStateRows(total = 2, updated = 2),
StopStream,
StartStream(),
AddData(inputData, "a", "b"), // should remove state for "a" and return count as -1
CheckNewAnswer(("a", "-1"), ("b", "2")),
assertNumStateRows(total = 1, updated = 2),
StopStream,
StartStream(),
AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1
CheckNewAnswer(("a", "1"), ("c", "1")),
assertNumStateRows(total = 3, updated = 2)
)
}
test("mapGroupsWithState - batch") {