redsanket commented on a change in pull request #28412:
URL: https://github.com/apache/spark/pull/28412#discussion_r437529862



##########
File path: 
core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
##########
@@ -1197,6 +1213,71 @@ private[history] class FsHistoryProvider(conf: 
SparkConf, clock: Clock)
     KVUtils.open(newStorePath, metadata)
   }
 
+  private def createHybridStore(
+      dm: HistoryServerDiskManager,
+      appId: String,
+      attempt: AttemptInfoWrapper,
+      metadata: AppStatusStoreMetadata): KVStore = {
+
+    val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath),
+      attempt.lastIndex)
+    val isCompressed = reader.compressionCodec.isDefined
+
+    val memoryUsage = approximateMemoryUsage(reader.totalSize, isCompressed)
+    if (currentInMemoryStoreUsage.get + memoryUsage > maxInMemoryStoreUsage) {
+      throw new IllegalStateException("Not enough in-memory storage to create 
hybrid store.")
+    }
+    currentInMemoryStoreUsage.addAndGet(memoryUsage)
+    logInfo(s"Attempt creating hybrid store to parse $appId / 
${attempt.info.attemptId}. " +
+      s"Requested ${Utils.bytesToString(memoryUsage)} in-memory storage 
quota.")
+
+    logInfo(s"Leasing disk manager space for app $appId / 
${attempt.info.attemptId}...")
+    val lease = dm.lease(reader.totalSize, isCompressed)
+    val isLeaseRolledBack = new 
java.util.concurrent.atomic.AtomicBoolean(false)
+    var store: HybridStore = null
+    try {
+      store = new HybridStore()
+      val levelDB = KVUtils.open(lease.tmpPath, metadata)
+      store.setLevelDB(levelDB)
+      store.setCachedQuantileKlass(classOf[CachedQuantile])
+      rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime())
+
+      // Start the background thread to dump data to levelDB when writing to
+      // InMemoryStore is completed.
+      store.switchingToLevelDB(new HybridStore.SwitchingToLevelDBListener {
+        override def onSwitchingToLevelDBSuccess: Unit = {
+          levelDB.close()
+          val newStorePath = lease.commit(appId, attempt.info.attemptId)
+          store.setLevelDB(KVUtils.open(newStorePath, metadata))
+          currentInMemoryStoreUsage.addAndGet(-memoryUsage)
+          logInfo(s"Completely switched to leveldb for app" +
+            s" $appId / ${attempt.info.attemptId}. " +
+            s"Released ${Utils.bytesToString(memoryUsage)} in-memory storage 
quota.")
+        }
+
+        override def onSwitchingToLevelDBFail(e: Exception): Unit = {
+          logWarning(s"Failed to switch to leveldb for app" +
+          s" $appId / ${attempt.info.attemptId}", e)
+          levelDB.close()
+          if (!isLeaseRolledBack.getAndSet(true)) {
+            lease.rollback()
+          }
+        }
+      })
+
+      store
+    } catch {
+      case e: Exception =>
+        store.close()
+        currentInMemoryStoreUsage.addAndGet(-memoryUsage)

Review comment:
       Ah missed it thanks

##########
File path: 
core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
##########
@@ -1167,6 +1172,17 @@ private[history] class FsHistoryProvider(conf: 
SparkConf, clock: Clock)
     // At this point the disk data either does not exist or was deleted 
because it failed to
     // load, so the event log needs to be replayed.
 
+    // If hybrid store is enabled, try it first.
+    if (hybridStoreEnabled) {
+      try {
+        return createHybridStore(dm, appId, attempt, metadata)
+      } catch {
+        case e: Exception =>
+          logInfo(s"Failed to create hybrid store for 
$appId/${attempt.info.attemptId}." +

Review comment:
       Should this be warning? may be having it as error would not be 
neccessary as leveldb would work?

##########
File path: core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.deploy.history
+
+import java.io.IOException
+import java.util.Collection
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicBoolean
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.util.kvstore._
+
+/**
+ * A implementation of KVStore that accelerates event logs loading.
+ *
+ * When rebuilding the application state from event logs, HybridStore will
+ * write data to InMemoryStore at first and use a background thread to dump
+ * data to LevelDB once the writing to InMemoryStore is completed.
+ */
+
+private[history] class HybridStore extends KVStore {
+
+  private val inMemoryStore = new InMemoryStore()
+
+  private var levelDB: LevelDB = null
+
+  // Flag to indicate whether we should use inMemoryStore or levelDB
+  private[history] val shouldUseInMemoryStore = new AtomicBoolean(true)
+
+  // A background thread that dumps data from inMemoryStore to levelDB
+  private var backgroundThread: Thread = null
+
+  // A hash map that stores all classes that had been writen to inMemoryStore
+  private val klassMap = new ConcurrentHashMap[Class[_], Boolean]
+
+  override def getMetadata[T](klass: Class[T]): T = {
+    getStore().getMetadata(klass)
+  }
+
+  override def setMetadata(value: Object): Unit = {
+    getStore().setMetadata(value)
+  }
+
+  override def read[T](klass: Class[T], naturalKey: Object): T = {
+    getStore().read(klass, naturalKey)
+  }
+
+  override def write(value: Object): Unit = {
+    val store = getStore()
+    store.write(value)
+
+    if (backgroundThread == null) {
+      // New classes won't be dumped once the background thread is started
+      klassMap.putIfAbsent(value.getClass(), true)
+    }
+  }
+
+  override def delete(klass: Class[_], naturalKey: Object): Unit = {
+    if (backgroundThread != null) {
+      throw new RuntimeException("delete() shouldn't be called after " +

Review comment:
       +1

##########
File path: 
core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
##########
@@ -1197,6 +1213,78 @@ private[history] class FsHistoryProvider(conf: 
SparkConf, clock: Clock)
     KVUtils.open(newStorePath, metadata)
   }
 
+  private def createHybridStore(
+      dm: HistoryServerDiskManager,
+      appId: String,
+      attempt: AttemptInfoWrapper,
+      metadata: AppStatusStoreMetadata): KVStore = {
+
+    var retried = false
+    var hybridStore: HybridStore = null
+    while (hybridStore == null) {
+      val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath),
+        attempt.lastIndex)
+      val isCompressed = reader.compressionCodec.isDefined
+
+      // Throws an exception if the memory space is not enough
+      memoryManager.lease(appId, attempt.info.attemptId, reader.totalSize, 
isCompressed)
+
+      logInfo(s"Leasing disk manager space for app $appId / 
${attempt.info.attemptId}...")
+      val lease = dm.lease(reader.totalSize, isCompressed)
+      val isLeaseRolledBack = new 
java.util.concurrent.atomic.AtomicBoolean(false)
+      var store: HybridStore = null
+      try {
+        store = new HybridStore()
+        val levelDB = KVUtils.open(lease.tmpPath, metadata)
+        store.setLevelDB(levelDB)
+        rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime())
+
+        // Start the background thread to dump data to levelDB when writing to
+        // InMemoryStore is completed.
+        store.switchToLevelDB(new HybridStore.SwitchToLevelDBListener {
+          override def onSwitchToLevelDBSuccess: Unit = {
+            levelDB.close()
+            val newStorePath = lease.commit(appId, attempt.info.attemptId)
+            store.setLevelDB(KVUtils.open(newStorePath, metadata))
+            memoryManager.release(appId, attempt.info.attemptId)
+            logInfo(s"Completely switched to LevelDB for app $appId / 
${attempt.info.attemptId}.")
+          }
+
+          override def onSwitchToLevelDBFail(e: Exception): Unit = {
+            levelDB.close()
+            if (!isLeaseRolledBack.getAndSet(true)) {
+              lease.rollback()
+            }

Review comment:
       Yeah LeaseRolledBack seems uneccessary

##########
File path: 
core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
##########
@@ -1197,6 +1213,78 @@ private[history] class FsHistoryProvider(conf: 
SparkConf, clock: Clock)
     KVUtils.open(newStorePath, metadata)
   }
 
+  private def createHybridStore(
+      dm: HistoryServerDiskManager,
+      appId: String,
+      attempt: AttemptInfoWrapper,
+      metadata: AppStatusStoreMetadata): KVStore = {
+
+    var retried = false
+    var hybridStore: HybridStore = null
+    while (hybridStore == null) {
+      val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath),
+        attempt.lastIndex)
+      val isCompressed = reader.compressionCodec.isDefined
+
+      // Throws an exception if the memory space is not enough
+      memoryManager.lease(appId, attempt.info.attemptId, reader.totalSize, 
isCompressed)
+
+      logInfo(s"Leasing disk manager space for app $appId / 
${attempt.info.attemptId}...")
+      val lease = dm.lease(reader.totalSize, isCompressed)
+      val isLeaseRolledBack = new 
java.util.concurrent.atomic.AtomicBoolean(false)
+      var store: HybridStore = null
+      try {
+        store = new HybridStore()
+        val levelDB = KVUtils.open(lease.tmpPath, metadata)
+        store.setLevelDB(levelDB)
+        rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime())
+
+        // Start the background thread to dump data to levelDB when writing to
+        // InMemoryStore is completed.
+        store.switchToLevelDB(new HybridStore.SwitchToLevelDBListener {
+          override def onSwitchToLevelDBSuccess: Unit = {
+            levelDB.close()
+            val newStorePath = lease.commit(appId, attempt.info.attemptId)
+            store.setLevelDB(KVUtils.open(newStorePath, metadata))
+            memoryManager.release(appId, attempt.info.attemptId)
+            logInfo(s"Completely switched to LevelDB for app $appId / 
${attempt.info.attemptId}.")
+          }
+
+          override def onSwitchToLevelDBFail(e: Exception): Unit = {
+            levelDB.close()
+            if (!isLeaseRolledBack.getAndSet(true)) {
+              lease.rollback()
+            }

Review comment:
       Yeah isLeaseRolledBack seems uneccessary

##########
File path: core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.deploy.history
+
+import java.io.IOException
+import java.util.Collection
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicBoolean
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.util.kvstore._
+
+/**
+ * A implementation of KVStore that accelerates event logs loading.
+ *
+ * When rebuilding the application state from event logs, HybridStore will
+ * write data to InMemoryStore at first and use a background thread to dump
+ * data to LevelDB once the writing to InMemoryStore is completed.
+ */
+
+private[history] class HybridStore extends KVStore {
+
+  private val inMemoryStore = new InMemoryStore()
+
+  private var levelDB: LevelDB = null
+
+  // Flag to indicate whether we should use inMemoryStore or levelDB
+  private[history] val shouldUseInMemoryStore = new AtomicBoolean(true)
+
+  // A background thread that dumps data from inMemoryStore to levelDB
+  private var backgroundThread: Thread = null
+
+  // A hash map that stores all classes that had been writen to inMemoryStore
+  private val klassMap = new ConcurrentHashMap[Class[_], Boolean]
+
+  override def getMetadata[T](klass: Class[T]): T = {
+    getStore().getMetadata(klass)
+  }
+
+  override def setMetadata(value: Object): Unit = {
+    getStore().setMetadata(value)
+  }
+
+  override def read[T](klass: Class[T], naturalKey: Object): T = {
+    getStore().read(klass, naturalKey)
+  }
+
+  override def write(value: Object): Unit = {
+    val store = getStore()
+    store.write(value)
+
+    if (backgroundThread == null) {
+      // New classes won't be dumped once the background thread is started
+      klassMap.putIfAbsent(value.getClass(), true)

Review comment:
       Do we expect a write after a background thread has started? We might 
want to throw an IIlegalStateException




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to