1818package org .apache .spark .sql .execution .streaming .state
1919
2020import java .io ._
21- import java .nio .channels .ClosedChannelException
2221import java .util .Locale
2322
2423import scala .collection .JavaConverters ._
2524import scala .collection .mutable
26- import scala .util .Random
2725import scala .util .control .NonFatal
2826
2927import com .google .common .io .ByteStreams
@@ -280,38 +278,49 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit
280278 if (loadedCurrentVersionMap.isDefined) {
281279 return loadedCurrentVersionMap.get
282280 }
283- val snapshotCurrentVersionMap = readSnapshotFile(version)
284- if (snapshotCurrentVersionMap.isDefined) {
285- synchronized { loadedMaps.put(version, snapshotCurrentVersionMap.get) }
286- return snapshotCurrentVersionMap.get
287- }
288281
289- // Find the most recent map before this version that we can.
290- // [SPARK-22305] This must be done iteratively to avoid stack overflow.
291- var lastAvailableVersion = version
292- var lastAvailableMap : Option [MapType ] = None
293- while (lastAvailableMap.isEmpty) {
294- lastAvailableVersion -= 1
282+ logWarning(s " The state for version $version doesn't exist in loadedMaps. " +
283+ " Reading snapshot file and delta files if needed..." +
284+ " Note that this is normal for the first batch of starting query." )
295285
296- if (lastAvailableVersion <= 0 ) {
297- // Use an empty map for versions 0 or less.
298- lastAvailableMap = Some (new MapType )
299- } else {
300- lastAvailableMap =
301- synchronized { loadedMaps.get(lastAvailableVersion) }
302- .orElse(readSnapshotFile(lastAvailableVersion))
286+ val (result, elapsedMs) = Utils .timeTakenMs {
287+ val snapshotCurrentVersionMap = readSnapshotFile(version)
288+ if (snapshotCurrentVersionMap.isDefined) {
289+ synchronized { loadedMaps.put(version, snapshotCurrentVersionMap.get) }
290+ return snapshotCurrentVersionMap.get
291+ }
292+
293+ // Find the most recent map before this version that we can.
294+ // [SPARK-22305] This must be done iteratively to avoid stack overflow.
295+ var lastAvailableVersion = version
296+ var lastAvailableMap : Option [MapType ] = None
297+ while (lastAvailableMap.isEmpty) {
298+ lastAvailableVersion -= 1
299+
300+ if (lastAvailableVersion <= 0 ) {
301+ // Use an empty map for versions 0 or less.
302+ lastAvailableMap = Some (new MapType )
303+ } else {
304+ lastAvailableMap =
305+ synchronized { loadedMaps.get(lastAvailableVersion) }
306+ .orElse(readSnapshotFile(lastAvailableVersion))
307+ }
308+ }
309+
310+ // Load all the deltas from the version after the last available one up to the target version.
311+ // The last available version is the one with a full snapshot, so it doesn't need deltas.
312+ val resultMap = new MapType (lastAvailableMap.get)
313+ for (deltaVersion <- lastAvailableVersion + 1 to version) {
314+ updateFromDeltaFile(deltaVersion, resultMap)
303315 }
304- }
305316
306- // Load all the deltas from the version after the last available one up to the target version.
307- // The last available version is the one with a full snapshot, so it doesn't need deltas.
308- val resultMap = new MapType (lastAvailableMap.get)
309- for (deltaVersion <- lastAvailableVersion + 1 to version) {
310- updateFromDeltaFile(deltaVersion, resultMap)
317+ synchronized { loadedMaps.put(version, resultMap) }
318+ resultMap
311319 }
312320
313- synchronized { loadedMaps.put(version, resultMap) }
314- resultMap
321+ logWarning(s " Loading state for $version takes $elapsedMs ms. " )
322+
323+ result
315324 }
316325
317326 private def writeUpdateToDeltaFile (
@@ -490,15 +499,18 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit
490499 /** Perform a snapshot of the store to allow delta files to be consolidated */
491500 private def doSnapshot (): Unit = {
492501 try {
493- val files = fetchFiles()
502+ val (files, e1) = Utils .timeTakenMs(fetchFiles())
503+ logDebug(s " fetchFiles() took $e1 ms. " )
504+
494505 if (files.nonEmpty) {
495506 val lastVersion = files.last.version
496507 val deltaFilesForLastVersion =
497508 filesForVersion(files, lastVersion).filter(_.isSnapshot == false )
498509 synchronized { loadedMaps.get(lastVersion) } match {
499510 case Some (map) =>
500511 if (deltaFilesForLastVersion.size > storeConf.minDeltasForSnapshot) {
501- writeSnapshotFile(lastVersion, map)
512+ val (_, e2) = Utils .timeTakenMs(writeSnapshotFile(lastVersion, map))
513+ logDebug(s " writeSnapshotFile() took $e2 ms. " )
502514 }
503515 case None =>
504516 // The last map is not loaded, probably some other instance is in charge
@@ -517,7 +529,9 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit
517529 */
518530 private [state] def cleanup (): Unit = {
519531 try {
520- val files = fetchFiles()
532+ val (files, e1) = Utils .timeTakenMs(fetchFiles())
533+ logDebug(s " fetchFiles() took $e1 ms. " )
534+
521535 if (files.nonEmpty) {
522536 val earliestVersionToRetain = files.last.version - storeConf.minVersionsToRetain
523537 if (earliestVersionToRetain > 0 ) {
@@ -527,9 +541,12 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit
527541 mapsToRemove.foreach(loadedMaps.remove)
528542 }
529543 val filesToDelete = files.filter(_.version < earliestFileToRetain.version)
530- filesToDelete.foreach { f =>
531- fm.delete(f.path)
544+ val (_, e2) = Utils .timeTakenMs {
545+ filesToDelete.foreach { f =>
546+ fm.delete(f.path)
547+ }
532548 }
549+ logDebug(s " deleting files took $e2 ms. " )
533550 logInfo(s " Deleted files older than ${earliestFileToRetain.version} for $this: " +
534551 filesToDelete.mkString(" , " ))
535552 }
0 commit comments