You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "zhouyifan279 (via GitHub)" <gi...@apache.org> on 2023/05/10 08:28:52 UTC

[GitHub] [spark] zhouyifan279 commented on pull request #41105: [SPARK-43403][UI] Ensure old SparkUI in HistoryServer has been detached before loading new one

zhouyifan279 commented on PR #41105:
URL: https://github.com/apache/spark/pull/41105#issuecomment-1541571622

   Here are some key code segments:
   
   1. Detaching of old SparkUI is called when it is removed from `appCache`
   
   **ApplicationCache**
   ```scala
   private val removalListener = new RemovalListener[CacheKey, CacheEntry] {
   
       /**
        * Removal event notifies the provider to detach the UI.
        * @param rm removal notification
        */
       override def onRemoval(rm: RemovalNotification[CacheKey, CacheEntry]): Unit = {
         metrics.evictionCount.inc()
         val key = rm.getKey
         logDebug(s"Evicting entry ${key}")
         operations.detachSparkUI(key.appId, key.attemptId, rm.getValue().loadedUI.ui)
       }
     }
   ```
   2. Execution of `removalListener#onRemoval` is not guarded by `appCache`'s lock
   
   **com.google.common.cache.LocalCache.Segment**
   ```java
       V lockedGetOrLoad(K key, int hash, CacheLoader<? super K, V> loader)
           throws ExecutionException {
         ReferenceEntry<K, V> e;
         ValueReference<K, V> valueReference = null;
         LoadingValueReference<K, V> loadingValueReference = null;
         boolean createNewEntry = true;
   
         lock();
         try {
            ...
         } finally {
           unlock();
           // `removalListener#onRemoval` executes inside postWriteCleanup()
           postWriteCleanup();
         }
         ...
       }
   ```
   
   3. New SparkUI is detached during detaching of old SparkUI
   
   **FsHistoryProvider**
   ```scala
     override def onUIDetached(appId: String, attemptId: Option[String], ui: SparkUI): Unit = {
       val uiOption = synchronized {
         // After new SparkUI is loaded, `activeUIs` contains new SparkUI
         activeUIs.remove((appId, attemptId))
       }
       uiOption.foreach { loadedUI =>
         loadedUI.lock.writeLock().lock()
         try {
           loadedUI.ui.store.close()
         } finally {
           loadedUI.lock.writeLock().unlock()
         }
   
         diskManager.foreach { dm =>
           // If the UI is not valid, delete its files from disk, if any. This relies on the fact that
           // ApplicationCache will never call this method concurrently with getAppUI() for the same
           // appId / attemptId.
           dm.release(appId, attemptId, delete = !loadedUI.valid)
         }
       }
     }
   ```


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org