You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by squito <gi...@git.apache.org> on 2015/12/08 19:01:39 UTC

[GitHub] spark pull request: [SPARK-7889] [CORE] HistoryServer to refresh c...

Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/6935#discussion_r46989032
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala ---
    @@ -0,0 +1,648 @@
    +/*
    + * 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.util.NoSuchElementException
    +
    +import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
    +import javax.servlet.{DispatcherType, Filter, FilterChain, FilterConfig, ServletException, ServletRequest, ServletResponse}
    +
    +import scala.collection.JavaConverters._
    +
    +import com.codahale.metrics.{Counter, MetricRegistry, Timer}
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache, RemovalListener, RemovalNotification}
    +import org.eclipse.jetty.servlet.FilterHolder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.metrics.source.Source
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Clock
    +
    +/**
    + * Cache for applications.
    + *
    + * Completed applications are cached for as long as there is capacity for them.
    + * Incompleted applications have their update time checked on every
    + * retrieval; if the cached entry is out of date, it is refreshed.
    + *
    + * @note there must be only one instance of [[ApplicationCache]] in a
    + * JVM at a time. This is because a static field in [[ApplicationCacheCheckFilterRelay]]
    + * keeps a reference to the cache so that HTTP requests on the attempt-specific web UIs
    + * can probe the current cache to see if the attempts have changed.
    + *
    + * Creating multiple instances will break this routing.
    + * @param operations implementation of record access operations
    + * @param refreshInterval interval between refreshes in milliseconds.
    + * @param retainedApplications number of retained applications
    + * @param clock time source
    + */
    +private[history] class ApplicationCache(val operations: ApplicationCacheOperations,
    +    val refreshInterval: Long,
    +    val retainedApplications: Int,
    +    val clock: Clock) extends Logging {
    +
    +  /**
    +   * Services the load request from the cache.
    +   */
    +  private val appLoader = new CacheLoader[CacheKey, CacheEntry] {
    +
    +    /** the cache key doesn't match an cached entry ... attempt to load it  */
    +    override def load(key: CacheKey): CacheEntry = {
    +      loadApplicationEntry(key.appId, key.attemptId)
    +    }
    +
    +  }
    +
    +  /**
    +   * Handler for callbacks from the cache of entry removal.
    +   */
    +  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().ui)
    +    }
    +  }
    +
    +  /**
    +   * The cache of applications.
    +   *
    +   * Tagged as `protected` so as to allow subclasses in tests to accesss it directly
    +   */
    +  protected val appCache: LoadingCache[CacheKey, CacheEntry] = CacheBuilder.newBuilder()
    +      .maximumSize(retainedApplications)
    +      .removalListener(removalListener)
    +      .build(appLoader)
    +
    +  /**
    +   * The metrics which are updated as the cache is used
    +   */
    +  val metrics = new CacheMetrics("history.cache")
    +
    +  init()
    +
    +  /**
    +   * Perform any startup operations.
    +   *
    +   * This includes declaring this instance as the cache to use in the
    +   * [[ApplicationCacheCheckFilterRelay]].
    +   */
    +  private def init(): Unit = {
    +    ApplicationCacheCheckFilterRelay.setApplicationCache(this)
    +  }
    +
    +  /**
    +   * Stop the cache.
    +   * This will reset the relay in [[ApplicationCacheCheckFilterRelay]].
    +   */
    +  def stop(): Unit = {
    +    ApplicationCacheCheckFilterRelay.resetApplicationCache()
    +  }
    +
    +  /**
    +   * Get an entry. Cache fetch/refresh will have taken place by
    +   * the time this method returns.
    +   * @param appAndAttempt application to look up in the format needed by the history server web UI,
    +   *                      `appId/attemptId` or `appId`.
    +   * @return the entry
    +   */
    +  def get(appAndAttempt: String): SparkUI = {
    +    val parts = splitAppAndAttemptKey(appAndAttempt)
    +    get(parts._1, parts._2)
    +  }
    +
    +  /**
    +   * Get the associated spark UI. Cache fetch/refresh will have taken place by
    +   * the time this method returns.
    +   * @param appId application ID
    +   * @param attemptId optional attempt ID
    +   * @return the entry
    +   */
    +  def get(appId: String, attemptId: Option[String]): SparkUI = {
    +    lookupAndUpdate(appId, attemptId)._1.ui
    +  }
    +
    +  /**
    +   * Look up the entry; update it if needed.
    +   * @param appId application ID
    +   * @param attemptId optional attempt ID
    +   * @return the underlying cache entry -which can have its timestamp changed, and a flag to
    +   *         indicate that the entry has changed
    +   */
    +  private def lookupAndUpdate(appId: String, attemptId: Option[String]): (CacheEntry, Boolean) = {
    +    metrics.lookupCount.inc()
    +    val cacheKey = CacheKey(appId, attemptId)
    +    var entry = appCache.getIfPresent(cacheKey)
    +    var updated = false
    +    if (entry == null) {
    +      // no entry, so fetch without any post-fetch probes for out-of-dateness
    +      entry = appCache.get(cacheKey)
    +    } else if (!entry.completed) {
    +      val now = clock.getTimeMillis()
    +      if (now - entry.probeTime > refreshInterval) {
    +        log.debug(s"Probing @time $now for updated application $cacheKey -> $entry")
    +        metrics.updateProbeCount.inc()
    +        updated = time(metrics.updateProbeTimer) {
    +          operations.isUpdated(appId, attemptId, entry.loadTime, entry.data)
    +        }
    +        if (updated) {
    +          logDebug(s"refreshing $cacheKey")
    +          metrics.updateTriggeredCount.inc()
    +          appCache.refresh(cacheKey)
    +          // and repeat the lookup
    +          entry = appCache.get(cacheKey)
    +        } else {
    +          // update the probe timestamp to the current time
    +          entry.probeTime = now
    +        }
    +      }
    +    }
    +    (entry, updated)
    +  }
    +
    +  /**
    +   * This method is visible for testing. It looks up the cached entry *and returns a clone of it*.
    +   * This ensures that the cached entries never leak
    +   * @param appId application ID
    +   * @param attemptId optional attempt ID
    +   * @return a new entry with shared SparkUI, but copies of the other fields.
    +   */
    +  def lookupCacheEntry(appId: String, attemptId: Option[String]): CacheEntry = {
    +    val entry = lookupAndUpdate(appId, attemptId)._1
    +    new CacheEntry(entry.ui, entry.completed, entry.data, entry.loadTime, entry.probeTime)
    +  }
    +
    +  /**
    +   * Probe for an application being updated
    +   * @param appId application ID
    +   * @param attemptId attempt ID
    +   * @return true if an update has been triggered
    +   */
    +  def checkForUpdates(appId: String, attemptId: Option[String]): Boolean = {
    +    val (entry, updated) = lookupAndUpdate(appId, attemptId)
    +    updated
    +  }
    +
    +  /**
    +   * Size probe, primarily for testing
    +   * @return size
    +   */
    +  def size(): Long = appCache.size()
    +
    +  /**
    +   * Emptiness predicate, primarily for testing
    +   * @return true if the cache is empty
    +   */
    +  def isEmpty: Boolean = appCache.size() == 0
    +
    +  /**
    +   * Time a closure, returning its output.
    +   * @param t timer
    +   * @param f function
    +   * @tparam T type of return value of time
    +   * @return the result of the function.
    +   */
    +  private def time[T](t: Timer)(f: => T): T = {
    +    val timeCtx = t.time()
    +    try {
    +      f
    +    } finally {
    +      timeCtx.close()
    +    }
    +  }
    +
    +  /**
    +   * Load the Spark UI via [[ApplicationCacheOperations.getAppUI()]],
    +   * then attach it to the web UI via [[ApplicationCacheOperations.attachSparkUI()]].
    +   * If the application is incomplete, it has the [[ApplicationCacheCheckFilter]]
    +   * added as a filter to the HTTP requests, so that queries on the UI will trigger
    +   * update checks.
    +   *
    +   * The generated entry contains the UI and the current timestamp.
    +   * The timer [[metrics.loadTimer]] tracks the time taken to load the UI.
    +   *
    +   * @param appId application ID
    +   * @param attemptId optional attempt ID
    +   * @return the cache entry
    +   * @throws NoSuchElementException if there is no matching element
    +   */
    +  @throws[NoSuchElementException]
    +  def loadApplicationEntry(appId: String, attemptId: Option[String]): CacheEntry = {
    +
    +    logDebug(s"Loading application Entry $appId/$attemptId")
    +    metrics.loadCount.inc()
    +    time(metrics.loadTimer) {
    +      operations.getAppUI(appId, attemptId) match {
    +        case Some(LoadedAppUI(ui, loadTime, data)) =>
    +          val completed = ui.getApplicationInfoList.exists(_.attempts.last.completed)
    +          if (completed) {
    +            // completed spark UIs are attached directly
    +            operations.attachSparkUI(appId, attemptId, ui, completed)
    +          } else {
    +            // incomplete UIs have the cache-check filter put in front of them.
    +            ApplicationCacheCheckFilterRelay.registerFilter(ui, appId, attemptId)
    +            operations.attachSparkUI(appId, attemptId, ui, completed)
    +          }
    +          // build the cache entry
    +          val now = clock.getTimeMillis()
    +          val entry = new CacheEntry(ui, completed, data, if (loadTime> 0) loadTime else now , now)
    +          logDebug(s"Loaded application $appId/$attemptId -> $entry")
    +          entry
    +        case None =>
    +          metrics.lookupFailureCount.inc()
    +          // guava's cache logs via java.util log, so is of limited use. Hence: our own message
    +          logInfo(s"Failed to load application attempt $appId/$attemptId")
    +          throw new NoSuchElementException(s"no application with application Id '$appId'" +
    +              attemptId.map { id => s" attemptId '$id'" }.getOrElse(" and no attempt Id"))
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Split up an `applicationId/attemptId` or `applicationId` key into the separate pieces.
    +   *
    +   * @param appAndAttempt combined key
    +   * @return a tuple of the application ID and, if present, the attemptID
    +   */
    +  def splitAppAndAttemptKey(appAndAttempt: String): (String, Option[String]) = {
    +    val parts = appAndAttempt.split("/")
    +    require(parts.length == 1 || parts.length == 2, s"Invalid app key $appAndAttempt")
    +    val appId = parts(0)
    +    val attemptId = if (parts.length > 1) Some(parts(1)) else None
    +    (appId, attemptId)
    +  }
    +
    +  /**
    +   * Merge an appId and optional attempt Id into a key of the form `applicationId/attemptId`
    +   * if there is an `attemptId`; `applicationId` if not.
    +   * @param appId application ID
    +   * @param attemptId optional attempt ID
    +   * @return a unified string
    +   */
    +  def mergeAppAndAttemptToKey(appId: String, attemptId: Option[String]): String = {
    +    appId + attemptId.map { id => s"/$id" }.getOrElse("")
    +  }
    +
    +  /**
    +   * String operator dumps the cache entries and metrics.
    +   * @return a string value, primarily for testing and diagnostics
    +   */
    +  override def toString: String = {
    +    val sb = new StringBuilder(s"ApplicationCache(refreshInterval=$refreshInterval," +
    +          s" retainedApplications= $retainedApplications)")
    +    sb.append(s"; time= ${clock.getTimeMillis()}")
    +    sb.append(s"; entry count= ${appCache.size()}\n")
    +    sb.append("----\n")
    +    appCache.asMap().asScala.foreach {
    +      case(key, entry) => sb.append(s"  $key -> $entry\n")
    +    }
    +    sb.append("----\n")
    +    sb.append(metrics)
    +    sb.append("----\n")
    +    sb.toString()
    +  }
    +}
    +
    +/**
    + * An entry in the cache.
    + *
    + * @param ui Spark UI
    + * @param completed: flag to indicated that the application has completed (and so
    + *                 does not need refreshing)
    + * @param probeTime timestamp in milliseconds. This may be updated during probes
    + */
    +private[history] final class CacheEntry(val ui: SparkUI, val completed: Boolean,
    +    val data: Option[HistoryProviderUpdateState],
    +    val loadTime: Long,
    +    var probeTime: Long) {
    +
    +  /** string value is for test assertions */
    +  override def toString: String = {
    +    s"UI $ui, data=$data, completed=$completed, loadTime=$loadTime probeTime=$probeTime"
    +  }
    +}
    +
    +/**
    + * Cache key: compares on App Id and then, if non-empty, attemptId.
    + * The [[hashCode()]] function uses the same fields.
    + * @param appId application ID
    + * @param attemptId attempt ID
    + */
    +private[history] final case class CacheKey(appId: String, attemptId: Option[String]) {
    +
    +  override def hashCode(): Int = {
    +    appId.hashCode() + attemptId.map(_.hashCode).getOrElse(0)
    +  }
    +
    +  override def equals(obj: Any): Boolean = {
    +    val that = obj.asInstanceOf[CacheKey]
    +    that.appId == appId && that.attemptId == attemptId
    +  }
    +
    +  override def toString: String = {
    +    appId + attemptId.map { id => s"/$id" }.getOrElse("")
    +  }
    +}
    +
    +private[history] class CacheMetrics(prefix: String) extends Source {
    +
    +  /* metrics: counters and timers */
    +  val lookupCount = new Counter()
    +  val lookupFailureCount = new Counter()
    +  val evictionCount = new Counter()
    +  val loadCount = new Counter()
    +  val loadTimer = new Timer()
    +  val updateProbeCount = new Counter()
    +  val updateProbeTimer = new Timer()
    +  val updateTriggeredCount = new Counter()
    +
    +  /** all the counters: for registration and string conversion. */
    +  private val counters = Seq(
    +    ("lookup.count", lookupCount),
    +    ("lookup.failure.count", lookupFailureCount),
    +    ("eviction.count", evictionCount),
    +    ("load.count", loadCount),
    +    ("update.probe.count", updateProbeCount),
    +    ("update.triggered.count", updateTriggeredCount))
    +
    +  /** all metrics, including timers */
    +  private val allMetrics = counters ++ Seq(
    +    ("load.timer", loadTimer),
    +    ("update.probe.timer", updateProbeTimer))
    +
    +  /**
    +   * Name of metric source
    +   */
    +  override val sourceName = "ApplicationCache"
    +
    +  override def metricRegistry: MetricRegistry = new MetricRegistry
    --- End diff --
    
    I think you want to store this in a `val` and not return a new one on every call


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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