@@ -43,7 +43,7 @@ private[sql] class RocksDBStateStoreProvider
43
43
with SupportsFineGrainedReplay {
44
44
import RocksDBStateStoreProvider ._
45
45
46
- class RocksDBStateStore (lastVersion : Long ) extends StateStore {
46
+ class RocksDBStateStore (lastVersion : Long , val stamp : Long ) extends StateStore {
47
47
/** Trait and classes representing the internal state of the store */
48
48
trait STATE
49
49
case object UPDATING extends STATE
@@ -58,6 +58,10 @@ private[sql] class RocksDBStateStoreProvider
58
58
59
59
@ volatile private var state : STATE = UPDATING
60
60
61
+ override def getReadStamp : Long = {
62
+ stamp
63
+ }
64
+
61
65
/**
62
66
* Validates the expected state, throws exception if state is not as expected.
63
67
* Returns the current state
@@ -81,6 +85,7 @@ private[sql] class RocksDBStateStoreProvider
81
85
private def validateAndTransitionState (transition : TRANSITION ): Unit = {
82
86
val newState = transition match {
83
87
case UPDATE =>
88
+ stateMachine.verifyStamp(stamp)
84
89
state match {
85
90
case UPDATING => UPDATING
86
91
case COMMITTED => throw StateStoreErrors .stateStoreOperationOutOfOrder(
@@ -90,14 +95,18 @@ private[sql] class RocksDBStateStoreProvider
90
95
}
91
96
case ABORT =>
92
97
state match {
93
- case UPDATING => ABORTED
98
+ case UPDATING =>
99
+ stateMachine.verifyStamp(stamp)
100
+ ABORTED
94
101
case COMMITTED => throw StateStoreErrors .stateStoreOperationOutOfOrder(
95
102
" Cannot abort after committed" )
96
103
case ABORTED => ABORTED
97
104
}
98
105
case COMMIT =>
99
106
state match {
100
- case UPDATING => COMMITTED
107
+ case UPDATING =>
108
+ stateMachine.verifyStamp(stamp)
109
+ COMMITTED
101
110
case COMMITTED => throw StateStoreErrors .stateStoreOperationOutOfOrder(
102
111
" Cannot commit after committed" )
103
112
case ABORTED => throw StateStoreErrors .stateStoreOperationOutOfOrder(
@@ -118,10 +127,14 @@ private[sql] class RocksDBStateStoreProvider
118
127
Option (TaskContext .get()).foreach(_.addTaskCompletionListener[Unit ] {
119
128
_ =>
120
129
try {
121
- abort()
130
+ if (state == UPDATING ) {
131
+ abort()
132
+ }
122
133
} catch {
123
134
case NonFatal (e) =>
124
135
logWarning(" Failed to abort state store" , e)
136
+ } finally {
137
+ stateMachine.releaseStore(stamp, throwEx = false )
125
138
}
126
139
})
127
140
@@ -318,15 +331,18 @@ private[sql] class RocksDBStateStoreProvider
318
331
}
319
332
320
333
var checkpointInfo : Option [StateStoreCheckpointInfo ] = None
334
+ private var storedMetrics : Option [RocksDBMetrics ] = None
335
+
321
336
override def commit (): Long = synchronized {
322
337
validateState(List (UPDATING ))
323
-
324
338
try {
325
339
verify(state == UPDATING , " Cannot commit after already committed or aborted" )
326
340
val (newVersion, newCheckpointInfo) = rocksDB.commit()
327
341
checkpointInfo = Some (newCheckpointInfo)
342
+ storedMetrics = rocksDB.metricsOpt
328
343
validateAndTransitionState(COMMIT )
329
- state = COMMITTED
344
+ stateMachine.releaseStore(stamp)
345
+
330
346
logInfo(log " Committed ${MDC (VERSION_NUM , newVersion)} " +
331
347
log " for ${MDC (STATE_STORE_ID , id)}" )
332
348
newVersion
@@ -342,6 +358,7 @@ private[sql] class RocksDBStateStoreProvider
342
358
log " for ${MDC (STATE_STORE_ID , id)}" )
343
359
rocksDB.rollback()
344
360
validateAndTransitionState(ABORT )
361
+ stateMachine.releaseStore(stamp)
345
362
}
346
363
}
347
364
@@ -541,15 +558,26 @@ private[sql] class RocksDBStateStoreProvider
541
558
542
559
override def stateStoreId : StateStoreId = stateStoreId_
543
560
561
+ private lazy val stateMachine : RocksDBStateStoreProviderStateMachine =
562
+ new RocksDBStateStoreProviderStateMachine (stateStoreId, RocksDBConf (storeConf))
563
+
544
564
override def getStore (version : Long , uniqueId : Option [String ] = None ): StateStore = {
545
565
try {
546
566
if (version < 0 ) {
547
567
throw QueryExecutionErrors .unexpectedStateStoreVersion(version)
548
568
}
549
- rocksDB.load(
550
- version,
551
- stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) uniqueId else None )
552
- new RocksDBStateStore (version)
569
+ val stamp = stateMachine.acquireStore()
570
+ try {
571
+ rocksDB.load(
572
+ version,
573
+ stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) uniqueId else None ,
574
+ readOnly = false )
575
+ new RocksDBStateStore (version, stamp)
576
+ } catch {
577
+ case e : Throwable =>
578
+ stateMachine.releaseStore(stamp)
579
+ throw e
580
+ }
553
581
}
554
582
catch {
555
583
case e : SparkException
@@ -564,16 +592,58 @@ private[sql] class RocksDBStateStoreProvider
564
592
}
565
593
}
566
594
567
- override def getReadStore (version : Long , uniqueId : Option [String ] = None ): StateStore = {
595
+ override def getWriteStore (
596
+ readStore : ReadStateStore ,
597
+ version : Long ,
598
+ uniqueId : Option [String ] = None ): StateStore = {
568
599
try {
569
600
if (version < 0 ) {
570
601
throw QueryExecutionErrors .unexpectedStateStoreVersion(version)
571
602
}
572
- rocksDB.load(
573
- version,
574
- stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) uniqueId else None ,
575
- readOnly = true )
576
- new RocksDBStateStore (version)
603
+ assert(version == readStore.version)
604
+ try {
605
+ rocksDB.load(
606
+ version,
607
+ stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) uniqueId else None ,
608
+ readOnly = false )
609
+ readStore match {
610
+ case stateStore : RocksDBStateStore =>
611
+ stateStore
612
+ case _ =>
613
+ throw new IllegalArgumentException
614
+ }
615
+ } catch {
616
+ case e : Throwable =>
617
+ stateMachine.releaseStore(readStore.getReadStamp)
618
+ throw e
619
+ }
620
+ } catch {
621
+ case e : SparkException
622
+ if Option (e.getCondition).exists(_.contains(" CANNOT_LOAD_STATE_STORE" )) =>
623
+ throw e
624
+ case e : OutOfMemoryError =>
625
+ throw QueryExecutionErrors .notEnoughMemoryToLoadStore(
626
+ stateStoreId.toString,
627
+ " ROCKSDB_STORE_PROVIDER" ,
628
+ e)
629
+ case e : Throwable => throw QueryExecutionErrors .cannotLoadStore(e)
630
+ }
631
+ }
632
+
633
+ override def getReadStore (version : Long , uniqueId : Option [String ] = None ): StateStore = {
634
+ try {
635
+ val stamp = stateMachine.acquireStore()
636
+ try {
637
+ rocksDB.load(
638
+ version,
639
+ stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) uniqueId else None ,
640
+ readOnly = true )
641
+ new RocksDBStateStore (version, stamp)
642
+ } catch {
643
+ case e : Throwable =>
644
+ stateMachine.releaseStore(stamp)
645
+ throw e
646
+ }
577
647
}
578
648
catch {
579
649
case e : SparkException
@@ -590,6 +660,7 @@ private[sql] class RocksDBStateStoreProvider
590
660
591
661
override def doMaintenance (): Unit = {
592
662
try {
663
+ stateMachine.maintenanceStore()
593
664
rocksDB.doMaintenance()
594
665
} catch {
595
666
// SPARK-46547 - Swallow non-fatal exception in maintenance task to avoid deadlock between
@@ -601,6 +672,7 @@ private[sql] class RocksDBStateStoreProvider
601
672
}
602
673
603
674
override def close (): Unit = {
675
+ stateMachine.closeStore()
604
676
rocksDB.close()
605
677
}
606
678
@@ -657,8 +729,15 @@ private[sql] class RocksDBStateStoreProvider
657
729
if (endVersion < snapshotVersion) {
658
730
throw QueryExecutionErrors .unexpectedStateStoreVersion(endVersion)
659
731
}
660
- rocksDB.loadFromSnapshot(snapshotVersion, endVersion)
661
- new RocksDBStateStore (endVersion)
732
+ val stamp = stateMachine.acquireStore()
733
+ try {
734
+ rocksDB.loadFromSnapshot(snapshotVersion, endVersion)
735
+ new RocksDBStateStore (endVersion, stamp)
736
+ } catch {
737
+ case e : Throwable =>
738
+ stateMachine.releaseStore(stamp)
739
+ throw e
740
+ }
662
741
}
663
742
catch {
664
743
case e : OutOfMemoryError =>
0 commit comments