18
18
package org .apache .spark .sql .execution .streaming .state
19
19
20
20
import java .util .UUID
21
- import java .util .concurrent .{ScheduledFuture , TimeUnit }
21
+ import java .util .concurrent .{ConcurrentLinkedQueue , ScheduledFuture , TimeUnit }
22
22
import javax .annotation .concurrent .GuardedBy
23
23
24
24
import scala .collection .mutable
25
+ import scala .collection .mutable .ArrayBuffer
25
26
import scala .util .control .NonFatal
26
27
27
28
import org .apache .hadoop .conf .Configuration
@@ -31,13 +32,14 @@ import org.json4s.JsonAST.JValue
31
32
import org .json4s .JsonDSL ._
32
33
import org .json4s .jackson .JsonMethods .{compact , render }
33
34
34
- import org .apache .spark .{SparkContext , SparkEnv , SparkException }
35
+ import org .apache .spark .{SparkContext , SparkEnv , SparkException , TaskContext }
35
36
import org .apache .spark .internal .{Logging , LogKeys , MDC }
36
37
import org .apache .spark .sql .catalyst .expressions .UnsafeRow
37
38
import org .apache .spark .sql .catalyst .util .UnsafeRowUtils
38
39
import org .apache .spark .sql .errors .QueryExecutionErrors
39
40
import org .apache .spark .sql .execution .metric .{SQLMetric , SQLMetrics }
40
41
import org .apache .spark .sql .execution .streaming .{StatefulOperatorStateInfo , StreamExecution }
42
+ import org .apache .spark .sql .execution .streaming .state .MaintenanceTaskType ._
41
43
import org .apache .spark .sql .types .StructType
42
44
import org .apache .spark .util .{NextIterator , ThreadUtils , Utils }
43
45
@@ -53,6 +55,14 @@ object StateStoreEncoding {
53
55
case object Avro extends StateStoreEncoding
54
56
}
55
57
58
+ sealed trait MaintenanceTaskType
59
+
60
+ object MaintenanceTaskType {
61
+ case object FromUnloadedProvidersQueue extends MaintenanceTaskType
62
+ case object FromTaskThread extends MaintenanceTaskType
63
+ case object FromLoadedProviders extends MaintenanceTaskType
64
+ }
65
+
56
66
/**
57
67
* Base trait for a versioned key-value store which provides read operations. Each instance of a
58
68
* `ReadStateStore` represents a specific version of state data, and such instances are created
@@ -554,7 +564,11 @@ trait StateStoreProvider {
554
564
*/
555
565
def stateStoreId : StateStoreId
556
566
557
- /** Called when the provider instance is unloaded from the executor */
567
+ /**
568
+ * Called when the provider instance is unloaded from the executor
569
+ * WARNING: IF PROVIDER FROM [[StateStore.loadedProviders ]],
570
+ * CLOSE MUST ONLY BE CALLED FROM MAINTENANCE THREAD!
571
+ */
558
572
def close (): Unit
559
573
560
574
/**
@@ -843,6 +857,9 @@ object StateStore extends Logging {
843
857
844
858
private val maintenanceThreadPoolLock = new Object
845
859
860
+ private val unloadedProvidersToClose =
861
+ new ConcurrentLinkedQueue [(StateStoreProviderId , StateStoreProvider )]
862
+
846
863
// This set is to keep track of the partitions that are queued
847
864
// for maintenance or currently have maintenance running on them
848
865
// to prevent the same partition from being processed concurrently.
@@ -1012,7 +1029,21 @@ object StateStore extends Logging {
1012
1029
if (! storeConf.unloadOnCommit) {
1013
1030
val otherProviderIds = loadedProviders.keys.filter(_ != storeProviderId).toSeq
1014
1031
val providerIdsToUnload = reportActiveStoreInstance(storeProviderId, otherProviderIds)
1015
- providerIdsToUnload.foreach(unload(_))
1032
+ val taskContextIdLogLine = Option (TaskContext .get()).map { tc =>
1033
+ log " taskId= ${MDC (LogKeys .TASK_ID , tc.taskAttemptId())}"
1034
+ }.getOrElse(log " " )
1035
+ providerIdsToUnload.foreach(id => {
1036
+ loadedProviders.remove(id).foreach( provider => {
1037
+ // Trigger maintenance thread to immediately do maintenance on and close the provider.
1038
+ // Doing maintenance first allows us to do maintenance for a constantly-moving state
1039
+ // store.
1040
+ logInfo(log " Submitted maintenance from task thread to close " +
1041
+ log " provider= ${MDC (LogKeys .STATE_STORE_PROVIDER_ID , id)}. " + taskContextIdLogLine +
1042
+ log " Removed provider from loadedProviders " )
1043
+ submitMaintenanceWorkForProvider(
1044
+ id, provider, storeConf, MaintenanceTaskType .FromTaskThread )
1045
+ })
1046
+ })
1016
1047
}
1017
1048
1018
1049
provider
@@ -1029,14 +1060,30 @@ object StateStore extends Logging {
1029
1060
}
1030
1061
}
1031
1062
1032
- /** Unload a state store provider */
1033
- def unload (storeProviderId : StateStoreProviderId ): Unit = loadedProviders.synchronized {
1034
- loadedProviders.remove(storeProviderId).foreach(_.close())
1063
+ /**
1064
+ * Unload a state store provider.
1065
+ * If alreadyRemovedFromLoadedProviders is None, provider will be
1066
+ * removed from loadedProviders and closed.
1067
+ * If alreadyRemovedFromLoadedProviders is Some, provider will be closed
1068
+ * using passed in provider.
1069
+ * WARNING: CAN ONLY BE CALLED FROM MAINTENANCE THREAD!
1070
+ */
1071
+ def removeFromLoadedProvidersAndClose (
1072
+ storeProviderId : StateStoreProviderId ,
1073
+ alreadyRemovedProvider : Option [StateStoreProvider ] = None ): Unit = {
1074
+ val providerToClose = alreadyRemovedProvider.orElse {
1075
+ loadedProviders.synchronized {
1076
+ loadedProviders.remove(storeProviderId)
1077
+ }
1078
+ }
1079
+ providerToClose.foreach { provider =>
1080
+ provider.close()
1081
+ }
1035
1082
}
1036
1083
1037
1084
/** Unload all state store providers: unit test purpose */
1038
1085
private [sql] def unloadAll (): Unit = loadedProviders.synchronized {
1039
- loadedProviders.keySet.foreach { key => unload (key) }
1086
+ loadedProviders.keySet.foreach { key => removeFromLoadedProvidersAndClose (key) }
1040
1087
loadedProviders.clear()
1041
1088
}
1042
1089
@@ -1075,7 +1122,7 @@ object StateStore extends Logging {
1075
1122
1076
1123
/** Unload and stop all state store providers */
1077
1124
def stop (): Unit = loadedProviders.synchronized {
1078
- loadedProviders.keySet.foreach { key => unload (key) }
1125
+ loadedProviders.keySet.foreach { key => removeFromLoadedProvidersAndClose (key) }
1079
1126
loadedProviders.clear()
1080
1127
_coordRef = null
1081
1128
stopMaintenanceTask()
@@ -1090,7 +1137,7 @@ object StateStore extends Logging {
1090
1137
if (SparkEnv .get != null && ! isMaintenanceRunning && ! storeConf.unloadOnCommit) {
1091
1138
maintenanceTask = new MaintenanceTask (
1092
1139
storeConf.maintenanceInterval,
1093
- task = { doMaintenance() }
1140
+ task = { doMaintenance(storeConf ) }
1094
1141
)
1095
1142
maintenanceThreadPool = new MaintenanceThreadPool (numMaintenanceThreads,
1096
1143
maintenanceShutdownTimeout)
@@ -1099,6 +1146,27 @@ object StateStore extends Logging {
1099
1146
}
1100
1147
}
1101
1148
1149
+ // Wait until this partition can be processed
1150
+ private def awaitProcessThisPartition (
1151
+ id : StateStoreProviderId ,
1152
+ timeoutMs : Long ): Boolean = maintenanceThreadPoolLock synchronized {
1153
+ val startTime = System .currentTimeMillis()
1154
+ val endTime = startTime + timeoutMs
1155
+
1156
+ // If immediate processing fails, wait with timeout
1157
+ var canProcessThisPartition = processThisPartition(id)
1158
+ while (! canProcessThisPartition && System .currentTimeMillis() < endTime) {
1159
+ maintenanceThreadPoolLock.wait(timeoutMs)
1160
+ canProcessThisPartition = processThisPartition(id)
1161
+ }
1162
+ val elapsedTime = System .currentTimeMillis() - startTime
1163
+ logInfo(log " Waited for ${MDC (LogKeys .TOTAL_TIME , elapsedTime)} ms to be able to process " +
1164
+ log " maintenance for partition ${MDC (LogKeys .STATE_STORE_PROVIDER_ID , id)}" )
1165
+ canProcessThisPartition
1166
+ }
1167
+
1168
+ private def doMaintenance (): Unit = doMaintenance(StateStoreConf .empty)
1169
+
1102
1170
private def processThisPartition (id : StateStoreProviderId ): Boolean = {
1103
1171
maintenanceThreadPoolLock.synchronized {
1104
1172
if (! maintenancePartitions.contains(id)) {
@@ -1114,56 +1182,42 @@ object StateStore extends Logging {
1114
1182
* Execute background maintenance task in all the loaded store providers if they are still
1115
1183
* the active instances according to the coordinator.
1116
1184
*/
1117
- private def doMaintenance (): Unit = {
1185
+ private def doMaintenance (storeConf : StateStoreConf ): Unit = {
1118
1186
logDebug(" Doing maintenance" )
1119
1187
if (SparkEnv .get == null ) {
1120
1188
throw new IllegalStateException (" SparkEnv not active, cannot do maintenance on StateStores" )
1121
1189
}
1190
+
1191
+ // Providers that couldn't be processed now and need to be added back to the queue
1192
+ val providersToRequeue = new ArrayBuffer [(StateStoreProviderId , StateStoreProvider )]()
1193
+
1194
+ // unloadedProvidersToClose are StateStoreProviders that have been removed from
1195
+ // loadedProviders, and can now be processed for maintenance. This queue contains
1196
+ // providers for which we weren't able to process for maintenance on the previous iteration
1197
+ while (! unloadedProvidersToClose.isEmpty) {
1198
+ val (providerId, provider) = unloadedProvidersToClose.poll()
1199
+
1200
+ if (processThisPartition(providerId)) {
1201
+ submitMaintenanceWorkForProvider(
1202
+ providerId, provider, storeConf, MaintenanceTaskType .FromUnloadedProvidersQueue )
1203
+ } else {
1204
+ providersToRequeue += ((providerId, provider))
1205
+ }
1206
+ }
1207
+
1208
+ if (providersToRequeue.nonEmpty) {
1209
+ logInfo(log " Had to requeue ${MDC (LogKeys .SIZE , providersToRequeue.size)} providers " +
1210
+ log " for maintenance in doMaintenance " )
1211
+ }
1212
+
1213
+ providersToRequeue.foreach(unloadedProvidersToClose.offer)
1214
+
1122
1215
loadedProviders.synchronized {
1123
1216
loadedProviders.toSeq
1124
1217
}.foreach { case (id, provider) =>
1125
1218
if (processThisPartition(id)) {
1126
- maintenanceThreadPool.execute(() => {
1127
- val startTime = System .currentTimeMillis()
1128
- try {
1129
- provider.doMaintenance()
1130
- if (! verifyIfStoreInstanceActive(id)) {
1131
- unload(id)
1132
- logInfo(log " Unloaded ${MDC (LogKeys .STATE_STORE_PROVIDER , provider)}" )
1133
- }
1134
- } catch {
1135
- case NonFatal (e) =>
1136
- logWarning(log " Error managing ${MDC (LogKeys .STATE_STORE_PROVIDER , provider)}, " +
1137
- log " unloading state store provider " , e)
1138
- // When we get a non-fatal exception, we just unload the provider.
1139
- //
1140
- // By not bubbling the exception to the maintenance task thread or the query execution
1141
- // thread, it's possible for a maintenance thread pool task to continue failing on
1142
- // the same partition. Additionally, if there is some global issue that will cause
1143
- // all maintenance thread pool tasks to fail, then bubbling the exception and
1144
- // stopping the pool is faster than waiting for all tasks to see the same exception.
1145
- //
1146
- // However, we assume that repeated failures on the same partition and global issues
1147
- // are rare. The benefit to unloading just the partition with an exception is that
1148
- // transient issues on a given provider do not affect any other providers; so, in
1149
- // most cases, this should be a more performant solution.
1150
- unload(id)
1151
- } finally {
1152
- val duration = System .currentTimeMillis() - startTime
1153
- val logMsg =
1154
- log " Finished maintenance task for " +
1155
- log " provider= ${MDC (LogKeys .STATE_STORE_PROVIDER_ID , id)}" +
1156
- log " in elapsed_time= ${MDC (LogKeys .TIME_UNITS , duration)}\n "
1157
- if (duration > 5000 ) {
1158
- logInfo(logMsg)
1159
- } else {
1160
- logDebug(logMsg)
1161
- }
1162
- maintenanceThreadPoolLock.synchronized {
1163
- maintenancePartitions.remove(id)
1164
- }
1165
- }
1166
- })
1219
+ submitMaintenanceWorkForProvider(
1220
+ id, provider, storeConf, MaintenanceTaskType .FromLoadedProviders )
1167
1221
} else {
1168
1222
logInfo(log " Not processing partition ${MDC (LogKeys .PARTITION_ID , id)} " +
1169
1223
log " for maintenance because it is currently " +
@@ -1172,6 +1226,108 @@ object StateStore extends Logging {
1172
1226
}
1173
1227
}
1174
1228
1229
+ /**
1230
+ * Submits maintenance work for a provider to the maintenance thread pool.
1231
+ *
1232
+ * @param id The StateStore provider ID to perform maintenance on
1233
+ * @param provider The StateStore provider instance
1234
+ */
1235
+ private def submitMaintenanceWorkForProvider (
1236
+ id : StateStoreProviderId ,
1237
+ provider : StateStoreProvider ,
1238
+ storeConf : StateStoreConf ,
1239
+ source : MaintenanceTaskType = FromLoadedProviders ): Unit = {
1240
+ maintenanceThreadPool.execute(() => {
1241
+ val startTime = System .currentTimeMillis()
1242
+ // Determine if we can process this partition based on the source
1243
+ val canProcessThisPartition = source match {
1244
+ case FromTaskThread =>
1245
+ // Provider from task thread needs to wait for lock
1246
+ // We potentially need to wait for ongoing maintenance to finish processing
1247
+ // this partition
1248
+ val timeoutMs = storeConf.stateStoreMaintenanceProcessingTimeout * 1000
1249
+ val ableToProcessNow = awaitProcessThisPartition(id, timeoutMs)
1250
+ if (! ableToProcessNow) {
1251
+ // Add to queue for later processing if we can't process now
1252
+ // This will be resubmitted for maintenance later by the background maintenance task
1253
+ unloadedProvidersToClose.add((id, provider))
1254
+ }
1255
+ ableToProcessNow
1256
+
1257
+ case FromUnloadedProvidersQueue =>
1258
+ // Provider from queue can be processed immediately
1259
+ // (we've already removed it from loadedProviders)
1260
+ true
1261
+
1262
+ case FromLoadedProviders =>
1263
+ // Provider from loadedProviders can be processed immediately
1264
+ // as it's in maintenancePartitions
1265
+ true
1266
+ }
1267
+
1268
+ if (canProcessThisPartition) {
1269
+ val awaitingPartitionDuration = System .currentTimeMillis() - startTime
1270
+ try {
1271
+ provider.doMaintenance()
1272
+ // Handle unloading based on source
1273
+ source match {
1274
+ case FromTaskThread | FromUnloadedProvidersQueue =>
1275
+ // Provider already removed from loadedProviders, just close it
1276
+ removeFromLoadedProvidersAndClose(id, Some (provider))
1277
+
1278
+ case FromLoadedProviders =>
1279
+ // Check if provider should be unloaded
1280
+ if (! verifyIfStoreInstanceActive(id)) {
1281
+ removeFromLoadedProvidersAndClose(id)
1282
+ }
1283
+ }
1284
+ logInfo(log " Unloaded ${MDC (LogKeys .STATE_STORE_PROVIDER_ID , id)}" )
1285
+ } catch {
1286
+ case NonFatal (e) =>
1287
+ logWarning(log " Error doing maintenance on provider: " +
1288
+ log " ${MDC (LogKeys .STATE_STORE_PROVIDER_ID , id)}. " +
1289
+ log " Could not unload state store provider " , e)
1290
+ // When we get a non-fatal exception, we just unload the provider.
1291
+ //
1292
+ // By not bubbling the exception to the maintenance task thread or the query execution
1293
+ // thread, it's possible for a maintenance thread pool task to continue failing on
1294
+ // the same partition. Additionally, if there is some global issue that will cause
1295
+ // all maintenance thread pool tasks to fail, then bubbling the exception and
1296
+ // stopping the pool is faster than waiting for all tasks to see the same exception.
1297
+ //
1298
+ // However, we assume that repeated failures on the same partition and global issues
1299
+ // are rare. The benefit to unloading just the partition with an exception is that
1300
+ // transient issues on a given provider do not affect any other providers; so, in
1301
+ // most cases, this should be a more performant solution.
1302
+ source match {
1303
+ case FromTaskThread | FromUnloadedProvidersQueue =>
1304
+ removeFromLoadedProvidersAndClose(id, Some (provider))
1305
+
1306
+ case FromLoadedProviders =>
1307
+ removeFromLoadedProvidersAndClose(id)
1308
+ }
1309
+ } finally {
1310
+ val duration = System .currentTimeMillis() - startTime
1311
+ val logMsg =
1312
+ log " Finished maintenance task for " +
1313
+ log " provider= ${MDC (LogKeys .STATE_STORE_PROVIDER_ID , id)}" +
1314
+ log " in elapsed_time= ${MDC (LogKeys .TIME_UNITS , duration)}" +
1315
+ log " and awaiting_partition_time= " +
1316
+ log " ${MDC (LogKeys .TIME_UNITS , awaitingPartitionDuration)}\n "
1317
+ if (duration > 5000 ) {
1318
+ logInfo(logMsg)
1319
+ } else {
1320
+ logDebug(logMsg)
1321
+ }
1322
+ maintenanceThreadPoolLock.synchronized {
1323
+ maintenancePartitions.remove(id)
1324
+ maintenanceThreadPoolLock.notifyAll()
1325
+ }
1326
+ }
1327
+ }
1328
+ })
1329
+ }
1330
+
1175
1331
private def reportActiveStoreInstance (
1176
1332
storeProviderId : StateStoreProviderId ,
1177
1333
otherProviderIds : Seq [StateStoreProviderId ]): Seq [StateStoreProviderId ] = {
0 commit comments