diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
index 0bc0cb1c15eb..a0af60879e5c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -228,6 +228,7 @@ object HistoryServer extends Logging {
val providerName = conf.getOption("spark.history.provider")
.getOrElse(classOf[FsHistoryProvider].getName())
+ logInfo(s"History provider class: $providerName")
val provider = Utils.classForName(providerName)
.getConstructor(classOf[SparkConf])
.newInstance(conf)
@@ -254,6 +255,8 @@ object HistoryServer extends Logging {
val principalName = conf.get("spark.history.kerberos.principal")
val keytabFilename = conf.get("spark.history.kerberos.keytab")
SparkHadoopUtil.get.loginUserFromKeytab(principalName, keytabFilename)
+ } else {
+ logDebug(s"Kerberos is not enabled")
}
}
diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index b796a44fe01a..9c86f5270142 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -157,6 +157,7 @@ private[spark] object JettyUtils extends Logging {
beforeRedirect(request)
// Make sure we don't end up with "//" in the middle
val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString
+ logDebug(s"Redirecting ${request.getRequestURL} to $newUrl")
response.sendRedirect(newUrl)
}
// SPARK-5983 ensure TRACE is not supported
diff --git a/docs/monitoring.md b/docs/monitoring.md
index cedceb295802..c6b1938739f1 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -74,9 +74,8 @@ follows:
spark.history.provider
org.apache.spark.deploy.history.FsHistoryProvider
-
Name of the class implementing the application history backend. Currently there is only
- one implementation, provided by Spark, which looks for application logs stored in the
- file system.
+
Name of the class implementing the application history backend. The default implementation,
+ is the FsHistoryProvider, retrieves application logs stored in the file system.
spark.history.fs.logDirectory
@@ -256,6 +255,271 @@ still required, though there is only one application available. Eg. to see the
running app, you would go to `http://localhost:4040/api/v1/applications/[app-id]/jobs`. This is to
keep the paths consistent in both modes.
+## Hadoop YARN Timeline service history provider
+
+As well as the Filesystem History Provider, Spark can integrate with the Hadoop YARN
+"Application Timeline Service". This is a service which runs in a YARN cluster, recording
+application- and YARN- published events to a database, retrieving them on request.
+
+Spark integrates with the timeline service by
+1. Publishing events to the timeline service as applications execute.
+1. Listing application histories published to the timeline service.
+1. Retrieving the details of specific application histories.
+
+### Configuring the Timeline Service
+
+For details on configuring and starting the timeline service, consult the Hadoop documentation.
+
+From the perspective of Spark, the key requirements are
+1. The YARN timeline service must be running.
+1. Its URL is known, and configured in the `yarn-site.xml` configuration file.
+1. The user has an Kerberos credentials required to interact with the service.
+
+The timeline service URL must be declared in the property `yarn.timeline-service.webapp.address`,
+or, if HTTPS is the protocol, `yarn.timeline-service.webapp.https.address`
+
+The choice between HTTP and HTTPS is made on the value of `yarn.http.policy`, which can be one of
+`http-only` (default), `https_only` or `http_and_https`; HTTP will be used unless the policy
+is `https_only`.
+
+Examples:
+
+
+
+ yarn.timeline-service.webapp.address
+ atshost.example.org:8188
+
+
+
+ yarn.timeline-service.enabled
+ true
+
+
+The root web page of the timeline service can be verified with a web browser,
+as an easy check that the service is live.
+
+### Saving Application History to the YARN Timeline Service
+
+To publish to the YARN Timeline Service, Spark applications executed in a YARN cluster
+must be configured to instantiate the `YarnHistoryService`. This is done
+by setting the spark configuration property `spark.yarn.services`
+to `org.apache.spark.deploy.history.yarn.YarnHistoryService`
+
+ spark.yarn.services org.apache.spark.deploy.history.yarn.YarnHistoryService
+
+Notes
+
+1. If the class-name is mis-spelled or cannot be instantiated, an error message will
+be logged; the application will still run.
+2. YARN history publishing can run alongside the filesystem history listener; both
+histories can be viewed by an appropriately configured history service.
+3. If the timeline service is disabled, that is `yarn.timeline-service.enabled` is not
+`true`, then the history will not be published: the application will still run.
+4. Similarly, in a cluster where the timeline service is disabled, the history server
+will simply show an empty history, while warning that the history service is disabled.
+5. In a secure cluster, the user must have the Kerberos credentials to interact
+with the timeline server. Being logged in via `kinit` or a keytab should suffice.
+6. If the application is killed it will be listed as incompleted. In an application
+started as a `--master yarn-client` this happens if the client process is stopped
+with a `kill -9` or process failure).
+Similarly, an application started with `--master yarn-cluster` will remain incompleted
+if killed without warning, if it fails, or it is killed via the `yarn kill` command.
+
+
+Specific configuration options:
+
+
+
Property Name
Default
Meaning
+
+
spark.hadoop.yarn.timeline.domain
+
+
+ If UI permissions are set through `spark.acls.enable` or `spark.ui.acls.enable` being true,
+ the optional name of a predefined timeline domain to use . If unset,
+ a value is created programmatically.
+
+
+
+
spark.hadoop.yarn.timeline.post.retry.interval
+
1s
+
+ Interval in milliseconds between POST retries. Every
+ failure adds another delay of this interval before the next retry
+ attempt. That is, first 1s, then 2s, 3s, ...
+
+
+
+
spark.hadoop.yarn.timeline.batch.size
+
3
+
+ How many events to batch up before submitting them to the timeline service.
+ This is a performance optimization.
+
+
+
+
spark.hadoop.yarn.timeline.post.limit
+
1000
+
+ Limit on number of queued events to posts. When exceeded
+ new events will be dropped. This is to place a limit on how much
+ memory will be consumed if the timeline server goes offline.
+
+
+
+
spark.hadoop.yarn.timeline.shutdown.waittime
+
30s
+
+ Maximum time in to wait for event posting to complete when the service stops.
+
+
+
+
spark.hadoop.yarn.timeline.listen
+
true
+
+ This flag exists for testing: if `false` the history publishing
+ service will not register for events with the spark context. As
+ a result, lifecycle events will not be picked up.
+
+
+
+
+
+### Viewing Application Histories via the YARN Timeline Service
+
+To retrieve and display history information in the YARN Timeline Service, the Spark history server must
+be configured to query the timeline service for the lists of running and completed applications.
+
+Note that the history server does not actually need to be deployed within the Hadoop cluster itself —it
+simply needs access to the REST API offered by the timeline service.
+
+To switch to the timeline history, set `spark.history.provider` to
+`org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider`:
+
+ spark.history.provider org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider
+
+The Timeline Server bindings in `yarn-site.xml` will also be needed, so that the YARN history
+provider can retrieve events from the YARN Timeline Service.
+
+The history provider retrieves data from the timeline service
+
+1. On startup.
+1. In the background at an interval set by the option
+ `spark.history.yarn.backround.refresh.interval`.
+1. When an HTTP request to the web UI or REST API is made and there has not been
+any update to the history view in the interval defined by
+`spark.history.yarn.manual.refresh.interval`. This triggers an asynchronous update,
+so the results are not visible in the HTTP request which triggered the update.
+
+The reason for this design is that in large YARN clusters, frequent polling of
+the timeline server to probe for updated applications can place excessive load
+on a shared resource.
+
+Together options can offer different policies. Here are some examples, two
+at the extremes and some more balanced
+
+#### Background refresh only; updated every minute
+
+ spark.history.yarn.backround.refresh.interval = 300s
+ spark.history.yarn.manual.refresh.interval = 0s
+
+The history is updated in the background, once a minute. The smaller
+the refresh interval, the higher the load on the timeline server.
+
+#### Manual refresh only; minimum interval one minute
+
+ spark.history.yarn.backround.refresh.interval = 0s
+ spark.history.yarn.manual.refresh.interval = 60s
+
+There is no backgroud update; a manual page refresh will trigger an asynchronous refresh.
+
+To get the most current history data, try refreshing the page more than once:
+the first to trigger the fetch of the latest data; the second to view the updated history.
+
+This configuration places no load on the YARN Timeline Service when there
+is no user of the Spark History Service, at the cost of a slightly less
+intuitive UI: because two refreshes are needed to get the updated information,
+the state of the system will not be immediately obvious.
+
+
+#### Manual and background refresh: responsive
+
+ spark.history.yarn.backround.refresh.interval = 60s
+ spark.history.yarn.manual.refresh.interval = 20s
+
+Here the background refresh interval is 60s, but a page refresh will trigger
+an update if the last refresh was more than 20 seconds ago.
+
+#### Manual and background refresh: low-load
+
+ spark.history.yarn.backround.refresh.interval = 300s
+ spark.history.yarn.manual.refresh.interval = 60s
+
+Here a background update takes place every five minutes; a refresh is
+also triggered on a page refresh if there has not been one in the last minute.
+
+What makes for the best configuration? It depends on cluster size. The smaller the cluster,
+the smaller the background refresh interval can be -and the manual refresh interval then set to zero,
+to disable that option entirely.
+
+#### YARN History Provider Configuration Options:
+
+
+
Property Name
Default
Meaning
+
+
spark.history.yarn.window.limit
+
24h
+
+ The earliest time to look for events. The default value is 24 hours.
+ This property limits the amount of data queried off the YARN timeline server;
+ applications started before this window will not be checked to see if they have completed.
+ Set this to 0 for no limits (and increased load on the timeline server).
+
+
+
+
spark.history.yarn.backround.refresh.interval
+
60s
+
+ The interval between background refreshes of the history data.
+ A value of 0s means "no background updates: manual refreshes only".
+
+
+
+
spark.history.yarn.manual.refresh.interval
+
30s
+
+ Minimum interval between manual refreshes of the history data; refreshing the
+ page before this limit will not trigger an update.
+ A value of 0s means "page refreshes do not trigger updates of the application list"
+
+
+
+
spark.history.yarn.event-fetch-limit
+
1000
+
+ Maximum number of application histories to fetch
+ from the timeline server in a single GET request.
+
+
+
+
spark.history.yarn.diagnostics
+
false
+
+ A flag to indicate whether low-level diagnostics information should be included in
+ status pages. This is for debugging and diagnostics.
+
+
+
+
+
spark.history.yarn.probe.running.applications
+
true
+
+ Should the history provider query the YARN Resource Manager to verify that
+ incompleted applications are actually still running.
+
+
+
+
+
# Metrics
Spark has a configurable metrics system based on the
diff --git a/pom.xml b/pom.xml
index 284c219519bc..b3899add146d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -212,6 +212,7 @@
64m512m512m
+ scalastyle-config.xml
@@ -695,6 +696,11 @@
jackson-annotations${fasterxml.jackson.version}
+
+ com.fasterxml.jackson.module
+ jackson-module-jaxb-annotations
+ ${fasterxml.jackson.version}
+
@@ -2270,7 +2276,7 @@
false${basedir}/src/main/scala${basedir}/src/test/scala
- scalastyle-config.xml
+ ${scalastyle.path}${basedir}/target/scalastyle-output.xml${project.build.sourceEncoding}${project.reporting.outputEncoding}
diff --git a/yarn/pom.xml b/yarn/pom.xml
index a8c122fd40a1..ad4dd159bd9f 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -187,8 +187,113 @@
libfb303test
+
+ com.fasterxml.jackson.module
+ jackson-module-jaxb-annotations
+ test
+
+
+
+
+ hadoop-2.6
+
+
+ com.sun.jersey
+ jersey-core
+
+
+ com.sun.jersey
+ jersey-json
+
+
+ com.sun.jersey
+ jersey-server
+
+
+
+
+
+ org.codehaus.mojo
+ build-helper-maven-plugin
+
+
+ add-source
+ generate-sources
+
+ add-source
+
+
+
+ src/history/main/scala
+
+
+
+
+ add-test-source
+ generate-test-sources
+
+ add-test-source
+
+
+
+ src/history/test/scala
+
+
+
+
+ add-test-resource
+ generate-test-resources
+
+ add-test-resource
+
+
+
+
+ src/history/test/resources
+
+
+
+
+
+
+
+ org.scalastyle
+ scalastyle-maven-plugin
+
+ false
+ true
+
+ true
+ false
+
+ ${basedir}/src/main/scala
+ ${basedir}/src/history/main/scala
+
+
+ ${basedir}/src/test/scala
+ ${basedir}/src/history/test/scala
+
+ ${basedir}/src/test/scala
+ ${scalastyle.path}
+ ${basedir}/target/scalastyle-output.xml
+ ${project.build.sourceEncoding}
+ ${project.reporting.outputEncoding}
+
+
+
+
+ check
+
+
+
+
+
+
+
+
+
target/scala-${scala.binary.version}/classestarget/scala-${scala.binary.version}/test-classes
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/ExtendedMetricsSource.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/ExtendedMetricsSource.scala
new file mode 100644
index 000000000000..ea4137f2a95b
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/ExtendedMetricsSource.scala
@@ -0,0 +1,61 @@
+/*
+ * 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.yarn
+
+import com.codahale.metrics.{Metric, Timer}
+
+import org.apache.spark.metrics.source.Source
+
+/**
+ * An extended metrics source with some operations to build up the registry, and
+ * to time a closure.
+ */
+private[history] trait ExtendedMetricsSource extends Source {
+
+ /**
+ * A map to build up of all metrics to register and include in the string value
+ * @return
+ */
+ def metricsMap: Map[String, Metric]
+
+ protected def init(): Unit = {
+ metricsMap.foreach(elt => metricRegistry.register(elt._1, elt._2))
+ }
+
+ override def toString: String = {
+ def sb = new StringBuilder()
+ metricsMap.foreach(elt => sb.append(s" ${elt._1} = ${elt._2}\n"))
+ sb.toString()
+ }
+
+ /**
+ * Time a closure, returning its output.
+ * @param t timer
+ * @param f function
+ * @tparam T type of return value of the function
+ * @return the result of the function.
+ */
+ def time[T](t: Timer)(f: => T): T = {
+ val timeCtx = t.time()
+ try {
+ f
+ } finally {
+ timeCtx.close()
+ }
+ }
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/YarnEventListener.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/YarnEventListener.scala
new file mode 100644
index 000000000000..14c02a97a20e
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/YarnEventListener.scala
@@ -0,0 +1,42 @@
+/*
+ * 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.yarn
+
+import org.apache.spark.{Logging, SparkContext, SparkFirehoseListener}
+import org.apache.spark.scheduler.SparkListenerEvent
+
+/**
+ * Spark listener which queues up all received events to the [[YarnHistoryService]] passed
+ * as a constructor. There's no attempt to filter event types at this point.
+ *
+ * @param sc context
+ * @param service service to forward events to
+ */
+private[spark] class YarnEventListener(sc: SparkContext, service: YarnHistoryService)
+ extends SparkFirehoseListener with Logging {
+
+ /**
+ * queue the event with the service, timestamped to the current time.
+ *
+ * @param event event to queue
+ */
+ override def onEvent(event: SparkListenerEvent): Unit = {
+ service.enqueue(event)
+ }
+
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/YarnHistoryService.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/YarnHistoryService.scala
new file mode 100644
index 000000000000..0edc4d41a893
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/YarnHistoryService.scala
@@ -0,0 +1,1322 @@
+/*
+ * 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.yarn
+
+import java.io.InterruptedIOException
+import java.net.{ConnectException, URI}
+import java.util.concurrent.{LinkedBlockingDeque, TimeUnit}
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicLong}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import com.codahale.metrics.{Metric, Counter, MetricRegistry, Timer}
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
+import org.apache.hadoop.yarn.api.records.timeline.{TimelineDomain, TimelineEntity, TimelineEvent}
+import org.apache.hadoop.yarn.client.api.TimelineClient
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+
+import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.scheduler.{SparkListenerApplicationEnd, SparkListenerApplicationStart, SparkListenerBlockUpdated, SparkListenerEvent, SparkListenerExecutorMetricsUpdate}
+import org.apache.spark.scheduler.cluster.{SchedulerExtensionService, SchedulerExtensionServiceBinding}
+import org.apache.spark.util.{SystemClock, Utils}
+
+/**
+ * A Yarn Extension Service to post lifecycle events to a registered YARN Timeline Server.
+ *
+ * Posting algorithm
+ *
+ * 1. The service subscribes to all events coming from the Spark Context.
+ * 1. These events are serialized into JSON objects for publishing to the timeline service through
+ * HTTP(S) posts.
+ * 1. Events are buffered into `pendingEvents` until a batch is aggregated into a
+ * [[TimelineEntity]] for posting.
+ * 1. That aggregation happens when a lifecycle event (application start/stop) takes place,
+ * or the number of pending events in a running application exceeds the limit set in
+ * `spark.hadoop.yarn.timeline.batch.size`.
+ * 1. Posting operations take place in a separate thread from the spark event listener.
+ * 1. If an attempt to post to the timeline server fails, the service sleeps and then
+ * it is re-attempted after the retry period defined by
+ * `spark.hadoop.yarn.timeline.post.retry.interval`.
+ * 1. If the number of events buffered in the history service exceed the limit set in
+ * `spark.hadoop.yarn.timeline.post.limit`, then further events other than application start/stop
+ * are dropped.
+ * 1. When the service is stopped, it will make a best-effort attempt to post all queued events.
+ * the call of [[stop()]] can block up to the duration of
+ * `spark.hadoop.yarn.timeline.shutdown.waittime` for this to take place.
+ * 1. No events are posted until the service receives a [[SparkListenerApplicationStart]] event.
+ *
+ * If the spark context has a metrics registry, then the internal counters of queued entities,
+ * post failures and successes, and the performance of the posting operation are all registered
+ * as metrics.
+ *
+ * The shutdown logic is somewhat convoluted, as the posting thread may be blocked on HTTP IO
+ * when the shutdown process begins. In this situation, the thread continues to be blocked, and
+ * will be interrupted once the wait time has expired. All time consumed during the ongoing
+ * operation will be counted as part of the shutdown time period.
+ */
+private[spark] class YarnHistoryService extends SchedulerExtensionService with Logging {
+
+ import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+
+ /** Simple state model implemented in an atomic integer. */
+ private val _serviceState = new AtomicInteger(CreatedState)
+
+ /** Get the current state. */
+ def serviceState: Int = {
+ _serviceState.get()
+ }
+
+ /**
+ * Atomic operatin to enter a new state, returning the old one.
+ * There are no checks on state model.
+ * @param state new state
+ * @return previous state
+ */
+ private def enterState(state: Int): Int = {
+ logDebug(s"Entering state $state from $serviceState")
+ _serviceState.getAndSet(state)
+ }
+
+ /** Spark context; valid once started. */
+ private var sparkContext: SparkContext = _
+
+ /** YARN configuration from the spark context. */
+ private var config: YarnConfiguration = _
+
+ /** Application ID. */
+ private[yarn] var applicationId: ApplicationId = _
+
+ /** Attempt ID -this will be null if the service is started in yarn-client mode. */
+ private var attemptId: Option[ApplicationAttemptId] = None
+
+ /** YARN timeline client. */
+ private var _timelineClient: Option[TimelineClient] = None
+
+ /** Registered event listener. */
+ private var listener: Option[YarnEventListener] = None
+
+ /** Application name from the spark start event. */
+ private var applicationName: String = _
+
+ /** Application ID received from a [[SparkListenerApplicationStart]]. */
+ private var sparkApplicationId: Option[String] = None
+
+ /** Optional Attempt ID string from [[SparkListenerApplicationStart]]. */
+ private var sparkApplicationAttemptId: Option[String] = None
+
+ /** User name as derived from `SPARK_USER` env var or [[Utils]]. */
+ private var userName = Utils.getCurrentUserName
+
+ /** Clock for recording time */
+ private val clock = new SystemClock
+
+ /** Start time of the application, as received in the start event. */
+ private var startTime: Long = _
+
+ /** Start time of the application, as received in the end event. */
+ private var endTime: Long = _
+
+ /** Number of events to batch up before posting. */
+ private[yarn] var batchSize = DEFAULT_BATCH_SIZE
+
+ /** Queue of entities to asynchronously post, plus the number of events in each entry. */
+ private val postingQueue = new LinkedBlockingDeque[PostQueueAction]()
+
+ /** Number of events in the post queue. */
+ private val postQueueEventSize = new AtomicLong
+
+ /** Limit on the total number of events permitted. */
+ private var postQueueLimit = DEFAULT_POST_EVENT_LIMIT
+
+ /** List of events which will be pulled into a timeline entity when created. */
+ private var pendingEvents = new mutable.LinkedList[TimelineEvent]()
+
+ /** The received application started event; `None` if no event has been received. */
+ private var applicationStartEvent: Option[SparkListenerApplicationStart] = None
+
+ /** The received application end event; `None` if no event has been received. */
+ private var applicationEndEvent: Option[SparkListenerApplicationEnd] = None
+
+ /** Has a start event been processed? */
+ private val appStartEventProcessed = new AtomicBoolean(false)
+
+ /** Has the application event event been processed? */
+ private val appEndEventProcessed = new AtomicBoolean(false)
+
+ /** Event handler thread. */
+ private var entityPostThread: Option[Thread] = None
+
+ /** Flag to indicate the queue is stopped; events aren't being processed. */
+ private val queueStopped = new AtomicBoolean(true)
+
+ /** Boolean to track when the post thread is active; Set and reset in the thread itself. */
+ private val postThreadActive = new AtomicBoolean(false)
+
+ /** How long to wait in millseconds for shutdown before giving up? */
+ private var shutdownWaitTime = 0L
+
+ /** What is the initial and incrementing interval for POST retries? */
+ private var retryInterval = 0L
+
+ /** Domain ID for entities: may be null. */
+ private var domainId: Option[String] = None
+
+ /** URI to timeline web application -valid after [[start()]]. */
+ private[yarn] var timelineWebappAddress: URI = _
+
+ /** Metric fields. Used in tests as well as metrics infrastructure. */
+ val metrics = new HistoryMetrics
+
+ /**
+ * Create a timeline client and start it. This does not update the
+ * `timelineClient` field, though it does verify that the field
+ * is unset.
+ *
+ * The method is private to the package so that tests can access it, which
+ * some of the mock tests do to override the timeline client creation.
+ * @return the timeline client
+ */
+ private[yarn] def createTimelineClient(): TimelineClient = {
+ require(_timelineClient.isEmpty, "timeline client already set")
+ YarnTimelineUtils.createTimelineClient(sparkContext)
+ }
+
+ /**
+ * Get the timeline client.
+ * @return the client
+ * @throws Exception if the timeline client is not currently running
+ */
+ def timelineClient: TimelineClient = {
+ synchronized { _timelineClient.get }
+ }
+
+ /**
+ * Get the total number of events dropped due to the queue of
+ * outstanding posts being too long.
+ * @return counter of events processed
+ */
+
+ def eventsDropped: Long = metrics.eventsDropped.getCount
+
+ /**
+ * Get the total number of processed events, those handled in the back-end thread without
+ * being rejected.
+ *
+ * @return counter of events processed
+ */
+ def eventsProcessed: Long = metrics.eventsProcessed.getCount
+
+ /**
+ * Get the total number of events queued.
+ *
+ * @return the total event count
+ */
+ def eventsQueued: Long = metrics.eventsQueued.getCount
+
+ /**
+ * Get the current size of the posting queue.
+ *
+ * @return the current queue length
+ */
+ def postingQueueSize: Int = postingQueue.size()
+
+ /**
+ * Query the counter of attempts to post entities to the timeline service.
+ *
+ * @return the current value
+ */
+ def postAttempts: Long = metrics.entityPostAttempts.getCount
+
+ /**
+ * Get the total number of failed post operations.
+ *
+ * @return counter of timeline post operations which failed
+ */
+ def postFailures: Long = metrics.entityPostFailures.getCount
+
+ /**
+ * Query the counter of successful post operations (this is not the same as the
+ * number of events posted).
+ *
+ * @return the number of successful post operations.
+ */
+ def postSuccesses: Long = metrics.entityPostSuccesses.getCount
+
+ /**
+ * Is the asynchronous posting thread active?
+ *
+ * @return true if the post thread has started; false if it has not yet/ever started, or
+ * if it has finished.
+ */
+ def isPostThreadActive: Boolean = postThreadActive.get
+
+ /**
+ * Reset the timeline client. Idempotent.
+ *
+ * 1. Stop the timeline client service if running.
+ * 2. set the `timelineClient` field to `None`
+ */
+ def stopTimelineClient(): Unit = {
+ synchronized {
+ _timelineClient.foreach(_.stop())
+ _timelineClient = None
+ }
+ }
+
+ /**
+ * Create the timeline domain.
+ *
+ * A Timeline Domain is a uniquely identified 'namespace' for accessing parts of the timeline.
+ * Security levels are are managed at the domain level, so one is created if the
+ * spark acls are enabled. Full access is then granted to the current user,
+ * all users in the configuration options `"spark.modify.acls"` and `"spark.admin.acls"`;
+ * read access to those users and those listed in `"spark.ui.view.acls"`
+ *
+ * @return an optional domain string. If `None`, then no domain was created.
+ */
+ private def createTimelineDomain(): Option[String] = {
+ val sparkConf = sparkContext.getConf
+ val aclsOn = sparkConf.getBoolean("spark.ui.acls.enable",
+ sparkConf.getBoolean("spark.acls.enable", false))
+ if (!aclsOn) {
+ logDebug("ACLs are disabled; not creating the timeline domain")
+ return None
+ }
+ val predefDomain = sparkConf.getOption(TIMELINE_DOMAIN)
+ if (predefDomain.isDefined) {
+ logDebug(s"Using predefined domain $predefDomain")
+ return predefDomain
+ }
+ val current = UserGroupInformation.getCurrentUser.getShortUserName
+ val adminAcls = stringToSet(sparkConf.get("spark.admin.acls", ""))
+ val viewAcls = stringToSet(sparkConf.get("spark.ui.view.acls", ""))
+ val modifyAcls = stringToSet(sparkConf.get("spark.modify.acls", ""))
+
+ val readers = (Seq(current) ++ adminAcls ++ modifyAcls ++ viewAcls).mkString(" ")
+ val writers = (Seq(current) ++ adminAcls ++ modifyAcls).mkString(" ")
+ val domain = DOMAIN_ID_PREFIX + applicationId
+ logInfo(s"Creating domain $domain with readers: $readers and writers: $writers")
+
+ // create the timeline domain with the reader and writer permissions
+ val timelineDomain = new TimelineDomain()
+ timelineDomain.setId(domain)
+ timelineDomain.setReaders(readers)
+ timelineDomain.setWriters(writers)
+ try {
+ timelineClient.putDomain(timelineDomain)
+ Some(domain)
+ } catch {
+ case e: Exception =>
+ logError(s"cannot create the domain $domain", e)
+ // fallback to default
+ None
+ }
+ }
+
+ /**
+ * Start the service.
+ *
+ * @param binding binding to the spark application and YARN
+ */
+ override def start(binding: SchedulerExtensionServiceBinding): Unit = {
+ val oldstate = enterState(StartedState)
+ if (oldstate != CreatedState) {
+ // state model violation
+ _serviceState.set(oldstate)
+ throw new IllegalArgumentException(s"Cannot start the service from state $oldstate")
+ }
+ val context = binding.sparkContext
+ val appId = binding.applicationId
+ val attemptId = binding.attemptId
+ require(context != null, "Null context parameter")
+ bindToYarnApplication(appId, attemptId)
+ this.sparkContext = context
+ this.config = new YarnConfiguration(context.hadoopConfiguration)
+ val sparkConf = sparkContext.conf
+
+ // work out the attempt ID from the YARN attempt ID. No attempt, assume "1".
+ val attempt1 = attemptId match {
+ case Some(attempt) => attempt.getAttemptId.toString
+ case None => CLIENT_BACKEND_ATTEMPT_ID
+ }
+ setContextAppAndAttemptInfo(Some(appId.toString), Some(attempt1))
+ batchSize = sparkConf.getInt(BATCH_SIZE, batchSize)
+ postQueueLimit = sparkConf.getInt(POST_EVENT_LIMIT, postQueueLimit)
+ retryInterval = 1000 * sparkConf.getTimeAsSeconds(POST_RETRY_INTERVAL,
+ DEFAULT_POST_RETRY_INTERVAL)
+ shutdownWaitTime = 1000 * sparkConf.getTimeAsSeconds(SHUTDOWN_WAIT_TIME,
+ DEFAULT_SHUTDOWN_WAIT_TIME)
+
+ // the full metrics integration happens if the spark context has a metrics system
+ val metricsSystem = sparkContext.metricsSystem
+ if (metricsSystem != null) {
+ metricsSystem.registerSource(metrics)
+ }
+
+ // set up the timeline service, unless it's been disabled for testing
+ if (timelineServiceEnabled) {
+ timelineWebappAddress = getTimelineEndpoint(config)
+
+ logInfo(s"Starting $this")
+ logInfo(s"Spark events will be published to the Timeline at $timelineWebappAddress")
+ _timelineClient = Some(createTimelineClient())
+ domainId = createTimelineDomain()
+ // declare that the processing is started
+ queueStopped.set(false)
+ val thread = new Thread(new EntityPoster(), "EventPoster")
+ entityPostThread = Some(thread)
+ thread.setDaemon(true)
+ thread.start()
+ } else {
+ logInfo("Timeline service is disabled")
+ }
+ if (registerListener()) {
+ logInfo(s"History Service listening for events: $this")
+ } else {
+ logInfo(s"History Service is not listening for events: $this")
+ }
+ }
+
+ /**
+ * Check the service configuration to see if the timeline service is enabled.
+ *
+ * @return true if `YarnConfiguration.TIMELINE_SERVICE_ENABLED` is set.
+ */
+ def timelineServiceEnabled: Boolean = {
+ YarnTimelineUtils.timelineServiceEnabled(config)
+ }
+
+ /**
+ * Return a summary of the service state to help diagnose problems
+ * during test runs, possibly even production.
+ *
+ * @return a summary of the current service state
+ */
+ override def toString(): String =
+ s"""YarnHistoryService for application $applicationId attempt $attemptId;
+ | state=$serviceState;
+ | endpoint=$timelineWebappAddress;
+ | bonded to ATS=$bondedToATS;
+ | listening=$listening;
+ | batchSize=$batchSize;
+ | flush count=$getFlushCount;
+ | total number queued=$eventsQueued, processed=$eventsProcessed;
+ | attempted entity posts=$postAttempts
+ | successful entity posts=$postSuccesses
+ | failed entity posts=$postFailures;
+ | events dropped=$eventsDropped;
+ | app start event received=$appStartEventProcessed;
+ | app end event received=$appEndEventProcessed;
+ """.stripMargin
+
+ /**
+ * Is the service listening to events from the spark context?
+ *
+ * @return true if it has registered as a listener
+ */
+ def listening: Boolean = {
+ listener.isDefined
+ }
+
+ /**
+ * Is the service hooked up to an ATS server?
+ *
+ * This does not check the validity of the link, only whether or not the service
+ * has been set up to talk to ATS.
+ *
+ * @return true if the service has a timeline client
+ */
+ def bondedToATS: Boolean = {
+ _timelineClient.isDefined
+ }
+
+ /**
+ * Set the YARN binding information.
+ *
+ * This is called during startup. It is private to the package so that tests
+ * may update this data.
+ * @param appId YARN application ID
+ * @param maybeAttemptId optional attempt ID
+ */
+ private[yarn] def bindToYarnApplication(appId: ApplicationId,
+ maybeAttemptId: Option[ApplicationAttemptId]): Unit = {
+ require(appId != null, "Null appId parameter")
+ applicationId = appId
+ attemptId = maybeAttemptId
+ }
+
+ /**
+ * Set the "spark" application and attempt information -the information
+ * provided in the start event. The attempt ID here may be `None`; even
+ * if set it may only be unique amongst the attempts of this application.
+ * That is: not unique enough to be used as the entity ID
+ *
+ * @param appId application ID
+ * @param attemptId attempt ID
+ */
+ private def setContextAppAndAttemptInfo(appId: Option[String],
+ attemptId: Option[String]): Unit = {
+ logDebug(s"Setting application ID to $appId; attempt ID to $attemptId")
+ sparkApplicationId = appId
+ sparkApplicationAttemptId = attemptId
+ }
+
+ /**
+ * Add the listener if it is not disabled.
+ * This is accessible in the same package purely for testing
+ *
+ * @return true if the register was enabled
+ */
+ private[yarn] def registerListener(): Boolean = {
+ assert(sparkContext != null, "Null context")
+ if (sparkContext.conf.getBoolean(REGISTER_LISTENER, true)) {
+ logDebug("Registering listener to spark context")
+ val l = new YarnEventListener(sparkContext, this)
+ listener = Some(l)
+ sparkContext.listenerBus.addListener(l)
+ true
+ } else {
+ false
+ }
+ }
+
+ /**
+ * Queue an action, or, if the service's `stopped` flag is set, discard it.
+ *
+ * This is the method called by the event listener when forward events to the service.
+ * @param event event to process
+ * @return true if the event was queued
+ */
+ def enqueue(event: SparkListenerEvent): Boolean = {
+ if (!queueStopped.get) {
+ metrics.eventsQueued.inc()
+ logDebug(s"Enqueue $event")
+ handleEvent(event)
+ true
+ } else {
+ // the service is stopped, so the event will not be processed.
+ if (timelineServiceEnabled) {
+ // if a timeline service was ever enabled, log the fact the event
+ // is being discarded. Don't do this if it was not, as it will
+ // only make the (test run) logs noisy.
+ logInfo(s"History service stopped; ignoring queued event : $event")
+ }
+ false
+ }
+ }
+
+ /**
+ * Stop the service; this triggers flushing the queue and, if not already processed,
+ * a pushing out of an application end event.
+ *
+ * This operation will block for up to `maxTimeToWaitOnShutdown` milliseconds
+ * to await the asynchronous action queue completing.
+ */
+ override def stop(): Unit = {
+ val oldState = enterState(StoppedState)
+ if (oldState != StartedState) {
+ // stopping from a different state
+ logDebug(s"Ignoring stop() request from state $oldState")
+ return
+ }
+ try {
+ stopQueue()
+ } finally {
+ if (sparkContext.metricsSystem != null) {
+ // unregister from metrics
+ sparkContext.metricsSystem.removeSource(metrics)
+ }
+ }
+ }
+
+ /**
+ * Stop the queue system.
+ */
+ private def stopQueue(): Unit = {
+ // if the queue is live
+ if (!queueStopped.get) {
+
+ if (appStartEventProcessed.get && !appEndEventProcessed.get) {
+ // push out an application stop event if none has been received
+ logDebug("Generating a SparkListenerApplicationEnd during service stop()")
+ enqueue(SparkListenerApplicationEnd(now()))
+ }
+
+ // flush out the events
+ asyncFlush()
+
+ // push out that queue stop event; this immediately sets the `queueStopped` flag
+ pushQueueStop(now(), shutdownWaitTime)
+
+ // Now await the halt of the posting thread.
+ var shutdownPosted = false
+ if (postThreadActive.get) {
+ postThreadActive.synchronized {
+ // check it hasn't switched state
+ if (postThreadActive.get) {
+ logDebug(s"Stopping posting thread and waiting $shutdownWaitTime mS")
+ shutdownPosted = true
+ postThreadActive.wait(shutdownWaitTime)
+ // then interrupt the thread if it is still running
+ if (postThreadActive.get) {
+ logInfo("Interrupting posting thread after $shutdownWaitTime mS")
+ entityPostThread.foreach(_.interrupt())
+ }
+ }
+ }
+ }
+ if (!shutdownPosted) {
+ // there was no running post thread, just stop the timeline client ourselves.
+ // (if there is a thread running, it must be the one to stop it)
+ stopTimelineClient()
+ logInfo(s"Stopped: $this")
+ }
+ }
+ }
+
+ /**
+ * Can an event be added?
+ *
+ * The policy is: only if the number of queued entities is below the limit, or the
+ * event marks the end of the application.
+ *
+ * @param isLifecycleEvent is this operation triggered by an application start/end?
+ * @return true if the event can be added to the queue
+ */
+ private def canAddEvent(isLifecycleEvent: Boolean): Boolean = {
+ isLifecycleEvent || metrics.eventsQueued.getCount < postQueueLimit
+ }
+
+ /**
+ * Add another event to the pending event list.
+ *
+ * Returns the size of the event list after the event was added
+ * (thread safe).
+ * @param event event to add
+ * @return the event list size
+ */
+ private def addPendingEvent(event: TimelineEvent): Int = {
+ pendingEvents.synchronized {
+ pendingEvents :+= event
+ pendingEvents.size
+ }
+ }
+
+ /**
+ * Publish next set of pending events if there are events to publish,
+ * and the application has been recorded as started.
+ *
+ * Builds the next event to push onto [[postingQueue]]; resets
+ * the current [[pendingEvents]] list and then adds a [[PostEntity]]
+ * operation to the queue.
+ *
+ * @return true if another entity was queued
+ */
+ private def publishPendingEvents(): Boolean = {
+ // verify that there are events to publish
+ val size = pendingEvents.synchronized {
+ pendingEvents.size
+ }
+ if (size > 0 && applicationStartEvent.isDefined) {
+ // push if there are events *and* the app is recorded as having started.
+ // -as the app name is needed for the the publishing.
+ metrics.flushCount.inc()
+ val timelineEntity = createTimelineEntity(
+ applicationId,
+ attemptId,
+ sparkApplicationId,
+ sparkApplicationAttemptId,
+ applicationName,
+ userName,
+ startTime,
+ endTime,
+ now())
+
+ // copy in pending events and then reset the list
+ pendingEvents.synchronized {
+ pendingEvents.foreach(timelineEntity.addEvent)
+ pendingEvents = new mutable.LinkedList[TimelineEvent]()
+ }
+ queueForPosting(timelineEntity)
+ true
+ } else {
+ false
+ }
+ }
+
+ /**
+ * Queue an asynchronous flush operation.
+ * @return if the flush event was queued
+ */
+ def asyncFlush(): Boolean = {
+ publishPendingEvents()
+ }
+
+ /**
+ * A `StopQueueAction` action has a size of 0
+ * @param currentTime time when action was queued.
+ * @param waitTime time for shutdown to wait
+ */
+ private def pushQueueStop(currentTime: Long, waitTime: Long): Unit = {
+ queueStopped.set(true)
+ postingQueue.add(StopQueueAction(currentTime, waitTime))
+ }
+
+ /**
+ * Queue an entity for posting; also increases
+ * [[postQueueEventSize]] by the size of the entity.
+ * @param timelineEntity entity to push
+ */
+ def queueForPosting(timelineEntity: TimelineEntity): Unit = {
+ // queue the entity for posting
+ preflightCheck(timelineEntity)
+ val e = new PostEntity(timelineEntity)
+ postQueueEventSize.addAndGet(e.size)
+ metrics.postQueueEventSize.inc(e.size)
+ postingQueue.add(e)
+ }
+
+ /**
+ * Push a `PostQueueAction` to the start of the queue; also increments
+ * [[postQueueEventSize]] by the size of the action.
+ * @param action action to push
+ */
+ private def pushToFrontOfQueue(action: PostQueueAction): Unit = {
+ postingQueue.push(action)
+ postQueueEventSize.addAndGet(action.size)
+ metrics.postQueueEventSize.inc(action.size)
+ }
+
+ /**
+ * Take from the posting queue; decrements [[postQueueEventSize]] by the size
+ * of the action.
+ * @return the action
+ */
+ private def takeFromPostingQueue(): PostQueueAction = {
+ val taken = postingQueue.take()
+ postQueueEventSize.addAndGet(-taken.size)
+ metrics.postQueueEventSize.dec(taken.size)
+ taken
+ }
+
+ /**
+ * Poll from the posting queue; decrements [[postQueueEventSize]] by the size
+ * of the action.
+ * @return
+ */
+ private def pollFromPostingQueue(mills: Long): Option[PostQueueAction] = {
+ val taken = postingQueue.poll(mills, TimeUnit.MILLISECONDS)
+ postQueueEventSize.addAndGet(-taken.size)
+ metrics.postQueueEventSize.dec(taken.size)
+ Option(taken)
+ }
+
+ /**
+ * Perform any preflight checks.
+ *
+ * This is just a safety check to catch regressions in the code which
+ * publish data that cannot be parsed at the far end.
+ * @param entity timeline entity to review.
+ */
+ private def preflightCheck(entity: TimelineEntity): Unit = {
+ require(entity.getStartTime != null,
+ s"No start time in ${describeEntity(entity)}")
+ }
+
+ /** Actions in the post queue */
+ private sealed trait PostQueueAction {
+ /**
+ * Number of events in this entry
+ * @return a natural number
+ */
+ def size: Int
+ }
+
+ /**
+ * A `StopQueueAction` action has a size of 0
+ * @param currentTime time when action was queued.
+ * @param waitTime time for shutdown to wait
+ */
+ private case class StopQueueAction(currentTime: Long, waitTime: Long) extends PostQueueAction {
+ override def size: Int = 0
+ def timeLimit: Long = currentTime + waitTime
+ }
+
+ /**
+ * A `PostEntity` action has a size of the number of listed events
+ */
+ private case class PostEntity(entity: TimelineEntity) extends PostQueueAction {
+ override def size: Int = entity.getEvents.size()
+ }
+
+ /**
+ * Post a single entity.
+ *
+ * Any network/connectivity errors will be caught and logged, and returned as the
+ * exception field in the returned tuple.
+ *
+ * Any posting which generates a response will result in the timeline response being
+ * returned. This response *may* contain errors; these are almost invariably going
+ * to re-occur when resubmitted.
+ *
+ * @param entity entity to post
+ * @return Any exception other than an interruption raised during the operation.
+ * @throws InterruptedException if an [[InterruptedException]] or [[InterruptedIOException]] is
+ * received. These exceptions may also get caught and wrapped in the ATS client library.
+ */
+ private def postOneEntity(entity: TimelineEntity): Option[Exception] = {
+ domainId.foreach(entity.setDomainId)
+ val entityDescription = describeEntity(entity)
+ logInfo(s"About to POST entity ${entity.getEntityId} with ${entity.getEvents.size()} events" +
+ s" to timeline service $timelineWebappAddress")
+ logDebug(s"About to POST $entityDescription")
+ val timeContext = metrics.postOperationTimer.time()
+ metrics.entityPostAttempts.inc()
+ try {
+ val response = timelineClient.putEntities(entity)
+ val errors = response.getErrors
+ if (errors.isEmpty) {
+ logDebug(s"entity successfully posted")
+ metrics.entityPostSuccesses.inc()
+ } else {
+ // The ATS service rejected the request at the API level.
+ // this is something we assume cannot be re-tried
+ metrics.entityPostRejections.inc()
+ logError(s"Failed to post $entityDescription")
+ errors.asScala.foreach { err =>
+ logError(describeError(err))
+ }
+ }
+ // whether accepted or rejected, this request is not re-issued
+ None
+ } catch {
+
+ case e: InterruptedException =>
+ // interrupted; this will break out of IO/Sleep operations and
+ // trigger a rescan of the stopped() event.
+ throw e
+
+ case e: ConnectException =>
+ // connection failure: network, ATS down, config problems, ...
+ metrics.entityPostFailures.inc()
+ logDebug(s"Connection exception submitting $entityDescription", e)
+ Some(e)
+
+ case e: Exception =>
+ val cause = e.getCause
+ if (cause != null && cause.isInstanceOf[InterruptedException]) {
+ // hadoop 2.7 retry logic wraps the interrupt
+ throw cause
+ }
+ // something else has gone wrong.
+ metrics.entityPostFailures.inc()
+ logDebug(s"Could not handle history entity: $entityDescription", e)
+ Some(e)
+
+ } finally {
+ timeContext.stop()
+ }
+ }
+
+ /**
+ * Wait for and then post entities until stopped.
+ *
+ * Algorithm.
+ *
+ * 1. The thread waits for events in the [[postingQueue]] until stopped or interrupted.
+ * 1. Failures result in the entity being queued for resending, after a delay which grows
+ * linearly on every retry.
+ * 1. Successful posts reset the retry delay.
+ * 1. If the process is interrupted, the loop continues with the `stopFlag` flag being checked.
+ *
+ * To stop this process then, first set the `stopFlag` flag, then interrupt the thread.
+ *
+ * @param retryInterval delay in milliseconds for the first retry delay; the delay increases
+ * by this value on every future failure. If zero, there is no delay, ever.
+ * @return the [[StopQueueAction]] received to stop the process.
+ */
+ private def postEntities(retryInterval: Long): StopQueueAction = {
+ var lastAttemptFailed = false
+ var currentRetryDelay = retryInterval
+ var result: StopQueueAction = null
+ while (result == null) {
+ takeFromPostingQueue() match {
+ case PostEntity(entity) =>
+ val ex = postOneEntity(entity)
+ if (ex.isDefined && !queueStopped.get()) {
+ // something went wrong
+ if (!lastAttemptFailed) {
+ // avoid filling up logs with repeated failures
+ logWarning(s"Exception submitting entity to $timelineWebappAddress", ex.get)
+ }
+ // log failure and queue for posting again
+ lastAttemptFailed = true
+ currentRetryDelay += retryInterval
+ if (!queueStopped.get()) {
+ // push back to the head of the queue
+ postingQueue.addFirst(PostEntity(entity))
+ if (currentRetryDelay > 0) {
+ Thread.sleep(currentRetryDelay)
+ }
+ }
+ } else {
+ // success; reset flags and retry delay
+ lastAttemptFailed = false
+ currentRetryDelay = retryInterval
+ }
+ case stop: StopQueueAction =>
+ logDebug("Queue stopped")
+ result = stop
+ }
+ }
+ result
+ }
+
+ /**
+ * Shutdown phase: continually post oustanding entities until the timeout has been exceeded.
+ * The interval between failures is the retryInterval: there is no escalation, and if
+ * is longer than the remaining time in the shutdown, the remaining time sets the limit.
+ *
+ * @param shutdown shutdown parameters.
+ * @param retryInterval delay in milliseconds for every delay.
+ */
+ private def postEntitiesShutdownPhase(shutdown: StopQueueAction, retryInterval: Long): Unit = {
+ val timeLimit = shutdown.timeLimit
+ val timestamp = YarnTimelineUtils.timeShort(timeLimit, "")
+ logDebug(s"Queue shutdown, time limit= $timestamp")
+ while (now() < timeLimit && !postingQueue.isEmpty) {
+ pollFromPostingQueue(timeLimit - now()) match {
+ case Some(PostEntity(entity)) =>
+ postOneEntity(entity).foreach { e =>
+ if (!e.isInstanceOf[InterruptedException] &&
+ !e.isInstanceOf[InterruptedIOException]) {
+ // failure, push back to try again
+ pushToFrontOfQueue(PostEntity(entity))
+ if (retryInterval > 0) {
+ Thread.sleep(retryInterval)
+ } else {
+ // there's no retry interval, so fail immediately
+ throw e
+ }
+ } else {
+ // this was an interruption. Throw it again
+ throw e
+ }
+ }
+ case Some(StopQueueAction(_, _)) =>
+ // ignore these
+ logDebug("Ignoring StopQueue action")
+
+ case None =>
+ // get here then the queue is empty; all is well
+ }
+ }
+ }
+
+ /**
+ * If the event reaches the batch size or flush is true, push events to ATS.
+ *
+ * @param event event. If null, no event is queued, but the post-queue flush logic still applies
+ */
+ private def handleEvent(event: SparkListenerEvent): Unit = {
+ // publish events unless stated otherwise
+ var publish = true
+ // don't trigger a push to the ATS
+ var push = false
+ // lifecycle events get special treatment: they are never discarded from the queues,
+ // even if the queues are full.
+ var isLifecycleEvent = false
+ val timestamp = now()
+ metrics.eventsProcessed.inc()
+ if (metrics.eventsProcessed.getCount() % 1000 == 0) {
+ logDebug(s"${metrics.eventsProcessed} events are processed")
+ }
+ event match {
+ case start: SparkListenerApplicationStart =>
+ // we already have all information,
+ // flush it for old one to switch to new one
+ logDebug(s"Handling application start event: $event")
+ if (!appStartEventProcessed.getAndSet(true)) {
+ applicationStartEvent = Some(start)
+ applicationName = start.appName
+ if (applicationName == null || applicationName.isEmpty) {
+ logWarning("Application does not have a name")
+ applicationName = applicationId.toString
+ }
+ userName = start.sparkUser
+ startTime = start.time
+ if (startTime == 0) {
+ startTime = timestamp
+ }
+ setContextAppAndAttemptInfo(start.appId, start.appAttemptId)
+ logDebug(s"Application started: $event")
+ isLifecycleEvent = true
+ push = true
+ } else {
+ logWarning(s"More than one application start event received -ignoring: $start")
+ publish = false
+ }
+
+ case end: SparkListenerApplicationEnd =>
+ if (!appStartEventProcessed.get()) {
+ // app-end events being received before app-start events can be triggered in
+ // tests, even if not seen in real applications.
+ // react by ignoring the event altogether, as an un-started application
+ // cannot be reported.
+ logError(s"Received application end event without application start $event -ignoring.")
+ } else if (!appEndEventProcessed.getAndSet(true)) {
+ // the application has ended
+ logDebug(s"Application end event: $event")
+ applicationEndEvent = Some(end)
+ // flush old entity
+ endTime = if (end.time > 0) end.time else timestamp
+ push = true
+ isLifecycleEvent = true
+ } else {
+ // another test-time only situation: more than one application end event
+ // received. Discard the later one.
+ logInfo(s"Discarding duplicate application end event $end")
+ publish = false
+ }
+
+ case update: SparkListenerBlockUpdated =>
+ publish = false
+
+ case update: SparkListenerExecutorMetricsUpdate =>
+ publish = false
+
+ case _ =>
+ }
+
+ if (publish) {
+ val tlEvent = toTimelineEvent(event, timestamp)
+ val eventCount = if (tlEvent.isDefined && canAddEvent(isLifecycleEvent)) {
+ addPendingEvent(tlEvent.get)
+ } else {
+ // discarding the event
+ logInfo(s"Discarding event $tlEvent")
+ metrics.eventsDropped.inc()
+ 0
+ }
+
+ // trigger a push if the batch limit is reached
+ // There's no need to check for the application having started, as that is done later.
+ push |= eventCount >= batchSize
+
+ logDebug(s"current event num: $eventCount")
+ if (push) {
+ logDebug("Push triggered")
+ publishPendingEvents()
+ }
+ }
+ }
+
+ /**
+ * Return the current time in milliseconds.
+ * @return system time in milliseconds
+ */
+ private def now(): Long = {
+ clock.getTimeMillis()
+ }
+
+ /**
+ * Get the number of flush events that have taken place.
+ *
+ * This includes flushes triggered by the event list being bigger the batch size,
+ * but excludes flush operations triggered when the action processor thread
+ * is stopped, or if the timeline service binding is disabled.
+ *
+ * @return count of processed flush events.
+ */
+ def getFlushCount: Long = {
+ metrics.flushCount.getCount
+ }
+
+ /**
+ * Post events until told to stop.
+ */
+ private class EntityPoster extends Runnable {
+
+ override def run(): Unit = {
+ postThreadActive.set(true)
+ try {
+ val shutdown = postEntities(retryInterval)
+ // getting here means the `stop` flag is true
+ postEntitiesShutdownPhase(shutdown, retryInterval)
+ logInfo(s"Stopping dequeue service, final queue size is ${postingQueue.size};" +
+ s" outstanding events to post count: ${postQueueEventSize.get()}")
+ } catch {
+ // handle exceptions triggering thread exit. Interruptes are good; others less welcome.
+ case ex: InterruptedException =>
+ logInfo("Entity Posting thread interrupted")
+ logDebug("Entity Posting thread interrupted", ex)
+
+ case ex: InterruptedIOException =>
+ logInfo("Entity Posting thread interrupted")
+ logDebug("Entity Posting thread interrupted", ex)
+
+ case ex: Exception =>
+ logError("Entity Posting thread exiting after exception raised", ex)
+ } finally {
+ stopTimelineClient()
+ postThreadActive synchronized {
+ // declare that this thread is no longer active
+ postThreadActive.set(false)
+ // and notify all listeners of this fact
+ postThreadActive.notifyAll()
+ }
+ }
+ }
+ }
+
+}
+
+/**
+ * Metrics integration: the various counters of activity
+ */
+private[yarn] class HistoryMetrics extends ExtendedMetricsSource {
+
+ /** Name for metrics: yarn_history */
+ override val sourceName = YarnHistoryService.METRICS_NAME
+
+ /** Metrics registry */
+ override val metricRegistry = new MetricRegistry()
+
+ /** Number of events in the post queue. */
+ val postQueueEventSize = new Counter()
+
+ /** Counter of events processed -that is have been through handleEvent() */
+ val eventsProcessed = new Counter()
+
+ /** Counter of events queued. */
+ val eventsQueued = new Counter()
+
+ /** Counter of number of attempts to post entities. */
+ val entityPostAttempts = new Counter()
+
+ /** Counter of number of successful entity post operations. */
+ val entityPostSuccesses = new Counter()
+
+ /** How many entity postings failed? */
+ val entityPostFailures = new Counter()
+
+ /** How many entity postings were rejected? */
+ val entityPostRejections = new Counter()
+
+ /** The number of events which were dropped as the backlog of pending posts was too big. */
+ val eventsDropped = new Counter()
+
+ /** How many flushes have taken place? */
+ val flushCount = new Counter()
+
+ /** Timer to build up statistics on post operation times */
+ val postOperationTimer = new Timer()
+
+ val metricsMap: Map[String, Metric] = Map(
+ "eventsDropped" -> eventsDropped,
+ "eventsProcessed" -> eventsProcessed,
+ "eventsQueued" -> eventsQueued,
+ "entityPostAttempts" -> entityPostAttempts,
+ "entityPostFailures" -> entityPostFailures,
+ "entityPostRejections" -> entityPostRejections,
+ "entityPostSuccesses" -> entityPostSuccesses,
+ "entityPostTimer" -> postOperationTimer,
+ "flushCount" -> flushCount
+ )
+
+ init()
+
+}
+
+/**
+ * Constants and defaults for the history service.
+ */
+private[spark] object YarnHistoryService {
+
+ /**
+ * Name of the entity type used to declare spark Applications.
+ */
+ val SPARK_EVENT_ENTITY_TYPE = "spark_event_v01"
+
+ /**
+ * Domain ID.
+ */
+ val DOMAIN_ID_PREFIX = "Spark_ATS_"
+
+ /**
+ * Time in millis to wait for shutdown on service stop.
+ */
+ val DEFAULT_SHUTDOWN_WAIT_TIME = "30s"
+
+ /**
+ * The maximum time in to wait for event posting to complete when the service stops.
+ */
+ val SHUTDOWN_WAIT_TIME = "spark.hadoop.yarn.timeline.shutdown.waittime"
+
+ /**
+ * Option to declare that the history service should register as a spark context
+ * listener. (default: true; this option is here for testing)
+ *
+ * This is a spark option, though its use of name will cause it to propagate down to the Hadoop
+ * Configuration.
+ */
+ val REGISTER_LISTENER = "spark.hadoop.yarn.timeline.listen"
+
+ /**
+ * Option for the size of the batch for timeline uploads. Bigger: less chatty.
+ * Smaller: history more responsive.
+ */
+ val BATCH_SIZE = "spark.hadoop.yarn.timeline.batch.size"
+
+ /**
+ * The default size of a batch
+ */
+ val DEFAULT_BATCH_SIZE = 10
+
+ /**
+ * Name of a domain for the timeline
+ */
+ val TIMELINE_DOMAIN = "spark.hadoop.yarn.timeline.domain"
+
+ /**
+ * Limit on number of posts in the outbound queue -when exceeded
+ * new events will be dropped
+ */
+ val POST_EVENT_LIMIT = "spark.hadoop.yarn.timeline.post.limit"
+
+ /**
+ * The default limit of events in the post queue
+ */
+ val DEFAULT_POST_EVENT_LIMIT = 1000
+
+ /**
+ * Interval in milliseconds between POST retries. Every
+ * failure causes the interval to increase by this value.
+ */
+ val POST_RETRY_INTERVAL = "spark.hadoop.yarn.timeline.post.retry.interval"
+
+ /**
+ * The default retry interval in millis
+ */
+ val DEFAULT_POST_RETRY_INTERVAL = "1000ms"
+
+ /**
+ * Primary key used for events
+ */
+ val PRIMARY_KEY = "spark_application_entity"
+
+ /**
+ * Entity `OTHER_INFO` field: start time
+ */
+ val FIELD_START_TIME = "startTime"
+
+ /**
+ * Entity `OTHER_INFO` field: last updated time.
+ */
+ val FIELD_LAST_UPDATED = "lastUpdated"
+
+ /**
+ * Entity `OTHER_INFO` field: end time. Not present if the app is running.
+ */
+ val FIELD_END_TIME = "endTime"
+
+ /**
+ * Entity `OTHER_INFO` field: application name from context.
+ */
+ val FIELD_APP_NAME = "appName"
+
+ /**
+ * Entity `OTHER_INFO` field: user.
+ */
+ val FIELD_APP_USER = "appUser"
+
+ /**
+ * Entity `OTHER_INFO` field: YARN application ID.
+ */
+ val FIELD_APPLICATION_ID = "applicationId"
+
+ /**
+ * Entity `OTHER_INFO` field: attempt ID from spark start event.
+ */
+ val FIELD_ATTEMPT_ID = "attemptId"
+
+ /**
+ * Entity `OTHER_INFO` field: a counter which is incremented whenever a new timeline entity
+ * is created in this JVM (hence, attempt). It can be used to compare versions of the
+ * current entity with any cached copy -it is less brittle than using timestamps.
+ */
+ val FIELD_ENTITY_VERSION = "entityVersion"
+
+ /**
+ * Entity `OTHER_INFO` field: Spark version.
+ */
+ val FIELD_SPARK_VERSION = "sparkVersion"
+
+ /**
+ * Entity filter field: to search for entities that have started.
+ */
+ val FILTER_APP_START = "startApp"
+
+ /**
+ * Value of the `startApp` filter field.
+ */
+ val FILTER_APP_START_VALUE = "SparkListenerApplicationStart"
+
+ /**
+ * Entity filter field: to search for entities that have ended.
+ */
+ val FILTER_APP_END = "endApp"
+
+ /**
+ * Value of the `endApp`filter field.
+ */
+ val FILTER_APP_END_VALUE = "SparkListenerApplicationEnd"
+
+ /**
+ * ID used in yarn-client attempts only.
+ */
+ val CLIENT_BACKEND_ATTEMPT_ID = "1"
+
+ /**
+ * The classname of the history service to instantiate in the YARN AM.
+ */
+ val CLASSNAME = "org.apache.spark.deploy.history.yarn.YarnHistoryService"
+
+ /**
+ * Name of metrics.
+ */
+ val METRICS_NAME = "yarn_history"
+
+ /**
+ * Enum value of application created state
+ */
+ val CreatedState = 0
+
+ /**
+ * Enum value of started state.
+ */
+ val StartedState = 1
+
+ /**
+ * Enum value of stopped state.
+ */
+ val StoppedState = 2
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/YarnTimelineUtils.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/YarnTimelineUtils.scala
new file mode 100644
index 000000000000..da105c52b1aa
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/YarnTimelineUtils.scala
@@ -0,0 +1,772 @@
+/*
+ * 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.yarn
+
+import java.{lang, util}
+import java.io.IOException
+import java.net.{InetSocketAddress, NoRouteToHostException, URI, URL}
+import java.text.DateFormat
+import java.util.{ArrayList => JArrayList, Collection => JCollection, Date, HashMap => JHashMap, Map => JMap}
+import java.util.concurrent.atomic.AtomicLong
+
+import scala.collection.mutable
+import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError
+import org.apache.hadoop.yarn.api.records.timeline.{TimelineEntity, TimelineEvent, TimelinePutResponse}
+import org.apache.hadoop.yarn.client.api.TimelineClient
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.json4s.{MappingException, JValue}
+import org.json4s.JsonAST.{JNull, JNothing, JArray, JBool, JDecimal, JDouble, JString, JInt, JObject}
+import org.json4s.jackson.JsonMethods._
+
+import org.apache.spark
+import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.scheduler.{SparkListenerApplicationEnd, SparkListenerApplicationStart, SparkListenerEvent, SparkListenerExecutorAdded, SparkListenerExecutorRemoved, SparkListenerJobEnd, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerStageSubmitted}
+import org.apache.spark.util.{JsonProtocol, Utils}
+
+/**
+ * Utility methods for timeline classes.
+ */
+private[spark] object YarnTimelineUtils extends Logging {
+
+ /**
+ * What attempt ID to use as the attempt ID field (not the entity ID) when
+ * there is no attempt info.
+ */
+ val SINGLE_ATTEMPT = "1"
+
+ /**
+ * Exception text when there is no event info data to unmarshall.
+ */
+ val E_NO_EVENTINFO = "No 'eventinfo' entry"
+
+ /**
+ * Exception text when there is event info entry in the timeline event, but it is empty.
+ */
+
+ val E_EMPTY_EVENTINFO = "Empty 'eventinfo' entry"
+
+ /**
+ * Counter incremented on every spark event to timeline event creation,
+ * so guaranteeing uniqueness of event IDs across a single application attempt
+ * (which is implicitly, one per JVM).
+ */
+ val eventCreateCounter = new AtomicLong(System.currentTimeMillis())
+
+ /**
+ * A counter incremented every time a new entity is created. This is included as an "other"
+ * field in the entity information -so can be used as a probe to determine if the entity
+ * has been updated since a previous check.
+ */
+ val entityVersionCounter = new AtomicLong(1)
+
+ /**
+ * Converts a Java object to its equivalent json4s representation.
+ */
+ def toJValue(obj: Object): JValue = {
+ obj match {
+ case str: String => JString(str)
+ case dbl: java.lang.Double => JDouble(dbl)
+ case dec: java.math.BigDecimal => JDecimal(dec)
+ case int: java.lang.Integer => JInt(BigInt(int))
+ case long: java.lang.Long => JInt(BigInt(long))
+ case bool: java.lang.Boolean => JBool(bool)
+ case map: JMap[_, _] =>
+ val jmap = map.asInstanceOf[JMap[String, Object]]
+ JObject(jmap.entrySet().asScala.map { e => e.getKey -> toJValue(e.getValue) }.toList)
+ case array: JCollection[_] =>
+ JArray(array.asInstanceOf[JCollection[Object]].asScala.map(o => toJValue(o)).toList)
+ case null => JNothing
+ }
+ }
+
+ /**
+ * Converts a JValue into its Java equivalent.
+ */
+ def toJavaObject(v: JValue): Object = {
+ v match {
+ case JNothing => null
+ case JNull => null
+ case JString(s) => s
+ case JDouble(num) => java.lang.Double.valueOf(num)
+ case JDecimal(num) => num.bigDecimal
+ case JInt(num) => java.lang.Long.valueOf(num.longValue())
+ case JBool(value) => java.lang.Boolean.valueOf(value)
+ case obj: JObject => toJavaMap(obj)
+ case JArray(vals) =>
+ val list = new JArrayList[Object]()
+ vals.foreach(x => list.add(toJavaObject(x)))
+ list
+ }
+ }
+
+ /**
+ * Converts a json4s list of fields into a Java Map suitable for serialization by Jackson,
+ * which is used by the ATS client library.
+ */
+ def toJavaMap(sourceObj: JObject): JHashMap[String, Object] = {
+ val map = new JHashMap[String, Object]()
+ sourceObj.obj.foreach { case (k, v) => map.put(k, toJavaObject(v)) }
+ map
+ }
+
+ /**
+ * Convert a timeline event to a spark one. Includes some basic checks for validity of
+ * the event payload.
+ * @param event timeline event
+ * @return an unmarshalled event
+ */
+ def toSparkEvent(event: TimelineEvent): SparkListenerEvent = {
+ val info = event.getEventInfo
+ if (info == null) {
+ throw new IOException(E_NO_EVENTINFO)
+ }
+ if (info.size() == 0) {
+ throw new IOException(E_EMPTY_EVENTINFO)
+ }
+ val payload = toJValue(info)
+ def jsonToString: String = {
+ val json = compact(render(payload))
+ val limit = 256
+ if (json.length < limit) {
+ json
+ } else {
+ json.substring(0, limit) + " ... }"
+ }
+ }
+ logDebug(s"toSparkEvent payload is $jsonToString")
+ val eventField = payload \ "Event"
+ if (eventField == JNothing) {
+ throw new IOException(s"No 'Event' entry in $jsonToString")
+ }
+
+ // now the real unmarshalling
+ try {
+ JsonProtocol.sparkEventFromJson(payload)
+ } catch {
+ // failure in the marshalling; include payload in the message
+ case ex: MappingException =>
+ logDebug(s"$ex while rendering $jsonToString", ex)
+ throw ex
+ }
+ }
+
+ /**
+ * Convert a spark event to a timeline event
+ * @param event handled spark event
+ * @return a timeline event if it could be marshalled
+ */
+ def toTimelineEvent(event: SparkListenerEvent, timestamp: Long): Option[TimelineEvent] = {
+ try {
+ val tlEvent = new TimelineEvent()
+ tlEvent.setEventType(Utils.getFormattedClassName(event)
+ + "-" + eventCreateCounter.incrementAndGet.toString)
+ tlEvent.setTimestamp(timestamp)
+ val kvMap = new JHashMap[String, Object]()
+ val json = JsonProtocol.sparkEventToJson(event)
+ val jObject = json.asInstanceOf[JObject]
+ // the timeline event wants a map of java objects for Jackson to serialize
+ val hashMap = toJavaMap(jObject)
+ tlEvent.setEventInfo(hashMap)
+ Some(tlEvent)
+ }
+ catch {
+ case e: MatchError =>
+ log.debug(s"Failed to convert $event to JSON: $e", e)
+ None
+ }
+ }
+
+ /**
+ * Describe the event for logging.
+ *
+ * @param event timeline event
+ * @return a description
+ */
+ def describeEvent(event: TimelineEvent): String = {
+ val sparkEventDetails = try {
+ toSparkEvent(event).toString
+ } catch {
+ case _: MappingException =>
+ "(cannot convert event details to spark exception)"
+ }
+ s"${event.getEventType()} @ ${new Date(event.getTimestamp())}" +
+ s"\n $sparkEventDetails"
+ }
+
+ /**
+ * Create details of a timeline entity, by describing every event inside it.
+ *
+ * @param entity entity containing a possibly empty or null list of events
+ * @return a list of event details, with a newline between each one
+ */
+ def eventDetails(entity: TimelineEntity): String = {
+ val events = entity.getEvents
+ if (events != null) {
+ events.asScala.map(describeEvent).mkString("\n")
+ } else {
+ ""
+ }
+ }
+
+ /**
+ * Describe a timeline entity.
+ * @param entity entity
+ * @return a string description.
+ */
+ def describeEntity(entity: TimelineEntity): String = {
+ val events: util.List[TimelineEvent] = entity.getEvents
+ val eventSummary = if (events != null) {
+ s"contains ${events.size()} event(s)"
+ } else {
+ "contains no events"
+ }
+
+ val domain = if (entity.getDomainId != null) s" Domain ${entity.getDomainId}" else ""
+ val header = s"${entity.getEntityType}/${entity.getEntityId} $domain"
+ try {
+ events.asScala.map(describeEvent).mkString("\n")
+ val otherInfo = entity.getOtherInfo.asScala.map {
+ case (k, v) => s" $k ='$v': ${v.getClass};"
+ }.mkString("\n")
+ s"Timeline Entity " + header +
+ " " + otherInfo + "\n" +
+ " started: " + timeFieldToString(entity.getStartTime, "start") + "\n" +
+ " " + eventSummary
+ } catch {
+ case e: MappingException =>
+ // failure to marshall/unmarshall; downgrade
+ s"Timeline Entity $header"
+ }
+ }
+
+ /**
+ * Convert a `java.lang.Long` reference to a string value, or, if the reference is null,
+ * to text declaring that the named field is empty.
+ *
+ * @param time time reference
+ * @param field field name for error message
+ * @return a string to describe the field
+ */
+ def timeFieldToString(time: lang.Long, field: String): String = {
+ if (time != null) {
+ new Date(time).toString
+ } else {
+ s"no $field time"
+ }
+ }
+
+ /**
+ * A verbose description of the entity which contains event details and info about
+ * primary/secondary keys.
+ *
+ * @param entity timeline entity
+ * @return a verbose description of the field
+ */
+ def describeEntityVerbose(entity: TimelineEntity): String = {
+ val header = describeEntity(entity)
+ val primaryFilters = entity.getPrimaryFilters.asScala.toMap
+ var filterElements = ""
+ for ((k, v) <- primaryFilters) {
+ filterElements = filterElements +
+ " filter " + k + ": [ " + v.asScala.foldLeft("")((s, o) => s + o.toString + " ") + "]\n"
+ }
+ val events = eventDetails(entity)
+ header + "\n" + filterElements + events
+ }
+
+ /**
+ * Split a comma separated String, filter out any empty items, and return a `Set` of strings.
+ */
+ def stringToSet(list: String): Set[String] = {
+ list.split(',').map(_.trim).filter(!_.isEmpty).toSet
+ }
+
+ /**
+ * Try to get the event time off an event. Not all events have the required information.
+ *
+ * @param event event to process
+ * @return the event time
+ */
+ def eventTime(event: SparkListenerEvent): Option[Long] = {
+ event match {
+ case evt: SparkListenerApplicationStart =>
+ Some(evt.time)
+ case evt: SparkListenerApplicationEnd =>
+ Some(evt.time)
+ case evt: SparkListenerJobStart =>
+ Some(evt.time)
+ case evt: SparkListenerJobEnd =>
+ Some(evt.time)
+ case evt: SparkListenerExecutorAdded =>
+ Some(evt.time)
+ case evt: SparkListenerExecutorRemoved =>
+ Some(evt.time)
+ case evt: SparkListenerStageSubmitted =>
+ evt.stageInfo.submissionTime
+ case evt: SparkListenerStageCompleted =>
+ evt.stageInfo.completionTime
+ case _ => None
+ }
+ }
+
+ /**
+ * Create and start a timeline client, using the configuration context to
+ * set up the binding.
+ *
+ * @param sparkContext spark context
+ * @return the started instance
+ */
+ def createTimelineClient(sparkContext: SparkContext): TimelineClient = {
+ val client = TimelineClient.createTimelineClient
+ client.init(sparkContext.hadoopConfiguration)
+ client.start()
+ client
+ }
+
+ /**
+ * The path for the V1 ATS REST API.
+ */
+ val TIMELINE_REST_PATH = s"/ws/v1/timeline/"
+
+ /**
+ * Build the URI to the base of the timeline web application
+ * from the Hadoop context.
+ *
+ * Raises an exception if the address cannot be determined or is considered invalid from
+ * a networking perspective.
+ *
+ * Does not perform any checks as to whether or not the timeline service is enabled
+ * @param conf configuration
+ * @return the URI to the timeline service.
+ */
+ def getTimelineEndpoint(conf: Configuration): URI = {
+ val isHttps = YarnConfiguration.useHttps(conf)
+ val address = if (isHttps) {
+ conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
+ YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS)
+ } else {
+ conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+ YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS)
+ }
+ val protocol = if (isHttps) "https://" else "http://"
+ require(address != null, s"No timeline service defined")
+ validateEndpoint(URI.create(s"$protocol$address$TIMELINE_REST_PATH"))
+ }
+
+ /**
+ * Create a URI to the history service. This uses the entity type of
+ * [[YarnHistoryService#ENTITY_TYPE]] for spark application histories.
+ * @param conf hadoop configuration to examine
+ * @return
+ */
+ def timelineWebappUri(conf: Configuration): URI = {
+ timelineWebappUri(conf, YarnHistoryService.SPARK_EVENT_ENTITY_TYPE)
+ }
+
+ /**
+ * Get the URI of a path under the timeline web UI.
+ *
+ * @param conf configuration
+ * @param subpath path under the root web UI
+ * @return a URI
+ */
+ def timelineWebappUri(conf: Configuration, subpath: String): URI = {
+ val base = getTimelineEndpoint(conf)
+ new URL(base.toURL, subpath).toURI
+ }
+
+ /**
+ * Check the service configuration to see if the timeline service is enabled.
+ *
+ * @return true if `YarnConfiguration.TIMELINE_SERVICE_ENABLED` is set.
+ */
+ def timelineServiceEnabled(conf: Configuration): Boolean = {
+ conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
+ YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)
+ }
+
+ /**
+ * Get the URI to an application under the timeline
+ * (this requires the applicationID to have been used to
+ * publish entities there)
+ * @param timelineUri timeline URI
+ * @param appId App ID (really, the entityId used to publish)
+ * @return the path
+ */
+ def applicationURI(timelineUri: URI, appId: String): URI = {
+ require(appId != null && !appId.isEmpty, "No application ID")
+ require(!appId.contains("/"), s"Illegal character '/' in $appId")
+ timelineUri.resolve(s"${timelineUri.getPath()}/$appId")
+ }
+
+ /**
+ * Map an error code to a string. For known codes, it returns
+ * a description; for others it just returns the error code.
+ *
+ * @param code error code
+ * @return a string description for error messages
+ */
+ def timelineErrorCodeToString(code: Int): String = {
+ code match {
+ case 0 => "0: no error"
+ case 1 => "No start time"
+ case 2 => "IO Exception"
+ case 3 => "System Filter Conflict"
+ case 4 => "Access Denied"
+ case 5 => "No Domain"
+ case 6 => "Forbidden Relation"
+ case other: Int => s"Error code $other"
+ }
+ }
+
+ /**
+ * Convert a timeline error response to a slightly more meaningful string.
+ * @param error error
+ * @return text for diagnostics
+ */
+ def describeError(error: TimelinePutError): String = {
+ s"Entity ID=${error.getEntityId()}; Entity type=${error.getEntityType}" +
+ s" Error code ${error.getErrorCode}" +
+ s": ${timelineErrorCodeToString(error.getErrorCode)}"
+ }
+
+ /**
+ * Describe a put response by enumerating and describing all errors.
+ * (if present. A null `errors` element is handled robustly).
+ *
+ * @param response response to describe
+ * @return text for diagnostics
+ */
+ def describePutResponse(response: TimelinePutResponse) : String = {
+ val responseErrs = response.getErrors
+ if (responseErrs != null) {
+ val errors = mutable.MutableList(s"TimelinePutResponse with ${responseErrs.size()} errors")
+ for (err <- responseErrs.asScala) {
+ errors += describeError(err)
+ }
+ errors.foldLeft("")((buff, elt) => buff + "\n" + elt)
+ } else {
+ s"TimelinePutResponse with null error list"
+ }
+ }
+
+ /**
+ * This is used to highlight an undefined field.
+ */
+ val UNDEFINED_FIELD = "Undefined"
+
+ /**
+ * Lookup a field in the `otherInfo` section of a [[TimelineEntity]].
+ *
+ * @param en entity
+ * @param name field name
+ * @return the value or the string [[UNDEFINED_FIELD]] if not
+ * @throws Exception if the field is not found
+ */
+ def field(en: TimelineEntity, name: String) : Object = {
+ fieldOption(en, name).getOrElse(UNDEFINED_FIELD)
+ }
+
+ /**
+ * Lookup a field in the `otherInfo` section of a [[TimelineEntity]].
+ *
+ * @param en entity
+ * @param name field name
+ * @return the value
+ * @throws Exception if the field is not found
+ */
+ def fieldOption(en: TimelineEntity, name: String) : Option[Object] = {
+ Option(en.getOtherInfo.get(name))
+ }
+
+ /**
+ * Lookup a field in the `otherInfo` section of a [[TimelineEntity]]
+ * @param en entity
+ * @param name field name
+ * @return the value converted to a string
+ * @throws Exception if the field is not found
+ */
+ def stringFieldOption(en: TimelineEntity, name: String): Option[String] = {
+ val value = en.getOtherInfo.get(name)
+ if (value != null ) {
+ Some(value.toString)
+ } else {
+ None
+ }
+ }
+
+ /**
+ * Lookup a numeric field in the `otherInfo` section of a [[TimelineEntity]],
+ * fall back to `defval` if the field is absent or cannot be parsed.
+ *
+ * @param en entity
+ * @param name field name
+ * @param defval default value; default is 0L
+ * @return the value
+ */
+ def numberField(en: TimelineEntity, name: String, defval: Long = 0L) : Number = {
+ try {
+ fieldOption(en, name) match {
+ case Some(n: Number) => n
+ case _ => defval
+ }
+ } catch {
+ case NonFatal(e) => defval
+ }
+ }
+
+ /**
+ * Take a sequence of timeline events and return an ordered list of spark events.
+ *
+ * Important: this reverses the input in the process.
+ * @param events event sequence
+ * @return spark event sequence
+ */
+ def asSparkEvents(events: Seq[TimelineEvent]): Seq[SparkListenerEvent] = {
+ events.reverse.map { event =>
+ toSparkEvent(event)
+ }
+ }
+
+ /**
+ * Build date for display in status messages.
+ *
+ * @param timestamp time in milliseconds post-Epoch
+ * @param unset string to use if timestamp == 0
+ * @return a string for messages
+ */
+ def humanDateCurrentTZ(timestamp: Long, unset: String) : String = {
+ if (timestamp == 0) {
+ unset
+ } else {
+ val dateFormatter = DateFormat.getDateTimeInstance(DateFormat.DEFAULT, DateFormat.LONG)
+ dateFormatter.format(timestamp)
+ }
+ }
+
+ /**
+ * Short formatted time.
+ *
+ * @param timestamp time in milliseconds post-Epoch
+ * @param unset string to use if timestamp == 0
+ * @return a string for messages
+ */
+ def timeShort(timestamp: Long, unset: String) : String = {
+ if (timestamp == 0) {
+ unset
+ } else {
+ val dateFormatter = DateFormat.getTimeInstance(DateFormat.SHORT)
+ dateFormatter.format(timestamp)
+ }
+ }
+
+ /**
+ * Generate the timeline entity ID from the application and attempt ID.
+ * This is required to be unique across all entities in the timeline server.
+ *
+ * @param yarnAppId yarn application ID as passed in during creation
+ * @param yarnAttemptId YARN attempt ID as passed in during creation
+ */
+ def buildEntityId(yarnAppId: ApplicationId,
+ yarnAttemptId: Option[ApplicationAttemptId]): String = {
+ yarnAttemptId match {
+ case Some(aid) => aid.toString
+ case None => yarnAppId.toString
+ }
+ }
+
+ /**
+ * Generate the application ID for use in entity fields from the application and attempt ID.
+ *
+ * @param yarnAppId yarn application ID as passed in during creation
+ */
+ def buildApplicationIdField(yarnAppId: ApplicationId): String = {
+ yarnAppId.toString
+ }
+
+ /**
+ * Generate an attempt ID for use in the timeline entity "other/app_id" field
+ * from the application and attempt ID.
+ *
+ * This is not guaranteed to be unique across all entities. It is
+ * only required to be unique across all attempts of an application.
+ *
+ * If the application doesn't have an attempt ID, then it is
+ * an application instance which, implicitly, is single-attempt.
+ * The value [[SINGLE_ATTEMPT]] is returned
+ * @param sparkAttemptId attempt ID
+ * @return the attempt ID.
+ */
+ def buildApplicationAttemptIdField(sparkAttemptId: Option[String]): String = {
+ sparkAttemptId.getOrElse(SINGLE_ATTEMPT)
+ }
+
+ /**
+ * Add a filter and field if the value is set.
+ *
+ * @param entity entity to update
+ * @param name filter/field name
+ * @param value optional value
+ */
+ private def addFilterAndField(entity: TimelineEntity,
+ name: String, value: Option[String]): Unit = {
+ value.foreach { v => addFilterAndField(entity, name, v) }
+ }
+
+ /**
+ * Add a filter and field.
+ *
+ * @param entity entity to update
+ * @param name filter/field name
+ * @param value value
+ */
+ private def addFilterAndField(entity: TimelineEntity, name: String, value: String): Unit = {
+ entity.addPrimaryFilter(name, value)
+ entity.addOtherInfo(name, value)
+ }
+
+ /**
+ * Generate the entity ID from the application and attempt ID.
+ * Current policy is to use the attemptId, falling back to the YARN application ID.
+ *
+ * @param appId yarn application ID as passed in during creation
+ * @param attemptId yarn application ID
+ * @param sparkApplicationId application ID as submitted in the application start event
+ * @param sparkApplicationAttemptId attempt ID, or `None`
+ * @param appName application name
+ * @param userName user name
+ * @param startTime time in milliseconds when this entity was started (must be non zero)
+ * @param endTime time in milliseconds when this entity was last updated (0 means not ended)
+ * @param lastUpdated time in milliseconds when this entity was last updated (0 leaves unset)
+ * @return the timeline entity
+ */
+ def createTimelineEntity(
+ appId: ApplicationId,
+ attemptId: Option[ApplicationAttemptId],
+ sparkApplicationId: Option[String],
+ sparkApplicationAttemptId: Option[String],
+ appName: String,
+ userName: String,
+ startTime: Long, endTime: Long,
+ lastUpdated: Long): TimelineEntity = {
+ require(appId != null, "no application Id")
+ require(appName != null, "no application name")
+ require(startTime > 0, "no start time")
+
+ val entity: TimelineEntity = new TimelineEntity()
+ val entityId = buildEntityId(appId, attemptId)
+ val appIdField = buildApplicationIdField(appId)
+ entity.setEntityType(SPARK_EVENT_ENTITY_TYPE)
+ entity.setEntityId(entityId)
+ // add app/attempt ID information
+ addFilterAndField(entity, FIELD_APPLICATION_ID, appIdField)
+
+ entity.addOtherInfo(FIELD_ATTEMPT_ID,
+ buildApplicationAttemptIdField(sparkApplicationAttemptId))
+ entity.addOtherInfo(FIELD_APP_NAME, appName)
+ entity.addOtherInfo(FIELD_APP_USER, userName)
+ entity.addOtherInfo(FIELD_SPARK_VERSION, spark.SPARK_VERSION)
+ entity.addOtherInfo(FIELD_ENTITY_VERSION, entityVersionCounter.getAndIncrement())
+ started(entity, startTime)
+ if (endTime != 0) {
+ entity.addPrimaryFilter(FILTER_APP_END, FILTER_APP_END_VALUE)
+ entity.addOtherInfo(FIELD_END_TIME, endTime)
+ }
+ if (lastUpdated != 0) {
+ entity.addOtherInfo(FIELD_LAST_UPDATED, lastUpdated)
+ }
+ entity
+ }
+
+ /**
+ * Add the information to declare that an application has finished and that
+ * it has a start time and an end time.
+ *
+ * @param entity entity to update
+ * @param startTime start time
+ * @param endtime end time
+ * @param sparkApplicationId app ID
+ * @param sparkApplicationAttemptId optional attempt ID
+ * @return the updated entity
+ */
+ def completed(
+ entity: TimelineEntity,
+ startTime: Long,
+ endtime: Long,
+ sparkApplicationId: Option[String],
+ sparkApplicationAttemptId: Option[String]): TimelineEntity = {
+ entity.addOtherInfo(FIELD_ATTEMPT_ID,
+ buildApplicationAttemptIdField(sparkApplicationAttemptId))
+ // set the start info
+ started(entity, startTime)
+ // add the end info
+ entity.addPrimaryFilter(FILTER_APP_END, FILTER_APP_END_VALUE)
+ entity.addOtherInfo(FIELD_END_TIME, endtime)
+ // this must be the end time
+ entity.addOtherInfo(FIELD_LAST_UPDATED, endtime)
+ entity
+ }
+
+ /**
+ * Add the information to declare that an application has started and that
+ * it has a start time.
+ *
+ * @param entity entity to update
+ * @param startTime start time.
+ * @return the updated entity
+ */
+ def started(entity: TimelineEntity, startTime: Long): TimelineEntity = {
+ entity.addPrimaryFilter(FILTER_APP_START, FILTER_APP_START_VALUE)
+ entity.setStartTime(startTime)
+ entity.addOtherInfo(FIELD_START_TIME, startTime)
+ entity.addOtherInfo(FIELD_LAST_UPDATED, startTime)
+ entity
+ }
+
+ /**
+ * Simple sanity checks for endpoint address, including hostname lookup.
+ *
+ * This can be used to help validate the address on startup, to postpone
+ * later delays.
+ *
+ * @param endpoint address of service to talk to
+ * @return the URL passed in
+ */
+ def validateEndpoint(endpoint: URI): URI = {
+ val host = endpoint.getHost
+ if (host == null || host == "0.0.0.0") {
+ throw new NoRouteToHostException(s"Invalid host in $endpoint" +
+ s" - see https://wiki.apache.org/hadoop/UnsetHostnameOrPort")
+ }
+ val port = endpoint.getPort
+ if (port == 0) {
+ throw new NoRouteToHostException(s"Invalid Port in $endpoint" +
+ s" - see https://wiki.apache.org/hadoop/UnsetHostnameOrPort")
+ }
+ // get the address; will trigger a hostname lookup failure if the
+ // host is not resolveable.
+ val addr = new InetSocketAddress(host, port)
+ endpoint
+ }
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/package.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/package.scala
new file mode 100644
index 000000000000..809dd399245a
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/package.scala
@@ -0,0 +1,43 @@
+/*
+ * 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
+
+/**
+ * Contains the classes needed to listen to spark events and publish them to a YARN application
+ * timeline service.
+ *
+ * How it works
+ *
+ * 1. `YarnEventListener` subscribes to events in the current spark context.
+ *
+ * 2. These are forwarded to an instance of `YarnHistoryService`.
+ *
+ * 3. This, if enabled, publishes events to the configured ATS server.
+ *
+ * 4. The Spark History Service, is configured to use `YarnHistoryProvider`
+ * as its provider of history information.
+ *
+ * 5. It enumerates application instances and attempts, for display in the web UI and access
+ * via the REST API.
+ *
+ * 6. When details of a specific attempt is requested, it is retrieved from the ATS server.
+ *
+ * See: [[http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/TimelineServer.html]]
+ */
+package object yarn {
+
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/HttpOperationResponse.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/HttpOperationResponse.scala
new file mode 100644
index 000000000000..9aa6c48132f2
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/HttpOperationResponse.scala
@@ -0,0 +1,62 @@
+/*
+ * 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.yarn.rest
+
+/**
+ * A response for use as a return value from operations
+ */
+private[spark] class HttpOperationResponse {
+ /** HTTP response code */
+ var responseCode: Int = 0
+
+ /** last modified timestamp */
+ var lastModified: Long = 0L
+
+ /** Content-type value */
+ var contentType: String = ""
+
+ /** reponse message text */
+ var responseMessage: String = ""
+
+ /** response body as byte array */
+ var data: Array[Byte] = new Array(0)
+
+ override def toString: String = {
+ s"$responseCode $responseMessage; last modified $lastModified," +
+ s" contentType $contentType" +
+ (if (data == null) "" else s"data[${data.length}]")
+ }
+
+ /**
+ * Calculate the current response line
+ * @return an HTTP response ilne
+ */
+ def responseLine: String = s"$responseCode $responseMessage"
+
+ /**
+ * Convert the response data into a string and return it.
+ *
+ * There must be some response data for this to work
+ * @return the body as a string.
+ */
+ def responseBody: String = {
+ require(data != null, s"no response body in $this")
+ new String(data)
+ }
+
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/HttpRequestException.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/HttpRequestException.scala
new file mode 100644
index 000000000000..e195a324abd4
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/HttpRequestException.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.yarn.rest
+
+import java.io.IOException
+
+/**
+ * Specific HTTP error request exception which includes the error code and other information.
+ *
+ * @param status HTTP status code
+ * @param verb HTTP verb
+ * @param url URL of request
+ * @param message non-null error message
+ * @param body optional body.
+ */
+private[spark] class HttpRequestException(val status: Int,
+ val verb: String,
+ val url: String,
+ val message: String,
+ val body: String = "")
+ extends IOException(message) {
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/JerseyBinding.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/JerseyBinding.scala
new file mode 100644
index 000000000000..6cd831b0087a
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/JerseyBinding.scala
@@ -0,0 +1,255 @@
+/*
+ * 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.yarn.rest
+
+import java.io.{FileNotFoundException, IOException}
+import java.lang.reflect.UndeclaredThrowableException
+import java.net.{HttpURLConnection, URI, URL}
+import javax.servlet.http.HttpServletResponse
+import javax.ws.rs.core.MediaType
+
+import com.sun.jersey.api.client.{Client, ClientHandlerException, ClientResponse, UniformInterfaceException}
+import com.sun.jersey.api.client.config.{ClientConfig, DefaultClientConfig}
+import com.sun.jersey.api.json.JSONConfiguration
+import com.sun.jersey.client.urlconnection.{HttpURLConnectionFactory, URLConnectionClientHandler}
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL
+import org.codehaus.jackson.jaxrs.JacksonJaxbJsonProvider
+import org.codehaus.jackson.map.ObjectMapper
+import org.codehaus.jackson.map.annotate.JsonSerialize.Inclusion
+import org.codehaus.jackson.xc.JaxbAnnotationIntrospector
+
+import org.apache.spark.Logging
+import org.apache.spark.util.Utils
+
+/**
+ * Jersey specific integration with the SPNEGO Auth
+ * @param conf configuration to build off
+ */
+private[spark] class JerseyBinding(conf: Configuration,
+ token: DelegationTokenAuthenticatedURL.Token)
+ extends Logging with HttpURLConnectionFactory {
+ private val connector = SpnegoUrlConnector.newInstance(conf, token)
+ private val handler = new URLConnectionClientHandler(this)
+
+ override def getHttpURLConnection(url: URL): HttpURLConnection = {
+ connector.getHttpURLConnection(url)
+ }
+
+ /**
+ * reset the token
+ */
+ def resetToken(): Unit = {
+ connector.resetAuthTokens()
+ }
+
+ /**
+ * Create a Jersey client with the UGI binding set up.
+ * @param conf Hadoop configuration
+ * @param clientConfig jersey client config
+ * @return a new client instance
+ */
+ def createClient(conf: Configuration, clientConfig: ClientConfig): Client = {
+ new Client(handler, clientConfig)
+ }
+}
+
+private[spark] object JerseyBinding extends Logging {
+
+ /**
+ * Translate exceptions, where possible. If not, it is passed through unchanged
+ * @param verb HTTP verb
+ * @param targetURL URL of operation
+ * @param thrown exception caught
+ * @return an exception to log, ingore, throw...
+ */
+ def translateException(
+ verb: String,
+ targetURL: URI,
+ thrown: Throwable): Throwable = {
+ thrown match {
+ case ex: ClientHandlerException =>
+ // client-side Jersey exception
+ translateException(verb, targetURL, ex)
+
+ case ex: UniformInterfaceException =>
+ // remote Jersey exception
+ translateException(verb, targetURL, ex)
+
+ case ex: UndeclaredThrowableException =>
+ // wrapped exception raised in a doAs() call. Extract cause and retry
+ translateException(verb, targetURL, ex.getCause)
+
+ case _ =>
+ // anything else
+ thrown
+ }
+ }
+
+ /**
+ * Handle a client-side Jersey exception by extracting the inner cause.
+ *
+ * If there's an inner IOException, return that.
+ *
+ * Otherwise: create a new wrapper IOE including verb and target details
+ * @param verb HTTP Verb used
+ * @param targetURL URL being targeted
+ * @param exception original exception
+ * @return an exception to throw
+ */
+ def translateException(
+ verb: String,
+ targetURL: URI,
+ exception: ClientHandlerException): IOException = {
+ val uri = if (targetURL !=null) targetURL.toString else "unknown URL"
+ exception.getCause match {
+ case ioe: IOException =>
+ // pass through
+ ioe
+ case other: Throwable =>
+ // get inner cause into exception text
+ val ioe = new IOException(s"$verb $uri failed: $exception - $other")
+ ioe.initCause(exception)
+ ioe
+ case _ =>
+ // no inner cause
+ val ioe = new IOException(s"$verb $uri failed: $exception")
+ ioe.initCause(exception)
+ ioe
+ }
+ }
+
+ /**
+ * Get the body of a response. A failure to extract the body is logged
+ * @param response response
+ * @param limit the limit: 0 means "the entire HTTP response"
+ * @return string body; may mean "" an empty body or the operation failed.
+ */
+ def bodyOfResponse(response: ClientResponse, limit: Int = 0) : String = {
+ var body: String = ""
+ Utils.tryLogNonFatalError {
+ if (response.hasEntity) {
+ body = response.getEntity(classOf[String])
+ }
+ }
+ // shorten the body
+ if (limit > 0) {
+ body.substring(0, Math.min(limit, body.length))
+ } else {
+ body
+ }
+ }
+
+ /**
+ * Convert Jersey exceptions into useful IOExceptions. This includes
+ * building an error message which include the URL, verb and status code,
+ * logging any text body, and wrapping in an IOException or subclass
+ * with that message and the response's exception as a nested exception.
+ * @param verb HTTP Verb used
+ * @param targetURL URL being targeted
+ * @param exception original exception
+ * @return a new exception, the original one nested as a cause
+ */
+ def translateException(
+ verb: String,
+ targetURL: URI,
+ exception: UniformInterfaceException): IOException = {
+ var ioe: IOException = null
+ val response = exception.getResponse
+ val url = if (targetURL != null) targetURL.toString else "unknown URL"
+ if (response != null) {
+ val status = response.getStatus
+ val body = bodyOfResponse(response, 256)
+ val errorText = s"Bad $verb request: status code $status against $url; $body"
+ status match {
+
+ case HttpServletResponse.SC_UNAUTHORIZED =>
+ ioe = new UnauthorizedRequestException(url,
+ s"Unauthorized (401) access to $url",
+ exception)
+
+ case HttpServletResponse.SC_FORBIDDEN =>
+ ioe = new UnauthorizedRequestException(url,
+ s"Forbidden (403) access to $url",
+ exception)
+
+ case HttpServletResponse.SC_BAD_REQUEST
+ | HttpServletResponse.SC_NOT_ACCEPTABLE
+ | HttpServletResponse.SC_UNSUPPORTED_MEDIA_TYPE =>
+ throw new IOException(errorText, exception)
+
+ case HttpServletResponse.SC_NOT_FOUND =>
+ ioe = new FileNotFoundException(s"$url; $body")
+ ioe.initCause(exception)
+
+ case resultCode: Int if resultCode >= 400 && resultCode < 500 =>
+ ioe = new FileNotFoundException(
+ s"Bad $verb request: status code $status against $url; $body")
+ ioe.initCause(exception)
+
+ case _ =>
+ ioe = new IOException(errorText, exception)
+
+ }
+ } else {
+ // no response
+ ioe = new IOException(s"$verb $url failed: $exception", exception)
+ }
+ ioe
+ }
+
+ /**
+ * Create a Jersey client with the UGI binding set up.
+ * @param conf Hadoop configuration
+ * @param clientConfig jersey client config
+ * @param token optional delegation token
+ * @return a new client instance
+ */
+ def createJerseyClient(
+ conf: Configuration,
+ clientConfig: ClientConfig,
+ token: DelegationTokenAuthenticatedURL.Token = new DelegationTokenAuthenticatedURL.Token)
+ : Client = {
+ new JerseyBinding(conf, token).createClient(conf, clientConfig)
+ }
+
+ /**
+ * Create the client config for Jersey.
+ * @return the client configuration
+ */
+ def createClientConfig(): ClientConfig = {
+ val cc = new DefaultClientConfig()
+ cc.getClasses().add(classOf[JsonJaxbBinding])
+ cc.getFeatures.put(JSONConfiguration.FEATURE_POJO_MAPPING, true)
+ cc
+ }
+}
+
+/**
+ * Define the jaxb binding for the Jersey client
+ */
+private[spark] class JsonJaxbBinding extends JacksonJaxbJsonProvider {
+
+ override def locateMapper(classtype: Class[_], mediaType: MediaType): ObjectMapper = {
+ val mapper = super.locateMapper(classtype, mediaType)
+ val introspector = new JaxbAnnotationIntrospector
+ mapper.setAnnotationIntrospector(introspector)
+ mapper.setSerializationInclusion(Inclusion.NON_NULL)
+ mapper
+ }
+
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/LoggingKerberosDelegationTokenAuthenticator.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/LoggingKerberosDelegationTokenAuthenticator.scala
new file mode 100644
index 000000000000..fce815098bde
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/LoggingKerberosDelegationTokenAuthenticator.scala
@@ -0,0 +1,76 @@
+/*
+ * 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.yarn.rest
+
+import java.net.URL
+
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL.{Token => AuthToken}
+import org.apache.hadoop.security.token.Token
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator
+
+import org.apache.spark.Logging
+
+/**
+ * Extend the Kerberos Delegation Token Authenticator with some logging.
+ */
+private[spark] class LoggingKerberosDelegationTokenAuthenticator
+ extends KerberosDelegationTokenAuthenticator with Logging {
+
+ override def authenticate(url: URL, token: AuthToken): Unit = {
+ val orig = tokenToString(token)
+ super.authenticate(url, token)
+ val updated = tokenToString(token)
+ if (updated != orig) {
+ logInfo(s"Token updated against $url")
+ logDebug(s"New Token: $updated")
+ }
+ }
+
+ /**
+ * Gets a token to a string. If the token value is null, the string "(unset)" is returned.
+ *
+ * In contrast, `Token.toString()` returns `null`
+ * @param token token to stringify
+ * @return a string value for logging
+ */
+ def tokenToString(token: AuthenticatedURL.Token): String = {
+ if (token.isSet) token.toString else "(unset)"
+ }
+
+ /**
+ * Low level token renewal operation
+ */
+ override def renewDelegationToken(
+ url: URL,
+ token: AuthToken,
+ dToken: Token[AbstractDelegationTokenIdentifier],
+ doAsUser: String): Long = {
+
+ val orig = dToken.toString
+ val user = if (doAsUser != null) s"user=$doAsUser" else ""
+ logInfo(s"Renewing token against $url $user")
+ val result = super.renewDelegationToken(url, token, dToken, doAsUser)
+ val updated = dToken.toString
+ if (updated != orig) {
+ logInfo(s"Delegation Token updated against $url")
+ logDebug(s"New Token: $updated")
+ }
+ result
+ }
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/PrivilegedFunction.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/PrivilegedFunction.scala
new file mode 100644
index 000000000000..7d84ec8e3cb1
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/PrivilegedFunction.scala
@@ -0,0 +1,33 @@
+/*
+ * 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.yarn.rest
+
+import java.security.PrivilegedExceptionAction
+
+/**
+ * Take any function `() => Type` and apply it as a privileged action.
+ *
+ * @param function function to apply
+ * @tparam T return type of the function
+ */
+private[spark] class PrivilegedFunction[T](function: (() => T))
+ extends PrivilegedExceptionAction[T] {
+ override def run(): T = {
+ function()
+ }
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/SpnegoUrlConnector.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/SpnegoUrlConnector.scala
new file mode 100644
index 000000000000..044b26b643b5
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/SpnegoUrlConnector.scala
@@ -0,0 +1,432 @@
+/*
+ * 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.yarn.rest
+
+import java.io.{FileNotFoundException, IOException}
+import java.net.{HttpURLConnection, NoRouteToHostException, URL, URLConnection}
+import javax.net.ssl.HttpsURLConnection
+import javax.servlet.http.HttpServletResponse
+
+import org.apache.commons.io.IOUtils
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.security.authentication.client.{AuthenticatedURL, AuthenticationException, Authenticator, ConnectionConfigurator, KerberosAuthenticator, PseudoAuthenticator}
+import org.apache.hadoop.security.ssl.SSLFactory
+import org.apache.hadoop.security.token.delegation.web.{DelegationTokenAuthenticatedURL, DelegationTokenAuthenticator, PseudoDelegationTokenAuthenticator}
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL.Token
+
+import org.apache.spark.Logging
+
+/**
+ * SPNEGO-backed URL connection factory for Java net, and hence Jersey.
+ * Based on WebHFDS client code in
+ * `org.apache.hadoop.hdfs.web.URLConnectionFactory`
+ */
+private[spark] class SpnegoUrlConnector(
+ connConfigurator: ConnectionConfigurator,
+ delegationToken: DelegationTokenAuthenticatedURL.Token) extends Logging {
+
+ val secure = UserGroupInformation.isSecurityEnabled
+ var authToken = new AuthenticatedURL.Token
+
+ // choose an authenticator based on security settings
+ val delegationAuthenticator: DelegationTokenAuthenticator =
+ if (secure) {
+ new LoggingKerberosDelegationTokenAuthenticator()
+ } else {
+ new PseudoDelegationTokenAuthenticator()
+ }
+
+ init()
+
+ /**
+ * Initialization operation sets up authentication
+ */
+ private def init(): Unit = {
+ logDebug(s"using $delegationAuthenticator for authentication")
+ delegationAuthenticator.setConnectionConfigurator(connConfigurator)
+ }
+
+ /**
+ * Opens a URL
+ *
+ * @param url URL to open
+ * @param isSpnego whether the url should be authenticated via SPNEGO
+ * @return URLConnection
+ * @throws IOException IO problems
+ * @throws AuthenticationException authentication failure
+ */
+
+ def openConnection(url: URL, isSpnego: Boolean): URLConnection = {
+ require(connConfigurator != null)
+ require(url.getPort != 0, "no port")
+ if (isSpnego) {
+ logDebug(s"open AuthenticatedURL connection $url")
+ UserGroupInformation.getCurrentUser.checkTGTAndReloginFromKeytab()
+ val authToken = new AuthenticatedURL.Token
+ val authurl = new AuthenticatedURL(KerberosUgiAuthenticator, connConfigurator)
+ authurl.openConnection(url, authToken)
+ } else {
+ logDebug(s"open URL connection $url")
+ val connection = url.openConnection()
+ connection match {
+ case connection1: HttpURLConnection =>
+ connConfigurator.configure(connection1)
+ case _ =>
+ }
+ connection
+ }
+ }
+
+ /**
+ * Opens a URL authenticated as the current user.
+ * A TGT check and login is done first, so the
+ * caller will always be logged in before the operation takes place
+ *
+ * @param url URL to open
+ * @return URLConnection
+ * @throws IOException problems.
+ */
+ def getHttpURLConnection(url: URL): HttpURLConnection = {
+ val callerUGI = UserGroupInformation.getCurrentUser
+ callerUGI.checkTGTAndReloginFromKeytab()
+ openConnection(url, callerUGI)
+ }
+
+ /**
+ * Open a connection as the specific user.
+ *
+ * @param url URL to open
+ * @param callerUGI identity of the caller
+ * @return the open connection
+ */
+ private def openConnection(url: URL, callerUGI: UserGroupInformation): HttpURLConnection = {
+ // special sanity check for 0.0.0.0, as the entire TCP stack forgot to do this.
+ if (url.getHost == "0.0.0.0") {
+ throw new NoRouteToHostException(s"Cannot connect to a host with no address: $url")
+ }
+ val conn = callerUGI.doAs(new PrivilegedFunction(
+ () => {
+ try {
+ new AuthenticatedURL(KerberosUgiAuthenticator, connConfigurator)
+ .openConnection(url, authToken)
+ } catch {
+ case ex: AuthenticationException =>
+ // auth failure
+ throw new UnauthorizedRequestException(url.toString,
+ s"Authentication failure as $callerUGI against $url: $ex", ex)
+ case other: Throwable =>
+ // anything else is rethrown
+ throw other
+ }
+ }))
+ conn.setUseCaches(false)
+ conn.setInstanceFollowRedirects(true)
+ conn
+ }
+
+ /**
+ * Get the current delegation token.
+ * @return the current delegation token
+ */
+ def getDelegationToken: Token = {
+ this.synchronized {
+ delegationToken
+ }
+ }
+
+ /**
+ * Get the auth token.
+ * @return the current auth token
+ */
+ def getAuthToken: AuthenticatedURL.Token = {
+ this.synchronized {
+ authToken
+ }
+ }
+
+ /**
+ * Reset the auth tokens.
+ */
+ def resetAuthTokens(): Unit = {
+ logInfo("Resetting the auth tokens")
+ this.synchronized {
+ authToken = new AuthenticatedURL.Token
+ delegationToken.setDelegationToken(null)
+ }
+ }
+
+ /**
+ * Execute an HTTP operation
+ * @param verb request method
+ * @param url URL
+ * @param payload payload, default is `null`
+ * @param payloadContentType content type. Required if payload != null
+ * @return the response
+ */
+ def execHttpOperation(
+ verb: String,
+ url: URL,
+ payload: Array[Byte] = null,
+ payloadContentType: String = ""): HttpOperationResponse = {
+ var conn: HttpURLConnection = null
+ val outcome = new HttpOperationResponse()
+ var resultCode = 0
+ var body: Array[Byte] = null
+ logDebug(s"$verb $url spnego=$secure")
+ val hasData = payload != null
+ try {
+ conn = getHttpURLConnection(url)
+ conn.setRequestMethod(verb)
+ conn.setDoOutput(hasData)
+ if (hasData) {
+ require(payloadContentType != null, "no content type")
+ conn.setRequestProperty("Content-Type", payloadContentType)
+ }
+ // connection
+ conn.connect()
+ if (hasData) {
+ // submit any data
+ val output = conn.getOutputStream
+ try {
+ IOUtils.write(payload, output)
+ } finally {
+ output.close()
+ }
+ }
+ resultCode = conn.getResponseCode
+ outcome.responseCode = resultCode
+ outcome.responseMessage = conn.getResponseMessage
+ outcome.lastModified = conn.getLastModified
+ outcome.contentType = conn.getContentType
+
+ var stream = conn.getErrorStream
+ if (stream == null) {
+ stream = conn.getInputStream
+ }
+ if (stream != null) {
+ outcome.data = IOUtils.toByteArray(stream)
+ } else {
+ log.debug("No body in response")
+ }
+ // now check to see if the response indicated that the delegation token had expired.
+ } finally {
+ if (conn != null) {
+ conn.disconnect()
+ }
+ }
+ if (SpnegoUrlConnector.delegationTokensExpired(outcome)) {
+ logInfo(s"Delegation token may have expired")
+ resetAuthTokens()
+ throw new UnauthorizedRequestException(url.toString,
+ s"Authentication failure: ${outcome.responseLine}")
+ }
+ SpnegoUrlConnector.uprateFaults(verb, url.toString, outcome)
+ outcome
+ }
+
+}
+
+
+/**
+ * Use UserGroupInformation as a fallback authenticator
+ * if the server does not use Kerberos SPNEGO HTTP authentication.
+ */
+private object KerberosUgiAuthenticator extends KerberosAuthenticator {
+
+ private object UgiAuthenticator extends PseudoAuthenticator {
+ protected override def getUserName: String = {
+ try {
+ UserGroupInformation.getLoginUser.getUserName
+ } catch {
+ case e: IOException => {
+ throw new SecurityException("Failed to obtain current username", e)
+ }
+ }
+ }
+ }
+
+ protected override def getFallBackAuthenticator: Authenticator = {
+ UgiAuthenticator
+ }
+}
+
+/**
+ * Default Connection Configurator: simply sets the socket timeout
+ */
+private object DefaultConnectionConfigurator extends ConnectionConfigurator {
+ override def configure(conn: HttpURLConnection): HttpURLConnection = {
+ SpnegoUrlConnector.setTimeouts(conn, SpnegoUrlConnector.DEFAULT_SOCKET_TIMEOUT)
+ conn
+ }
+}
+
+private[spark] object SpnegoUrlConnector extends Logging {
+
+ /**
+ * Timeout for socket connects and reads
+ */
+ val DEFAULT_SOCKET_TIMEOUT: Int = 1 * 60 * 1000
+
+ /**
+ * Sets timeout parameters on the given URLConnection.
+ *
+ * @param connection URLConnection to set
+ * @param socketTimeout the connection and read timeout of the connection.
+ */
+ def setTimeouts(connection: URLConnection, socketTimeout: Int) {
+ connection.setConnectTimeout(socketTimeout)
+ connection.setReadTimeout(socketTimeout)
+ }
+
+ /**
+ * Construct a new URLConnectionFactory based on the configuration.
+ *
+ * @param conf configuration
+ * @param token delegation token
+ * @return a new instance
+ */
+ def newInstance(conf: Configuration,
+ token: DelegationTokenAuthenticatedURL.Token): SpnegoUrlConnector = {
+ var conn: ConnectionConfigurator = null
+ try {
+ conn = newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf)
+ } catch {
+ case e: Exception => {
+ logInfo("Cannot load customized SSL related configuration." +
+ " Fallback to system-generic settings.")
+ logDebug("Exception details", e)
+ conn = DefaultConnectionConfigurator
+ }
+ }
+ new SpnegoUrlConnector(conn, token)
+ }
+
+ /**
+ * Create a new ConnectionConfigurator for SSL connections.
+ *
+ * This uses `org.apache.hadoop.security.ssl.SSLFactory` to build an
+ * SSL factory based on information provided in the Hadoop configuration
+ * object passed in, including the hostname verifier specified
+ * in the configuration options.
+ *
+ * @param timeout timeout in millis
+ * @param conf configuration
+ * @return a [[SSLConnConfigurator]] instance
+ */
+ def newSslConnConfigurator(timeout: Int, conf: Configuration): ConnectionConfigurator = {
+ val factory = new SSLFactory(SSLFactory.Mode.CLIENT, conf)
+ factory.init()
+ new SSLConnConfigurator(timeout, factory )
+ }
+
+ /**
+ * The SSL connection configurator
+ * @param timeout connection timeout
+ * @param factory socket factory
+ */
+ private[rest] class SSLConnConfigurator(timeout: Int, factory: SSLFactory)
+ extends ConnectionConfigurator {
+ val sf = factory.createSSLSocketFactory
+ val hv = factory.getHostnameVerifier
+
+ @throws(classOf[IOException])
+ def configure(conn: HttpURLConnection): HttpURLConnection = {
+ conn match {
+ case c: HttpsURLConnection =>
+ c.setSSLSocketFactory(sf)
+ c.setHostnameVerifier(hv)
+ case _ =>
+ }
+ setTimeouts(conn, timeout)
+ conn
+ }
+ }
+
+ /**
+ * Uprate error codes 400 and up into exceptions, which are then thrown
+ *
+ * 1. 404 is converted to a `FileNotFoundException`
+ * 2. 401 and 404 to `UnauthorizedRequestException`
+ * 3. All others above 400: `IOException`
+ * 4. Any error code under 400 is not considered a failure; this function will return normally.
+ *
+ * @param verb HTTP Verb used
+ * @param url URL as string
+ * @param response response from the request
+ * @throws IOException if the result code was 400 or higher
+ */
+ @throws(classOf[IOException])
+ def uprateFaults(verb: String, url: String, response: HttpOperationResponse): Unit = {
+ val resultCode: Int = response.responseCode
+ val body: Array[Byte] = response.data
+ val errorText = s"$verb $url"
+ resultCode match {
+
+ // 401 & 403
+ case HttpServletResponse.SC_UNAUTHORIZED | HttpServletResponse.SC_FORBIDDEN =>
+ throw new UnauthorizedRequestException(url, errorText)
+
+ case 404 =>
+ throw new FileNotFoundException(errorText)
+
+ case resultCode: Int if resultCode >= 400 =>
+ val bodyText = if (body != null && body.length > 0) {
+ new String(body)
+ } else {
+ ""
+ }
+ val message = s"$errorText failed with exit code $resultCode, body length" +
+ s" ${bodyText.length }\n${bodyText }"
+ logError(message)
+ throw new HttpRequestException(resultCode, verb, url, message, bodyText)
+
+ case _ =>
+ // success
+ }
+ }
+
+ /** Error text received warning anonymous requests are forbidden */
+ val ANONYMOUS_REQUESTS_DISALLOWED = "Anonymous requests are disallowed"
+
+ /** Error text received on an invalid signature */
+ val INVALID_SIGNATURE = "Invalid signature"
+
+ /**
+ * Look for specific error codes or messages as a cue triggering token renewal.
+ * Looking for specific text in exception codes is awful, but it is what other
+ * Hadoop SPNEGO client libraries do.
+ *
+ * @param response response to analyse
+ * @return true if this is a 40x exception that may be addressed by renewing
+ * the delegation tokens
+ */
+ def delegationTokensExpired(response: HttpOperationResponse): Boolean = {
+ val message = response.responseMessage
+ response.responseCode match {
+ case HttpURLConnection.HTTP_UNAUTHORIZED =>
+ true
+ case HttpURLConnection.HTTP_FORBIDDEN
+ if message == ANONYMOUS_REQUESTS_DISALLOWED || message.contains(INVALID_SIGNATURE) =>
+ true
+ case _ =>
+ false
+ }
+ }
+
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/UnauthorizedRequestException.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/UnauthorizedRequestException.scala
new file mode 100644
index 000000000000..8d0a1f1f5ab6
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/UnauthorizedRequestException.scala
@@ -0,0 +1,31 @@
+/*
+ * 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.yarn.rest
+
+import java.io.IOException
+
+/**
+ * Special exception to indicate a request was forbidden/unauthorized.
+ * This implies that it's potentially a token-renewal-related failure.
+ * @param msg text to use in the message
+ * @param cause optional cause
+ */
+private[spark] class UnauthorizedRequestException(
+ url: String,
+ private val msg: String, private val cause: Throwable = null)
+ extends IOException(msg, cause) {
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/package.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/package.scala
new file mode 100644
index 000000000000..4e7df0a87091
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/rest/package.scala
@@ -0,0 +1,32 @@
+/*
+ * 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.yarn
+
+/**
+ * This package contains a Jersey client to Hadoop REST services.
+ *
+ * The majority of the complexity of this code is to deal with Kerberos-based authentication
+ * over a SPNEGO channel. It is derived from *multiple* implementations inside the Hadoop
+ * source tree, and is designed to work from a client which has the Kerberos credentials
+ * to request an Authentication Token from the service endpoint.
+ *
+ * That is: *delegation tokens are not supported*.
+ */
+package object rest {
+
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/ApplicationListingResults.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/ApplicationListingResults.scala
new file mode 100644
index 000000000000..4fcbea383e2e
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/ApplicationListingResults.scala
@@ -0,0 +1,115 @@
+/*
+ * 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.yarn.server
+
+import org.apache.spark.Logging
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+
+/**
+ * (Immutable) results of a list operation.
+ *
+ * @param timestamp timestamp in milliseconds
+ * @param applications application listing. These must be pre-sorted
+ * @param failureCause optional exception raised (implies operation was a failure)
+ */
+private[spark] class ApplicationListingResults(
+ val timestamp: Long,
+ val applications: Seq[TimelineApplicationHistoryInfo],
+ val failureCause: Option[Throwable]) extends Logging {
+
+ /**
+ * Predicate which is true if the listing failed; that there
+ * is a failure cause value.
+ *
+ * @return true if the listing failed
+ */
+ def failed: Boolean = { failureCause.isDefined }
+
+ /**
+ * Did the listing operation succeed? The opposite of [[failed]]
+ * @return true if the last operation did not fail
+ */
+ def succeeded: Boolean = { !failed }
+
+ /**
+ * Get an updated time for display.
+ * @return a printable date/time value or or "never"
+ */
+ def updated: String = {
+ humanDateCurrentTZ(timestamp, YarnHistoryProvider.TEXT_NEVER_UPDATED)
+ }
+
+ /**
+ * The number of applications in the list.
+ *
+ * @return the list size
+ */
+ def size: Int = {
+ applications.size
+ }
+
+ /**
+ * Look up an application by its ID
+ * @param applicationId application ID
+ * @return (app, attempt) options.
+ */
+ def lookup(applicationId: String): Option[TimelineApplicationHistoryInfo] = {
+ applications.find(_.id == applicationId)
+ }
+
+ /**
+ * Look up an attempt from the attempt ID passed down in the spark history
+ * server `getSparkUI()` operations; return the tuple of `(app, attempt, attempts)`.
+ *
+ * `None` for app means: the application wasn't found; in this case
+ * `attempt` is always `None`, `attempts` empty.
+ *
+ * If the application is set, but the attempt undefined, then the application could be found,
+ * but not the specific attempt. In this case, the `attempts` sequence will list all known
+ * attempts.
+ * @param appId application ID
+ * @param attemptId attempt ID
+ * @return (app, attempt, attempt) options.
+ */
+ def lookupAttempt(appId: String, attemptId: Option[String])
+ : (Option[TimelineApplicationHistoryInfo], Option[TimelineApplicationAttemptInfo],
+ List[TimelineApplicationAttemptInfo] ) = {
+ val foundApp = lookup(appId)
+ if (foundApp.isEmpty) {
+ return (None, None, Nil)
+ }
+ val attempts = foundApp.get.attempts
+ if (attempts.isEmpty) {
+ return (foundApp, None, attempts)
+ }
+ // henceforth attempts is always non-empty.
+ // look to see if there's an attempt ID
+
+ if (attemptId.isEmpty) {
+ return (foundApp, Some(attempts.head), attempts)
+ }
+
+ // here there i
+ val entityId = attemptId.get
+
+ // scan the list of app attempts to ensure that the attempt is associated
+ // with the app; return no match if it is not
+ val attemptInfo = attempts.find( _.attemptId.get == entityId)
+ (foundApp, attemptInfo, attempts)
+ }
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/TimelineApplicationAttemptInfo.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/TimelineApplicationAttemptInfo.scala
new file mode 100644
index 000000000000..128794747ff4
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/TimelineApplicationAttemptInfo.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.yarn.server
+
+import org.apache.spark.deploy.history.{ApplicationHistoryInfo, ApplicationAttemptInfo}
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+
+/**
+ * Extend [[ApplicationAttemptInfo]] with information about the entityID; this allows
+ * the attemptId to be set to something in the web UI for people, rather than
+ * display the YARN attempt ID used to retrieve it from the timeline server.
+ *
+ * @param attemptId attemptID for GUI
+ * @param startTime start time in millis
+ * @param endTime end time in millis (or 0)
+ * @param lastUpdated updated time in millis
+ * @param sparkUser user
+ * @param completed flag true if completed
+ * @param entityId ID of the YARN timeline server entity containing the data
+ */
+private[spark] class TimelineApplicationAttemptInfo(
+ attemptId: Option[String],
+ startTime: Long,
+ endTime: Long,
+ lastUpdated: Long,
+ sparkUser: String,
+ completed: Boolean,
+ val entityId: String,
+ val sparkAttemptId: Option[String],
+ val version: Long = 0)
+ extends ApplicationAttemptInfo(attemptId,
+ startTime,
+ endTime,
+ lastUpdated,
+ sparkUser,
+ completed) {
+
+ /**
+ * Describe the application history, including timestamps and completed flag.
+ *
+ * @return a string description
+ */
+ override def toString: String = {
+ val never = "-"
+ s"""TimelineApplicationAttemptInfo: attemptId $attemptId,
+ | completed = $completed,
+ | sparkAttemptId $sparkAttemptId,
+ | started ${timeShort(startTime, never)},
+ | ended ${timeShort(endTime, never)},
+ | updated ${timeShort(lastUpdated, never)},
+ | sparkUser = $sparkUser,
+ | version = $version,
+ """.stripMargin
+ }
+
+}
+
+private[spark] class TimelineApplicationHistoryInfo(
+ override val id: String,
+ override val name: String,
+ override val attempts: List[TimelineApplicationAttemptInfo])
+ extends ApplicationHistoryInfo(id, name, attempts)
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/TimelineQueryClient.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/TimelineQueryClient.scala
new file mode 100644
index 000000000000..4b4156814f0a
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/TimelineQueryClient.scala
@@ -0,0 +1,428 @@
+/*
+ * 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.yarn.server
+
+import java.io.{Closeable, FileNotFoundException}
+import java.net.{URI, URL}
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
+import javax.servlet.http.HttpServletResponse
+import javax.ws.rs.core.MediaType
+
+import scala.collection.JavaConverters._
+
+import com.sun.jersey.api.client.{Client, ClientResponse, WebResource}
+import com.sun.jersey.api.client.config.ClientConfig
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL
+import org.apache.hadoop.yarn.api.records.timeline.{TimelineEntities, TimelineEntity}
+import org.codehaus.jackson.annotate.{JsonAnySetter, JsonIgnoreProperties}
+import org.json4s.JString
+import org.json4s.jackson.JsonMethods._
+
+import org.apache.spark.Logging
+import org.apache.spark.deploy.history.yarn.rest.{HttpRequestException, JerseyBinding, UnauthorizedRequestException}
+
+/**
+ * A class to make queries of the Timeline sever through a Jersey client.
+ * Exceptions raised by Jersey are extracted and potentially translated.
+ * @param timelineURI URI of the timeline service. This is the base URI,
+ * without the `ws/...` subpath
+ * @param conf Hadoop configuration
+ * @param jerseyClientConfig Jersey client configuration
+ */
+private[spark] class TimelineQueryClient(
+ timelineURI: URI,
+ conf: Configuration,
+ jerseyClientConfig: ClientConfig)
+ extends Logging with Closeable {
+ require(timelineURI != null, "Null timelineURI")
+
+ /**
+ * bool to stop `close()` executing more than once
+ */
+ private val closed = new AtomicBoolean(false)
+ private val timelineURL = timelineURI.toURL
+ private val retryLimit = 3
+ private val retry_interval = 100
+ private val APPLICATION_JSON = "application/json"
+
+ /**
+ * the delegation token (unused until delegation support implemented)
+ */
+ private var token = new DelegationTokenAuthenticatedURL.Token
+
+ /**
+ * The last time there was a token renewal operation.
+ */
+ private val _lastTokenRenewal = new AtomicLong(0)
+ private val _tokenRenewalCount = new AtomicLong(0)
+
+ /**
+ * Jersey binding -this exposes the method to reset the token
+ */
+ private val jerseyBinding = new JerseyBinding(conf, token)
+
+ /**
+ * Jersey Client using config from constructor
+ */
+ private val jerseyClient: Client = jerseyBinding.createClient(conf, jerseyClientConfig)
+
+ /**
+ * Base resource of ATS
+ */
+ private val timelineResource = jerseyClient.resource(timelineURI)
+
+ /**
+ * Trigger initialization
+ */
+ init()
+
+ private def init(): Unit = {
+ logDebug("logging in ")
+ // this operation has side effects including triggering a refresh thread, if not already running
+ val user = UserGroupInformation.getLoginUser()
+ logInfo(s"User = $user")
+ // now do an initial checkin
+ UserGroupInformation.getCurrentUser.checkTGTAndReloginFromKeytab()
+ }
+
+ /**
+ * When this instance is closed, the jersey client is stopped
+ */
+ override def close(): Unit = {
+ if (!closed.getAndSet(true)) {
+ jerseyClient.destroy()
+ }
+ }
+
+ /**
+ * Get the timeline URI
+ * @return
+ */
+ def getTimelineURI() : URI = { timelineURI }
+
+ /**
+ * Construct a URI under the timeline service URI
+ * @param path subpath
+ * @return a new URI
+ */
+ def uri(path: String): URI = {
+ new URL(timelineURL, path).toURI
+ }
+
+ /**
+ * Get a resource under the service
+ * @param path path
+ * @return a new resource
+ */
+ def subresource(path: String): WebResource = {
+ timelineResource.path(path)
+ }
+
+ /**
+ * Execute a GET operation against a specific URI, uprating jersey faults
+ * into more specific exceptions
+ * @param uri URI (used when handling exceptions)
+ * @param action action to perform
+ * @tparam T type of response
+ * @return the result of the action
+ */
+ def get[T](uri: URI, action: (() => T)): T = {
+ exec("GET", uri, action)
+ }
+
+ /**
+ * Execute an HTTP operation against a specific URI, uprating jersey faults
+ * into more specific exceptions.
+ * Some operations may fail-fast, if they are considered to be non-recoverable
+ * @param uri URI (used when generating text reporting exceptions)
+ * @param action action to perform
+ * @param retries number of retries on any failed operation
+ * @tparam T type of response
+ * @return the result of the action
+ */
+ def exec[T](verb: String, uri: URI, action: (() => T), retries: Int = retryLimit): T = {
+ logDebug(s"$verb $uri")
+ try {
+ innerExecAction(action)
+ } catch {
+ case e: Exception =>
+ val exception = JerseyBinding.translateException(verb, uri, e)
+ logDebug(s"$verb $uri failed: $exception", exception)
+ exception match {
+ case notFound: FileNotFoundException =>
+ logInfo(s"Not found: $uri")
+ // it's not likely to come back: fail fast
+ throw notFound
+
+ case ure: UnauthorizedRequestException =>
+ // possible expiry
+ logInfo(s"Renewing Auth token due to $exception")
+ resetConnection()
+
+ case other: Exception =>
+ logWarning(s"$verb $uri failed: $exception")
+ logDebug(s"detail", exception)
+ }
+ if (retries > 0) {
+ logInfo(s"Retrying -remaining attempts: $retries")
+ Thread.sleep(retry_interval)
+ exec(verb, uri, action, retries - 1)
+ } else {
+ throw exception
+ }
+ }
+ }
+
+ /**
+ * Reset the delegation token. Also triggers a TGT login,
+ * just for completeness
+ */
+ def resetConnection(): Unit = {
+ logInfo("Resetting connection")
+ UserGroupInformation.getCurrentUser.checkTGTAndReloginFromKeytab()
+ jerseyBinding.resetToken()
+ _lastTokenRenewal.set(System.currentTimeMillis())
+ _tokenRenewalCount.incrementAndGet()
+ }
+
+ /**
+ * Invoke the action without any failure handling.
+ *
+ * This is intended as a point for subclasses to simulate failures
+ * and so verify the failure handling code paths.
+ * @param action action to perform
+ * @tparam T type of response
+ * @return the result of the action
+ */
+ protected def innerExecAction[T](action: () => T): T = {
+ action()
+ }
+
+ /**
+ * Peform an "about" query.
+ * @return information about the service.
+ */
+ def about(): String = {
+ val aboutURI = uri("")
+ val resource = jerseyClient.resource(aboutURI)
+ val body = get(aboutURI, () => resource.accept(APPLICATION_JSON).get(classOf[String]))
+ val json = parse(body)
+ json \ "About" match {
+ case s: JString =>
+ s.toString
+ case _ =>
+ throw new HttpRequestException(200, "GET", aboutURI.toString, body)
+ }
+ }
+
+ /**
+ * This is a low-cost, non-side-effecting timeline service
+ * endpoint check operation.
+ *
+ * It does a GET of the about URL, and verifies
+ * it for validity (response type, body).
+ * @throws Exception on a failure
+ */
+ def endpointCheck(): Unit = {
+ val aboutURI = uri("")
+ val resource = jerseyClient.resource(aboutURI)
+
+ val clientResponse = get(aboutURI,
+ () => {
+ val response = resource.get(classOf[ClientResponse])
+ val status = response.getClientResponseStatus
+ if (status.getStatusCode != HttpServletResponse.SC_OK) {
+ // error code. Repeat looking for a string and so
+ // trigger a failure and the exception conversion logic
+ resource.get(classOf[String])
+ }
+ response
+ })
+
+ val endpoint = aboutURI.toString
+ val status = clientResponse.getClientResponseStatus.getStatusCode
+ val body = clientResponse.getEntity(classOf[String])
+
+ // validate the content type is JSON; if not its usually the wrong URL
+ val contentType = clientResponse.getType
+ if (MediaType.APPLICATION_JSON_TYPE != contentType) {
+ throw new HttpRequestException(status, "GET", endpoint ,
+ s"Wrong content type: expected application/json but got $contentType. " +
+ TimelineQueryClient.MESSAGE_CHECK_URL + s": $aboutURI",
+ body)
+ }
+ // an empty body is a sign of other problems
+ if (body.isEmpty) {
+ throw new HttpRequestException(status, "GET", endpoint,
+ TimelineQueryClient.MESSAGE_EMPTY_RESPONSE + s": $aboutURI")
+ }
+ // finally, issue an about() operation again to force the JSON parse
+ about()
+ }
+
+ /**
+ * Add a new query param if the option contains a value; the stringified value of the optional
+ * is used as the query parameter value
+ * @param resource resource to extend
+ * @param name parameter name
+ * @param opt option
+ * @return a new resource
+ */
+ private def applyOptionalParam(resource: WebResource,
+ name: String,
+ opt: Option[Any]): WebResource = {
+ opt match {
+ case Some(value) => resource.queryParam(name, value.toString)
+ case None => resource
+ }
+ }
+
+
+ /**
+ * Get entities matching the entity type and any optional filters.
+ * All parameters other than entityType have
+ * default values; None for optional, empty
+ * collections for the others.
+ * @param entityType entity type
+ * @param primaryFilter primary filter
+ * @param secondaryFilters map of secondary filters
+ * @param fields list of fields to retrieve
+ * @param limit limit on how many to retrieve
+ * @param windowStart time window to start retrieval
+ * @param windowEnd time window to stop retrieval
+ * @param fromId optional ID to start from
+ * @param fromTs optional timestamp to start from
+ * @return a possibly empty list of entities
+ */
+ def listEntities(
+ entityType: String,
+ primaryFilter: Option[(String, String)] = None,
+ secondaryFilters: Map[String, String] = Map(),
+ fields: Seq[String] = Nil,
+ limit: Option[Long] = None,
+ windowStart: Option[Long] = None,
+ windowEnd: Option[Long] = None,
+ fromId: Option[String] = None,
+ fromTs: Option[Long] = None): List[TimelineEntity] = {
+ require(!entityType.isEmpty, "no entity type")
+ var resource = entityResource(entityType)
+ // build the resource
+ // every application returns a new result, which complicates applying map and list arguments
+ // to it. hence the use of a variable
+ resource = primaryFilter match {
+ case Some((key, value)) =>
+ resource.queryParam("primaryFilter", s"$key:$value")
+ case None =>
+ resource
+ }
+ secondaryFilters foreach
+ ((t: (String, String)) =>
+ resource = resource.queryParam("secondaryFilter", s"${t._1}:${t._2}"))
+ resource = applyOptionalParam(resource, "windowStart", windowStart)
+ resource = applyOptionalParam(resource, "windowEnd", windowEnd)
+ resource = applyOptionalParam(resource, "limit", limit)
+ resource = applyOptionalParam(resource, "fromId", fromId)
+ resource = applyOptionalParam(resource, "fromTs", fromTs)
+ if (fields.nonEmpty) {
+ resource = resource.queryParam("fields", fields.mkString(","))
+ }
+ // execute the request
+ val response = get(resource.getURI,
+ () => resource.accept(APPLICATION_JSON).get(classOf[TimelineEntities]))
+ response.getEntities.asScala.toList
+ }
+
+ /**
+ * Get the resource of an entity type
+ * @param entityType entity type
+ * @return resource
+ */
+ def entityResource(entityType: String): WebResource = {
+ subresource(entityType)
+ }
+
+ /**
+ * Get an entity
+ * @param entityType type
+ * @param entityId the entity
+ * @return the entity if it was found
+ */
+ def getEntity(entityType: String, entityId: String): TimelineEntity = {
+ require(!entityId.isEmpty, "no entity ID")
+ val resource = entityResource(entityType).path(entityId)
+ get(resource.getURI,
+ () => resource.accept(APPLICATION_JSON).get(classOf[TimelineEntity]))
+ }
+
+ /**
+ * toString method returns the URI of the timeline service
+ * @return
+ */
+ override def toString: String = {
+ s"Timeline Query Client against $timelineURI"
+ }
+
+ /**
+ * Get the time the token was last renewed
+ * @return a system timestamp of the last renewal; 0 on startup
+ */
+ def lastTokenRenewal: Long = {
+ _lastTokenRenewal.get()
+ }
+
+ /**
+ * Get count of token renewals
+ * @return
+ */
+ def tokenRenewalCount: Long = {
+ _tokenRenewalCount.get()
+ }
+}
+
+/**
+ * Simple About response. The timeline V1 API keeps this type hidden in the server code, even though
+ * it is tagged `@Public`
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+private[spark] class AboutResponse {
+
+ var other: Map[String, Object] = Map()
+
+ var About: String = _
+
+ @JsonAnySetter
+ def handleUnknown(key: String, value: Object): Unit = {
+ other += (key -> value)
+ }
+
+}
+
+/**
+ * Constants associated with the Query API
+ */
+private[spark] object TimelineQueryClient {
+ val MESSAGE_CHECK_URL = "Check the URL of the timeline service:"
+ val MESSAGE_EMPTY_RESPONSE = s"No data in the response"
+ val EVENTS = "EVENTS"
+ val LAST_EVENT_ONLY = "LASTEVENTONLY"
+ val OTHER_INFO = "OTHERINFO"
+ val PRIMARY_FILTERS = "PRIMARYFILTERS"
+ val RELATED_ENTITIES = "RELATEDENTITIES"
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/YarnHistoryProvider.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/YarnHistoryProvider.scala
new file mode 100644
index 000000000000..4bb939f7f630
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/YarnHistoryProvider.scala
@@ -0,0 +1,1389 @@
+/*
+ * 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.yarn.server
+
+import java.io.{FileNotFoundException, IOException, InterruptedIOException}
+import java.net.URI
+import java.util.Date
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
+import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
+import java.util.zip.ZipOutputStream
+
+import scala.collection.JavaConverters._
+
+import com.codahale.metrics.{Counter, Gauge, Metric, MetricRegistry, Timer}
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.service.{Service, ServiceOperations}
+import org.apache.hadoop.yarn.api.records.ApplicationReport
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity
+import org.apache.hadoop.yarn.client.api.YarnClient
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+
+import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException}
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.history.{ApplicationHistoryProvider, HistoryServer}
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.{ExtendedMetricsSource, YarnTimelineUtils}
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.rest.JerseyBinding
+import org.apache.spark.deploy.history.yarn.server.TimelineQueryClient._
+import org.apache.spark.deploy.history.yarn.server.YarnProviderUtils._
+import org.apache.spark.scheduler.{ApplicationEventListener, ReplayListenerBus}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.util.{Clock, SystemClock}
+
+/**
+ * A History provider which reads in the history from the YARN Application Timeline Service.
+ *
+ * The service is a remote HTTP service, so failure modes are different from simple file IO.
+ *
+ * 1. Application listings are asynchronous, and made on a schedule, though
+ * they can be forced (and the schedule disabled).
+ * 2. The results are cached and can be retrieved with [[getApplications]].
+ * 3. The most recent failure of any operation is stored,
+ * The [[getLastFailure]] call will return the last exception
+ * or `None`. It is shared across threads so is primarily there for
+ * tests and basic diagnostics.
+ * 4. Listing the details of a single application in [[getAppUI()]]
+ * is synchronous and *not* cached.
+ * 5. the [[maybeCheckEndpoint()]] call performs an endpoint check as the initial
+ * binding operation of this instance. This call invokes [[TimelineQueryClient.endpointCheck()]]
+ * for better diagnostics on binding failures -particularly configuration problems.
+ * 6. Every REST call, synchronous or asynchronous, will invoke [[maybeCheckEndpoint()]] until
+ * the endpoint check eventually succeeds.
+ *
+ * If the timeline is not enabled, the API calls used by the web UI
+ * downgrade gracefully (returning empty entries), rather than fail.
+ *
+ * @param sparkConf configuration of the provider
+ */
+private[spark] class YarnHistoryProvider(sparkConf: SparkConf)
+ extends ApplicationHistoryProvider with Logging {
+
+ // import all the constants
+ import org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider._
+
+ /**
+ * The configuration here is a YarnConfiguration built off the spark configuration
+ * supplied in the constructor; this operation ensures that `yarn-default.xml`
+ * and `yarn-site.xml` are pulled in. Options in the spark conf will override
+ * those in the -default and -site XML resources which are not marked as final.
+ */
+ private val yarnConf = {
+ new YarnConfiguration(SparkHadoopUtil.get.newConfiguration(sparkConf))
+ }
+
+ /**
+ * UI ACL option.
+ */
+ private val uiAclsEnabled = sparkConf.getBoolean("spark.history.ui.acls.enable", false)
+
+ /**
+ * Flag/option which triggers providing detailed diagnostics info on the property
+ * map of the history provider.
+ */
+ private val detailedInfo = sparkConf.getBoolean(OPTION_DETAILED_INFO, false)
+
+ /**
+ * The minimum interval in milliseconds between each check for event log updates.
+ */
+ val manualRefreshInterval = sparkConf.getTimeAsSeconds(OPTION_MANUAL_REFRESH_INTERVAL,
+ s"${DEFAULT_MANUAL_REFRESH_INTERVAL_SECONDS}s") * 1000
+
+ /**
+ * The interval for manual intervals.
+ */
+ val backgroundRefreshInterval = sparkConf.getTimeAsSeconds(OPTION_BACKGROUND_REFRESH_INTERVAL,
+ s"${DEFAULT_BACKGROUND_REFRESH_INTERVAL_SECONDS}s") * 1000
+
+ /**
+ * Window limit in milliseconds.
+ */
+ private val windowLimitMs = sparkConf.getTimeAsSeconds(OPTION_WINDOW_LIMIT,
+ DEFAULT_WINDOW_LIMIT) * 1000
+
+ /**
+ * Number of events to get.
+ */
+ private val eventFetchLimit = sparkConf.getLong(OPTION_EVENT_FETCH_LIMIT,
+ DEFAULT_EVENT_FETCH_LIMIT)
+
+ /**
+ * Convert the limit to an option where 0 is mapped to None.
+ */
+ private val eventFetchOption: Option[Long] =
+ if (eventFetchLimit > 0) Some(eventFetchLimit) else None
+
+ /**
+ * Clock. Subclasses may override for testing.
+ */
+ protected var clock: Clock = new SystemClock()
+
+ /**
+ * YARN client used to list running apps and so infer which of the incomplete apps have really
+ * finished.
+ */
+ private var yarnClient: YarnClient = _
+
+ /**
+ * Are liveness checks enabled?
+ * That is, is the option [[OPTION_YARN_LIVENESS_CHECKS]] set?
+ */
+ val _livenessChecksEnabled = sparkConf.getBoolean(OPTION_YARN_LIVENESS_CHECKS, true)
+
+ /**
+ * Start time in milliseconds.
+ */
+ val serviceStartTime = clock.getTimeMillis()
+
+ /**
+ * Timeline endpoint URI.
+ */
+ protected val timelineEndpoint = createTimelineEndpoint()
+
+ /**
+ * Create a [[TimelineQueryClient]] instance to talk to the timeline service running
+ * at [[timelineEndpoint]].
+ *
+ */
+ protected val timelineQueryClient = {
+ createTimelineQueryClient()
+ }
+
+ /**
+ * Override point: create the timeline endpoint.
+ *
+ * @return a URI to the timeline web service
+ */
+ protected def createTimelineEndpoint(): URI = {
+ getTimelineEndpoint(yarnConf)
+ }
+
+ /**
+ * Create the timeline query client.
+ *
+ * This is called during instance creation; tests may override this
+ * @return a timeline query client for use for the duration
+ * of this instance
+ */
+ protected def createTimelineQueryClient(): TimelineQueryClient = {
+ new TimelineQueryClient(timelineEndpoint, yarnConf, JerseyBinding.createClientConfig())
+ }
+
+ /**
+ * The empty listing, with a timestamp to indicate that the listing
+ * has never taken place.
+ */
+ private val EmptyListing = new ApplicationListingResults(0, Nil, None)
+
+ /**
+ * List of applications. Initial result is empty.
+ */
+ private var applications: ApplicationListingResults = EmptyListing
+
+ /**
+ * Last exception seen and when.
+ */
+ protected var lastFailureCause: Option[(Throwable, Date)] = None
+
+ /**
+ * Flag to indicate an endpoint diagnostics probe should take place.
+ */
+ protected val endpointCheckExecuted = new AtomicBoolean(false)
+
+ /**
+ * Flag set to indicate refresh is in progress.
+ */
+ private val _refreshInProgress = new AtomicBoolean(false)
+
+ /**
+ * How long did that last refresh take?
+ */
+ private val _lastRefreshDuration = new AtomicLong(0)
+
+ /**
+ * Enabled flag.
+ */
+ private val _enabled = timelineServiceEnabled(yarnConf)
+
+ /**
+ * Atomic boolean used to signal to the refresh thread that it
+ * must exit its loop.
+ */
+ private val stopRefresh = new AtomicBoolean(false)
+
+ /**
+ * Refresher thread.
+ */
+ private[yarn] val refresher = new Refresher()
+
+
+ /**
+ * Time in milliseconds *before* the last application report to set the window
+ * start on the next query. Keeping this earlier than the last report ensures that
+ * changes to that application are picked up.
+ */
+ private val StartWindowOffsetMillis = 1000
+
+ /**
+ * Component Metrics. Accessible for the benefit of tests.
+ */
+ val metrics = new YarnHistoryProviderMetrics(this)
+
+ /**
+ * Initialize the provider.
+ */
+ init()
+
+ /**
+ * Check the configuration and log whether or not it is enabled;
+ * if it is enabled then the refresh thread starts.
+ *
+ * Starting threads in a constructor is considered awful practice as
+ * it can leak reference and cause chaos and confusion with subclasses.
+ * But in the absence of some start() method for history providers, it's
+ * all there is.
+ */
+ private def init(): Unit = {
+ if (!enabled) {
+ logError(TEXT_SERVICE_DISABLED)
+ } else {
+ logInfo(TEXT_SERVICE_ENABLED)
+ logInfo(KEY_SERVICE_URL + ": " + timelineEndpoint)
+ logDebug(sparkConf.toDebugString)
+ // get the thread time
+ logInfo(s"Manual refresh interval $manualRefreshInterval milliseconds")
+ validateInterval(OPTION_MANUAL_REFRESH_INTERVAL, manualRefreshInterval, 0)
+ // check the background refresh interval if there is one
+ if (backgroundRefreshInterval > 0) {
+ validateInterval(OPTION_BACKGROUND_REFRESH_INTERVAL, backgroundRefreshInterval,
+ MIN_BACKGROUND_REFRESH_INTERVAL)
+ logInfo(s"Background refresh interval $backgroundRefreshInterval milliseconds")
+ } else {
+ logInfo(s"Background refresh interval=0: manual refreshes only")
+ }
+ initYarnClient()
+ startRefreshThread()
+ }
+ }
+
+ /**
+ * Verify that a time interval is at or above the minimum allowed.
+ *
+ * @param prop property for use in exception text
+ * @param interval interval to valide
+ * @param min minimum allowed value
+ */
+ private def validateInterval(prop: String, interval: Long, min: Long): Unit = {
+ if (interval < min) {
+ throw new IllegalArgumentException(TEXT_INVALID_UPDATE_INTERVAL +
+ prop + s": ${interval / 1000}; minimum allowed = ${min / 1000}")
+ }
+ }
+
+ /**
+ * Stop the service. After this point operations will fail.
+ */
+ override def stop(): Unit = {
+ logDebug(s"Stopping $this")
+ // attempt to stop the refresh thread
+ if (enabled) {
+ if (!stopRefreshThread()) {
+ closeQueryClient()
+ ServiceOperations.stop(yarnClient)
+ }
+ }
+ }
+
+ /**
+ * Close the query client.
+ */
+ private def closeQueryClient(): Unit = {
+ logDebug("Stopping Timeline client")
+ timelineQueryClient.close()
+ }
+
+ /**
+ * Is the timeline service (and therefore this provider) enabled.
+ * (override point for tests)?
+ *
+ * Important: this is called during construction, so test-time subclasses
+ * will be invoked before their own construction has taken place.
+ * Code appropriately.
+ *
+ * @return true if the provider/YARN configuration enables the timeline
+ * service.
+ */
+ def enabled: Boolean = {
+ _enabled
+ }
+
+ /**
+ * Are liveness checks enabled?
+ * @return true if the liveness checks on the service endpoint take place.
+ */
+ def livenessChecksEnabled: Boolean = {
+ _livenessChecksEnabled
+ }
+
+ /**
+ * Is a refresh in progress?
+ *
+ * @return flag to indicate a refresh is in progress
+ */
+ def refreshInProgress: Boolean = {
+ _refreshInProgress.get()
+ }
+
+ /**
+ * The duration of last refresh.
+ * @return a duration in millis; 0 if no refresh has taken place
+ */
+ def lastRefreshDuration: Long = {
+ _lastRefreshDuration.get()
+ }
+
+ /**
+ * Get the timeline query client. Used internally to ease testing
+ * @return the client.
+ */
+ def getTimelineQueryClient: TimelineQueryClient = {
+ timelineQueryClient
+ }
+
+ /**
+ * Set the last exception.
+ * @param ex exception seen
+ */
+ private def setLastFailure(ex: Throwable): Unit = {
+ setLastFailure(ex, now())
+ }
+
+ /**
+ * Set the last exception.
+ *
+ * @param ex exception seen
+ * @param timestamp the timestamp of the failure
+ */
+ private def setLastFailure(ex: Throwable, timestamp: Long): Unit = {
+ synchronized {
+ lastFailureCause = Some((ex, new Date(timestamp)))
+ }
+ }
+
+ /**
+ * Reset the failure info.
+ */
+ private def resetLastFailure(): Unit = {
+ synchronized {
+ lastFailureCause = None
+ }
+ }
+
+ /**
+ * Get the last exception.
+ *
+ * @return the last exception or null
+ */
+ def getLastFailure: Option[(Throwable, Date)] = {
+ synchronized {
+ lastFailureCause
+ }
+ }
+
+ /**
+ * Thread safe accessor to the application list.
+ *
+ * @return a list of applications
+ */
+ def getApplications: ApplicationListingResults = {
+ synchronized {
+ applications
+ }
+ }
+
+ /**
+ * Thread safe call to update the application results.
+ *
+ * @param newVal new value
+ */
+ private def setApplications(newVal: ApplicationListingResults): Unit = {
+ synchronized {
+ applications = newVal
+ }
+ }
+
+ /**
+ * Reachability check to call before any other operation is attempted.
+ * This is atomic, using the `shouldCheckEndpoint` flag to check.
+ * If the endpoint check failes then the
+ * `endpointCheckExecuted` flag is reset to false and an exception thrown.
+ *
+ * @return true if the check took place
+ */
+ protected def maybeCheckEndpoint(): Boolean = {
+ if (!endpointCheckExecuted.getAndSet(true)) {
+ val client = getTimelineQueryClient
+ try {
+ client.endpointCheck()
+ true
+ } catch {
+ case e: Exception =>
+ // failure
+ logWarning(s"Endpoint check of $client failed", e)
+ setLastFailure(e)
+ // reset probe so another caller may attempt it.
+ endpointCheckExecuted.set(false)
+ // propagate the failure
+ throw e
+ }
+ } else {
+ false
+ }
+ }
+
+ /**
+ * Query for the endpoint check being successful.
+ *
+ * Note: also true if the check is in progress.
+ *
+ * @return true if the check has succeeded, or it is actually ongoing (it may fail)
+ */
+ def endpointCheckSuccess(): Boolean = {
+ endpointCheckExecuted.get()
+ }
+
+ /**
+ * Start the refresh thread with the given interval.
+ *
+ * When this thread exits, it will close the `timelineQueryClient`
+ * instance
+ */
+ def startRefreshThread(): Unit = {
+ logInfo(s"Starting timeline refresh thread")
+ val thread = new Thread(refresher, s"YarnHistoryProvider Refresher")
+ thread.setDaemon(true)
+ refresher.start(thread)
+ }
+
+ /**
+ * Stop the refresh thread if there is one.
+ *
+ * This does not guarantee an immediate halt to the thread.
+ *
+ * @return true if there was a refresh thread to stop
+ */
+ private def stopRefreshThread(): Boolean = {
+ refresher.stopRefresher()
+ }
+
+ /**
+ * Probe for the refresh thread running.
+ *
+ * @return true if the refresh thread has been created and is still alive
+ */
+ def isRefreshThreadRunning: Boolean = {
+ refresher.isRunning
+ }
+
+ /**
+ * Get the number of times an attempt was made to refresh the listing.
+ * This is incremented on every operation, irrespective of the outcome.
+ *
+ * @return the current counter of refresh attempts.
+ */
+ def refreshCount: Long = {
+ metrics.refreshCount.getCount
+ }
+
+ /**
+ * Get number of failed refreshes.
+ *
+ * Invariant: always equal to or less than [[refreshCount]]
+ *
+ * @return the number of times refreshes failed
+ */
+ def refreshFailedCount: Long = {
+ metrics.refreshFailedCount.getCount()
+ }
+
+ /**
+ * List applications.
+ *
+ * If the timeline is not enabled, returns `emptyListing`
+ * @return the result of the last successful listing operation,
+ * or a listing with no history events if there has been a failure
+ */
+ def listApplications(
+ limit: Option[Long] = None,
+ windowStart: Option[Long] = None,
+ windowEnd: Option[Long] = None): ApplicationListingResults = {
+ if (!enabled) {
+ // Timeline is disabled: return the empty listing
+ return EmptyListing
+ }
+ try {
+ maybeCheckEndpoint()
+ val client = getTimelineQueryClient
+ logInfo(s"getListing from: $client")
+ // get the timestamp after any endpoint check
+ val timestamp = now()
+ // list the entities, excluding the events -this is critical for performance reasons
+ val timelineEntities = client.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ windowStart = windowStart,
+ windowEnd = windowEnd,
+ limit = limit,
+ fields = Seq(PRIMARY_FILTERS, OTHER_INFO)
+ )
+
+ // build one history info entry for each entity in the least (implicitly, one
+ // attempt per history info entry
+ val umergedHistory = timelineEntities.flatMap { en =>
+ try {
+ val historyInfo = toApplicationHistoryInfo(en)
+ logDebug(s"${describeApplicationHistoryInfo(historyInfo)}")
+ Some(historyInfo)
+ } catch {
+ case e: Exception =>
+ logWarning(s"Failed to parse entity. ${YarnTimelineUtils.describeEntity(en)}", e)
+ // skip this result
+ None
+ }
+ }
+ // merge the results so that multiple attempts are combined into
+ // single history entries
+ val histories = combineResults(Nil, umergedHistory)
+ val incomplete = countIncompleteApplications(histories)
+ logInfo(s"Found ${histories.size} application(s): " +
+ s"${histories.size - incomplete} complete and $incomplete incomplete")
+ new ApplicationListingResults(timestamp, histories, None)
+ } catch {
+ case e: IOException =>
+ logWarning(s"Failed to list entities from $timelineEndpoint", e)
+ new ApplicationListingResults(now(), Nil, Some(e))
+ }
+ }
+
+ /**
+ * List applications.
+ *
+ * Also updates the cached values of the listing/last failure, depending
+ * upon the outcome.
+ *
+ * If the timeline is not enabled, returns an empty list.
+ *
+ * @param startup a flag to indicate this is the startup retrieval with different window policy
+ * @return List of all known applications.
+ */
+ def listAndCacheApplications(startup: Boolean): ApplicationListingResults = {
+ metrics.refreshCount.inc()
+ _refreshInProgress.set(true)
+ val history = getApplications.applications
+
+ val refreshStartTime = now()
+ metrics.time(metrics.refreshDuration) {
+ try {
+ // work out the start of the new window
+ val nextWindowStart = if (startup) {
+ None
+ } else {
+ findStartOfWindow(history) map { app =>
+ // inclusive on the one retrieved last time.
+ // Why? we need to include the oldest incomplete entry in our range
+ val inclusiveWindow = startTime(app) - StartWindowOffsetMillis
+ // sanity check on window size
+ val earliestWindow = if (windowLimitMs > 0) refreshStartTime - windowLimitMs else 0
+ Math.max(earliestWindow, inclusiveWindow)
+ }
+ }
+ val results = listApplications(windowStart = nextWindowStart)
+ synchronized {
+ if (results.succeeded) {
+ // on a success, the existing application list is merged
+ // creating a new aggregate application list
+ logDebug(s"Listed application count: ${results.size}")
+ val merged = combineResults(history, results.applications)
+ logDebug(s"Existing count: ${history.size}; merged = ${merged.size} ")
+ val updated = if (livenessChecksEnabled) {
+ updateAppsFromYARN(merged)
+ } else {
+ merged
+ }
+ val sorted = sortApplicationsByStartTime(updated)
+ // and a final result
+ setApplications(new ApplicationListingResults(results.timestamp, sorted, None))
+ resetLastFailure()
+ } else {
+ // on a failure, the failure cause is updated
+ setLastFailure(results.failureCause.get, results.timestamp)
+ // and the failure counter
+ metrics.refreshFailedCount.inc()
+ }
+ }
+ results
+ } finally {
+ _refreshInProgress.set(false)
+ _lastRefreshDuration.set(now() - refreshStartTime)
+ }
+ }
+ }
+
+ /**
+ * List applications.
+ *
+ * If the timeline is not enabled, returns an empty list
+ * @return List of all known applications.
+ */
+ override def getListing(): Seq[TimelineApplicationHistoryInfo] = {
+ // get the current list
+ val listing = getApplications.applications
+ // and queue another refresh
+ triggerRefresh()
+ listing
+ }
+
+ /**
+ * Trigger a refresh.
+ */
+ private[yarn] def triggerRefresh(): Unit = {
+ refresher.refresh(now())
+ }
+
+ /**
+ * Return the current time.
+ * @return the time in milliseconds.
+ */
+ private[yarn] def now(): Long = {
+ clock.getTimeMillis()
+ }
+
+ /**
+ * Get the last refresh attempt (Which may or may not have been successful).
+ *
+ * @return the last refresh time
+ */
+ def lastRefreshAttemptTime: Long = {
+ refresher.lastRefreshAttemptTime
+ }
+
+ /**
+ * Look up the timeline entity.
+ *
+ * @param entityId application ID
+ * @return the entity associated with the given application
+ * @throws FileNotFoundException if no entry was found
+ */
+ def getTimelineEntity(entityId: String): TimelineEntity = {
+ logDebug(s"GetTimelineEntity $entityId")
+ metrics.time(metrics.attemptFetchDuration){
+ maybeCheckEndpoint()
+ getTimelineQueryClient.getEntity(SPARK_EVENT_ENTITY_TYPE, entityId)
+ }
+ }
+
+ /**
+ * Returns the Spark UI for a specific application.
+ *
+ * If the timeline is not enabled, returns `None`
+ *
+ * @param appId The application ID.
+ * @param attemptId The application attempt ID (or `None` if there is no attempt ID).
+ * @return The application's UI, or `None` if application is not found.
+ */
+ override def getAppUI(appId: String, attemptId: Option[String]): Option[SparkUI] = {
+
+ logInfo(s"Request UI with appId $appId attempt $attemptId")
+ if (!enabled) {
+ // Timeline is disabled: return nothing
+ return None
+ }
+ maybeCheckEndpoint()
+ metrics.attemptLoadCount.inc()
+ if (attemptId.isEmpty) {
+ // empty attempts are rejected
+ metrics.attemptLoadFailureCount.inc()
+ return None
+ }
+
+ // the URL generation/linking code in the History Server appears fairly brittle,
+ // so provide as diagnostics information in the logs to help understand what
+ // is happending
+ val (foundApp, attempt, attempts) = getApplications.lookupAttempt(appId, attemptId)
+ if (foundApp.isEmpty) {
+ log.error(s"Application $appId not found")
+ metrics.attemptLoadFailureCount.inc()
+ return None
+ }
+ val attemptInfo = attempt.getOrElse {
+ log.error(s"Attempt $attemptId not found under application $appId")
+ log.error(s"${attempts.length} attempts: " + attempts.mkString("[", ", ", "]"))
+ metrics.attemptLoadFailureCount.inc()
+ return None
+ }
+
+ val entityId = attemptInfo.entityId
+ val appInfo = foundApp.get
+
+ metrics.time(metrics.attemptLoadDuration) {
+ try {
+ val attemptEntity = getTimelineEntity(entityId)
+ if (log.isDebugEnabled) {
+ logDebug(describeEntity(attemptEntity))
+ }
+ val bus = new ReplayListenerBus()
+ val appListener = new ApplicationEventListener()
+ bus.addListener(appListener)
+ val conf = this.sparkConf.clone()
+ val attemptURI = HistoryServer.getAttemptURI(appId, attemptId)
+ val name = appInfo.name
+ val ui = SparkUI.createHistoryUI(conf,
+ bus,
+ new SecurityManager(conf),
+ name,
+ attemptURI,
+ attemptInfo.startTime)
+ ui.setAppId(appId)
+ logInfo(s"Building Application UI $name attempt $attemptId under ${ui.basePath}")
+
+ // replay all the events
+ val events = attemptEntity.getEvents.asScala
+ logInfo(s"App $appId history contains ${events.length} events")
+
+ metrics.time(metrics.attemptReplayDuration) {
+ events.reverse.foreach { event =>
+ val sparkEvent = toSparkEvent(event)
+ bus.postToAll(sparkEvent)
+ }
+ }
+
+ ui.getSecurityManager.setAcls(uiAclsEnabled)
+ // make sure to set admin acls before view acls so they are properly picked up
+ ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse(""))
+ ui.getSecurityManager.setViewAcls(appListener.sparkUser.getOrElse(""),
+ appListener.viewAcls.getOrElse(""))
+ Some(ui)
+ } catch {
+
+ case e: FileNotFoundException =>
+ logInfo(s"Unknown attempt $entityId", e)
+ setLastFailure(e)
+ metrics.attemptLoadFailureCount.inc()
+ None
+
+ case e: Exception =>
+ logWarning(s"Failed to get attempt information for $appId attempt $entityId", e)
+ setLastFailure(e)
+ metrics.attemptLoadFailureCount.inc()
+ throw e
+ }
+ }
+ }
+
+ /**
+ * Get configuration information for the Web UI.
+ *
+ * @return A map with the configuration data. Data is shown in the order returned by the map.
+ */
+ override def getConfig(): Map[String, String] = {
+ val timelineURI = getEndpointURI
+ logDebug(s"getConfig $timelineURI")
+ synchronized {
+ val applications = getApplications
+ val failure = getLastFailure
+ var state = Map(
+ KEY_PROVIDER_NAME -> PROVIDER_DESCRIPTION,
+ KEY_START_TIME -> humanDateCurrentTZ(serviceStartTime, "(not started)"),
+ KEY_SERVICE_URL -> s"$timelineURI",
+ KEY_ENABLED -> (if (enabled) TEXT_SERVICE_ENABLED else TEXT_SERVICE_DISABLED),
+ KEY_LAST_UPDATED -> applications.updated,
+ KEY_CURRENT_TIME -> humanDateCurrentTZ(now(), "unknown")
+ )
+ // in a secure cluster, list the user name
+ if (UserGroupInformation.isSecurityEnabled) {
+ state += (KEY_USERNAME -> UserGroupInformation.getCurrentUser.getUserName)
+ }
+
+ // on a failure, add failure specifics to the operations
+ failure foreach {
+ case (ex , date) =>
+ state = state ++
+ Map(
+ KEY_LAST_FAILURE_TIME -> humanDateCurrentTZ(date.getTime, TEXT_NEVER_UPDATED),
+ KEY_LAST_FAILURE -> ex.toString)
+ }
+ // add detailed information if enabled
+ if (detailedInfo) {
+ state = state ++ Map(
+ KEY_X_TOKEN_RENEWAL ->
+ humanDateCurrentTZ(timelineQueryClient.lastTokenRenewal, TEXT_NEVER_UPDATED),
+ KEY_X_TOKEN_RENEWAL_COUNT -> timelineQueryClient.tokenRenewalCount.toString,
+ KEY_X_INTERNAL_STATE -> s"$this",
+ KEY_X_MIN_REFRESH_INTERVAL -> s"$manualRefreshInterval mS",
+ KEY_X_BACKGROUND_REFRESH_INTERVAL -> s"$backgroundRefreshInterval mS",
+ KEY_X_EVENT_FETCH_LIMIT -> eventFetchLimit.toString,
+ KEY_X_ENTITY_LISTING ->
+ (timelineQueryClient.entityResource(SPARK_EVENT_ENTITY_TYPE).getURI.toString +
+ s"?fields=$PRIMARY_FILTERS,$OTHER_INFO"),
+ KEY_X_REFRESH_IN_PROGRESS -> refreshInProgress.toString,
+ KEY_X_LAST_REFRESH_DURATION -> s"$lastRefreshDuration ms"
+ )
+ }
+ state
+ }
+ }
+
+ /**
+ * Get the URI of the root of the timeline server.
+ *
+ * @return URI of the timeline endpoint
+ */
+ def getEndpointURI: URI = {
+ timelineEndpoint.resolve("/")
+ }
+
+ /**
+ * Stub implementation of the "write event logs" operation, which isn't supported
+ * by the timeline service
+ * @throws SparkException always
+ */
+ override def writeEventLogs(appId: String,
+ attemptId: Option[String],
+ zipStream: ZipOutputStream): Unit = {
+ throw new SparkException("Unsupported Feature")
+ }
+
+ override def toString(): String = {
+ s"YarnHistoryProvider bound to history server at $timelineEndpoint," +
+ s" enabled = $enabled;" +
+ s" refresh count = $refreshCount; failed count = $refreshFailedCount;" +
+ s" last update ${applications.updated};" +
+ s" history size ${applications.size};" +
+ s" $refresher"
+ }
+
+ /**
+ * Comparison function that defines the sort order for the application listing.
+ *
+ * @return Whether `i1` should precede `i2`.
+ */
+ private def compareAppInfo(
+ i1: TimelineApplicationHistoryInfo,
+ i2: TimelineApplicationHistoryInfo): Boolean = {
+ val a1 = i1.attempts.head
+ val a2 = i2.attempts.head
+ if (a1.endTime != a2.endTime) a1.endTime >= a2.endTime else a1.startTime >= a2.startTime
+ }
+
+ /**
+ * Initialize the YARN client
+ */
+ protected def initYarnClient(): Unit = {
+ require(yarnClient == null, "YARN client already initialized")
+ if (livenessChecksEnabled) {
+ logDebug("Creating YARN Client")
+ yarnClient = YarnClient.createYarnClient()
+ yarnClient.init(yarnConf)
+ yarnClient.start()
+ } else {
+ logInfo("YARN liveness checks disabled.")
+ }
+ }
+
+ /**
+ * List spark applications known by the the YARN RM.
+ *
+ * This includes known failed/halted/killed applications as well as those in running/scheduled
+ * states. Sometimes on AM restart the application appears to drop out of the list briefly.
+ *
+ * @return the list of running spark applications, which can then be filtered against
+ */
+ private[yarn] def listYarnSparkApplications(): Map[String, ApplicationReport] = {
+ if (isYarnClientRunning) {
+ val reports = yarnClient.getApplications(Set(SPARK_YARN_APPLICATION_TYPE).asJava).asScala
+ reportsToMap(reports)
+ } else {
+ Map()
+ }
+ }
+
+ /**
+ * Is the [[yarnClient]] running?
+ *
+ * @return true if it is non-null and running
+ */
+ private[yarn] def isYarnClientRunning: Boolean = {
+ yarnClient != null && yarnClient.isInState(Service.STATE.STARTED)
+ }
+
+ /**
+ * Filter out all incomplete applications that are not in the list of running YARN applications.
+ *
+ * This filters out apps which have failed without any notification event.
+ * @param apps list of applications
+ * @return list of apps which are marked as incomplete but no longer running
+ */
+ private[yarn] def updateAppsFromYARN(apps: Seq[TimelineApplicationHistoryInfo])
+ : Seq[TimelineApplicationHistoryInfo] = {
+
+ if (isYarnClientRunning) {
+ completeAppsFromYARN(apps, listYarnSparkApplications(), now(), DEFAULT_LIVENESS_WINDOW_I)
+ } else {
+ apps
+ }
+ }
+
+ /**
+ * This is the implementation of the triggered refresh logic.
+ * It awaits events, including one generated on a schedule
+ */
+ private[yarn] class Refresher extends Runnable {
+
+ private[yarn] sealed trait RefreshActions
+
+ /** start the refresh */
+ private[yarn] case class Start() extends RefreshActions
+
+ /** refresh requested at the given time */
+ private[yarn] case class RefreshRequest(time: Long) extends RefreshActions
+
+ /** stop */
+ private[yarn] case class StopExecution() extends RefreshActions
+
+ /**
+ * Poll operation timed out.
+ */
+ private[yarn] case class PollTimeout() extends RefreshActions
+
+ private val queue = new LinkedBlockingQueue[RefreshActions]()
+ private val running = new AtomicBoolean(false)
+ private var self: Thread = _
+ private val _lastRefreshAttemptTime = new AtomicLong(0)
+
+ /**
+ * Bond to the thread then start it.
+ * @param t thread
+ */
+ def start(t: Thread): Unit = {
+ synchronized {
+ self = t
+ running.set(true)
+ queue.add(Start())
+ t.start()
+ }
+ }
+
+ /**
+ * Request a refresh. If the request queue is empty, a refresh request
+ * is queued.
+ * @param time time request was made
+ */
+ def refresh(time: Long): Unit = {
+ if (queue.isEmpty) {
+ queue.add(RefreshRequest(time))
+ }
+ }
+
+ /**
+ * Stop operation.
+ * @return true if the stop was scheduled
+ */
+ def stopRefresher(): Boolean = {
+ synchronized {
+ if (isRunning) {
+ // yes, more than one stop may get issued. but it will
+ // replace the previous one.
+ queue.clear()
+ queue.add(StopExecution())
+ self.interrupt()
+ true
+ } else {
+ false
+ }
+ }
+ }
+
+ /**
+ * Thread routine.
+ */
+ override def run(): Unit = {
+ try {
+ var stopped = false
+ while (!stopped) {
+ try {
+ // get next event
+ val event = if (backgroundRefreshInterval > 0) {
+ // background interval set => poll for an event
+ poll(backgroundRefreshInterval)
+ } else {
+ // no interval: block for manually triggered refresh
+ take()
+ }
+ event match {
+ case StopExecution() =>
+ // stop: exit the loop
+ stopped = true
+ case Start() =>
+ // initial read; may be bigger
+ logDebug("Startup refresh")
+ doRefresh(true, false)
+ case RefreshRequest(time) =>
+ // requested refresh operation
+ logDebug("Triggered refresh")
+ doRefresh(false, false)
+ case PollTimeout() =>
+ // poll timeout. Do a refresh
+ logDebug("Background Refresh")
+ doRefresh(false, true)
+ }
+ // it is only after processing the
+ // message that the message process counter
+ // is incremented
+ metrics.backgroundOperationsProcessed.inc()
+
+ } catch {
+ case ex: InterruptedException =>
+ // raised during stop process to interrupt IO
+ logDebug("Interrupted ", ex)
+ case ex: InterruptedIOException =>
+ // a common wrapper for interrupted exceptions in the Hadoop stack.
+ logDebug("Interrupted ", ex)
+ case ex: Exception =>
+ // something else. Log at warn
+ logWarning("Exception in refresh thread", ex)
+ }
+ }
+ } finally {
+ closeQueryClient()
+ ServiceOperations.stop(yarnClient)
+ running.set(false)
+ logInfo("Background Refresh Thread exited")
+ }
+ }
+
+ /**
+ * Do the refresh.
+ *
+ * This contains the decision making as when to refresh, which can happen if
+ * any of the following conditions were met:
+ *
+ * 1. the refresh interval == 0 (always)
+ * 2. the last refresh was outside the window.
+ * 3. this is a background refresh
+ *
+ * There isn't a special check for "never updated", as this
+ * would only be inside the window in test cases with a small
+ * simulated clock.
+ *
+ * @param startup a flag to indicate this is the startup retrieval with different window policy
+ */
+ private def doRefresh(startup: Boolean, background: Boolean): Unit = {
+ val t = now()
+ if (manualRefreshInterval == 0
+ || background
+ || ((t - _lastRefreshAttemptTime.get) >= manualRefreshInterval )) {
+ logDebug(s"refresh triggered at $t")
+ listAndCacheApplications(startup)
+ // set refresh time to after the operation, so that even if the operation
+ // is slow, refresh intervals don't come too often.
+ _lastRefreshAttemptTime.set(now())
+ }
+ }
+
+ /**
+ * Get the next action -blocking until it is ready.
+ *
+ * @return the next action
+ */
+ private def take(): Refresher.this.RefreshActions = {
+ queue.take
+ }
+
+ /**
+ * Poll for the next action; return the head of the queue as soon as it is available,
+ * or, after the timeout, a [[PollTimeout]] message.
+ *
+ * @param millis poll time
+ * @return an action
+ */
+ private def poll(millis: Long): Refresher.this.RefreshActions = {
+ val result = queue.poll(millis, TimeUnit.MILLISECONDS)
+ if (result == null) PollTimeout() else result
+ }
+
+ /**
+ * Flag to indicate the refresher thread is running.
+ * @return true if the refresher is running
+ */
+ def isRunning: Boolean = {
+ running.get
+ }
+
+ /**
+ * Get the last refresh time.
+ * @return the last refresh time
+ */
+ def lastRefreshAttemptTime: Long = {
+ _lastRefreshAttemptTime.get
+ }
+
+ /**
+ * Get count of messages processed.
+ *
+ * This will be at least equal to the number of refreshes executed
+ * @return processed count
+ */
+ def messagesProcessed: Long = {
+ metrics.backgroundOperationsProcessed.getCount
+ }
+
+ /**
+ * String value is for diagnostics in tests.
+ *
+ * @return a description of the current state
+ */
+ override def toString: String = {
+ s"Refresher running = $isRunning queue size = ${queue.size};" +
+ s" min refresh interval = $manualRefreshInterval mS;" +
+ s" background refresh interval = $backgroundRefreshInterval mS;" +
+ s" processed = $messagesProcessed;" +
+ s" last refresh attempt = " + timeShort(lastRefreshAttemptTime, "never") + ";\n" +
+ metrics.toString
+ }
+ }
+
+}
+
+/**
+ * All the metrics for the YARN history provider
+ * @param owner owning class
+ */
+private[history] class YarnHistoryProviderMetrics(owner: YarnHistoryProvider)
+ extends ExtendedMetricsSource {
+ override val sourceName = "yarn.history.provider"
+
+ override val metricRegistry = new MetricRegistry()
+
+ /** How many applications? */
+ val applicationGauge = new Gauge[Int] {
+ override def getValue: Int = { owner.getApplications.size }
+ }
+
+ /** How many application attempts? */
+ val applicationAttemptGauge = new Gauge[Int] {
+ override def getValue: Int = {
+ owner.getApplications.applications.foldLeft(0) {
+ (acc, info) => acc + info.attempts.length}
+ }
+ }
+
+ /** Flag to indicate whether or not a refresh is in progress. */
+ val refreshInProgress = new Gauge[Int] {
+ override def getValue: Int = { if (owner.refreshInProgress) 1 else 0 }
+ }
+
+ /** Timer of background refresh operations. */
+ val refreshDuration = new Timer()
+
+ /** Counter of number of application attempts that have been loaded. */
+ val attemptLoadCount = new Counter()
+
+ /** Counter of how many times applicaton attempts failed to load. */
+ val attemptLoadFailureCount = new Counter()
+
+ /** Timer of how long it it taking to load app attempt. */
+ val attemptLoadDuration = new Timer()
+
+ /** Timer of how long it has taken to fetch app attempts from ATS. */
+ val attemptFetchDuration = new Timer()
+
+ /** How long has it taken to replay fetched app attempts? */
+ val attemptReplayDuration = new Timer()
+
+ /** Counter of refresh operations. */
+ val refreshCount = new Counter()
+
+ /** Counter of failed refresh operations. */
+ val refreshFailedCount = new Counter()
+
+ /** Number of operations processed asynchronously. */
+ val backgroundOperationsProcessed = new Counter()
+
+ val metricsMap: Map[String, Metric] = Map(
+ "applications" -> applicationGauge,
+ "application.attempts" -> applicationAttemptGauge,
+ "app.attempt.load.count" -> attemptLoadCount,
+ "app.attempt.load.duration" -> attemptLoadDuration,
+ "app.attempt.load.failure.count" -> attemptLoadFailureCount,
+ "app.attempt.load.fetch.duration" -> attemptFetchDuration,
+ "app.attempt.load.replay.duration" -> attemptReplayDuration,
+ "background.operations.processed" -> backgroundOperationsProcessed,
+ "refresh.count" -> refreshCount,
+ "refresh.duration" -> refreshDuration,
+ "refresh.failed.count" -> refreshFailedCount,
+ "refresh.in.progress" -> refreshInProgress
+ )
+
+ init()
+}
+
+/**
+ * Constants to go with the history provider.
+ *
+ * 1. Any with the prefix `KEY_` are for configuration (key, value) pairs, so can be
+ * searched for after scraping the History server web page.
+ *
+ * 2. Any with the prefix `OPTION_` are options from the configuration.
+ *
+ * 3. Any with the prefix `DEFAULT_` are the default value of options
+ *
+ * 4. Any with the prefix `TEXT_` are text messages which may appear in web pages
+ * and other messages (and so can be scanned for in tests)
+ */
+private[spark] object YarnHistoryProvider {
+
+ /** Name of the class to use in configuration strings. */
+ val YARN_HISTORY_PROVIDER_CLASS = classOf[YarnHistoryProvider].getName()
+
+ /** Message when the timeline service is enabled. */
+ val TEXT_SERVICE_ENABLED = "Timeline service is enabled"
+
+ /** Message when the timeline service is disabled. */
+ val TEXT_SERVICE_DISABLED =
+ "Timeline service is disabled: application history cannot be retrieved"
+
+ /** Message when a "last updated" field has never been updated. */
+ val TEXT_NEVER_UPDATED = "Never"
+
+ /** Message when reporting an invalid update interval. */
+ val TEXT_INVALID_UPDATE_INTERVAL = s"Invalid update interval defined in "
+
+ /** What is the app type to ask for when listing apps on YARN? */
+ val SPARK_YARN_APPLICATION_TYPE = "SPARK"
+
+ /**
+ * Option for the interval for listing timeline refreshes. Bigger: less chatty.
+ * Smaller: history more responsive.
+ */
+ val OPTION_MANUAL_REFRESH_INTERVAL = "spark.history.yarn.manual.refresh.interval"
+ val DEFAULT_MANUAL_REFRESH_INTERVAL_SECONDS = 30
+
+ /**
+ * Interval for background refreshes.
+ */
+ val OPTION_BACKGROUND_REFRESH_INTERVAL = "spark.history.yarn.background.refresh.interval"
+ val DEFAULT_BACKGROUND_REFRESH_INTERVAL_SECONDS = 60
+
+ /**
+ * Minimum allowed refresh interval in milliseconds.
+ */
+ val MIN_BACKGROUND_REFRESH_INTERVAL = 1000
+
+ /**
+ * Option for the number of events to retrieve.
+ */
+ val OPTION_EVENT_FETCH_LIMIT = "spark.history.yarn.event-fetch-limit"
+ val DEFAULT_EVENT_FETCH_LIMIT = 1000
+
+ /**
+ * Expiry age window. This is only going to be turned from a constant
+ * to an option if the default is found to be inadequate. It's needed
+ * primarily because killing a YARN app can sometimes cause it to get
+ * its state mixed up on a listing operation.
+ */
+ val OPTION_EXPIRY_AGE = "spark.history.yarn.liveness.window"
+ val DEFAULT_LIVENESS_WINDOW_I = 60 * 60
+ val DEFAULT_LIVENESS_WINDOW = s"${DEFAULT_LIVENESS_WINDOW_I}s"
+
+ /**
+ * Maximum timeline of the window when getting updates.
+ * If set to zero, there's no limit
+ */
+ val OPTION_WINDOW_LIMIT = "spark.history.yarn.window.limit"
+ val DEFAULT_WINDOW_LIMIT = "24h"
+
+ /**
+ * Option to enabled detailed/diagnostics view.
+ */
+ val OPTION_DETAILED_INFO = "spark.history.yarn.diagnostics"
+
+ /**
+ * Option to enable YARN probes for running applications.
+ */
+ val OPTION_YARN_LIVENESS_CHECKS = "spark.history.yarn.probe.running.applications"
+
+ /** Current time. */
+ val KEY_CURRENT_TIME = "Current Time"
+
+ /** Entry to use in the 'enabled' status line. */
+ val KEY_ENABLED = "Timeline Service"
+
+ /** Key for the last operation failure entry. */
+ val KEY_LAST_FAILURE = "Last Operation Failure"
+
+ /** Key for reporting the time of the last operation failure. */
+ val KEY_LAST_FAILURE_TIME = "Last Operation Failed"
+
+ /** Key for the last updated entry. */
+ val KEY_LAST_UPDATED = "Last Updated"
+
+ /** Key for reporting update interval. */
+ val KEY_LISTING_REFRESH_INTERVAL = "Update Interval"
+
+ /** Key of the [[KEY_PROVIDER_NAME]] entry. */
+ val PROVIDER_DESCRIPTION = "Apache Hadoop YARN Timeline Service"
+
+ /** Key used to identify the history provider */
+ val KEY_PROVIDER_NAME = "History Provider"
+
+ /** Key used when listing the URL of the ATS instance. */
+ val KEY_SERVICE_URL = "Timeline Service Location"
+
+ /** Key for service start time entry. */
+ val KEY_START_TIME = "Service Started"
+
+ /** Key for username; shown in secure clusters. */
+ val KEY_USERNAME = "User"
+
+ /** key on background refresh interval (millis). */
+ val KEY_X_BACKGROUND_REFRESH_INTERVAL = "x-Background refresh interval"
+
+ /** Detailed-view URL to timeline entity listing path. */
+ val KEY_X_ENTITY_LISTING = "x-Entity Listing URL"
+
+ /** Detailed view of entity fetch limit . */
+ val KEY_X_EVENT_FETCH_LIMIT = "x-" + OPTION_EVENT_FETCH_LIMIT
+
+ /** Detailed view of internal state of history provider; the `toString()` value. */
+ val KEY_X_INTERNAL_STATE = "x-Internal State"
+
+ /** Detailed view of the duration of the last refresh operation. */
+ val KEY_X_LAST_REFRESH_DURATION = "x-Last Refresh Duration"
+
+ /** Detailed view of the minimum interval between manual refreshes. */
+ val KEY_X_MIN_REFRESH_INTERVAL = "x-manual refresh interval"
+
+ /** Detailed view: is a refresh in progress? */
+ val KEY_X_REFRESH_IN_PROGRESS = "x-Refresh in Progress"
+
+ /** Detailed view: has a security token ever been renewed? */
+ val KEY_X_TOKEN_RENEWAL = "x-Token Renewed"
+
+ /** Detailed view: how many timeline server tokens have been renewed? */
+ val KEY_X_TOKEN_RENEWAL_COUNT = "x-Token Renewal Count"
+
+}
diff --git a/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/YarnProviderUtils.scala b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/YarnProviderUtils.scala
new file mode 100644
index 000000000000..faf4e483a940
--- /dev/null
+++ b/yarn/src/history/main/scala/org/apache/spark/deploy/history/yarn/server/YarnProviderUtils.scala
@@ -0,0 +1,517 @@
+/*
+ * 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.yarn.server
+
+import scala.collection.mutable
+
+import org.apache.hadoop.yarn.api.records.{ApplicationReport, YarnApplicationState}
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity
+
+import org.apache.spark.Logging
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+
+/**
+ * Utils on the history server-side.
+ */
+private[spark] object YarnProviderUtils extends Logging {
+
+ /**
+ * What should the value be for start/end times which are not set.
+ */
+ val UNSET_TIME_VALUE = 0L
+
+ /**
+ * Build an [[TimelineApplicationHistoryInfo]] instance from a [[TimelineEntity]].
+ *
+ * @param en the entity
+ * @return an history info structure. The completed bit is true if the entity has an end time.
+ * @throws Exception if the conversion failed
+ */
+ def toApplicationHistoryInfo(en: TimelineEntity): TimelineApplicationHistoryInfo = {
+ val endTime = numberField(en, FIELD_END_TIME, UNSET_TIME_VALUE).longValue
+ val startTime = numberField(en, FIELD_START_TIME, UNSET_TIME_VALUE).longValue
+ val lastTimestamp = Math.max(startTime, endTime)
+ var lastUpdated = numberField(en, FIELD_LAST_UPDATED).longValue
+ if (lastUpdated < lastTimestamp) {
+ logDebug(s"lastUpdated field $lastUpdated < latest timestamp $lastTimestamp; overwriting")
+ lastUpdated = lastTimestamp
+ }
+ val name = field(en, FIELD_APP_NAME).asInstanceOf[String]
+ val sparkUser = field(en, FIELD_APP_USER).asInstanceOf[String]
+ val completed = endTime > 0
+
+ val entityId = en.getEntityId()
+ val appId = fieldOption(en, FIELD_APPLICATION_ID) match {
+ case Some(value) => value.asInstanceOf[String]
+ case None => entityId
+ }
+ val version = numberField(en, FIELD_ENTITY_VERSION, 0).longValue
+
+ // the spark attempt ID; only unique amongst entities
+ val sparkAttemptId = stringFieldOption(en, FIELD_ATTEMPT_ID)
+
+ val attemptId = if (sparkAttemptId.isDefined) {
+ sparkAttemptId
+ } else {
+ Some(entityId)
+ }
+
+ val attempt = new TimelineApplicationAttemptInfo(
+ attemptId,
+ startTime,
+ endTime,
+ lastUpdated,
+ sparkUser,
+ completed,
+ entityId,
+ sparkAttemptId,
+ version)
+
+ // return the single attempt which can be built from this entity
+ new TimelineApplicationHistoryInfo(appId, name, attempt :: Nil)
+ }
+
+ /**
+ * Describe the application history, including timestamps and completed flag.
+ *
+ * @param info history info to describe
+ * @return a string description
+ */
+ def describeApplicationHistoryInfo(info: TimelineApplicationHistoryInfo): String = {
+ val core = s"TimelineApplicationHistoryInfo [${info.id}] ${info.name}"
+ if (info.attempts.isEmpty) {
+ s"$core : no attempts"
+ } else {
+ val attempt = info.attempts.head
+ s"$core : " + info.attempts.map(describeAttempt).mkString("[{", "}, {", "}]")
+ }
+ }
+
+ /**
+ * Describe an individual attempt.
+ * @param attempt attempt
+ * @return a string description
+ */
+ def describeAttempt(attempt: TimelineApplicationAttemptInfo): String = {
+ val never = "-"
+
+ s"Attempt ID ${attempt.attemptId }" +
+ s" started ${timeShort(attempt.startTime, never) }," +
+ s" ended ${timeShort(attempt.endTime, never) }" +
+ s" updated ${timeShort(attempt.lastUpdated, never) }" +
+ s" completed = ${attempt.completed }"
+ }
+
+ /**
+ * Build a combined list with the policy of all original values come first,
+ * followed by the later ones. Unless there is a later entry of the same ID...
+ * In which case, that later entry appears.
+ *
+ * @param original original list of entries
+ * @param latest later list of entries
+ * @return a combined list.
+ */
+ def combineResults(
+ original: Seq[TimelineApplicationHistoryInfo],
+ latest: Seq[TimelineApplicationHistoryInfo]): Seq[TimelineApplicationHistoryInfo] = {
+ // build map of original
+ val results = new scala.collection.mutable.HashMap[String, TimelineApplicationHistoryInfo]
+ original.map((elt) => results.put(elt.id, elt))
+ // then merge in the new values, a combination of appending and adding
+ latest.foreach(history => {
+ val id = history.id
+ results.put(id, results.get(id) match {
+ case Some(old) => mergeAttempts(old, history)
+ case None => history
+ })
+ })
+ results.values.toList
+ }
+
+ /**
+ * Merge two attempts. It is critical that the older event comes first,
+ * so that the ordering is consistent.
+ *
+ * It's not enough to simply append values, as if the same attempt were merged
+ * the result would be a history with duplicate attempts. Every attempt needs to
+ * be unique.
+ *
+ * Note that `None` may be an attempt. Two entries with that as their attempt ID are
+ * treated as equal and merged
+ * @param old the older history
+ * @param latest the latest attempt
+ * @return the merged set
+ */
+ def mergeAttempts(old: TimelineApplicationHistoryInfo, latest: TimelineApplicationHistoryInfo)
+ : TimelineApplicationHistoryInfo = {
+ val oldAttempts = old.attempts
+ val latestAttempts = latest.attempts
+ new TimelineApplicationHistoryInfo(old.id, old.name,
+ mergeAttemptInfoLists(oldAttempts, latestAttempts))
+ }
+
+ /**
+ * Merge the lists of two attempts. Factored out for ease of testing.
+ *
+ * @param oldAttempts list of old attempts -assuming no duplicate attempts
+ * @param latestAttempts list of later attempts; this may include duplicate attempt entries.
+ * @return an ordered list of attempts with original attempt entries removed if a later
+ * version updated the event information.
+ */
+ def mergeAttemptInfoLists(
+ oldAttempts: List[TimelineApplicationAttemptInfo],
+ latestAttempts: List[TimelineApplicationAttemptInfo])
+ : List[TimelineApplicationAttemptInfo] = {
+
+ // build map of id->attempt which will be updated during merge
+ var attemptMap = mutable.Map[Option[String], TimelineApplicationAttemptInfo]()
+ oldAttempts foreach {
+ a => attemptMap += (a.attemptId -> a)
+ }
+
+ latestAttempts foreach (a => {
+ val id = a.attemptId
+ attemptMap get id match {
+ case None =>
+ // no match: insert into the map
+ // and add to the map of attempts
+ attemptMap += (id -> a)
+
+ case Some(existing) =>
+ // existing match, so merge.
+ // this will also match Some(None), meaning there is an attempt ID with the of `None`,
+ attemptMap += (id -> mostRecentAttempt(existing, a))
+ }
+ })
+ val finalMapValues = attemptMap.values.toList
+ val orderedAttemptList = sortAttempts(finalMapValues)
+ orderedAttemptList
+ }
+
+ /**
+ * Compare two entries of information about the same application attempt -and decide which
+ * is the most recent one.
+ *
+ * @param attempt1 attempt 1
+ * @param attempt2 attempt 2
+ * @return the preferred outcome
+ */
+ def mostRecentAttempt
+ (attempt1: TimelineApplicationAttemptInfo,
+ attempt2: TimelineApplicationAttemptInfo): TimelineApplicationAttemptInfo = {
+ (attempt1, attempt2) match {
+ case (a1, a2) if a1.version > 0 && a2.version > 0 =>
+ // use the version field if set
+ if (a2.version > a1.version) a2 else a1
+ case (a1, a2) if a1.completed => a1
+ case (a1, a2) if a2.completed => a2
+ case (a1, a2) if a2.lastUpdated >= a1.lastUpdated => a2
+ case (a1, _) => a1
+ }
+ }
+
+ /**
+ * Comparator to find which attempt is newer than the other.
+ *
+ * @param attempt1 attempt 1
+ * @param attempt2 attempt 2
+ * @return true if attempt1 is considered newer than attempt2
+ */
+ def attemptNewerThan(
+ attempt1: TimelineApplicationAttemptInfo,
+ attempt2: TimelineApplicationAttemptInfo): Boolean = {
+ if (attempt1.version > 0 && attempt2.version > 0) {
+ attempt1.version > attempt2.version
+ } else {
+ attempt1.lastUpdated > attempt2.lastUpdated
+ }
+ }
+
+ /**
+ * Sort an attempt list using the such that newer attempts come first.
+ *
+ * It is critical for the web UI that completed events come before incomplete ones, so if
+ * a completed one is found, it takes priority.
+ * @param attempts attempt list to sort
+ * @return a sorted list
+ */
+ def sortAttempts(attempts: List[TimelineApplicationAttemptInfo])
+ : List[TimelineApplicationAttemptInfo] = {
+ // sort attempts
+ attempts.sortWith(attemptNewerThan)
+ }
+
+ /**
+ * Get the start time of an application.
+ *
+ * For multiple attempts, the first attempt is chosen as the start time, so
+ * that if sorting a list of application, the one attempted first is considered
+ * the oldest.
+ * @param info history info
+ */
+ def startTime(info: TimelineApplicationHistoryInfo): Long = {
+ info.attempts match {
+ case Nil => 0L
+ case (h :: _) => h.startTime
+ }
+ }
+
+ /**
+ * Sort a list of applications by their start time.
+ *
+ * @param history history list
+ * @return a new, sorted list
+ */
+ def sortApplicationsByStartTime(history: Seq[TimelineApplicationHistoryInfo]):
+ Seq[TimelineApplicationHistoryInfo] = {
+ history.sortBy(startTime)
+ }
+
+ /**
+ * Find the latest application in the list. Scans the list once, so is O(n) even if
+ * the list is already sorted.
+ *
+ * @param history history to scan (which can be an empty list
+ * @return the latest element in the list
+ */
+ def findLatestApplication(history: Seq[TimelineApplicationHistoryInfo])
+ : Option[TimelineApplicationHistoryInfo] = {
+ history match {
+ case Nil => None
+ case l => Some(l.reduceLeft((x, y) => if (startTime(x) < startTime(y)) y else x))
+ }
+ }
+
+ /**
+ * Find the latest application in the list. Scans the list once, so is `O(n)` even if
+ * the list is already sorted.
+ *
+ * @param history history to scan (which can be an empty list
+ * @return the element in the list which started first
+ */
+ def findOldestApplication(history: Seq[TimelineApplicationHistoryInfo])
+ : Option[TimelineApplicationHistoryInfo] = {
+ history match {
+ case Nil => None
+ case l => Some(l.reduceLeft((x, y) => if (startTime(x) <= startTime(y)) x else y))
+ }
+ }
+
+ /**
+ * Find the application that represents the start of the update window.
+ *
+ * First it locates the oldest incomplete application in the list.
+ * If there are no incomplete entries, then the latest completed entry is picked up.
+ *
+ * @param history history to scan (which can be an empty list)
+ * @return the latest element in the list, or `None` for no match
+ */
+ def findStartOfWindow(history: Seq[TimelineApplicationHistoryInfo])
+ : Option[TimelineApplicationHistoryInfo] = {
+ findIncompleteApplications(history) match {
+ // no incomplete apps; use latest
+ case Nil => findLatestApplication(history)
+ case incomplete => findOldestApplication(incomplete)
+ }
+ }
+
+ /**
+ * Is an application completed?
+ *
+ * @param info info to examine
+ * @return true if its last attempt completed
+ */
+ def isCompleted(info: TimelineApplicationHistoryInfo): Boolean = {
+ if (info.attempts.isEmpty) {
+ // no events, implicitly not complete
+ false
+ } else {
+ info.attempts.head.completed
+ }
+ }
+
+ /**
+ * Get the last update time; 0 if there are no attempts.
+ *
+ * @param info info to examine
+ * @return the last update time of the attempt first in the list; 0 if none found
+ */
+ def lastUpdated(info: TimelineApplicationHistoryInfo): Long = {
+ if (info.attempts.isEmpty) {
+ // no events, implicitly not complete
+ 0
+ } else {
+ info.attempts.head.lastUpdated
+ }
+ }
+
+ /**
+ * Build the list of all incomplete applications.
+ *
+ * @param history history to scan (which can be an empty list)
+ * @return a filtered list containing only those applications which are considered incompleted.
+ */
+ def findIncompleteApplications(history: Seq[TimelineApplicationHistoryInfo])
+ : Seq[TimelineApplicationHistoryInfo] = {
+ history.filter(x => !isCompleted(x))
+ }
+
+ /**
+ * Count the number of incomplete applications in the sequence.
+ *
+ * @param history history to scan (which can be an empty list)
+ * @return the number of incomplete applications found
+ */
+ def countIncompleteApplications(history: Seq[TimelineApplicationHistoryInfo]): Int = {
+ findIncompleteApplications(history).size
+ }
+
+ /**
+ * Find an application by its ID.
+ *
+ * @param history history to scan (which can be an empty list)
+ * @param id app Id
+ * @return the application or `None`
+ */
+ def findAppById(history: Seq[TimelineApplicationHistoryInfo], id: String)
+ : Option[TimelineApplicationHistoryInfo] = {
+ history.find(_.id == id)
+ }
+
+ /**
+ * Get the finish time of the application, if it finished.
+ *
+ * @param appReport YARN application report
+ * @return the finish time, or `None` if the application has not finished
+ */
+ def finishTime(appReport: ApplicationReport): Option[Long] = {
+ if (appReport.getFinishTime > 0) Some(appReport.getFinishTime) else None
+ }
+
+ /**
+ * Has an application finished?
+ *
+ * @param appReport application report
+ * @return true if the application is in some terminated state
+ */
+ def isFinished(appReport: ApplicationReport): Boolean = {
+ appReport.getYarnApplicationState match {
+ case YarnApplicationState.FINISHED => true
+ case YarnApplicationState.FAILED => true
+ case YarnApplicationState.KILLED => true
+ case _ => false
+ }
+ }
+
+ /**
+ * Given a sequence of application reports, create a map of them
+ * mapped by the string value of their application ID.
+ *
+ * @param reports list of reports
+ * @return mapped set
+ */
+ def reportsToMap(reports: Seq[ApplicationReport]): Map[String, ApplicationReport] = {
+ var map: Map[String, ApplicationReport] = Map()
+ reports.foreach(r => map = map + (r.getApplicationId.toString -> r))
+ map
+ }
+
+ /**
+ * Compare the list of applications with the YARN list and convert any that
+ * are incomplete to completed state if they aren't found in the
+ * list of running apps -or they are there, but failed.
+ *
+ * @param apps list of applications
+ * @param recordMap map of app id to Application report entries
+ * @param currentTime the current time in millis
+ * @param livenessWindow the window in millis within which apps are considered automatically live
+ * @return list of apps which are marked as incomplete but no longer running
+ */
+ private[yarn] def completeAppsFromYARN(
+ apps: Seq[TimelineApplicationHistoryInfo],
+ recordMap: Map[String, ApplicationReport],
+ currentTime: Long,
+ livenessWindow: Long): Seq[TimelineApplicationHistoryInfo] = {
+
+ // complete an application, finishTime is optional finish time from report
+ def complete(appInfo: TimelineApplicationHistoryInfo,
+ finishTime: Option[Long]): TimelineApplicationHistoryInfo = {
+ val incomplete = appInfo.attempts.head
+ val endTime = finishTime match {
+ case Some(t) => t
+ case None => incomplete.lastUpdated
+ }
+ val updated = new TimelineApplicationAttemptInfo(
+ incomplete.attemptId,
+ incomplete.startTime,
+ endTime,
+ incomplete.lastUpdated,
+ incomplete.sparkUser,
+ true,
+ incomplete.entityId,
+ incomplete.sparkAttemptId)
+ logDebug(s"Marking application ${appInfo.id} completed: ${describeAttempt(incomplete)}")
+ new TimelineApplicationHistoryInfo(appInfo.id, appInfo.name, updated :: appInfo.attempts.tail)
+ }
+
+ // complete an application from a report
+ def completeFromReport(appInfo: TimelineApplicationHistoryInfo, report: ApplicationReport):
+ TimelineApplicationHistoryInfo = {
+
+ complete(appInfo, finishTime(report))
+ }
+
+ // build and return an updated set of applications
+ apps.map(app =>
+ if (isCompleted(app)) {
+ // app finished: return
+ app
+ } else {
+ // app is incomplete, so look up the record.
+ val id = app.id
+ recordMap.get(id) match {
+ case Some(report) if isFinished(report) =>
+ // report found and app is actually finished
+ logDebug(s"Incomplete app $id has halted as ${report.getYarnApplicationState}")
+ completeFromReport(app, report)
+
+ case Some(report) =>
+ // in progress
+ logDebug(s"App $id is in running state ${report.getYarnApplicationState}")
+ app
+
+ case None =>
+ // app not found in the map. Outcome now depends on when it happened, to avoid
+ // over-reacting to incomplete applications which are between AM instances and
+ // briefly unlisted.
+ val updated = lastUpdated(app)
+ val humanTime = humanDateCurrentTZ(updated, "(never)")
+ logDebug(s"Incomplete app $id updated at ${humanTime}is not in list of running apps")
+ if ((currentTime - updated) > livenessWindow) {
+ complete(app, None)
+ } else {
+ // complete it at the last update time
+ app
+ }
+ }
+ }
+ )
+ }
+
+}
diff --git a/yarn/src/history/test/resources/org/apache/spark/deploy/history/yarn/integration/history-1.json b/yarn/src/history/test/resources/org/apache/spark/deploy/history/yarn/integration/history-1.json
new file mode 100644
index 000000000000..80169fffe01f
--- /dev/null
+++ b/yarn/src/history/test/resources/org/apache/spark/deploy/history/yarn/integration/history-1.json
@@ -0,0 +1,381 @@
+{
+ "entities": [
+ {
+ "events": [
+ {
+ "timestamp": 1443746071340,
+ "eventtype": "SparkListenerApplicationStart-3",
+ "eventinfo": {
+ "User": "root",
+ "App ID": "application_1443668830514_0009",
+ "Event": "SparkListenerApplicationStart",
+ "App Name": "SparkSQL::10.0.0.143",
+ "App Attempt ID": null,
+ "Timestamp": 1443746021367
+ }
+ },
+ {
+ "timestamp": 1443746071307,
+ "eventtype": "SparkListenerEnvironmentUpdate-2",
+ "eventinfo": {
+ "System Properties": {
+ "SPARK_YARN_MODE": "true",
+ "java.vm.version": "24.79-b02",
+ "hive.server2.thrift.http.path": "/",
+ "sun.jnu.encoding": "UTF-8",
+ "java.vendor.url": "http://java.oracle.com/",
+ "java.vm.info": "mixed mode",
+ "user.dir": "/usr/hadoop/spark/bin",
+ "sun.cpu.isalist": "",
+ "java.awt.graphicsenv": "sun.awt.X11GraphicsEnvironment",
+ "sun.os.patch.level": "unknown",
+ "hive.server2.transport.mode": "http",
+ "sun.nio.ch.bugLevel": "",
+ "java.io.tmpdir": "/tmp",
+ "user.home": "/root",
+ "java.awt.printerjob": "sun.print.PSPrinterJob",
+ "java.version": "1.7.0_79",
+ "file.encoding.pkg": "sun.io",
+ "java.vendor.url.bug": "http://bugreport.sun.com/bugreport/",
+ "file.encoding": "UTF-8",
+ "sun.java.command": "org.apache.spark.deploy.SparkSubmit --master yarn-client --conf spark.driver.memory=1g --class org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 --driver-cores 1 --executor-memory 1g --num-executors 1 spark-internal --hiveconf hive.server2.transport.mode=http --hiveconf hive.server2.thrift.http.port=10002 --hiveconf hive.server2.thrift.http.path=/",
+ "line.separator": "\n",
+ "hive.server2.thrift.http.port": "10002",
+ "java.vm.specification.vendor": "Oracle Corporation",
+ "java.vm.vendor": "Oracle Corporation",
+ "SPARK_SUBMIT": "true",
+ "sun.io.unicode.encoding": "UnicodeLittle",
+ "user.name": "root",
+ "os.arch": "amd64",
+ "user.language": "en",
+ "java.runtime.version": "1.7.0_79-b14",
+ "sun.boot.class.path": "/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/rhino.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/classes",
+ "sun.cpu.endian": "little",
+ "awt.toolkit": "sun.awt.X11.XToolkit",
+ "sun.boot.library.path": "/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/amd64",
+ "java.home": "/usr/lib/jvm/java-7-openjdk-amd64/jre",
+ "java.vm.name": "OpenJDK 64-Bit Server VM",
+ "java.endorsed.dirs": "/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/endorsed",
+ "sun.management.compiler": "HotSpot 64-Bit Tiered Compilers",
+ "java.runtime.name": "OpenJDK Runtime Environment",
+ "java.library.path": "/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib",
+ "file.separator": "/",
+ "java.specification.vendor": "Oracle Corporation",
+ "java.vm.specification.version": "1.7",
+ "sun.java.launcher": "SUN_STANDARD",
+ "user.timezone": "Etc/UTC",
+ "os.name": "Linux",
+ "path.separator": ":",
+ "java.ext.dirs": "/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext",
+ "sun.arch.data.model": "64",
+ "java.specification.name": "Java Platform API Specification",
+ "os.version": "3.13.0-63-generic",
+ "java.class.version": "51.0",
+ "user.country": "US",
+ "java.vendor": "Oracle Corporation",
+ "java.vm.specification.name": "Java Virtual Machine Specification",
+ "java.specification.version": "1.7"
+ },
+ "JVM Information": {
+ "Scala Version": "version 2.10.4",
+ "Java Home": "/usr/lib/jvm/java-7-openjdk-amd64/jre",
+ "Java Version": "1.7.0_79 (Oracle Corporation)"
+ },
+ "Event": "SparkListenerEnvironmentUpdate",
+ "Spark Properties": {
+ "spark.yarn.queue": "default",
+ "spark.scheduler.mode": "FIFO",
+ "spark.history.kerberos.principal": "none",
+ "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS": "headnode0.example.net,headnode1.example.net",
+ "spark.yarn.applicationMaster.waitTries": "10",
+ "spark.master": "yarn-client",
+ "spark.ui.filters": "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter",
+ "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES": "http://headnode0.example.net:8088/proxy/application_1443668830514_0009,http://headnode1.example.net:8088/proxy/application_1443668830514_0009",
+ "spark.yarn.scheduler.heartbeat.interval-ms": "5000",
+ "spark.history.yarn.diagnostics": "true",
+ "spark.history.provider": "org.apache.spark.deploy.yarn.history.YarnHistoryProvider",
+ "spark.kryo.referenceTracking": "false",
+ "spark.executor.memory": "1g",
+ "spark.executor.instances": "1",
+ "spark.app.id": "application_1443668830514_0009",
+ "spark.eventLog.enabled": "true",
+ "spark.driver.memory": "1g",
+ "spark.yarn.submit.file.replication": "3",
+ "spark.yarn.executor.memoryOverhead": "390",
+ "spark.jars": "",
+ "spark.serializer": "org.apache.spark.serializer.KryoSerializer",
+ "spark.history.fs.logDirectory": "/tmp/spark-events",
+ "spark.yarn.preserve.staging.files": "false",
+ "spark.yarn.max.executor.failures": "4",
+ "spark.yarn.services": "org.apache.spark.deploy.yarn.history.YarnHistoryService",
+ "spark.yarn.am.extraJavaOptions": "-Dhdp.version=2.3.2.0-2937",
+ "spark.history.retainedApplications": "0",
+ "spark.externalBlockStore.folderName": "spark-1c27ac67-8503-4dfc-9ac7-871c343409d5",
+ "spark.history.kerberos.keytab": "none",
+ "spark.driver.extraJavaOptions": "-Dhdp.version=2.3.2.0-2937",
+ "spark.driver.appUIAddress": "http://10.0.0.143:4040",
+ "spark.driver.host": "10.0.0.143",
+ "spark.fileserver.uri": "http://10.0.0.143:50237",
+ "spark.yarn.driver.memoryOverhead": "390",
+ "spark.yarn.containerLauncherMaxThreads": "25",
+ "spark.eventLog.dir": "/tmp/spark-events",
+ "spark.yarn.historyServer.address": "headnode0.example.net:18080",
+ "spark.app.name": "SparkSQL::10.0.0.143",
+ "spark.driver.port": "37375",
+ "spark.executor.id": "driver",
+ "spark.history.ui.port": "18080"
+ },
+ "Classpath Entries": {
+ "/usr/hadoop/spark/lib/datanucleus-core-3.2.10.jar": "System Classpath",
+ "/usr/hadoop/spark/sbin/../conf/": "System Classpath",
+ "/usr/hadoop/current/hadoop-client/conf/": "System Classpath",
+ "/usr/hadoop/current/hadoop-client/lib/azure-storage-2.2.0.jar": "System Classpath",
+ "/usr/hadoop/current/hadoop-client/hadoop-azure.jar": "System Classpath",
+ "/usr/hadoop/spark/lib/datanucleus-rdbms-3.2.9.jar": "System Classpath",
+ "/usr/hadoop/spark/lib/spark-assembly-1.4.1.2.3.2.0-2937-hadoop2.7.1.2.3.2.0-2937.jar": "System Classpath",
+ "/usr/hadoop/spark/lib/datanucleus-api-jdo-3.2.6.jar": "System Classpath"
+ }
+ }
+ },
+ {
+ "timestamp": 1443746071268,
+ "eventtype": "SparkListenerBlockManagerAdded-1",
+ "eventinfo": {
+ "Block Manager ID": {
+ "Port": 42692,
+ "Host": "10.0.0.143",
+ "Executor ID": "driver"
+ },
+ "Event": "SparkListenerBlockManagerAdded",
+ "Maximum Memory": 556038881,
+ "Timestamp": 1443746070770
+ }
+ }
+ ],
+ "entitytype": "spark_event_v01",
+ "entity": "application_1443668830514_0009",
+ "starttime": 1443746071268,
+ "domain": "DEFAULT",
+ "relatedentities": {},
+ "primaryfilters": {
+ "appName": [
+ "SparkSQL::10.0.0.143"
+ ],
+ "appUser": [
+ "root"
+ ],
+ "startApp": [
+ "SparkListenerApplicationStart"
+ ]
+ },
+ "otherinfo": {
+ "startTime": 1443746021367,
+ "appName": "SparkSQL::10.0.0.143",
+ "appUser": "root",
+ "lastUpdated": 1443746071352
+ }
+ },
+ {
+ "events": [
+ {
+ "timestamp": 1443746012909,
+ "eventtype": "SparkListenerApplicationEnd-419",
+ "eventinfo": {
+ "Event": "SparkListenerApplicationEnd",
+ "Timestamp": 1443746012909
+ }
+ },
+ {
+ "timestamp": 1443741864403,
+ "eventtype": "SparkListenerBlockManagerAdded-4",
+ "eventinfo": {
+ "Block Manager ID": {
+ "Port": 40348,
+ "Host": "10.0.0.231",
+ "Executor ID": "1"
+ },
+ "Event": "SparkListenerBlockManagerAdded",
+ "Maximum Memory": 556038881,
+ "Timestamp": 1443741864358
+ }
+ },
+ {
+ "timestamp": 1443741820933,
+ "eventtype": "SparkListenerApplicationStart-3",
+ "eventinfo": {
+ "User": "root",
+ "App ID": "application_1443668830514_0008",
+ "Event": "SparkListenerApplicationStart",
+ "App Name": "SparkSQL::10.0.0.143",
+ "App Attempt ID": null,
+ "Timestamp": 1443741773688
+ }
+ },
+ {
+ "timestamp": 1443741820913,
+ "eventtype": "SparkListenerEnvironmentUpdate-2",
+ "eventinfo": {
+ "System Properties": {
+ "SPARK_YARN_MODE": "true",
+ "java.vm.version": "24.79-b02",
+ "hive.server2.thrift.http.path": "/",
+ "sun.jnu.encoding": "UTF-8",
+ "java.vendor.url": "http://java.oracle.com/",
+ "java.vm.info": "mixed mode",
+ "user.dir": "/usr/hadoop/spark/sbin",
+ "sun.cpu.isalist": "",
+ "java.awt.graphicsenv": "sun.awt.X11GraphicsEnvironment",
+ "sun.os.patch.level": "unknown",
+ "hive.server2.transport.mode": "http",
+ "sun.nio.ch.bugLevel": "",
+ "java.io.tmpdir": "/tmp",
+ "user.home": "/root",
+ "java.awt.printerjob": "sun.print.PSPrinterJob",
+ "java.version": "1.7.0_79",
+ "file.encoding.pkg": "sun.io",
+ "java.vendor.url.bug": "http://bugreport.sun.com/bugreport/",
+ "file.encoding": "UTF-8",
+ "sun.java.command": "org.apache.spark.deploy.SparkSubmit --master yarn-client --conf spark.driver.memory=1g --class org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 --driver-cores 1 --executor-memory 1g --num-executors 1 spark-internal --hiveconf hive.server2.transport.mode=http --hiveconf hive.server2.thrift.http.port=10002 --hiveconf hive.server2.thrift.http.path=/",
+ "line.separator": "\n",
+ "hive.server2.thrift.http.port": "10002",
+ "java.vm.specification.vendor": "Oracle Corporation",
+ "java.vm.vendor": "Oracle Corporation",
+ "SPARK_SUBMIT": "true",
+ "sun.io.unicode.encoding": "UnicodeLittle",
+ "user.name": "root",
+ "os.arch": "amd64",
+ "user.language": "en",
+ "java.runtime.version": "1.7.0_79-b14",
+ "sun.boot.class.path": "/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/rhino.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-7-openjdk-amd64/jre/classes",
+ "sun.cpu.endian": "little",
+ "awt.toolkit": "sun.awt.X11.XToolkit",
+ "sun.boot.library.path": "/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/amd64",
+ "java.home": "/usr/lib/jvm/java-7-openjdk-amd64/jre",
+ "java.vm.name": "OpenJDK 64-Bit Server VM",
+ "java.endorsed.dirs": "/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/endorsed",
+ "sun.management.compiler": "HotSpot 64-Bit Tiered Compilers",
+ "java.runtime.name": "OpenJDK Runtime Environment",
+ "java.library.path": "/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib",
+ "file.separator": "/",
+ "java.specification.vendor": "Oracle Corporation",
+ "java.vm.specification.version": "1.7",
+ "sun.java.launcher": "SUN_STANDARD",
+ "user.timezone": "Etc/UTC",
+ "os.name": "Linux",
+ "path.separator": ":",
+ "hdp.version": "2.3.2.0-2937",
+ "java.ext.dirs": "/usr/lib/jvm/java-7-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext",
+ "sun.arch.data.model": "64",
+ "java.specification.name": "Java Platform API Specification",
+ "os.version": "3.13.0-63-generic",
+ "java.class.version": "51.0",
+ "user.country": "US",
+ "java.vendor": "Oracle Corporation",
+ "java.vm.specification.name": "Java Virtual Machine Specification",
+ "java.specification.version": "1.7"
+ },
+ "JVM Information": {
+ "Scala Version": "version 2.10.4",
+ "Java Home": "/usr/lib/jvm/java-7-openjdk-amd64/jre",
+ "Java Version": "1.7.0_79 (Oracle Corporation)"
+ },
+ "Event": "SparkListenerEnvironmentUpdate",
+ "Spark Properties": {
+ "spark.yarn.queue": "default",
+ "spark.scheduler.mode": "FIFO",
+ "spark.history.kerberos.principal": "none",
+ "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS": "headnode0.example.net,headnode1.example.net",
+ "spark.yarn.applicationMaster.waitTries": "10",
+ "spark.master": "yarn-client",
+ "spark.ui.filters": "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter",
+ "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES": "http://headnode0.example.net:8088/proxy/application_1443668830514_0008,http://headnode1.example.net:8088/proxy/application_1443668830514_0008",
+ "spark.yarn.scheduler.heartbeat.interval-ms": "5000",
+ "spark.history.yarn.diagnostics": "true",
+ "spark.history.provider": "org.apache.spark.deploy.yarn.history.YarnHistoryProvider",
+ "spark.kryo.referenceTracking": "false",
+ "spark.executor.memory": "1g",
+ "spark.executor.instances": "1",
+ "spark.app.id": "application_1443668830514_0008",
+ "spark.eventLog.enabled": "true",
+ "spark.driver.memory": "1g",
+ "spark.yarn.submit.file.replication": "3",
+ "spark.yarn.executor.memoryOverhead": "390",
+ "spark.jars": "",
+ "spark.serializer": "org.apache.spark.serializer.KryoSerializer",
+ "spark.history.fs.logDirectory": "/tmp/spark-events",
+ "spark.yarn.preserve.staging.files": "false",
+ "spark.yarn.max.executor.failures": "4",
+ "spark.yarn.services": "org.apache.spark.deploy.yarn.history.YarnHistoryService",
+ "spark.yarn.am.extraJavaOptions": "-Dhdp.version=2.3.2.0-2937",
+ "spark.history.retainedApplications": "0",
+ "spark.externalBlockStore.folderName": "spark-58c6e87e-4ced-44ee-b7cc-3aa354626aa5",
+ "spark.history.kerberos.keytab": "none",
+ "spark.driver.extraJavaOptions": "-Dhdp.version=2.3.2.0-2937",
+ "spark.driver.appUIAddress": "http://10.0.0.143:4040",
+ "spark.driver.host": "10.0.0.143",
+ "spark.fileserver.uri": "http://10.0.0.143:59458",
+ "spark.yarn.driver.memoryOverhead": "390",
+ "spark.yarn.containerLauncherMaxThreads": "25",
+ "spark.eventLog.dir": "/tmp/spark-events",
+ "spark.yarn.historyServer.address": "headnode0.example.net:18080",
+ "spark.app.name": "SparkSQL::10.0.0.143",
+ "spark.driver.port": "42736",
+ "spark.executor.id": "driver",
+ "spark.history.ui.port": "18080"
+ },
+ "Classpath Entries": {
+ "/usr/hadoop/spark/lib/datanucleus-core-3.2.10.jar": "System Classpath",
+ "/usr/hadoop/spark/sbin/../conf/": "System Classpath",
+ "/usr/hadoop/current/hadoop-client/conf/": "System Classpath",
+ "/usr/hadoop/current/hadoop-client/lib/azure-storage-2.2.0.jar": "System Classpath",
+ "/usr/hadoop/current/hadoop-client/hadoop-azure.jar": "System Classpath",
+ "/usr/hadoop/spark/lib/datanucleus-rdbms-3.2.9.jar": "System Classpath",
+ "/usr/hadoop/spark/lib/spark-assembly-1.4.1.2.3.2.0-2937-hadoop2.7.1.2.3.2.0-2937.jar": "System Classpath",
+ "/usr/hadoop/spark/lib/datanucleus-api-jdo-3.2.6.jar": "System Classpath"
+ }
+ }
+ },
+ {
+ "timestamp": 1443741820869,
+ "eventtype": "SparkListenerBlockManagerAdded-1",
+ "eventinfo": {
+ "Block Manager ID": {
+ "Port": 37616,
+ "Host": "10.0.0.143",
+ "Executor ID": "driver"
+ },
+ "Event": "SparkListenerBlockManagerAdded",
+ "Maximum Memory": 556038881,
+ "Timestamp": 1443741820309
+ }
+ }
+ ],
+ "entitytype": "spark_event_v01",
+ "entity": "application_1443668830514_0008",
+ "starttime": 1443741820869,
+ "domain": "DEFAULT",
+ "relatedentities": {},
+ "primaryfilters": {
+ "appName": [
+ "SparkSQL::10.0.0.143"
+ ],
+ "appUser": [
+ "root"
+ ],
+ "endApp": [
+ "SparkListenerApplicationEnd"
+ ],
+ "startApp": [
+ "SparkListenerApplicationStart"
+ ]
+ },
+ "otherinfo": {
+ "startTime": 1443741773688,
+ "appName": "SparkSQL::10.0.0.143",
+ "appUser": "root",
+ "lastUpdated": 1443746012912,
+ "endTime": 1443746012909
+ }
+ }
+ ]
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/DisabledProviderDiagnosticsSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/DisabledProviderDiagnosticsSuite.scala
new file mode 100644
index 000000000000..e0904c4d069f
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/DisabledProviderDiagnosticsSuite.scala
@@ -0,0 +1,62 @@
+/*
+ * 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.yarn.failures
+
+import java.net.{URI, URL}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.integration.AbstractHistoryIntegrationTests
+import org.apache.spark.deploy.history.yarn.rest.JerseyBinding
+import org.apache.spark.deploy.history.yarn.server.{TimelineQueryClient, YarnHistoryProvider}
+
+class DisabledProviderDiagnosticsSuite extends AbstractHistoryIntegrationTests {
+
+ /**
+ * Create a history provider instance.
+ * @param conf configuration
+ * @return the instance
+ */
+ override protected def createHistoryProvider(conf: SparkConf): YarnHistoryProvider = {
+ val yarnConf = sc.hadoopConfiguration
+
+ val client = new TimelineQueryClient(timelineRootEndpoint(),
+ yarnConf,
+ JerseyBinding.createClientConfig())
+ new DisabledFailingYarnHistoryProvider(client, false, client.getTimelineURI(), conf)
+ }
+
+ def timelineRootEndpoint(): URI = {
+ val realTimelineEndpoint = getTimelineEndpoint(sc.hadoopConfiguration).toURL
+ new URL(realTimelineEndpoint, "/").toURI
+ }
+
+ /**
+ * When the UI is disabled, the GET works but there's an error message
+ * warning of the fact. The endpoint check is not reached.
+ */
+ test("Probe Disabled UI") {
+ def probeDisabledUI(webUI: URL, provider: YarnHistoryProvider): Unit = {
+ val fp = provider.asInstanceOf[FailingYarnHistoryProvider]
+ probeEmptyWebUI(webUI, provider)
+ val body = getHtmlPage(webUI, YarnHistoryProvider.TEXT_SERVICE_DISABLED :: Nil)
+ }
+ webUITest("Probe Disabled UI", probeDisabledUI)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/FailingTimelineQueryClient.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/FailingTimelineQueryClient.scala
new file mode 100644
index 000000000000..a90cc5c066d8
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/FailingTimelineQueryClient.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.yarn.failures
+
+import java.io.ByteArrayInputStream
+import java.net.{NoRouteToHostException, URI}
+
+import com.sun.jersey.api.client.{ClientResponse, UniformInterfaceException}
+import com.sun.jersey.api.client.config.ClientConfig
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.deploy.history.yarn.server.TimelineQueryClient
+
+/**
+ * A timeline query client that always throws an exception
+ */
+class FailingTimelineQueryClient(timelineURI: URI,
+ conf: Configuration,
+ jerseyClientConfig: ClientConfig)
+ extends TimelineQueryClient(timelineURI, conf, jerseyClientConfig) {
+
+ /**
+ * Throw the exception
+ */
+ override def innerExecAction[T](action: () => T): T = {
+ throw new NoRouteToHostException(FailingTimelineQueryClient.ERROR_TEXT)
+ }
+
+ /**
+ * toString method returns the URI of the timeline service
+ * @return
+ */
+ override def toString: String = {
+ "Failing " + super.toString
+ }
+
+}
+
+object FailingTimelineQueryClient {
+ val ERROR_TEXT = "No-route-to-host"
+}
+
+/**
+ * Client which returns a wrapped HTTP status code
+ */
+class ClientResponseTimelineQueryClient(status: Int, text: String,
+ timelineURI: URI, conf: Configuration, jerseyClientConfig: ClientConfig)
+ extends TimelineQueryClient(timelineURI, conf, jerseyClientConfig) {
+
+ val response = new ClientResponse(status,
+ null,
+ new ByteArrayInputStream(text.getBytes("UTF-8")),
+ null)
+
+ /**
+ * Throw the exception
+ */
+ override def innerExecAction[T](action: () => T): T = {
+ throw new UniformInterfaceException(response, false)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/FailingYarnHistoryProvider.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/FailingYarnHistoryProvider.scala
new file mode 100644
index 000000000000..9bef7f95fca6
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/FailingYarnHistoryProvider.scala
@@ -0,0 +1,159 @@
+/*
+ * 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.yarn.failures
+
+import java.net.URI
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.deploy.history.yarn.rest.JerseyBinding
+import org.apache.spark.deploy.history.yarn.server.{TimelineQueryClient, YarnHistoryProvider}
+
+/**
+ * This is a YARN history provider that can be given
+ * a (possibly failing) query client, and can be configured
+ * as to whether to start with an endpoint check.
+ * @param queryClient query client
+ * @param skipEndpointCheck should the initial endpoint
+ * check be skipped? It will if this
+ * is true
+ * @param endpoint URI of the service.
+ */
+class FailingYarnHistoryProvider(
+ queryClient: TimelineQueryClient,
+ skipEndpointCheck: Boolean,
+ endpoint: URI,
+ sparkConf: SparkConf,
+ refreshEnabled: Boolean = false) extends YarnHistoryProvider(sparkConf) with Logging {
+
+ init()
+
+ /**
+ * Is the timeline service (and therefore this provider) enabled.
+ * @return true : always
+ */
+ override def enabled: Boolean = {
+ true
+ }
+
+ /**
+ * Any initialization logic
+ */
+ private def init(): Unit = {
+ setEndpointChecked(skipEndpointCheck)
+ }
+
+ /**
+ *
+ * @return the endpoint
+ */
+ override def getEndpointURI: URI = {
+ endpoint
+ }
+
+ /**
+ * @return the `queryClient` field.
+ */
+ override protected def createTimelineQueryClient(): TimelineQueryClient = {
+ queryClient
+ }
+
+ /**
+ * @return the `queryClient` field.
+ */
+ override def getTimelineQueryClient: TimelineQueryClient = {
+ queryClient
+ }
+
+ /**
+ * Set the endpoint checked flag to the desired value
+ * @param b new value
+ */
+ def setEndpointChecked(b: Boolean): Unit = {
+ endpointCheckExecuted.set(b)
+ }
+
+
+ /**
+ * export the endpoint check for testing
+ */
+ override def maybeCheckEndpoint(): Boolean = {
+ super.maybeCheckEndpoint()
+ }
+
+ /**
+ * Start the refresh thread with the given interval.
+ *
+ * When this thread exits, it will close the `timelineQueryClient`
+ * instance
+ */
+ override def startRefreshThread(): Unit = {
+ if (refreshEnabled) {
+ super.startRefreshThread()
+ }
+ }
+}
+
+/**
+ * A failing yarn history provider that returns enabled=false, always
+ * @param queryClient query client
+ * @param endpointCheckExecuted should the initial endpoint
+ * check be skipped? It will if this
+ * is true
+ * @param endpoint URI of the service.
+ * @param sparkConf spark configuration to use
+ */
+class DisabledFailingYarnHistoryProvider(queryClient: TimelineQueryClient,
+ endpointCheckExecuted: Boolean,
+ endpoint: URI,
+ sparkConf: SparkConf) extends FailingYarnHistoryProvider(
+ queryClient, endpointCheckExecuted, endpoint, sparkConf) {
+
+ /**
+ * false -always
+ */
+ override def enabled: Boolean = {
+ false
+ }
+}
+
+/**
+ * Some operations to help the failure tests
+ */
+object FailingYarnHistoryProvider extends Logging {
+
+ def createQueryClient(): FailingTimelineQueryClient = {
+ new FailingTimelineQueryClient(new URI("http://localhost:80/"),
+ new Configuration(), JerseyBinding.createClientConfig())
+ }
+
+ /**
+ * This inner provider calls most of its internal methods.
+ * @return a failing instance
+ */
+ def createFailingProvider(sparkConf: SparkConf,
+ endpointCheckExecuted: Boolean = false,
+ refreshEnabled: Boolean = false): YarnHistoryProvider = {
+ new FailingYarnHistoryProvider(createQueryClient(),
+ endpointCheckExecuted,
+ new URI("http://localhost:80/"),
+ sparkConf,
+ refreshEnabled)
+ }
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/TimelineQueryFailureSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/TimelineQueryFailureSuite.scala
new file mode 100644
index 000000000000..0d0a3031bd19
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/TimelineQueryFailureSuite.scala
@@ -0,0 +1,101 @@
+/*
+ * 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.yarn.failures
+
+import java.net.{NoRouteToHostException, URI}
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.YarnHistoryService
+import org.apache.spark.deploy.history.yarn.integration.AbstractHistoryIntegrationTests
+import org.apache.spark.deploy.history.yarn.rest.{JerseyBinding, UnauthorizedRequestException}
+import org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+class TimelineQueryFailureSuite extends AbstractHistoryIntegrationTests {
+
+ /**
+ * Create the client and the app server
+ * @param conf the hadoop configuration
+ */
+ override protected def startTimelineClientAndAHS(conf: Configuration): Unit = {
+ }
+
+ /**
+ * Create a history provider instance
+ * @param conf configuration
+ * @return the instance
+ */
+ override protected def createHistoryProvider(conf: SparkConf): YarnHistoryProvider = {
+ FailingYarnHistoryProvider.createFailingProvider(conf, false)
+ }
+
+ /**
+ * Verifies that failures are propagated
+ */
+ test("ClientGETFails") {
+ val failingClient = FailingYarnHistoryProvider.createQueryClient()
+ intercept[NoRouteToHostException] {
+ failingClient.get(new URI("http://localhost:80/"),
+ () => "failed" )
+ }
+ }
+
+ test("ClientListFails") {
+ val failingClient = FailingYarnHistoryProvider.createQueryClient()
+ intercept[NoRouteToHostException] {
+ failingClient.listEntities(YarnHistoryService.SPARK_EVENT_ENTITY_TYPE)
+ }
+ }
+
+ test("UnauthedClientListFails") {
+ val failingClient = new ClientResponseTimelineQueryClient(
+ 401, "401",
+ new URI("http://localhost:80/"),
+ new Configuration(),
+ JerseyBinding.createClientConfig())
+
+ intercept[UnauthorizedRequestException] {
+ failingClient.listEntities(YarnHistoryService.SPARK_EVENT_ENTITY_TYPE)
+ }
+ }
+
+ test("getTimelineEntity to fail") {
+ describe("getTimelineEntity to fail")
+
+ // not using intercept[] for better diagnostics on failure (i.e. rethrow the unwanted
+ // exception
+ var provider: FailingYarnHistoryProvider = null
+ try {
+ provider = createHistoryProvider(new SparkConf()).asInstanceOf[FailingYarnHistoryProvider]
+ provider.setEndpointChecked(true)
+ val entity = provider.getTimelineEntity("app1")
+ fail(s"Expected failure, got $entity")
+ } catch {
+ case ioe: NoRouteToHostException =>
+ logInfo(s"expected exception caught: $ioe")
+
+ } finally {
+ if (provider != null) {
+ provider.stop()
+ }
+ }
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/WebsiteDiagnosticsSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/WebsiteDiagnosticsSuite.scala
new file mode 100644
index 000000000000..8c34e8f5b717
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/WebsiteDiagnosticsSuite.scala
@@ -0,0 +1,97 @@
+/*
+ * 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.yarn.failures
+
+import java.net.{URI, URL}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.integration.AbstractHistoryIntegrationTests
+import org.apache.spark.deploy.history.yarn.rest.{HttpRequestException, JerseyBinding}
+import org.apache.spark.deploy.history.yarn.server.{TimelineQueryClient, YarnHistoryProvider}
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+/**
+ * Test reporting of connectivity problems to the caller, specifically how
+ * the `YarnHistoryProvider` handles the initial binding & reporting of problems.
+ *
+ */
+class WebsiteDiagnosticsSuite extends AbstractHistoryIntegrationTests {
+
+ var failingHistoryProvider: FailingYarnHistoryProvider = _
+
+ /**
+ * Create a failing history provider instance, with the flag set to say "the initial
+ * endpoint check" has not been executed.
+ * @param conf configuration
+ * @return the instance
+ */
+ override protected def createHistoryProvider(conf: SparkConf): YarnHistoryProvider = {
+ val yarnConf = sc.hadoopConfiguration
+
+ val client = new TimelineQueryClient(timelineRootEndpoint(),
+ yarnConf, JerseyBinding.createClientConfig())
+ failingHistoryProvider = new
+ FailingYarnHistoryProvider(client, false, client.getTimelineURI(), conf)
+ failingHistoryProvider
+ }
+
+ def timelineRootEndpoint(): URI = {
+ val realTimelineEndpoint = getTimelineEndpoint(sc.hadoopConfiguration).toURL
+ new URL(realTimelineEndpoint, "/").toURI
+ }
+
+ /**
+ * Issue a GET request against the Web UI and expect it to fail with an error
+ * message indicating that `text/html` is not a supported type.
+ * with error text indicating it was in the endpoint check
+ * @param webUI URL to the web UI
+ * @param provider the provider
+ */
+ def expectApplicationLookupToFailInEndpointCheck(webUI: URL,
+ provider: YarnHistoryProvider): Unit = {
+ val connector = createUrlConnector()
+ val appURL = new URL(webUI, "/history/app-0001")
+ describe(s"Expecting endpoint checks to fail while retrieving $appURL")
+ awaitURL(webUI, TEST_STARTUP_DELAY)
+ try {
+ assert(!failingHistoryProvider.endpointCheckSuccess())
+ val body = getHtmlPage(appURL, Nil)
+ fail(s"Expected a failure from GET $appURL -but got\n$body")
+ } catch {
+ case ex: HttpRequestException =>
+ assertContains(ex.toString, TimelineQueryClient.MESSAGE_CHECK_URL)
+ }
+ }
+
+ test("Probe UI with Endpoint check") {
+ def probeUIWithFailureCaught(webUI: URL, provider: YarnHistoryProvider): Unit = {
+ awaitURL(webUI, TEST_STARTUP_DELAY)
+ getHtmlPage(webUI, YarnHistoryProvider.TEXT_NEVER_UPDATED :: Nil)
+ }
+ webUITest("Probe UI with Endpoint check", probeUIWithFailureCaught)
+ }
+
+ test("Probe App ID with Endpoint check") {
+ def expectAppIdToFail(webUI: URL, provider: YarnHistoryProvider): Unit = {
+ expectApplicationLookupToFailInEndpointCheck(webUI, provider)
+ }
+ webUITest("Probe App ID with Endpoint check", expectAppIdToFail)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/WebsiteFailureSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/WebsiteFailureSuite.scala
new file mode 100644
index 000000000000..03af0eb1285e
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/failures/WebsiteFailureSuite.scala
@@ -0,0 +1,126 @@
+/*
+ * 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.yarn.failures
+
+import java.net.{NoRouteToHostException, URL}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.integration.AbstractHistoryIntegrationTests
+import org.apache.spark.deploy.history.yarn.rest.HttpRequestException
+import org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+class WebsiteFailureSuite extends AbstractHistoryIntegrationTests {
+
+ /**
+ * Create a failing history provider instance with the endpoint check bypassed.
+ * @param conf configuration
+ * @return the instance
+ */
+ override protected def createHistoryProvider(conf: SparkConf): YarnHistoryProvider = {
+ FailingYarnHistoryProvider.createFailingProvider(conf, true, true)
+ }
+
+ /**
+ * this is the probe for if exceptions are swallowed/handled
+ * @param webUI web UI
+ * @param provider provider
+ */
+ def expectFailuresToBeSwallowed(webUI: URL, provider: YarnHistoryProvider): Unit = {
+ val connector = createUrlConnector()
+
+ awaitRefreshExecuted(provider, true, TEST_STARTUP_DELAY)
+ awaitURL(webUI, TEST_STARTUP_DELAY)
+ awaitURLContainsText(connector, webUI,
+ FailingTimelineQueryClient.ERROR_TEXT, TEST_STARTUP_DELAY)
+
+ // get that last exception
+ provider.getLastFailure match {
+ case Some((ex, date) ) =>
+ // success
+ case None =>
+ fail("expected an exception, got no failure")
+ }
+ }
+
+ /**
+ * this is the probe for exceptions passed back to the caller.
+ * @param url URL to GET
+ */
+ def expectFailuresToPropagate(url: URL): Unit = {
+ val connector = createUrlConnector()
+ try {
+ val outcome = connector.execHttpOperation("GET", url, null, "")
+ fail(s"Expected an exception, got $outcome")
+ } catch {
+ case ex: HttpRequestException if ex.status == 500 =>
+ logInfo(s"received exception ", ex)
+ val body = ex.body
+ assert(!body.isEmpty, s"Empty body from exception $ex")
+ assertContains(body, FailingTimelineQueryClient.ERROR_TEXT)
+ }
+ }
+
+ test("WebUI swallows failures") {
+ webUITest("WebUI swallows failures", expectFailuresToBeSwallowed)
+ }
+
+ test("listAndCacheApplications failure handling") {
+ describe("Checking listAndCacheApplications behavior")
+ val provider = createHistoryProvider(sc.conf)
+
+ try {
+ assertResult(0, "initial applications.timestamp not zero") {
+ provider.getApplications.timestamp
+ }
+
+ logDebug("Asking for a listing")
+
+ val listing = provider.listAndCacheApplications(false)
+ assertResult(0, "Non-empty listing") {
+ listing.applications.size
+ }
+ assert(listing.failed, "listing did not fail")
+ assert(listing.timestamp > 0, "zero timestamp")
+ provider.getLastFailure match {
+ case Some((ex: NoRouteToHostException, time)) =>
+ assert(time.getTime > 0, s"zero time value")
+ case Some((ex, _)) =>
+ // wrong exception
+ throw ex
+ case None =>
+ fail("no failure logged")
+ }
+
+ // the inner application listing has a timestamp of zero
+ val applications = provider.getApplications
+ assert(0 === applications.timestamp, "updated applications.timestamp not zero")
+
+ val config = provider.getConfig()
+ assertMapValueContains(config,
+ YarnHistoryProvider.KEY_LAST_FAILURE,
+ FailingTimelineQueryClient.ERROR_TEXT)
+ assertMapValueContains(config,
+ YarnHistoryProvider.KEY_LAST_UPDATED,
+ YarnHistoryProvider.TEXT_NEVER_UPDATED)
+ } finally {
+ provider.stop()
+ }
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/AbstractHistoryIntegrationTests.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/AbstractHistoryIntegrationTests.scala
new file mode 100644
index 000000000000..8d697d5f742f
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/AbstractHistoryIntegrationTests.scala
@@ -0,0 +1,486 @@
+/*
+ * 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.yarn.integration
+
+import java.net.URL
+
+import scala.collection.mutable
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL.Token
+import org.apache.hadoop.service.ServiceOperations
+import org.apache.hadoop.yarn.api.records.timeline.{TimelineEntity, TimelineEvent, TimelinePutResponse}
+import org.apache.hadoop.yarn.client.api.TimelineClient
+import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer
+
+import org.apache.spark.{SecurityManager, SparkConf}
+import org.apache.spark.deploy.history.{ApplicationHistoryProvider, FsHistoryProvider, HistoryServer}
+import org.apache.spark.deploy.history.yarn.{YarnHistoryService, YarnTimelineUtils}
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.rest.JerseyBinding._
+import org.apache.spark.deploy.history.yarn.rest.SpnegoUrlConnector
+import org.apache.spark.deploy.history.yarn.server.{TimelineQueryClient, YarnHistoryProvider}
+import org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider._
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.deploy.history.yarn.testtools.{AbstractYarnHistoryTests, FreePortFinder, HistoryServiceNotListeningToSparkContext, TimelineServiceEnabled}
+import org.apache.spark.scheduler.SparkListenerEvent
+import org.apache.spark.scheduler.cluster.{SchedulerExtensionServices, StubApplicationAttemptId}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.util.Utils
+
+/**
+ * Integration tests with history services setup and torn down
+ */
+abstract class AbstractHistoryIntegrationTests
+ extends AbstractYarnHistoryTests
+ with FreePortFinder
+ with HistoryServiceNotListeningToSparkContext
+ with TimelineServiceEnabled
+ with IntegrationTestUtils {
+
+ protected var _applicationHistoryServer: ApplicationHistoryServer = _
+ protected var _timelineClient: TimelineClient = _
+ protected var historyService: YarnHistoryService = _
+ protected var sparkHistoryServer: HistoryServer = _
+
+ protected val incomplete_flag = "&showIncomplete=true"
+ protected val page1_flag = "&page=1"
+ protected val page1_incomplete_flag = "&page=1&showIncomplete=true"
+
+ protected val attemptId1 = new StubApplicationAttemptId(applicationId, 1)
+ protected val attemptId2 = new StubApplicationAttemptId(applicationId, 222)
+ protected val attemptId3 = new StubApplicationAttemptId(applicationId, 333)
+
+ protected val attempt1SparkId = "1"
+ protected val attempt2SparkId = "2"
+ protected val attempt3SparkId = "3"
+
+ protected val no_completed_applications = "No completed applications found!"
+ protected val no_incomplete_applications = "No incomplete applications found!"
+
+ // a list of actions to fail with
+ protected var failureActions: mutable.MutableList[() => Unit] = mutable.MutableList()
+
+ def applicationHistoryServer: ApplicationHistoryServer = {
+ _applicationHistoryServer
+ }
+
+ def timelineClient: TimelineClient = {
+ _timelineClient
+ }
+
+ /*
+ * Setup phase creates a local ATS server and a client of it
+ */
+
+ override def setup(): Unit = {
+ // abort the tests if the server is offline
+ cancelIfOffline()
+ super.setup()
+ startTimelineClientAndAHS(sc.hadoopConfiguration)
+ }
+
+ /**
+ * Set up base configuratin for integration tests, including
+ * classname bindings in publisher & provider, refresh intervals and a port for the UI
+ * @param sparkConf spark configuration
+ * @return the expanded configuration
+ */
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ super.setupConfiguration(sparkConf)
+ sparkConf.set(SchedulerExtensionServices.SPARK_YARN_SERVICES, YarnHistoryService.CLASSNAME)
+ sparkConf.set(SPARK_HISTORY_PROVIDER, YarnHistoryProvider.YARN_HISTORY_PROVIDER_CLASS)
+ sparkConf.set(OPTION_MANUAL_REFRESH_INTERVAL, "1ms")
+ sparkConf.set(OPTION_BACKGROUND_REFRESH_INTERVAL, "0s")
+ sparkConf.set(OPTION_YARN_LIVENESS_CHECKS, "false")
+ sparkConf.set(OPTION_WINDOW_LIMIT, "0")
+ sparkConf.setAppName(APP_NAME)
+ }
+
+ /**
+ * Stop all services, including, if set, anything in
+ * historyService
+ */
+ override def afterEach(): Unit = {
+ describe("Teardown of history server, timeline client and history service")
+ stopHistoryService(historyService)
+ historyService = null
+ ServiceOperations.stopQuietly(_applicationHistoryServer)
+ _applicationHistoryServer = null
+ ServiceOperations.stopQuietly(_timelineClient)
+ _timelineClient = null
+ super.afterEach()
+ }
+
+ /**
+ * Stop a history service. This includes flushing its queue,
+ * blocking until that queue has been flushed and closed, then
+ * stopping the YARN service.
+ * @param hservice history service to stop
+ */
+ def stopHistoryService(hservice: YarnHistoryService): Unit = {
+ if (hservice != null && hservice.serviceState == YarnHistoryService.StartedState) {
+ flushHistoryServiceToSuccess()
+ hservice.stop()
+ awaitServiceThreadStopped(hservice, TEST_STARTUP_DELAY, false)
+ }
+ }
+
+ /**
+ * Add an action to execute on failures (if the test runs it
+ * @param action action to execute
+ */
+ def addFailureAction(action: () => Unit) : Unit = {
+ failureActions += action
+ }
+
+ /**
+ * Execute all the failure actions in order.
+ */
+ def executeFailureActions(): Unit = {
+ if (failureActions.nonEmpty) {
+ logError("== Executing failure actions ==")
+ }
+ failureActions.foreach{ action =>
+ try {
+ action()
+ } catch {
+ case _ : Exception =>
+ }
+ }
+ }
+
+ /**
+ * Failure action to log history service details at INFO
+ */
+ def dumpYarnHistoryService(): Unit = {
+ if (historyService != null) {
+ logError(s"-- History Service --\n$historyService")
+ }
+ }
+
+ /**
+ * Curryable Failure action to dump provider state
+ * @param provider the provider
+ */
+ def dumpProviderState(provider: YarnHistoryProvider)(): Unit = {
+ logError(s"-- Provider --\n$provider")
+ val results = provider.getApplications
+ results.applications.foreach{
+ app =>
+ logError(s" $app")
+ }
+ results.failureCause.foreach{ e =>
+ logError("Failed", e)
+ }
+ }
+
+ /**
+ * Curryable Failure action to log all timeline entities
+ * @param provider the provider bonded to the endpoint
+ */
+ def dumpTimelineEntities(provider: YarnHistoryProvider)(): Unit = {
+ logError("-- Dumping timeline entities --")
+ val entities = provider.getTimelineQueryClient
+ .listEntities(YarnHistoryService.SPARK_EVENT_ENTITY_TYPE)
+ entities.foreach{ e =>
+ logError(describeEntity(e))
+ }
+ }
+
+ /**
+ * Create a SPNEGO-enabled URL Connector.
+ * Picks up the hadoop configuration from `sc`, so the context
+ * must be live/non-null
+ * @return a URL connector for issuing HTTP requests
+ */
+ protected def createUrlConnector(): SpnegoUrlConnector = {
+ val hadoopConfiguration = sc.hadoopConfiguration
+ createUrlConnector(hadoopConfiguration)
+ }
+
+ /**
+ * Create a SPNEGO-enabled URL Connector.
+ * @param hadoopConfiguration the configuration to use
+ * @return a URL connector for issuing HTTP requests
+ */
+ def createUrlConnector(hadoopConfiguration: Configuration): SpnegoUrlConnector = {
+ SpnegoUrlConnector.newInstance(hadoopConfiguration, new Token)
+ }
+
+ /**
+ * Create the client and the app server
+ * @param conf the hadoop configuration
+ */
+ protected def startTimelineClientAndAHS(conf: Configuration): Unit = {
+ ServiceOperations.stopQuietly(_applicationHistoryServer)
+ ServiceOperations.stopQuietly(_timelineClient)
+ _timelineClient = TimelineClient.createTimelineClient()
+ _timelineClient.init(conf)
+ _timelineClient.start()
+ _applicationHistoryServer = new ApplicationHistoryServer()
+ _applicationHistoryServer.init(_timelineClient.getConfig)
+ _applicationHistoryServer.start()
+ // Wait for AHS to come up
+ val endpoint = YarnTimelineUtils.timelineWebappUri(conf, "")
+ awaitURL(endpoint.toURL, TEST_STARTUP_DELAY)
+ }
+
+ protected def createTimelineQueryClient(): TimelineQueryClient = {
+ new TimelineQueryClient(historyService.timelineWebappAddress,
+ sc.hadoopConfiguration, createClientConfig())
+ }
+
+ /**
+ * Put a timeline entity to the timeline client; this is expected
+ * to eventually make it to the history server
+ * @param entity entity to put
+ * @return the response
+ */
+ def putTimelineEntity(entity: TimelineEntity): TimelinePutResponse = {
+ assertNotNull(_timelineClient, "timelineClient")
+ _timelineClient.putEntities(entity)
+ }
+
+ /**
+ * Marshall and post a spark event to the timeline; return the outcome
+ * @param sparkEvt event
+ * @param time event time
+ * @return a triple of the wrapped event, marshalled entity and the response
+ */
+ protected def postEvent(sparkEvt: SparkListenerEvent, time: Long):
+ (TimelineEvent, TimelineEntity, TimelinePutResponse) = {
+ val event = toTimelineEvent(sparkEvt, time).get
+ val entity = newEntity(time)
+ entity.addEvent(event)
+ val response = putTimelineEntity(entity)
+ val description = describePutResponse(response)
+ logInfo(s"response: $description")
+ assert(response.getErrors.isEmpty, s"errors in response: $description")
+ (event, entity, response)
+ }
+
+ /**
+ * flush the history service of its queue, await it to complete,
+ * then assert that there were no failures
+ */
+ protected def flushHistoryServiceToSuccess(): Unit = {
+ flushHistoryServiceToSuccess(historyService)
+ }
+
+ /**
+ * Flush a history service to success
+ * @param history service to flush
+ * @param delay time to wait for an empty queue
+ */
+ def flushHistoryServiceToSuccess(
+ history: YarnHistoryService,
+ delay: Int = TEST_STARTUP_DELAY): Unit = {
+ assertNotNull(history, "null history queue")
+ historyService.asyncFlush()
+ awaitEmptyQueue(history, delay)
+ assert(0 === history.postFailures, s"Post failure count: $history")
+ }
+
+ /**
+ * Create a history provider instance, following the same process
+ * as the history web UI itself: querying the configuration for the
+ * provider and falling back to the [[FsHistoryProvider]]. If
+ * that falback does take place, however, and assertion is raised.
+ * @param conf configuration
+ * @return the instance
+ */
+ protected def createHistoryProvider(conf: SparkConf): YarnHistoryProvider = {
+ val providerName = conf.getOption("spark.history.provider")
+ .getOrElse(classOf[FsHistoryProvider].getName())
+ val provider = Utils.classForName(providerName)
+ .getConstructor(classOf[SparkConf])
+ .newInstance(conf)
+ .asInstanceOf[ApplicationHistoryProvider]
+ assert(provider.isInstanceOf[YarnHistoryProvider],
+ s"Instantiated $providerName to get $provider")
+
+ provider.asInstanceOf[YarnHistoryProvider]
+ }
+
+ /**
+ * Crete a history server and maching provider, execute the
+ * probe against it. After the probe completes, the history server
+ * is stopped.
+ * @param probe probe to run
+ */
+ def webUITest(name: String, probe: (URL, YarnHistoryProvider) => Unit): Unit = {
+ val (_, server, webUI, provider) = createHistoryServer(findPort())
+ try {
+ sparkHistoryServer = server
+ server.bind()
+ describe(name)
+ probe(webUI, provider)
+ } catch {
+ case ex: Exception =>
+ executeFailureActions()
+ throw ex
+ } finally {
+ describe("stopping history service")
+ Utils.tryLogNonFatalError {
+ server.stop()
+ }
+ sparkHistoryServer = null
+ }
+ }
+
+ /**
+ * Probe the empty web UI for not having any completed apps; expect
+ * a text/html response with specific text and history provider configuration
+ * elements.
+ * @param webUI web UI
+ * @param provider provider
+ */
+ def probeEmptyWebUI(webUI: URL, provider: YarnHistoryProvider): String = {
+ val body: String = getHtmlPage(webUI,
+ "History Server"
+ :: no_completed_applications
+ :: YarnHistoryProvider.KEY_PROVIDER_NAME
+ :: YarnHistoryProvider.PROVIDER_DESCRIPTION
+ :: Nil)
+ logInfo(s"$body")
+ body
+ }
+
+ /**
+ * Get an HTML page. Includes a check that the content type is `text/html`
+ * @param page web UI
+ * @param checks list of strings to assert existing in the response
+ * @return the body of the response
+ */
+ protected def getHtmlPage(page: URL, checks: List[String]): String = {
+ val outcome = createUrlConnector().execHttpOperation("GET", page, null, "")
+ logDebug(s"$page => $outcome")
+ assert(outcome.contentType.startsWith("text/html"), s"content type of $outcome")
+ val body = outcome.responseBody
+ assertStringsInBody(body, checks)
+ body
+ }
+
+ /**
+ * Assert that a list of checks are in the HTML body
+ * @param body body of HTML (or other string)
+ * @param checks list of strings to assert are present
+ */
+ def assertStringsInBody(body: String, checks: List[String]): Unit = {
+ var missing: List[String] = Nil
+ var text = "[ "
+ checks foreach { check =>
+ if (!body.contains(check)) {
+ missing = check :: missing
+ text = text +"\"" + check +"\" "
+ }
+ }
+ text = text + "]"
+ if (missing.nonEmpty) {
+ fail(s"Did not find $text in\n$body")
+ }
+ }
+
+ /**
+ * Create a [[HistoryServer]] instance with a coupled history provider.
+ * @param defaultPort a port to use if the property `spark.history.ui.port` isn't
+ * set in the spark context. (default: 18080)
+ * @return (port, server, web UI URL, history provider)
+ */
+ protected def createHistoryServer(defaultPort: Int = 18080):
+ (Int, HistoryServer, URL, YarnHistoryProvider) = {
+ val conf = sc.conf
+ val securityManager = new SecurityManager(conf)
+ val args: List[String] = Nil
+ val port = conf.getInt(SPARK_HISTORY_UI_PORT, defaultPort)
+ val provider = createHistoryProvider(sc.getConf)
+ val server = new HistoryServer(conf, provider, securityManager, port)
+ val webUI = new URL("http", "localhost", port, "/")
+ (port, server, webUI, provider)
+ }
+
+ /**
+ * closing context generates an application stop
+ */
+ def stopContextAndFlushHistoryService(): Unit = {
+ describe("stopping context")
+ resetSparkContext()
+ flushHistoryServiceToSuccess()
+ }
+
+ /**
+ * Create and queue a new [HandleSparkEvent] from the data
+ * @param sparkEvent spark event
+ */
+ def enqueue(sparkEvent: SparkListenerEvent): Unit = {
+ eventTime(sparkEvent).getOrElse {
+ throw new RuntimeException(s"No time from $sparkEvent")
+ }
+ assert(historyService.enqueue(sparkEvent))
+ }
+
+ /**
+ * Post up multiple attempts with the second one a success
+ */
+ def postMultipleAttempts(): Unit = {
+ logDebug("posting app start")
+ val startTime = 10000
+ historyService = startHistoryService(sc, applicationId, Some(attemptId1))
+ val start1 = appStartEvent(startTime,
+ sc.applicationId,
+ Utils.getCurrentUserName(),
+ Some(attempt1SparkId))
+ enqueue(start1)
+ enqueue(jobStartEvent(10001, 1))
+ enqueue(jobFailureEvent(10002, 1, new scala.RuntimeException("failed")))
+ enqueue(appStopEvent(10003))
+ flushHistoryServiceToSuccess()
+ stopHistoryService(historyService)
+
+ // second attempt
+ val start2Time = 20000
+ historyService = startHistoryService(sc, applicationId, Some(attemptId2))
+ val start2 = appStartEvent(start2Time,
+ sc.applicationId,
+ Utils.getCurrentUserName(),
+ Some(attempt2SparkId))
+ enqueue(start2)
+
+ enqueue(jobStartEvent(20000, 1))
+ enqueue(jobSuccessEvent(20002, 1))
+ enqueue(appStopEvent(20003))
+
+ awaitEmptyQueue(historyService, TEST_STARTUP_DELAY)
+ }
+
+ /**
+ * Get the provider UI with an assertion failure if none came back
+ * @param provider provider
+ * @param appId app ID
+ * @param attemptId optional attempt ID
+ * @return the provider UI retrieved
+ */
+ def getAppUI(provider: YarnHistoryProvider,
+ appId: String,
+ attemptId: Option[String]): SparkUI = {
+ val ui = provider.getAppUI(appId, attemptId)
+ assertSome(ui, s"Failed to retrieve App UI under ID $appId attempt $attemptId from $provider")
+ ui.get
+ }
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/AsyncRefreshSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/AsyncRefreshSuite.scala
new file mode 100644
index 000000000000..3cc1e7cd3d49
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/AsyncRefreshSuite.scala
@@ -0,0 +1,164 @@
+/*
+ * 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.yarn.integration
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.server.{YarnHistoryProviderMetrics, YarnHistoryProvider}
+import org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider._
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+/**
+ * Tests to verify async refreshes work as expected.
+ * These tests disable background updates
+ */
+class AsyncRefreshSuite extends AbstractHistoryIntegrationTests {
+
+ val SLEEP_INTERVAL = 100
+ val EVENT_PROCESSED_TIMEOUT = 5000
+ val WINDOW_SECONDS = YarnHistoryProvider.DEFAULT_MANUAL_REFRESH_INTERVAL_SECONDS
+ val WINDOW_MILLIS = WINDOW_SECONDS * 1000
+
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ super.setupConfiguration(sparkConf)
+ sparkConf.set(OPTION_MANUAL_REFRESH_INTERVAL, s"${WINDOW_SECONDS}s")
+ sparkConf.set(OPTION_BACKGROUND_REFRESH_INTERVAL, "0")
+ }
+
+ /**
+ * Start time. This has to be big enough that `now()-lastRefreshTime > window`,
+ * even for the initial last refresh time value of 0
+ */
+ val CLOCK_START_TIME = 10 * 60 * 1000
+ val TICK_TIME = 1000
+
+ /**
+ * Create time managed history provider whose clock can be set
+ * by the tests themselves
+ * @param conf configuration
+ * @return the instance
+ */
+ override protected def createHistoryProvider(conf: SparkConf): YarnHistoryProvider = {
+ new TimeManagedHistoryProvider(conf, CLOCK_START_TIME, TICK_TIME)
+ }
+
+ def metrics()(implicit history: YarnHistoryProvider): YarnHistoryProviderMetrics = {
+ history.metrics
+ }
+
+ /**
+ * This test verifies that `YarnHistoryProvider.getListing()` events
+ * trigger asynchronous refreshes, except when the last refresh
+ * was within the limits of the refresh window.
+ *
+ * To do this it makes use of the counter in the refresher recording the number
+ * of events received in its action queue; so verifying when the (async) refresh
+ * actions are received and processed.
+ */
+ test("RefreshInterval") {
+ describe("verify reset interval logic")
+ implicit val history = createHistoryProvider(sc.conf)
+ .asInstanceOf[TimeManagedHistoryProvider]
+ // checks that the time reading/parsing logic works
+ assert(WINDOW_MILLIS === history.manualRefreshInterval, s"refresh interval in $history")
+ val refresher = history.refresher
+ assert(refresher.isRunning, s"refresher not running in $history")
+ awaitRefreshMessageProcessed(history, 0, EVENT_PROCESSED_TIMEOUT, s"expecting startup refresh")
+
+ val attempt0 = history.lastRefreshAttemptTime
+ assert(CLOCK_START_TIME === attempt0, s"initial event in $refresher")
+ // the refresh is expected to succeed, but a small delay is needed for the async nature of the
+ // GET operation
+ Thread.sleep(5000)
+ assert(0 === history.refreshFailedCount, s"Failed refreshes in $history")
+ assert(1 === metrics.refreshCount.getCount,
+ s"Refreshes successfully executed in $refresher")
+
+ def messagesProcessed: Long = { metrics.backgroundOperationsProcessed.getCount }
+ val count0 = messagesProcessed
+ // add thirty seconds
+ history.incrementTime(15000)
+ history.getListing()
+ // yield a bit for the refresher thread
+ awaitRefreshMessageProcessed(history, count0, EVENT_PROCESSED_TIMEOUT, "in-window")
+
+ // expect no refresh
+ assert(history.lastRefreshAttemptTime === CLOCK_START_TIME,
+ s"last refresh was not clock time of $CLOCK_START_TIME in $refresher")
+
+ // now move clock forwards a minute
+ history.incrementTime(2 * WINDOW_SECONDS * 1000)
+ // and expect the new listing to trigger another refresh
+ val count2 = messagesProcessed
+ val time2 = history.now()
+ history.getListing()
+ awaitRefreshMessageProcessed(history, count2, EVENT_PROCESSED_TIMEOUT, "out of window")
+ val refreshAttemptTime2 = history.lastRefreshAttemptTime
+ val time3 = history.now()
+ assert(2 === history.metrics.refreshCount.getCount, s"refreshes executed in $refresher")
+ assert(time2 <= refreshAttemptTime2,
+ s"refresh time in $refresher was $refreshAttemptTime2; time2 was $time2; current =$time3")
+ }
+
+ test("Size Zero refresh window") {
+ describe("Test with a zero size window to verify that the refresh always takes ")
+ val conf = sc.conf
+ conf.set(OPTION_MANUAL_REFRESH_INTERVAL, "0")
+ implicit val history = createHistoryProvider(conf).asInstanceOf[TimeManagedHistoryProvider]
+ val refresher = history.refresher
+ assert(refresher.isRunning, s"refresher not running in $history")
+ awaitRefreshMessageProcessed(history, 0, EVENT_PROCESSED_TIMEOUT, s"startup of $refresher")
+
+ val attempt0 = history.lastRefreshAttemptTime
+ assert(attempt0 === CLOCK_START_TIME, s"initial event in $refresher")
+ val count0 = refresher.messagesProcessed
+ val t2 = history.tick()
+ history.getListing()
+ // yield a bit for the refresher thread
+ awaitRefreshMessageProcessed(history, count0, EVENT_PROCESSED_TIMEOUT, "in-window")
+
+ // expect no refresh
+ assert(history.lastRefreshAttemptTime === t2, s"refresh didn't happen in $refresher")
+ }
+
+ /**
+ * Wait for the refresh count to increment by at least one iteration
+ * @param provider provider
+ * @param timeout timeout
+ * @return the successful listing
+ */
+ def awaitRefreshMessageProcessed (
+ provider: TimeManagedHistoryProvider,
+ initialCount: Long,
+ timeout: Long,
+ text: String): Unit = {
+ val refresher = provider.refresher
+
+ def listingProbe(): Outcome = {
+ outcomeFromBool(refresher.messagesProcessed > initialCount)
+ }
+
+ def failure(outcome: Outcome, attempts: Int, timedOut: Boolean): Unit = {
+ val outcome = if (timedOut) "timeout" else "failed"
+ fail(s"$text -$outcome after $attempts attempts," +
+ s" refresh count is <= $initialCount in: $refresher")
+ }
+ require(provider.isRefreshThreadRunning, s"refresher is not running in $provider")
+ spinForState("await refresh message count", SLEEP_INTERVAL, timeout, listingProbe, failure)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/ContextToHistoryProviderSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/ContextToHistoryProviderSuite.scala
new file mode 100644
index 000000000000..40b4ccacf899
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/ContextToHistoryProviderSuite.scala
@@ -0,0 +1,96 @@
+/*
+ * 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.yarn.integration
+
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.rest.JerseyBinding._
+import org.apache.spark.deploy.history.yarn.server.{TimelineQueryClient, YarnHistoryProvider}
+import org.apache.spark.deploy.history.yarn.testtools.{HistoryServiceListeningToSparkContext, TimelineSingleEntryBatchSize}
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.util.Utils
+
+/**
+ * full hookup from spark context to timeline then reread.
+ */
+class ContextToHistoryProviderSuite
+ extends AbstractHistoryIntegrationTests
+ with HistoryServiceListeningToSparkContext
+ with TimelineSingleEntryBatchSize {
+
+ test("Stop Event via Context") {
+ describe("Stop Event via Context")
+ var provider: YarnHistoryProvider = null
+ try {
+ // hook up to spark context
+ historyService = startHistoryService(sc)
+ assert(historyService.listening, s"listening $historyService")
+ assertResult(1, s"batch size in $historyService") {
+ historyService.batchSize
+ }
+ assert(historyService.bondedToATS, s"not bonded to ATS: $historyService")
+ // post in an app start
+ var flushes = 0
+ logDebug("posting app start")
+ val startTime = now()
+ val event = appStartEvent(startTime, sc.applicationId, Utils.getCurrentUserName())
+ enqueue(event)
+ flushes += 1
+ awaitEmptyQueue(historyService, TEST_STARTUP_DELAY)
+ // closing context generates an application stop
+ describe("stopping context")
+ sc.stop()
+ flushHistoryServiceToSuccess()
+
+ val timeline = historyService.timelineWebappAddress
+ val queryClient = new TimelineQueryClient(timeline,
+ sc.hadoopConfiguration, createClientConfig())
+ val entities = queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE)
+ logInfo(s"Entity listing returned ${entities.size} entities")
+ entities.foreach { en =>
+ logInfo(describeEntityVerbose(en))
+ }
+ assertResult(1, "number of listed entities (unfiltered)") {
+ entities.size
+ }
+ assertResult(1, "entities listed by app end filter") {
+ queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ primaryFilter = Some((FILTER_APP_END, FILTER_APP_END_VALUE))).size
+ }
+
+ assertResult(1, "entities listed by app start filter") {
+ queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ primaryFilter = Some((FILTER_APP_START, FILTER_APP_START_VALUE))).size
+ }
+
+ // now read it in via history provider
+ provider = new YarnHistoryProvider(sc.conf)
+ val history = awaitApplicationListingSize(provider, 1, TEST_STARTUP_DELAY)
+ val info = history.head
+ val attempt = info.attempts.head
+ assert(attempt.completed, s"application not flagged as completed")
+ provider.getAppUI(info.id, attempt.attemptId)
+ } finally {
+ describe("teardown")
+ if (provider != null) {
+ provider.stop()
+ }
+ }
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/HistoryPlaybackSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/HistoryPlaybackSuite.scala
new file mode 100644
index 000000000000..f937bd4e6847
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/HistoryPlaybackSuite.scala
@@ -0,0 +1,113 @@
+/*
+ * 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.yarn.integration
+
+import java.net.URL
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.YarnHistoryService
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.failures.FailingYarnHistoryProvider
+import org.apache.spark.deploy.history.yarn.rest.JerseyBinding
+import org.apache.spark.deploy.history.yarn.server.{TimelineQueryClient, YarnHistoryProvider}
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+/**
+ * Plays back in a real history from a resource
+ */
+class HistoryPlaybackSuite extends AbstractHistoryIntegrationTests {
+
+ /** path to the resource with the history */
+ val History1 = "org/apache/spark/deploy/history/yarn/integration/history-1.json"
+ val EntityCount = 2
+ var historyProvider: FailingYarnHistoryProvider = _
+
+ /**
+ * Create a history provider bonded to the resource entity list
+ * @param conf configuration
+ * @return the instance
+ */
+ override protected def createHistoryProvider(conf: SparkConf): YarnHistoryProvider = {
+ val client = createTimelineQueryClient()
+ historyProvider = new FailingYarnHistoryProvider(client, true, client.getTimelineURI(),
+ conf, true)
+ historyProvider
+ }
+
+ protected override def createTimelineQueryClient(): TimelineQueryClient = {
+ new ResourceDrivenTimelineQueryClient(
+ History1,
+ new URL(getTimelineEndpoint(sc.hadoopConfiguration).toURL, "/").toURI,
+ sc.hadoopConfiguration,
+ JerseyBinding.createClientConfig())
+ }
+
+ test("Publish Events and GET the web UI") {
+ def examineHistory(webUI: URL, provider: YarnHistoryProvider): Unit = {
+
+ val connector = createUrlConnector()
+ val queryClient = historyProvider.getTimelineQueryClient
+
+ val timelineEntities =
+ queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE)
+ assert(EntityCount === timelineEntities.size,
+ s"entities listed count = ${timelineEntities.size}")
+ val yarnAppId = "application_1443668830514_0008"
+
+ assertNotNull(yarnAppId, s"Null entityId from $yarnAppId")
+ val entity = queryClient.getEntity(YarnHistoryService.SPARK_EVENT_ENTITY_TYPE, yarnAppId)
+ assertNotNull(entity, s"Null entity from $yarnAppId")
+
+ // listing
+ awaitApplicationListingSize(provider, EntityCount, TEST_STARTUP_DELAY)
+
+ // resolve to entry
+ provider.getAppUI(yarnAppId, Some(yarnAppId)) match {
+ case Some(yarnAppUI) =>
+ // success
+ case None => fail(s"Did not get a UI for $yarnAppId")
+ }
+
+ // and look for the complete app
+ awaitURL(webUI, TEST_STARTUP_DELAY)
+ val completeBody = awaitURLDoesNotContainText(connector, webUI,
+ no_completed_applications, TEST_STARTUP_DELAY)
+ logInfo(s"GET /\n$completeBody")
+ // look for the link
+ assertContains(completeBody,
+ "")
+
+ val appPath = s"/history/$yarnAppId/$yarnAppId"
+ // GET the app
+ val appURL = new URL(webUI, appPath)
+ val appUI = connector.execHttpOperation("GET", appURL, null, "")
+ val appUIBody = appUI.responseBody
+ logInfo(s"Application\n$appUIBody")
+ assertContains(appUIBody, "SparkSQL::10.0.0.143")
+ connector.execHttpOperation("GET", new URL(appURL, s"$appPath/jobs"), null, "")
+ connector.execHttpOperation("GET", new URL(appURL, s"$appPath/stages"), null, "")
+ connector.execHttpOperation("GET", new URL(appURL, s"$appPath/storage"), null, "")
+ connector.execHttpOperation("GET", new URL(appURL, s"$appPath/environment"), null, "")
+ connector.execHttpOperation("GET", new URL(appURL, s"$appPath/executors"), null, "")
+ }
+
+ webUITest("submit and check", examineHistory)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/IncompleteApplicationsSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/IncompleteApplicationsSuite.scala
new file mode 100644
index 000000000000..062514b55352
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/IncompleteApplicationsSuite.scala
@@ -0,0 +1,142 @@
+/*
+ * 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.yarn.integration
+
+import java.net.URL
+
+import org.apache.spark.deploy.history.yarn.{YarnEventListener, YarnHistoryService}
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.util.Utils
+
+/**
+ * Test handling/logging of incomplete applications.
+ *
+ * This implicitly tests some of the windowing logic. Specifically, do completed
+ * applications get picked up?
+ */
+class IncompleteApplicationsSuite extends AbstractHistoryIntegrationTests {
+
+ val EVENT_PROCESSED_TIMEOUT = 2000
+
+ test("WebUI incomplete view") {
+ def checkEmptyIncomplete(webUI: URL, provider: YarnHistoryProvider): Unit = {
+ val connector = createUrlConnector()
+ val url = new URL(webUI, "/?" + page1_incomplete_flag)
+ val incompleted = connector.execHttpOperation("GET", url, null, "")
+ val body = incompleted.responseBody
+ logInfo(s"$url => $body")
+ assertContains(body, no_incomplete_applications, s"In $url")
+ }
+
+ webUITest("incomplete view", checkEmptyIncomplete)
+ }
+
+ test("Get the web UI of an incomplete application") {
+ def submitAndCheck(webUI: URL, provider: YarnHistoryProvider): Unit = {
+ val connector = createUrlConnector()
+ val incompleteURL = new URL(webUI, "/?" + incomplete_flag)
+ awaitURL(incompleteURL, TEST_STARTUP_DELAY)
+
+ def listIncompleteApps(): String = {
+ connector.execHttpOperation("GET", incompleteURL, null, "").responseBody
+ }
+ historyService = startHistoryService(sc, applicationId, Some(attemptId1))
+ val timeline = historyService.timelineWebappAddress
+ val listener = new YarnEventListener(sc, historyService)
+ // initial view has no incomplete applications
+ assertContains(listIncompleteApps(), no_incomplete_applications,
+ "initial incomplete page is empty")
+
+ val startTime = now()
+ val expectedAppId = historyService.applicationId.toString
+ val attemptId = attemptId1.toString
+ val sparkAttemptId = "1"
+ val started = appStartEvent(startTime,
+ expectedAppId,
+ Utils.getCurrentUserName(),
+ Some(sparkAttemptId))
+ listener.onApplicationStart(started)
+ val jobId = 2
+ listener.onJobStart(jobStartEvent(startTime + 1, jobId))
+ awaitEventsProcessed(historyService, 2, EVENT_PROCESSED_TIMEOUT)
+ flushHistoryServiceToSuccess()
+
+ // await for a refresh
+
+ // listing
+ awaitApplicationListingSize(provider, 1, EVENT_PROCESSED_TIMEOUT)
+
+ val queryClient = createTimelineQueryClient()
+
+ // check for work in progress
+ assertDoesNotContain(listIncompleteApps(), no_incomplete_applications,
+ "'incomplete application' list empty")
+
+ logInfo("Ending job and application")
+ // job completion event
+ listener.onJobEnd(jobSuccessEvent(startTime + 1, jobId))
+ // stop the app
+ historyService.stop()
+ awaitEmptyQueue(historyService, EVENT_PROCESSED_TIMEOUT)
+
+ // validate ATS has it
+ val timelineEntities = queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ primaryFilter = Some((FILTER_APP_END, FILTER_APP_END_VALUE)))
+ assert(1 === timelineEntities.size, "entities listed by app end filter")
+ assert(attemptId === timelineEntities.head.getEntityId,
+ "attemptId === timelineEntities.head.getEntityId")
+
+ queryClient.getEntity(YarnHistoryService.SPARK_EVENT_ENTITY_TYPE, attemptId)
+
+ // at this point the REST UI is happy. Check the provider level
+
+ // listing
+ awaitApplicationListingSize(provider, 1, EVENT_PROCESSED_TIMEOUT)
+
+ // and look for the complete app
+ awaitURL(webUI, EVENT_PROCESSED_TIMEOUT)
+ val completeBody = awaitURLDoesNotContainText(connector, webUI,
+ no_completed_applications, EVENT_PROCESSED_TIMEOUT,
+ s"Awaiting completed applications in the web UI listing $webUI")
+ logDebug(completeBody)
+ // look for the link
+ assertContains(completeBody, s"$expectedAppId",
+ "expecting app listed in completed page")
+ assertContains(completeBody, s"$expectedAppId/$sparkAttemptId",
+ "expecting app attempt URL listed in completed page")
+
+ val appPath = s"/history/$expectedAppId/$sparkAttemptId"
+ // GET the app
+ val appURL = new URL(webUI, appPath)
+ val appUI = connector.execHttpOperation("GET", appURL, null, "")
+ val appUIBody = appUI.responseBody
+ logInfo(s"Application\n$appUIBody")
+ assertContains(appUIBody, APP_NAME, "application name in app body")
+ // look for the completed job
+ assertContains(appUIBody, completedJobsMarker, "expected to list completed jobs")
+
+ // final view has no incomplete applications
+ assertContains(listIncompleteApps(), no_incomplete_applications)
+ }
+
+ webUITest("submit and check", submitAndCheck)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/IncompleteSparkUISuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/IncompleteSparkUISuite.scala
new file mode 100644
index 000000000000..fe478ee9743d
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/IncompleteSparkUISuite.scala
@@ -0,0 +1,128 @@
+/*
+ * 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.yarn.integration
+
+import java.net.URL
+
+import org.apache.spark.deploy.history.yarn.YarnEventListener
+import org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider
+import org.apache.spark.deploy.history.yarn.server.YarnProviderUtils._
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.util.Utils
+
+/**
+ * test to see that incomplete spark UIs are handled
+ */
+class IncompleteSparkUISuite extends AbstractHistoryIntegrationTests {
+
+ test("incomplete UI must not be cached") {
+ def submitAndCheck(webUI: URL, provider: YarnHistoryProvider): Unit = {
+ val connector = createUrlConnector()
+ val incompleteAppsURL = new URL(webUI, "/?" + incomplete_flag)
+ def listIncompleteApps: String = {
+ connector.execHttpOperation("GET", incompleteAppsURL, null, "").responseBody
+ }
+ historyService = startHistoryService(sc)
+ val listener = new YarnEventListener(sc, historyService)
+ // initial view has no incomplete applications
+ assertContains(listIncompleteApps, no_incomplete_applications)
+
+ val startTime = now()
+
+ val started = appStartEvent(startTime,
+ sc.applicationId,
+ Utils.getCurrentUserName(),
+ None)
+ listener.onApplicationStart(started)
+ val jobId = 2
+ listener.onJobStart(jobStartEvent(startTime + 1 , jobId))
+ awaitEventsProcessed(historyService, 2, 2000)
+ flushHistoryServiceToSuccess()
+
+ // listing
+ awaitApplicationListingSize(provider, 1, TEST_STARTUP_DELAY)
+
+ // here we can do a GET of the application history and expect to see something incomplete
+
+ // check for work in progress
+ awaitURLDoesNotContainText(connector, incompleteAppsURL,
+ no_completed_applications, TEST_STARTUP_DELAY, "expecting incomplete app listed")
+
+ val yarnAttemptId = attemptId.toString
+ val webAttemptId = "1"
+ val webAppId = historyService.applicationId.toString
+
+ val attemptPath = s"/history/$webAppId/$webAttemptId"
+ // GET the app
+ val attemptURL = new URL(webUI, attemptPath)
+ val appUIBody = connector.execHttpOperation("GET", attemptURL).responseBody
+ assertContains(appUIBody, APP_NAME, s"application name in $attemptURL")
+ // look for active jobs marker
+ assertContains(appUIBody, activeJobsMarker, s"active jobs string in $attemptURL")
+
+ logInfo("Ending job and application")
+ // job completion event
+ listener.onJobEnd(jobSuccessEvent(startTime + 1, jobId))
+ // stop the app
+ historyService.stop()
+ awaitEmptyQueue(historyService, TEST_STARTUP_DELAY)
+
+ flushHistoryServiceToSuccess()
+
+ // spin for a refresh event
+ awaitRefreshExecuted(provider, true, TEST_STARTUP_DELAY)
+
+ // root web UI declares it complete
+ awaitURLDoesNotContainText(connector, webUI,
+ no_completed_applications, TEST_STARTUP_DELAY,
+ s"expecting application listed as completed in $historyService")
+
+ // final view has no incomplete applications
+ assertContains(listIncompleteApps, no_incomplete_applications,
+ s"incomplete applications still in list view in $historyService")
+
+ // the underlying timeline entity
+ val entity = provider.getTimelineEntity(yarnAttemptId)
+
+ val history = awaitApplicationListingSize(provider, 1, TEST_STARTUP_DELAY).head
+
+ val historyDescription = describeApplicationHistoryInfo(history)
+ assert(1 === history.attempts.size, "wrong number of app attempts ")
+ val attempt1 = history.attempts.head
+ assert(attempt1.completed,
+ s"application attempt considered incomplete: $historyDescription")
+
+ // get the final app UI
+ val finalAppUIPage = connector.execHttpOperation("GET", attemptURL, null, "").responseBody
+ assertContains(finalAppUIPage, APP_NAME, s"Application name $APP_NAME not found" +
+ s" at $attemptURL")
+
+/* DISABLED. SPARK-7889
+ // the active jobs section must no longer exist
+ assertDoesNotContain(finalAppUIPage, activeJobsMarker,
+ s"Web UI $attemptURL still declared active")
+
+ // look for the completed job
+ assertContains(finalAppUIPage, completedJobsMarker,
+ s"Web UI $attemptURL does not declare completed jobs")
+*/
+ }
+ webUITest("submit and check", submitAndCheck)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/IntegrationTestUtils.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/IntegrationTestUtils.scala
new file mode 100644
index 000000000000..020b6eae8077
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/IntegrationTestUtils.scala
@@ -0,0 +1,311 @@
+/*
+ * 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.yarn.integration
+
+import java.io.IOException
+import java.net.URL
+
+import org.apache.hadoop.yarn.api.records.{ApplicationId, ApplicationReport, YarnApplicationState}
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl
+import org.scalatest.exceptions.TestFailedException
+
+import org.apache.spark.deploy.history.yarn.rest.SpnegoUrlConnector
+import org.apache.spark.deploy.history.yarn.server.{TimelineApplicationHistoryInfo, TimelineApplicationAttemptInfo, YarnHistoryProvider}
+import org.apache.spark.deploy.history.yarn.server.YarnProviderUtils._
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.scheduler.cluster.{StubApplicationAttemptId, StubApplicationId}
+
+/**
+ * Utils for the integration test, including provider-related code
+ */
+private[yarn] trait IntegrationTestUtils {
+
+ def appHistoryInfo(
+ id: String,
+ attempts: List[TimelineApplicationAttemptInfo]): TimelineApplicationHistoryInfo = {
+ new TimelineApplicationHistoryInfo(id, id, attempts)
+ }
+
+ def appHistoryInfo(
+ id: String,
+ attempt: TimelineApplicationAttemptInfo): TimelineApplicationHistoryInfo = {
+ new TimelineApplicationHistoryInfo(id, id, attempt :: Nil)
+ }
+
+ def attempt(
+ id: String,
+ startTime: Long,
+ endTime: Long,
+ lastUpdated: Long,
+ completed: Boolean = false): TimelineApplicationAttemptInfo = {
+ new TimelineApplicationAttemptInfo(Some(id), startTime, endTime, lastUpdated, "user",
+ completed, id, Some(id))
+ }
+
+ /**
+ * Create a stub application report
+ * @param id integer app Id (essentially the RM counter)
+ * @param clusterTimestamp timestamp for application Id
+ * @param attempt attempt ID
+ * @param state app state
+ * @param startTime start time
+ * @param finishTime finish time or 0
+ * @return the report
+ */
+ def stubApplicationReport(
+ id: Int,
+ clusterTimestamp: Long,
+ attempt: Int,
+ state: YarnApplicationState,
+ startTime: Long,
+ finishTime: Long = 0): ApplicationReport = {
+ val yarnId = new StubApplicationId(id, clusterTimestamp)
+ // this is tagged as hadoop private. The alternate tactic: create your own implementation,
+ // is brittle against Hadoop versions, as new fields are added. Using this
+ // class does at least ensure that it is current.
+ val report = new ApplicationReportPBImpl()
+ report.setApplicationId(yarnId)
+ report.setCurrentApplicationAttemptId(new StubApplicationAttemptId(yarnId, attempt))
+ report.setYarnApplicationState(state)
+ report.setStartTime(startTime)
+ report.setFinishTime(finishTime)
+ report
+ }
+
+ /**
+ * Create an attempt from an application report;
+ * @param report source report
+ * @param endTime end time
+ * @param completed completed flag
+ * @param updateTime update time, will be taken from report start time otherwise
+ * @return
+ */
+ def attemptFromAppReport(
+ report: ApplicationReport,
+ endTime: Long,
+ completed: Boolean,
+ updateTime: Long = 0): TimelineApplicationAttemptInfo = {
+
+ val entityId = report.getCurrentApplicationAttemptId.toString
+ val updated = if (updateTime > 0) updateTime else report.getStartTime
+ new TimelineApplicationAttemptInfo(
+ Some(entityId),
+ report.getStartTime,
+ endTime,
+ updated,
+ "user",
+ completed,
+ entityId,
+ Some(entityId)
+ )
+ }
+
+ def appHistoryInfoFromAppReport(report: ApplicationReport, endTime: Long, completed: Boolean)
+ : TimelineApplicationHistoryInfo = {
+ val attempt = attemptFromAppReport(report, endTime, completed)
+ val id = report.getApplicationId.toString
+ new TimelineApplicationHistoryInfo(id, id, List(attempt))
+ }
+
+ /**
+ * Wait for the listing size to match that desired.
+ * @param provider provider
+ * @param size size to require
+ * @param timeout timeout
+ * @return the successful listing
+ */
+ def awaitApplicationListingSize(provider: YarnHistoryProvider, size: Long, timeout: Long)
+ : Seq[TimelineApplicationHistoryInfo] = {
+ def listingProbe(): Outcome = {
+ val listing = provider.getListing()
+ outcomeFromBool(listing.size == size)
+ }
+ def failure(outcome: Outcome, i: Int, b: Boolean): Unit = {
+ fail(s"after $i attempts, provider listing size !=$size: ${provider.getListing()}\n" +
+ s"$provider")
+ }
+ spinForState(s"await listing size=$size", 100, timeout, listingProbe, failure)
+ provider.getListing()
+ }
+
+ /**
+ * Wait for the application to be listed with the given number of application attempts.
+ *
+ * The function retries if the application is not found in the listing or if the
+ * number of attempts is not that required
+ * @param provider history provider
+ * @param appId application ID
+ * @param attempts number of required attempts which the entry must have
+ * @param timeout timeout
+ * @return the application details.
+ */
+ def awaitListingEntry(
+ provider: YarnHistoryProvider,
+ appId: String,
+ attempts: Int,
+ timeout: Long): TimelineApplicationHistoryInfo = {
+ def probe(): Outcome = {
+ findAppById(provider.getListing(), appId) match {
+ case Some(applicationInfo) => outcomeFromBool(applicationInfo.attempts.size == attempts)
+ case None => Retry()
+ }
+ }
+ // complex failure reporting
+ def failure(outcome: Outcome, i: Int, b: Boolean): Unit = {
+ val listing = provider.getListing()
+ findAppById(listing, appId) match {
+ case Some(info) =>
+ fail(s"After $timeout mS, application $appId did not have $attempts attempts:" +
+ s" ${describeApplicationHistoryInfo(info)}")
+ case None =>
+ fail(s"After $timeout mS, application $appId was not found in ${provider.getListing()}")
+ }
+ }
+ spinForState(s"await listing entry=$appId", 100, timeout, probe, failure)
+ lookupApplication(provider.getListing(), appId)
+ }
+
+ /**
+ * Wait for the refresh count to increment by at least one iteration
+ * @param provider provider
+ * @param timeout timeout
+ * @return the successful listing
+ */
+ def awaitRefreshExecuted(
+ provider: YarnHistoryProvider,
+ triggerRefresh: Boolean,
+ timeout: Long): Unit = {
+ val initialCount = provider.refreshCount
+ def listingProbe(): Outcome = {
+ outcomeFromBool(provider.refreshCount > initialCount)
+ }
+ def failure(outcome: Outcome, i: Int, b: Boolean): Unit = {
+ fail(s"After $i attempts, refresh count is $initialCount: $provider")
+ }
+ require(provider.isRefreshThreadRunning,
+ s"refresh thread is not running in $provider")
+ if (triggerRefresh) {
+ provider.triggerRefresh()
+ }
+ spinForState("await refresh count", 100, timeout, listingProbe, failure)
+ }
+
+ /**
+ * Spin awaiting a URL to not contain some text
+ * @param connector connector to use
+ * @param url URL to probe
+ * @param text text which must not be present
+ * @param timeout timeout in mils
+ */
+ def awaitURLDoesNotContainText(
+ connector: SpnegoUrlConnector,
+ url: URL,
+ text: String,
+ timeout: Long,
+ message: String = ""): String = {
+ def get: String = {
+ connector.execHttpOperation("GET", url, null, "").responseBody
+ }
+ def probe(): Outcome = {
+ try {
+ outcomeFromBool(!get.contains(text))
+ } catch {
+ case ioe: IOException =>
+ Retry()
+ case ex: Exception =>
+ throw ex
+ }
+ }
+
+
+ // failure action is simply to attempt the connection without
+ // catching the exception raised
+ def failure(outcome: Outcome, iterations: Int, timeout: Boolean): Unit = {
+ assertDoesNotContain(get, text, message)
+ }
+
+ spinForState(s"Awaiting a response from URL $url",
+ interval = 50, timeout = timeout, probe = probe, failure = failure)
+
+ get
+ }
+
+ /**
+ * Spin awaiting a URL to contain some text
+ * @param connector connector to use
+ * @param url URL to probe
+ * @param text text which must be present
+ * @param timeout timeout in mils
+ */
+ def awaitURLContainsText(
+ connector: SpnegoUrlConnector,
+ url: URL,
+ text: String,
+ timeout: Long): String = {
+ def get: String = {
+ connector.execHttpOperation("GET", url, null, "").responseBody
+ }
+ def probe(): Outcome = {
+ try {
+ outcomeFromBool(get.contains(text))
+ } catch {
+ case ioe: IOException =>
+ Retry()
+ case ex: Exception =>
+ throw ex
+ }
+ }
+
+ /*
+ failure action is simply to attempt the connection without
+ catching the exception raised
+ */
+ def failure(outcome: Outcome, iterations: Int, timeout: Boolean): Unit = {
+ assertContains(get, text)
+ }
+
+ spinForState(s"Awaiting a response from URL $url",
+ interval = 50, timeout = timeout, probe = probe, failure = failure)
+
+ get
+ }
+
+ def lookupApplication(listing: Seq[TimelineApplicationHistoryInfo], id: ApplicationId)
+ : TimelineApplicationHistoryInfo = {
+ lookupApplication(listing, id.toString)
+ }
+
+ def lookupApplication(listing: Seq[TimelineApplicationHistoryInfo], id: String)
+ : TimelineApplicationHistoryInfo = {
+ findAppById(listing, id) match {
+ case Some(applicationInfo2) =>
+ applicationInfo2
+ case None =>
+ throw new TestFailedException(s"Did not find $id entry in $listing", 4)
+ }
+ }
+
+ def assertAppCompleted(info: TimelineApplicationHistoryInfo, text: String = ""): Unit = {
+ assert(isCompleted(info), s"$text $info not flagged as completed")
+ }
+
+ def assertCompletedAttempt(attempt: TimelineApplicationAttemptInfo, text: String = ""): Unit = {
+ assert(attempt.completed, s"$text attempt not flagged as completed $attempt")
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/JerseyBindingSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/JerseyBindingSuite.scala
new file mode 100644
index 000000000000..5d3da7565376
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/JerseyBindingSuite.scala
@@ -0,0 +1,134 @@
+/*
+ * 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.yarn.integration
+
+import java.io.{ByteArrayInputStream, FileNotFoundException, IOException}
+import java.net.URI
+
+import com.sun.jersey.api.client.{ClientHandlerException, ClientResponse, UniformInterfaceException}
+
+import org.apache.spark.deploy.history.yarn.rest.{JerseyBinding, UnauthorizedRequestException}
+import org.apache.spark.deploy.history.yarn.testtools.AbstractYarnHistoryTests
+
+/**
+ * Unit test of how well the Jersey Binding works -especially some error handling logic
+ * Which can follow different paths
+ */
+class JerseyBindingSuite extends AbstractYarnHistoryTests {
+
+ val uriPath = "http://spark.apache.org"
+ val uri = new URI(uriPath)
+
+ def translate(ex: Throwable): Throwable = {
+ JerseyBinding.translateException("GET", uri, ex)
+ }
+
+ /**
+ * Build a [[UniformInterfaceException]] with the given string body
+ * @param status status code
+ * @param body body message
+ * @param buffer buffer flag
+ * @return new instance
+ */
+ def newUIE(status: Int, body: String, buffer: Boolean ): UniformInterfaceException = {
+ val response = new ClientResponse(status,
+ null,
+ new ByteArrayInputStream(body.getBytes("UTF-8")),
+ null)
+ new UniformInterfaceException(response, buffer)
+ }
+
+ /**
+ * If a [[ClientHandlerException]] contains an IOE, it
+ * is unwrapped and returned
+ */
+ test("UnwrapIOEinClientHandler") {
+ val fnfe = new FileNotFoundException("/tmp")
+ val che = new ClientHandlerException(fnfe)
+ assertResult(fnfe) {
+ translate(che)
+ }
+ }
+
+ /**
+ * If a [[ClientHandlerException]] does not contains an IOE, it
+ * is wrapped, but the inner text is extracted
+ */
+ test("BuildIOEinClientHandler") {
+ val npe = new NullPointerException("oops")
+ val che = new ClientHandlerException(npe)
+ val ex = translate(che)
+ assert(che === ex.getCause)
+ assertExceptionDetails(ex, "oops", uriPath)
+ }
+
+ /**
+ * If a [[ClientHandlerException]] does not contains an IOE, it
+ * is unwrapped and returned
+ */
+ test("EmptyClientHandlerException") {
+ val che = new ClientHandlerException("che")
+ val ex = translate(che)
+ assert(che === ex.getCause)
+ assertExceptionDetails(ex, "che", uriPath)
+ }
+
+ /**
+ * If the URI passed into translating a CHE is null, no
+ * URI is printed
+ */
+ test("Null URI for ClientHandlerException") {
+ val che = new ClientHandlerException("che")
+ val ex = JerseyBinding.translateException("POST", null, che)
+ assert(che === ex.getCause)
+ assertExceptionDetails(ex, "POST", "unknown")
+ }
+
+ test("UniformInterfaceException null response") {
+ // bufferResponseEntity must be false to avoid triggering NPE in constructor
+ val uie = new UniformInterfaceException("uae", null, false)
+ val ex = translate(uie)
+ assert(uie === ex.getCause)
+ assertExceptionDetails(ex, "uae", uriPath)
+ }
+
+ test("UniformInterfaceException 404 no body response") {
+ val uie = newUIE(404, "", false)
+ val ex = translate(uie)
+ assert(uie === ex.getCause)
+ assert(ex.isInstanceOf[FileNotFoundException], s"not FileNotFoundException: $ex")
+ assertExceptionDetails(ex, uriPath, uriPath)
+ }
+
+ test("UniformInterfaceException 403 forbidden") {
+ val uie = newUIE(403, "forbidden", false)
+ val ex = translate(uie)
+ assert(uie === ex.getCause)
+ assert(ex.isInstanceOf[UnauthorizedRequestException], s"not UnauthorizedRequestException: $ex")
+ assertExceptionDetails(ex, "Forbidden", uriPath)
+ }
+
+ test("UniformInterfaceException 500 response") {
+ val uie = newUIE(500, "internal error", false)
+ val ex = translate(uie)
+ assert(uie === ex.getCause)
+ assert(ex.isInstanceOf[IOException], s"not IOException: $ex")
+ assertExceptionDetails(ex, "500", uriPath)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/MultiAttemptWebSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/MultiAttemptWebSuite.scala
new file mode 100644
index 000000000000..8f047990cc9a
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/MultiAttemptWebSuite.scala
@@ -0,0 +1,96 @@
+/*
+ * 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.yarn.integration
+
+import java.net.URL
+
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.server.TimelineQueryClient._
+import org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider
+import org.apache.spark.deploy.history.yarn.server.YarnProviderUtils._
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+/**
+ * Create a completed app from multiple app attempts and fetch from Web UI
+ */
+class MultiAttemptWebSuite extends AbstractHistoryIntegrationTests {
+
+ test("Multi-attempt web UI") {
+ def submitAndCheck(webUI: URL, provider: YarnHistoryProvider): Unit = {
+
+ postMultipleAttempts()
+ val queryClient = createTimelineQueryClient()
+ val conf = sc.hadoopConfiguration
+ stopContextAndFlushHistoryService()
+
+ val expectedAppId = historyService.applicationId.toString
+
+ // validate ATS has it
+ val timelineEntities = queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ fields = Seq(PRIMARY_FILTERS, OTHER_INFO))
+ assert(2 === timelineEntities.size, "entities listed by app end filter")
+ val head = timelineEntities.head
+ val attempt1 = attemptId1.toString
+ val attempt2 = attemptId2.toString
+ assert(attempt1 === head.getEntityId || attempt2 === head.getEntityId,
+ s"wrong entity id in ${describeEntity(head)}")
+
+ queryClient.getEntity(SPARK_EVENT_ENTITY_TYPE, attempt1)
+ queryClient.getEntity(SPARK_EVENT_ENTITY_TYPE, attempt2)
+
+ // at this point the REST API is happy. Check the provider level
+
+ // listing must eventually contain two attempts
+ val appHistory = awaitListingEntry(provider, expectedAppId, 2, TEST_STARTUP_DELAY)
+ val historyDescription = describeApplicationHistoryInfo(appHistory)
+ // check the provider thinks that it has completed
+ assert(isCompleted(appHistory), s"App is not completed $historyDescription")
+
+ // resolve to entries
+ getAppUI(provider, expectedAppId, Some(attempt1SparkId))
+ getAppUI(provider, expectedAppId, Some(attempt2SparkId))
+
+ // then look for the complete app
+ awaitURL(webUI, TEST_STARTUP_DELAY)
+ val connector = createUrlConnector(conf)
+
+ val completeBody = awaitURLDoesNotContainText(connector, webUI,
+ no_completed_applications, TEST_STARTUP_DELAY)
+ logInfo(s"GET /\n$completeBody")
+ // look for the link
+ assertContains(completeBody, s"$expectedAppId")
+
+ val appPath = s"/history/$expectedAppId/$attempt1SparkId"
+ // GET the app
+ val appURL = new URL(webUI, appPath)
+ val appUI = connector.execHttpOperation("GET", appURL, null, "")
+ val appUIBody = appUI.responseBody
+ logInfo(s"Application\n$appUIBody")
+ assertContains(appUIBody, APP_NAME)
+ connector.execHttpOperation("GET", new URL(appURL, s"$appPath/jobs"), null, "")
+ connector.execHttpOperation("GET", new URL(appURL, s"$appPath/stages"), null, "")
+ connector.execHttpOperation("GET", new URL(appURL, s"$appPath/storage"), null, "")
+ connector.execHttpOperation("GET", new URL(appURL, s"$appPath/environment"), null, "")
+ connector.execHttpOperation("GET", new URL(appURL, s"$appPath/executors"), null, "")
+ }
+
+ webUITest("submit and check", submitAndCheck)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/MultipleAttemptSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/MultipleAttemptSuite.scala
new file mode 100644
index 000000000000..9db1dc5451c3
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/MultipleAttemptSuite.scala
@@ -0,0 +1,107 @@
+/*
+ * 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.yarn.integration
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.rest.JerseyBinding._
+import org.apache.spark.deploy.history.yarn.server.{TimelineApplicationAttemptInfo, TimelineQueryClient, YarnHistoryProvider}
+import org.apache.spark.deploy.history.yarn.testtools.{HistoryServiceNotListeningToSparkContext, TimelineSingleEntryBatchSize}
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+/**
+ * Test handling of multiple attempts in timeline
+ */
+class MultipleAttemptSuite
+ extends AbstractHistoryIntegrationTests
+ with HistoryServiceNotListeningToSparkContext
+ with TimelineSingleEntryBatchSize {
+
+ var provider: YarnHistoryProvider = null
+
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ super.setupConfiguration(sparkConf)
+ // no window limit, so no windowing of requests
+ sparkConf.set(YarnHistoryProvider.OPTION_WINDOW_LIMIT, "0")
+ }
+
+ test("Multiple Attempts") {
+ describe("Multiple Attempts")
+
+ postMultipleAttempts()
+ val hadoopConfiguration = sc.hadoopConfiguration
+ val sparkConf = sc.conf
+
+ stopContextAndFlushHistoryService()
+
+ val timeline = historyService.timelineWebappAddress
+ val queryClient = new TimelineQueryClient(timeline,
+ hadoopConfiguration,
+ createClientConfig())
+
+ describe("Querying history service via REST API")
+
+ val entities = awaitListSize(2,
+ s"number of entities of type $SPARK_EVENT_ENTITY_TYPE",
+ TEST_STARTUP_DELAY,
+ () => queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE))
+
+ // now expect/require there to be > 1 job attempt in there
+ val timelineEntity = entities.head
+ val timelineEntityDescription = describeEntityVerbose(timelineEntity)
+ // verify that the first entity has 4 events, that is: a single lifecycle
+ val timelineEvents = timelineEntity.getEvents
+ assertResult(4,
+ s"Number of timeline events in $timelineEntityDescription") {
+ timelineEvents.size
+ }
+
+ describe("Building attempt history")
+
+ // now read it in via history provider
+ provider = new YarnHistoryProvider(sparkConf)
+ val history = awaitApplicationListingSize(provider, 1, TEST_STARTUP_DELAY)
+ val info = history.head
+ val attempts = info.attempts.map(_.asInstanceOf[TimelineApplicationAttemptInfo])
+ assertListSize(attempts, 2, s"number of attempts in $info")
+ val attemptListAsText = attempts.mkString("[", ", ", "]")
+ val (elt1 :: elt2 :: _) = attempts
+ assertCompletedAttempt(elt1)
+ assert(elt1.attemptId !== elt2.attemptId)
+ // we expect the events to be sorted
+ assert(attemptId2.toString === elt1.entityId,
+ s"elt1 ID in $attemptListAsText wrong -sort order?")
+ assert(attemptId1.toString === elt2.entityId, s"elt2 ID in $attemptListAsText")
+
+ // verify this is picked up
+ assertAppCompleted(info, "retrieved info")
+ getAppUI(provider, info.id, elt1.attemptId)
+
+ // get a UI from an attempt that doesn't exist
+ assertNone(provider.getAppUI(info.id, Some("Nonexistent attempt")), "UI of nonexistent attempt")
+ }
+
+ override def afterEach(): Unit = {
+ if (provider != null) {
+ provider.stop()
+ }
+ super.afterEach()
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/ResourceDrivenTimelineQueryClient.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/ResourceDrivenTimelineQueryClient.scala
new file mode 100644
index 000000000000..08bc624f5f28
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/ResourceDrivenTimelineQueryClient.scala
@@ -0,0 +1,112 @@
+/*
+ * 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.yarn.integration
+
+import java.io.{FileNotFoundException, InputStream}
+import java.net.URI
+
+import scala.collection.JavaConverters._
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.databind.`type`.TypeFactory
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector
+import com.fasterxml.jackson.module.scala.experimental.ScalaObjectMapper
+import com.sun.jersey.api.client.config.ClientConfig
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.records.timeline.{TimelineEntities, TimelineEntity}
+
+import org.apache.spark.Logging
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.server.TimelineQueryClient
+
+/**
+ * A timeline query client which is driven by a resource.
+ *
+ * All entities are expected to be of the entity type used in list requests; there's no
+ * attempt to support multiple entity types.
+ *
+ * @param resource resource within test classpath
+ * @param timelineURI URI of the timeline service
+ * @param conf configuration
+ * @param jerseyClientConfig jersey client config
+ */
+class ResourceDrivenTimelineQueryClient(
+ resource: String,
+ timelineURI: URI,
+ conf: Configuration,
+ jerseyClientConfig: ClientConfig)
+ extends TimelineQueryClient(timelineURI, conf, jerseyClientConfig) with Logging {
+
+ /** Loaded timeline entity instance */
+ private var entities: TimelineEntities = _
+
+ /** list of entities within */
+ private var entityList: List[TimelineEntity] = _
+
+ init()
+
+ /**
+ * load in the JSON
+ */
+ private def init(): Unit = {
+ val in = this.getClass.getClassLoader.getResourceAsStream(resource)
+ require(in != null, s"Failed to load resource $resource")
+ val mapper = new ObjectMapper() with ScalaObjectMapper
+ // make deserializer use JAXB annotations (only)
+ mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(TypeFactory.defaultInstance))
+ entities = mapper.readValue[TimelineEntities](in)
+ entityList = entities.getEntities.asScala.toList
+
+ logDebug(s"$entities entities")
+ entityList.foreach { e =>
+ log.debug(s"${describeEntity(e)}")
+ }
+ }
+
+ override def endpointCheck (): Unit = {}
+
+ /**
+ * List entities. No filtering is performed.
+ * @return a possibly empty list of entities
+ */
+ override def listEntities(
+ entityType: String,
+ primaryFilter: Option[(String, String)],
+ secondaryFilters: Map[String, String],
+ fields: Seq[String],
+ limit: Option[Long],
+ windowStart: Option[Long],
+ windowEnd: Option[Long],
+ fromId: Option[String],
+ fromTs: Option[Long]): List[TimelineEntity] = {
+ entityList
+ }
+
+ /**
+ * Get an entity
+ * @param entityType type
+ * @param entityId the entity
+ * @return the entity if it was found
+ */
+ override def getEntity(entityType: String, entityId: String): TimelineEntity = {
+ entityList.find(_.getEntityId == entityId) match {
+ case Some(e) => e
+ case None => throw new FileNotFoundException(s"No entity $entityId")
+ }
+ }
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/TimeManagedHistoryProvider.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/TimeManagedHistoryProvider.scala
new file mode 100644
index 000000000000..dffd11b31244
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/TimeManagedHistoryProvider.scala
@@ -0,0 +1,110 @@
+/*
+ * 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.yarn.integration
+
+import java.util.concurrent.atomic.AtomicLong
+
+import org.apache.hadoop.yarn.api.records.ApplicationReport
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.server.YarnHistoryProvider
+import org.apache.spark.deploy.history.yarn.server.YarnProviderUtils._
+
+/**
+ * A subclass of the history provider which uses the `time` field rather than
+ * the current clock. This is needed to reliably test windowed operations
+ * and other actions in which the clock is checked.
+ *
+ * It also stubs the YARN client calls of the [[YarnHistoryProvider]], and doesn't need
+ * a YARN cluster connection to work. It also permits tests to alter the list of running apps
+ * to simulate failures.
+ *
+ * @param sparkConf configuration of the provider
+ * @param startTime the start time (millis)
+ * @param tickInterval amount to increase on a `tick()`
+ * @param _livenessChecksEnabled are liveness checks enabled
+ */
+class TimeManagedHistoryProvider(
+ sparkConf: SparkConf,
+ var startTime: Long = 0L,
+ var tickInterval: Long = 1000L,
+ _livenessChecksEnabled: Boolean = false)
+ extends YarnHistoryProvider(sparkConf){
+
+ private val time = new AtomicLong(startTime)
+
+ /**
+ * Is the timeline service (and therefore this provider) enabled.
+ * @return true : always
+ */
+ override def enabled: Boolean = {
+ true
+ }
+
+ /**
+ * Return the current time
+ * @return
+ */
+ override def now(): Long = {
+ time.get()
+ }
+
+ def setTime(t: Long): Unit = {
+ time.set(t)
+ }
+
+ /**
+ * Increase the time by one tick
+ * @return the new value
+ */
+ def tick(): Long = {
+ incrementTime(tickInterval)
+ }
+
+ def incrementTime(t: Long): Long = {
+ time.addAndGet(t)
+ }
+
+ private var runningApps: Seq[ApplicationReport] = List()
+
+ override protected def initYarnClient(): Unit = {}
+
+ override def livenessChecksEnabled: Boolean = {
+ _livenessChecksEnabled
+ }
+
+ /**
+ * List spark applications
+ * @return the list of running spark applications, which can then be filtered against
+ */
+ override def listYarnSparkApplications(): Map[String, ApplicationReport] = {
+ synchronized {
+ reportsToMap(runningApps)
+ }
+ }
+
+ /**
+ * List spark applications
+ * @return the list of running spark applications, which can then be filtered against
+ */
+ def setRunningApplications(apps: Seq[ApplicationReport]): Unit = {
+ synchronized {
+ runningApps = apps
+ }
+ }
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/TimelineListenerSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/TimelineListenerSuite.scala
new file mode 100644
index 000000000000..c6a7d22608ce
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/TimelineListenerSuite.scala
@@ -0,0 +1,218 @@
+/*
+ * 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.yarn.integration
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.deploy.history.yarn.{YarnEventListener, YarnHistoryService}
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.server.{TimelineApplicationAttemptInfo, YarnHistoryProvider}
+import org.apache.spark.deploy.history.yarn.server.YarnProviderUtils._
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.scheduler.SparkListenerApplicationStart
+import org.apache.spark.util.Utils
+
+/**
+ * Set up a listener and feed events in; verify they get through to ATS
+ */
+class TimelineListenerSuite extends AbstractHistoryIntegrationTests {
+
+ private val appStartFilter = Some((FILTER_APP_START, FILTER_APP_START_VALUE))
+
+ private val appEndFilter = Some((FILTER_APP_END, FILTER_APP_END_VALUE))
+
+ test("Listener Events") {
+ describe("Listener events pushed out")
+ // listener is still not hooked up to spark context
+ historyService = startHistoryService(sc)
+ val listener = new YarnEventListener(sc, historyService)
+ val startTime = now()
+ val contextAppId = sc.applicationId
+ val started = appStartEvent(startTime,
+ contextAppId,
+ Utils.getCurrentUserName())
+ listener.onApplicationStart(started)
+ awaitEventsProcessed(historyService, 1, TEST_STARTUP_DELAY)
+ flushHistoryServiceToSuccess()
+ historyService.stop()
+ awaitEmptyQueue(historyService, TEST_STARTUP_DELAY)
+ describe("reading events back")
+
+ val queryClient = createTimelineQueryClient()
+
+ // list all entries
+ val entities = queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE)
+ assertResult(1, "number of listed entities") { entities.size }
+ assertResult(1, "entities listed by app start filter") {
+ queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ primaryFilter = appStartFilter).size
+ }
+ val timelineEntities = queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ primaryFilter = appEndFilter)
+ assert(1 === timelineEntities.size, "entities listed by app end filter")
+ val expectedAppId = historyService.applicationId.toString
+ val expectedEntityId = attemptId.toString
+ val entry = timelineEntities.head
+ val entryDetails = describeEntity(entry)
+ assertResult(expectedEntityId,
+ s"no entity of id $expectedEntityId - found $entryDetails") {
+ entry.getEntityId
+ }
+ queryClient.getEntity(SPARK_EVENT_ENTITY_TYPE, expectedEntityId)
+
+ // here the events should be in the system
+ val provider = new YarnHistoryProvider(sc.conf)
+ val history = awaitApplicationListingSize(provider, 1, TEST_STARTUP_DELAY)
+ val info = history.head
+ logInfo(s"App history = $info")
+ val attempt = info.attempts.head.asInstanceOf[TimelineApplicationAttemptInfo]
+ // validate received data matches that saved
+ assertResult(started.sparkUser, s"username in $info") {
+ attempt.sparkUser
+ }
+ assertResult(startTime, s"started.time != startTime") {
+ started.time
+ }
+ assertResult(started.time, s"info.startTime != started.time in $info") {
+ attempt.startTime
+ }
+ assertResult(expectedAppId, s"info.id != expectedAppId in $info") {
+ info.id
+ }
+ assert(attempt.endTime> 0, s"end time is ${attempt.endTime} in $info")
+ // on a completed app, lastUpdated is the end time
+ assert(attempt.lastUpdated >= attempt.endTime,
+ s"attempt.lastUpdated < attempt.endTime time in $info")
+ assertResult(started.appName, s"info.name != started.appName in $info") {
+ info.name
+ }
+ // fecth the Spark UI - no attempt ID
+ provider.getAppUI(info.id, None)
+
+ // hit the underlying attempt
+ val timelineEntity = queryClient.getEntity(SPARK_EVENT_ENTITY_TYPE, attempt.entityId)
+ val events = timelineEntity.getEvents.asScala.toList
+ assertResult(2, s"number of events in ${describeEntity(timelineEntity)}") {
+ events.size
+ }
+ // first event must be the start one
+ val sparkListenerEvents = events.map(toSparkEvent).reverse
+ val (firstEvent :: secondEvent :: Nil) = sparkListenerEvents
+ val fetchedStartEvent = firstEvent.asInstanceOf[SparkListenerApplicationStart]
+ assert(started.time === fetchedStartEvent.time, "start time")
+
+ // direct retrieval using Spark context attempt
+ queryClient.getEntity(SPARK_EVENT_ENTITY_TYPE, expectedEntityId)
+
+ }
+
+ /**
+ * This test is like its predecessor except the application is queried while
+ * it has not yet completed; we are looking at how that intermediate state is
+ * described
+ */
+ test("Last-Updated time of incompleted app") {
+ describe("Last-Updated time of incompleted app")
+ // listener is still not hooked up to spark context
+ historyService = startHistoryService(sc)
+ val timeline = historyService.timelineWebappAddress
+ val listener = new YarnEventListener(sc, historyService)
+ val startTime = now()
+ val userName = Utils.getCurrentUserName()
+ val yarnAppId = applicationId.toString()
+ val attemptId = attemptId1.toString
+ val started = appStartEvent(startTime, appId = yarnAppId,
+ user = userName, attempt = Some(attemptId))
+ // initial checks to make sure the event is fully inited
+ assert(userName === started.sparkUser, s"started.sparkUser")
+ assert(Some(yarnAppId) === started.appId, s"started.appId")
+ assert(APP_NAME === started.appName, s"started.appName")
+ listener.onApplicationStart(started)
+ awaitEventsProcessed(historyService, 1, TEST_STARTUP_DELAY)
+ flushHistoryServiceToSuccess()
+
+ awaitEmptyQueue(historyService, TEST_STARTUP_DELAY)
+ describe("reading events back")
+ val queryClient = createTimelineQueryClient()
+
+ // list all entries
+ val entities = queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE)
+ assert(1 === entities.size, "number of listed entities")
+ val timelineEntities = queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ primaryFilter = appStartFilter)
+ assert(1 === timelineEntities.size, "entities listed by app start filter")
+ assertResult(0, "entities listed by app end filter") {
+ queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE, primaryFilter = appEndFilter).size
+ }
+ val headEntity = timelineEntities.head
+ assertResult(attemptId, s"no entry of id $yarnAppId in ${describeEntity(headEntity)}") {
+ headEntity.getEntityId
+ }
+
+ // first grab the initial entity and extract it manually
+ // this helps isolate any unmarshalling problems
+ val entity = queryClient.getEntity(YarnHistoryService.SPARK_EVENT_ENTITY_TYPE, attemptId)
+ val entityDetails = describeEntity(entity)
+ logInfo(s"Timeline Event = $entityDetails")
+ logInfo(s"Timeline Event = ${eventDetails(entity) }")
+ val unmarshalledEntity = toApplicationHistoryInfo(entity)
+ assert(started.appName === unmarshalledEntity.name,
+ s"unmarshalledEntity.name != started.appName in $unmarshalledEntity")
+ assert(userName === unmarshalledEntity.attempts.head.sparkUser,
+ s"unmarshalledEntity.sparkUser != username in $unmarshalledEntity")
+
+ // here the events should be in the system
+ val provider = new YarnHistoryProvider(sc.conf)
+ val history = awaitApplicationListingSize(provider, 1, TEST_STARTUP_DELAY)
+ val info = history.head
+
+ logInfo(s"App history = $info")
+ val attempt = info.attempts.head
+
+ // validate received data matches that saved
+
+ assert(startTime === started.time, s"started.time != startTime")
+ assert(started.time === attempt.startTime, s"attempt.startTime != started.time in $info")
+ assert(yarnAppId === info.id, s"info.id != yarnAppId in $info")
+ assert(Some(attemptId) === attempt.attemptId, s"Attempt ID in head attempt")
+ assert(attempt.endTime === 0, s"end time must be zero in in incompete app $info")
+ // on a completed app, lastUpdated is the end time
+ assert(attempt.lastUpdated !== 0,
+ s"attempt.lastUpdated must be non-zero in incompete app $info")
+ assert(started.appName === info.name, s"info.name != started.appName in $info")
+ assert(userName === attempt.sparkUser, s"attempt.sparkUser != username in $info")
+ // now get the event.
+
+ getAppUI(provider, info.id, attempt.attemptId)
+
+ val timelineEntity = queryClient.getEntity(SPARK_EVENT_ENTITY_TYPE, attemptId)
+ val events = timelineEntity.getEvents.asScala.toList
+ assert(1 === events.size, s"number of events in ${describeEntity(timelineEntity) }")
+ // first event must be the start one
+ val sparkListenerEvents = events.map(toSparkEvent).reverse
+ val (firstEvent :: Nil) = sparkListenerEvents
+ val fetchedStartEvent = firstEvent.asInstanceOf[SparkListenerApplicationStart]
+ assert(started.time === fetchedStartEvent.time, "start time")
+
+ // finally, a couple of checks for invalid data
+ assertNone(provider.getAppUI("unknown", attempt.attemptId), "Get UI with unknown app")
+ assertNone(provider.getAppUI(info.id, Some("unknown Attempt")), "Get UI with unknown attempt")
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/TimelineQueryClientSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/TimelineQueryClientSuite.scala
new file mode 100644
index 000000000000..6e1480c5522e
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/TimelineQueryClientSuite.scala
@@ -0,0 +1,133 @@
+/*
+ * 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.yarn.integration
+
+import java.io.IOException
+import java.net.{URI, URL}
+
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity
+
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.rest.HttpRequestException
+import org.apache.spark.deploy.history.yarn.rest.JerseyBinding._
+import org.apache.spark.deploy.history.yarn.server.TimelineQueryClient
+import org.apache.spark.deploy.history.yarn.server.TimelineQueryClient._
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+class TimelineQueryClientSuite extends AbstractHistoryIntegrationTests {
+
+ private var timeline: URI = _
+ var queryClient: TimelineQueryClient = _
+
+ override def setup(): Unit = {
+ super.setup()
+ historyService = startHistoryService(sc)
+ timeline = historyService.timelineWebappAddress
+ queryClient = createTimelineQueryClient()
+ }
+
+ test("About") {
+ val response = queryClient.about()
+ logInfo(s"$timeline/about => \n$response")
+ assertNotNull(response, s"$queryClient about()")
+ assertContains(response, "Timeline")
+ }
+
+ def assertNilQuery(filter: String, fields: Seq[String] = Nil): Unit = {
+ assertNil(queryClient.listEntities(filter), s"list with filter $filter")
+ }
+
+ test("ListNoEntityTypes") {
+ assertNilQuery(SPARK_EVENT_ENTITY_TYPE)
+ }
+
+ test("List LAST_EVENT_ONLY") {
+ assertNilQuery(SPARK_EVENT_ENTITY_TYPE, Seq(LAST_EVENT_ONLY))
+ }
+
+ test("List RELATED_ENTITIES") {
+ assertNilQuery(SPARK_EVENT_ENTITY_TYPE, Seq(RELATED_ENTITIES))
+ }
+
+ test("List LAST_EVENT_ONLY | PRIMARY_FILTERS") {
+ assertNil(queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ fields = Seq(LAST_EVENT_ONLY, TimelineQueryClient.PRIMARY_FILTERS)),
+ "List LAST_EVENT_ONLY | PRIMARY_FILTERS")
+ }
+
+ test("List OTHER_INFO") {
+ assertNilQuery(SPARK_EVENT_ENTITY_TYPE, Seq(OTHER_INFO))
+ }
+
+ test("List PRIMARY_FILTERS") {
+ assertNilQuery(SPARK_EVENT_ENTITY_TYPE, Seq(PRIMARY_FILTERS))
+ }
+
+ test("List EVENTS") {
+ assertNilQuery(SPARK_EVENT_ENTITY_TYPE, Seq(EVENTS))
+ }
+
+ test("PostEntity") {
+ describe("post an entity and then retrieve it")
+ val te = new TimelineEntity
+ te.setStartTime(now())
+ te.setEntityId("SPARK-0001")
+ te.setEntityType(SPARK_EVENT_ENTITY_TYPE)
+ te.addPrimaryFilter(FILTER_APP_START, FILTER_APP_START_VALUE)
+
+ val timelineClient = historyService.timelineClient
+ timelineClient.putEntities(te)
+ val timelineEntities: List[TimelineEntity] =
+ queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE)
+ assert(timelineEntities.size === 1, "empty TimelineEntity list")
+ assertEntitiesEqual(te, timelineEntities.head)
+
+ val entity2 = queryClient.getEntity(SPARK_EVENT_ENTITY_TYPE, te.getEntityId() )
+ assertEntitiesEqual(te, entity2)
+
+ val listing2 = queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ primaryFilter = Some((FILTER_APP_START, FILTER_APP_START_VALUE)))
+ assert(1 === listing2.size, s"filtering on $FILTER_APP_START:$FILTER_APP_START_VALUE")
+ // filtered query
+ assertEntitiesEqual(te, listing2.head)
+ }
+
+ def createTimelineClientRootPath(): TimelineQueryClient = {
+ val realTimelineEndpoint = historyService.timelineWebappAddress.toURL
+ val rootTimelineServer = new URL(realTimelineEndpoint, "/").toURI
+ new TimelineQueryClient(rootTimelineServer,
+ sc.hadoopConfiguration,
+ createClientConfig())
+ }
+
+ test("Client about() Against Wrong URL") {
+ intercept[IOException] {
+ createTimelineClientRootPath().about()
+ }
+ }
+
+ test("Client endpointcheck() Against Wrong URL") {
+ val client: TimelineQueryClient = createTimelineClientRootPath()
+ val ex = intercept[HttpRequestException] {
+ client.endpointCheck()
+ }
+ log.debug(s"GET $client", ex)
+ assertContains(ex.toString(), "text/html")
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/WebsiteIntegrationSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/WebsiteIntegrationSuite.scala
new file mode 100644
index 000000000000..6ecc98dcb29e
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/WebsiteIntegrationSuite.scala
@@ -0,0 +1,145 @@
+/*
+ * 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.yarn.integration
+
+import java.io.FileNotFoundException
+import java.net.URL
+
+import org.apache.spark.deploy.history.HistoryServer
+import org.apache.spark.deploy.history.yarn.{YarnEventListener, YarnHistoryService}
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.rest.HttpOperationResponse
+import org.apache.spark.deploy.history.yarn.server.{TimelineApplicationAttemptInfo, YarnHistoryProvider}
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.util.Utils
+
+/**
+ * Complete integration test: lifecycle events through to web site
+ */
+class WebsiteIntegrationSuite extends AbstractHistoryIntegrationTests {
+
+ test("Instantiate HistoryProvider") {
+ val provider = createHistoryProvider(sc.conf)
+ provider.stop()
+ }
+
+ test("WebUI hooked up") {
+ def probeEmptyWebUIVoid(webUI: URL, provider: YarnHistoryProvider): Unit = {
+ probeEmptyWebUI(webUI, provider)
+ }
+ webUITest("WebUI hooked up", probeEmptyWebUIVoid)
+ }
+
+ test("Get the web UI of a completed application") {
+ def submitAndCheck(webUI: URL, provider: YarnHistoryProvider): Unit = {
+
+ historyService = startHistoryService(sc)
+ val listener = new YarnEventListener(sc, historyService)
+ val startTime = now()
+
+ val ctxAppId = sc.applicationId
+ val started = appStartEvent(startTime, ctxAppId, Utils.getCurrentUserName())
+ listener.onApplicationStart(started)
+ awaitEventsProcessed(historyService, 1, TEST_STARTUP_DELAY)
+ flushHistoryServiceToSuccess()
+
+ // now stop the app
+ historyService.stop()
+ awaitEmptyQueue(historyService, SERVICE_SHUTDOWN_DELAY)
+ val expectedAppId = historyService.applicationId.toString
+ val expectedAttemptId = attemptId.toString
+
+ // validate ATS has it
+ val queryClient = createTimelineQueryClient()
+ val timelineEntities = queryClient.listEntities(SPARK_EVENT_ENTITY_TYPE,
+ primaryFilter = Some((FILTER_APP_END, FILTER_APP_END_VALUE)))
+ assert(1 === timelineEntities.size, "entities listed by app end filter")
+ val entry = timelineEntities.head
+ assert(expectedAttemptId === entry.getEntityId,
+ s"head entry id!=$expectedAttemptId: ${describeEntity(entry)} ")
+
+ queryClient.getEntity(YarnHistoryService.SPARK_EVENT_ENTITY_TYPE, expectedAttemptId)
+
+ // at this point the REST UI is happy. Check the provider level
+
+ val listing = awaitApplicationListingSize(provider, 1, TEST_STARTUP_DELAY)
+ val appInListing = listing.find(_.id == expectedAppId)
+ assertSome(appInListing, s"Application $expectedAppId not found in listing $listing")
+ val attempts = appInListing.get.attempts
+ assertNotEmpty( attempts, s"App attempts empty")
+ val firstAttempt = attempts.head.asInstanceOf[TimelineApplicationAttemptInfo]
+ val expectedWebAttemptId = firstAttempt.attemptId.get
+
+ // and look for the complete app
+ awaitURL(webUI, TEST_STARTUP_DELAY)
+ val connector = createUrlConnector()
+
+ val completeBody = awaitURLDoesNotContainText(connector, webUI,
+ no_completed_applications, TEST_STARTUP_DELAY)
+ logInfo(s"GET /\n$completeBody")
+ // look for the anchor body
+ assertContains(completeBody, s"$expectedAppId")
+ // look for the tail of the link
+ assertContains(completeBody, s"/history/$expectedAppId/$expectedWebAttemptId")
+
+ val appPath = HistoryServer.getAttemptURI(expectedAppId, Some(expectedWebAttemptId))
+ // GET the app
+ val attemptURL = getAttemptURL(webUI, expectedAppId, Some(expectedWebAttemptId), "")
+ logInfo(s"Fetching Application attempt from $attemptURL")
+ val appUI = connector.execHttpOperation("GET", attemptURL, null, "")
+ val appUIBody = appUI.responseBody
+ logInfo(s"Application\n$appUIBody")
+ assertContains(appUIBody, APP_NAME)
+
+ def GET(component: String): HttpOperationResponse = {
+ val url = new URL(attemptURL, s"$appPath" + component)
+ logInfo(s"GET $url")
+ connector.execHttpOperation("GET", url)
+ }
+ GET("")
+ GET("/jobs")
+ GET("/stages")
+ GET("/storage")
+ GET("/environment")
+ GET("/executors")
+
+ // then try to resolve the app on its own and expect a failure
+ intercept[FileNotFoundException] {
+ val appURL = new URL(webUI, s"/history/$expectedAppId")
+ connector.execHttpOperation("GET", appURL)
+ }
+ }
+
+ webUITest("submit and check", submitAndCheck)
+ }
+
+ /**
+ * Get the full URL to an application/application attempt
+ * @param webUI base URL of the history server
+ * @param appId application ID
+ * @param attemptId attempt ID
+ * @param item optional path under the URL
+ * @return A URL which can be used to access the spark UI
+ */
+ def getAttemptURL(webUI: URL, appId: String, attemptId: Option[String], item: String = "")
+ : URL = {
+ val path = HistoryServer.getAttemptURI(appId, attemptId) + (if (item == "") "" else s"/$item")
+ new URL(webUI, path)
+ }
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/YarnHistoryProviderSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/YarnHistoryProviderSuite.scala
new file mode 100644
index 000000000000..e35e55242131
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/YarnHistoryProviderSuite.scala
@@ -0,0 +1,209 @@
+/*
+ * 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.yarn.integration
+
+import org.apache.hadoop.yarn.api.records.{ApplicationReport, YarnApplicationState}
+
+import org.apache.spark.deploy.history.yarn.server.{TimelineApplicationHistoryInfo, YarnHistoryProvider}
+import org.apache.spark.deploy.history.yarn.server.YarnProviderUtils._
+import org.apache.spark.deploy.history.yarn.testtools.AbstractYarnHistoryTests
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.ui.SparkUI
+
+/**
+ * Basic lifecycle/method calls on the history provider.
+ * It's not quite an integration test, but kept together as it is part
+ * of the server-side code.
+ */
+class YarnHistoryProviderSuite extends AbstractYarnHistoryTests with IntegrationTestUtils {
+
+ var provider: YarnHistoryProvider = _
+
+ val app1FailedReport = stubApplicationReport(1, 1, 1, YarnApplicationState.FAILED, 1000, 1500)
+ val app1Incomplete = appHistoryInfoFromAppReport(app1FailedReport, 0, false)
+ val app2FinishedReport = stubApplicationReport(2, 2, 1, YarnApplicationState.FINISHED, 2000, 2500)
+ val app2Complete = appHistoryInfoFromAppReport(app2FinishedReport, 2500, true)
+
+ val app3RunningReport = stubApplicationReport(3, 3, 1, YarnApplicationState.RUNNING, 3000, 0)
+ val app3Running = appHistoryInfoFromAppReport(app3RunningReport, 0, false)
+
+ val app4AcceptedReport = stubApplicationReport(4, 4, 2, YarnApplicationState.ACCEPTED, 4000, 0)
+ val app4FirstAttemptReport = stubApplicationReport(4, 4, 1, YarnApplicationState.RUNNING, 4000, 0)
+
+ // a more complex history of two attempts; first one failed
+ val app4Running = new TimelineApplicationHistoryInfo(app4AcceptedReport.getApplicationId.toString,
+ app4AcceptedReport.getApplicationId.toString,
+ List(attemptFromAppReport(app4AcceptedReport, 0, false, 4500),
+ attemptFromAppReport(app4FirstAttemptReport, 0, false, 4000)))
+
+ val allApps = reportsToMap(List(app1FailedReport, app2FinishedReport,
+ app3RunningReport, app4AcceptedReport))
+ val runningApps = reportsToMap(List(app3RunningReport, app4AcceptedReport))
+
+ /*
+ * Setup creates the spark context
+ */
+ override protected def setup(): Unit = {
+ super.setup()
+ provider = new YarnHistoryProvider(sc.conf)
+ }
+
+ /**
+ * Stop the provider
+ */
+ override def afterEach(): Unit = {
+ if (provider != null) {
+ provider.stop()
+ }
+ super.afterEach()
+ }
+
+ test("Empty Provider List") {
+ describe("Provider listing")
+ assertResult(Nil) {
+ provider.getListing()
+ }
+ provider.stop()
+ provider.stop()
+ }
+
+ test("Provider Stop+Stop") {
+ describe("Provider Stop+Stop")
+ val historyProvider = new YarnHistoryProvider(sc.conf)
+ // check there are no re-entrancy bugs here
+ historyProvider.stop()
+ historyProvider.stop()
+ }
+
+ test("getAppUi(unknown-app)") {
+ describe("getAppUi(unknown-app) -> none")
+ assertResult(None) {
+ getAppUI("unknown-app")
+ }
+ }
+
+ test("getAppUi(unknown-app, Some(attempt)") {
+ describe("getAppUi(unknown-app, Some(attempt)) -> none")
+ assertResult(None) {
+ getAppUI("unknown-app", Some("attempt"))
+ }
+ }
+
+ test("getAppUi(\"\")") {
+ describe("getAppUi(\"\")")
+ assertResult(None) {
+ getAppUI("")
+ }
+ }
+
+ test("getAppUi('',Some(attempt)") {
+ describe("getAppUi('',Some(attempt)")
+ assertResult(None) {
+ getAppUI("", Some("attempt"))
+ }
+ }
+
+ test("CompleteNoRunning") {
+ assertUnmodified(app2Complete, Map())
+ }
+
+ test("CompleteOtherRunning") {
+ assertUnmodified(app2Complete, runningApps)
+ }
+
+ test("CompleteInReport") {
+ assertUnmodified(app2Complete, allApps)
+ }
+
+ test("IncompleteRunning") {
+ // There's an incomplete App, as it is running all is well
+ assertUnmodified(app3Running, allApps)
+ }
+ test("IncompleteAccepted") {
+ // There's an incomplete App, as it is running all is well
+ assertUnmodified(app4Running, allApps)
+ }
+
+ test("Incomplete with failure reported") {
+ assertBecomesCompleted(app1Incomplete, allApps)
+ }
+
+ test("Incomplete with no entry outside window") {
+ // as it is incomplete but with no entry: culled
+ assertBecomesCompleted(app4Running, Map())
+ }
+
+ test("Incomplete with entry inside window") {
+ // do a scan with the window expanded to 3 seconds, enough for the
+ // last attempt updated field to be in range
+ val (head :: tail) = completeAppsFromYARN(List(app4Running), Map(), 5000, 3000)
+ assert(app4Running === head,
+ s"history was modified to ${describeApplicationHistoryInfo(head) }")
+
+ }
+
+ test("state.FINISHED is completion event") {
+ assertBecomesCompleted(app2Complete, allApps)
+ }
+
+ test("Completion Probes") {
+ assert(isFinished(app2FinishedReport))
+ assert(isFinished(app1FailedReport))
+ assert(!isFinished(app3RunningReport))
+ assert(!isFinished(app4FirstAttemptReport))
+ assert(!isFinished(app4AcceptedReport))
+ }
+
+ /**
+ * Assert that an application history entry becomes completed in
+ * [[completeAppsFromYARN()]]
+ * @param info history info
+ * @param reports list of applications known by the RM
+ */
+ def assertBecomesCompleted(info: TimelineApplicationHistoryInfo,
+ reports: Map[String, ApplicationReport]): Unit = {
+ val (head:: tail) = completeAppsFromYARN(List(info), reports, 5000, 100)
+ assert(isCompleted(head))
+ assert(info.id === head.id )
+ }
+
+ /**
+ * Assert that an application history entry is not modified by
+ * [[completeAppsFromYARN()]]
+ * @param info history info
+ * @param reports list of applications known by the RM
+ */
+ def assertUnmodified(info: TimelineApplicationHistoryInfo,
+ reports: Map[String, ApplicationReport]): Unit = {
+ val (head :: tail) = completeAppsFromYARN(List(info), reports, 5000, 100)
+ assert(info === head, s"history was modified to ${describeApplicationHistoryInfo(head)}")
+ }
+
+ /**
+ * Create the provider, get the app UI.
+ *
+ * The provider member variable will be set as a side-effect
+ * @param appid application ID
+ * @return the result of provider.getAppUI
+ */
+ def getAppUI(appid: String, attemptId: Option[String] = None): Option[SparkUI] = {
+ logDebug(s"GET appID =$appid attemptId=$attemptId" )
+ provider.getAppUI(appid, attemptId)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/YarnHistoryProviderWindowSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/YarnHistoryProviderWindowSuite.scala
new file mode 100644
index 000000000000..7c9c9efcc7a7
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/YarnHistoryProviderWindowSuite.scala
@@ -0,0 +1,145 @@
+/*
+ * 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.yarn.integration
+
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
+
+import org.apache.spark.deploy.history.yarn.YarnHistoryService
+import org.apache.spark.deploy.history.yarn.server.YarnProviderUtils._
+import org.apache.spark.deploy.history.yarn.testtools.{HistoryServiceNotListeningToSparkContext, TimelineSingleEntryBatchSize}
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.scheduler.cluster.StubApplicationAttemptId
+import org.apache.spark.util.Utils
+
+/**
+ * check windowed providder.
+ *
+ * More than one history service is started here, each publishing their own events, with
+ * their own app ID. For this to work they are set up to not listen to context events.
+ */
+class YarnHistoryProviderWindowSuite
+ extends AbstractHistoryIntegrationTests
+ with HistoryServiceNotListeningToSparkContext
+ with TimelineSingleEntryBatchSize {
+ val minute = 60000
+ val start1Time = minute
+ val start2Time = start1Time + minute
+ val appReport1 = stubApplicationReport(1, 0, 1, YarnApplicationState.RUNNING, start1Time, 0)
+ val appReport2 = stubApplicationReport(2, 0, 1, YarnApplicationState.RUNNING, start2Time, 0)
+
+ val appId1 = appReport1.getApplicationId.toString
+ val appId2 = appReport2.getApplicationId.toString
+ val user = Utils.getCurrentUserName()
+
+ test("YarnHistoryProviderWindow") {
+ describe("Windowed publishing across apps")
+ var history2: YarnHistoryService = null
+ var provider: TimeManagedHistoryProvider = null
+ try {
+ logDebug("Start application 1")
+ val expectedAppId1 = appReport1.getApplicationId
+ historyService = startHistoryService(sc, expectedAppId1,
+ Some(appReport1.getCurrentApplicationAttemptId))
+ assert(!historyService.listening, s"listening $historyService")
+ assert(historyService.bondedToATS, s"not bonded to ATS: $historyService")
+ // post in an app start
+ val start1 = appStartEvent(start1Time, appId1, user, Some("1"))
+ enqueue(start1)
+ flushHistoryServiceToSuccess(historyService)
+
+ describe("application 2")
+ // the second application starts then stops after the first one
+ val applicationId2 = appReport2.getApplicationId
+ val attemptId2 = new StubApplicationAttemptId(applicationId2, 2)
+ val expectedAppId2 = applicationId2.toString
+ history2 = startHistoryService(sc, applicationId2,
+ Some(appReport2.getCurrentApplicationAttemptId))
+
+
+ val start2 = appStartEvent(start2Time, appId2, user, Some("1"))
+ history2.enqueue(start2)
+ val end2Time = start2Time + minute
+ val end2 = appStopEvent(end2Time)
+ history2.enqueue(end2)
+ // stop the second application
+ history2.stop()
+ flushHistoryServiceToSuccess(history2)
+ history2 = null
+
+ // here there is one incomplete application, and a completed one
+ // which started and stopped after the incomplete one started
+ provider = new TimeManagedHistoryProvider(sc.conf, end2Time, minute)
+ provider.setRunningApplications(List(appReport1, appReport2))
+ addFailureAction(dumpProviderState(provider))
+ addFailureAction(dumpTimelineEntities(provider))
+
+ // now read it in via history provider
+ describe("read in listing")
+
+ val listing1 = awaitApplicationListingSize(provider, 2, TEST_STARTUP_DELAY)
+ logInfo(s"Listing 1: $listing1")
+ assertAppCompleted(lookupApplication(listing1, expectedAppId2),
+ s"app2 ID $expectedAppId2, in listing1 $listing1")
+ val applicationInfo1_1 = lookupApplication(listing1, expectedAppId1)
+ assert(!isCompleted(applicationInfo1_1), s"$applicationInfo1_1 completed in L1 $listing1")
+
+ describe("stop application 1")
+ val end3Time = provider.tick()
+ val end3 = appStopEvent(end3Time)
+ historyService.enqueue(end3)
+ historyService.stop()
+ flushHistoryServiceToSuccess()
+
+ // move time forwards
+ provider.incrementTime(5 * minute)
+ // Now await a refresh
+ describe("read in listing #2")
+
+ awaitRefreshExecuted(provider, true, TEST_STARTUP_DELAY)
+ awaitRefreshExecuted(provider, true, TEST_STARTUP_DELAY)
+ awaitRefreshExecuted(provider, true, TEST_STARTUP_DELAY)
+
+ logDebug("Refreshes executed; extracting application listing")
+ val allApps = provider.listApplications()
+ logInfo(s"allApps : ${allApps.applications}")
+
+ // get a new listing
+ val listing2 = provider.getListing()
+ logInfo(s"Listing 2: $listing2")
+ // which had better be updated or there are refresh problems
+ assert(listing1 !== listing2, s"updated listing was unchanged from $provider")
+ // get the updated value and expect it to be complete
+ assertAppCompleted(lookupApplication(listing2, expectedAppId1), s"app1 in L2 $listing2")
+ assertAppCompleted(lookupApplication(listing2, expectedAppId1), s"app2 in L2 $listing2")
+ provider.stop()
+ } catch {
+ case ex: Exception =>
+ executeFailureActions()
+ throw ex
+ } finally {
+ describe("teardown")
+ if (history2 != null) {
+ history2.stop()
+ }
+ if (provider != null) {
+ provider.stop()
+ }
+ }
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/YarnProviderUtilsSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/YarnProviderUtilsSuite.scala
new file mode 100644
index 000000000000..b7d6601a791d
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/integration/YarnProviderUtilsSuite.scala
@@ -0,0 +1,401 @@
+/*
+ * 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.yarn.integration
+
+import org.scalatest.Matchers
+
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.server.{TimelineApplicationHistoryInfo, TimelineApplicationAttemptInfo}
+import org.apache.spark.deploy.history.yarn.server.YarnProviderUtils._
+import org.apache.spark.deploy.history.yarn.testtools.ExtraAssertions
+import org.apache.spark.scheduler.cluster.{StubApplicationAttemptId, StubApplicationId}
+import org.apache.spark.{Logging, SparkFunSuite}
+
+/**
+ * Test of utility methods in [[org.apache.spark.deploy.history.yarn.server.YarnProviderUtils]]
+ */
+class YarnProviderUtilsSuite extends SparkFunSuite with Logging
+ with ExtraAssertions with Matchers {
+
+ def historyInfo(
+ id: String,
+ started: Long,
+ ended: Long,
+ complete: Boolean): TimelineApplicationHistoryInfo = {
+ historyInfo(id, Some(id), started, ended, ended, complete )
+ }
+
+ def historyInfo(
+ appId: String,
+ attemptId: Option[String],
+ started: Long,
+ ended: Long,
+ updated: Long,
+ complete: Boolean): TimelineApplicationHistoryInfo = {
+ val updated = Math.max(started, ended)
+ val attempt = new TimelineApplicationAttemptInfo(attemptId,
+ started, ended, updated, "user", complete , attemptId.get, attemptId)
+ new TimelineApplicationHistoryInfo(appId, appId, List(attempt))
+ }
+
+ def historyInfo(
+ old: TimelineApplicationHistoryInfo,
+ attempts: List[TimelineApplicationAttemptInfo]): TimelineApplicationHistoryInfo = {
+ new TimelineApplicationHistoryInfo(old.id, old.name, attempts)
+ }
+
+ val yarnAppId = new StubApplicationId(5, 0)
+ val yarnAttemptId = new StubApplicationAttemptId(yarnAppId, 1)
+
+ val h12 = historyInfo("h12", 1, 2, true)
+ val h22 = historyInfo("h22", 2, 2, true)
+ val i20 = historyInfo("i20", 2, 0, false)
+ val i30 = historyInfo("i30", 3, 0, false)
+ val h33 = historyInfo("h30", 3, 3, true)
+ val h44 = historyInfo("h44", 4, 4, true)
+ val iA10_incomplete = historyInfo("iA", 1, 0, false)
+ val iA11_completed = historyInfo("iA", 1, 1, true)
+ val a1_attempt_1 = historyInfo("iA", Some("attempt_1"), 100, 102, 102, false)
+ val a1_attempt_2 = historyInfo("iA", Some("attempt_2"), 200, 202, 202, true)
+ val none_incomplete = new TimelineApplicationAttemptInfo(None, 100, 0, 102, "spark", false,
+ "001", None)
+ val none_completed = new TimelineApplicationAttemptInfo(None, 200, 202, 202, "spark", true,
+ "001", None)
+ val none_completed_orig_time = new TimelineApplicationAttemptInfo(None, 100, 0, 102, "spark",
+ true, "001", None)
+ // app attempt started @ 100, updated @102, version 1
+ val attempt_1_1 = new TimelineApplicationAttemptInfo(Some("attempt_1_1"),
+ 100, 0, 102, "spark", false, "001", None, 1)
+ // attempt 1.1 updated at time = 102; no version field
+ val attempt_1_1_updated = new TimelineApplicationAttemptInfo(Some("attempt_1_1"),
+ 100, 0, 150, "spark", false, "001", None)
+ // attempt 1.1 with the version field updated to 2; it should always be newer
+ val attempt_1_1_updated_version = new TimelineApplicationAttemptInfo(Some("attempt_1_1"),
+ 100, 0, 300, "spark", false, "001", None, 2)
+ val attempt_1_1_completed_v4 = new TimelineApplicationAttemptInfo(Some("attempt_1_1"),
+ 100, 400, 400, "spark", true, "001", None, 4)
+ val attempt_1_2 = new TimelineApplicationAttemptInfo(Some("attempt_1_2"), 200, 202, 202,
+ "spark", true, "001", None, 3)
+
+ test("timeShort") {
+ assert("unset" === timeShort(0, "unset"))
+ assert("unset" !== timeShort(System.currentTimeMillis(), "unset"))
+ }
+
+ test("findOldest") {
+ assert(Some(h12) === findOldestApplication(List(h12, h22, i20)))
+ }
+
+ test("findOldest-2") {
+ assert(Some(h22) === findOldestApplication(List(h44, h22, i20)))
+ }
+
+ test("findOldest-3") {
+ assert(Some(i20) === findOldestApplication(List(h44, h33, i20)))
+ }
+
+ test("findOldest-4") {
+ assert(None === findOldestApplication(Nil))
+ }
+
+ test("findIncomplete") {
+ assert(List(i20, i30) === findIncompleteApplications(List(h44, i20, i30, h33)))
+ }
+
+ test("findIncomplete-2") {
+ assert(Nil === findIncompleteApplications(Nil))
+ }
+
+ test("findIncomplete-3") {
+ assert(Nil === findIncompleteApplications(List(h44, h33)))
+ }
+
+ test("countIncomplete") {
+ assert(2 === countIncompleteApplications(List(h44, i20, i30, h33)))
+ }
+
+ test("countIncomplete-2") {
+ assert(0 === countIncompleteApplications(Nil))
+ }
+
+ test("countIncomplete-3") {
+ assert(0 === countIncompleteApplications(List(h44, h33)))
+ }
+
+ test("findStartOfWindow") {
+ assert(Some(i20) === findStartOfWindow(List(h44, i20, i30, h33)))
+ }
+
+ test("findStartOfWindow-2") {
+ assert(Some(h44) === findStartOfWindow(List(h44, h12, h33)))
+ }
+
+ test("combineResults-list-nil") {
+ assert((h44 :: Nil) === combineResults(List(h44), Nil))
+ }
+
+ test("combineResults-2-Nil-list") {
+ assert((h44 :: Nil) === combineResults(Nil, List(h44)))
+ }
+
+ test("combineResults-3-Nil-lists") {
+ assert(Nil === combineResults(Nil, Nil))
+ }
+
+ test("combineResults-5") {
+ assert((h44 :: i20 :: Nil) === combineResults(List(h44), List(i20)))
+ }
+
+ test("combineResults-6-merge-duplicate-to-one") {
+ assert(List(h44) === combineResults(List(h44), List(h44)))
+ }
+
+ test("combineResults-7-completed") {
+ assert(List(iA11_completed) === combineResults(List(iA10_incomplete), List(iA11_completed)))
+ }
+
+ test("merge-multiple_attempts") {
+ assert(List(historyInfo(a1_attempt_1, a1_attempt_2.attempts ++ a1_attempt_1.attempts))
+ === combineResults(List(a1_attempt_1), List(a1_attempt_2)))
+ }
+
+ test("SortApplications-1") {
+ assert((h33 :: h44 :: Nil) === sortApplicationsByStartTime(List(h44, h33)))
+ }
+
+ test("SortApplications-2") {
+ assert((h22 :: i20 :: h33 :: Nil) === sortApplicationsByStartTime(List(h22, i20, h33)))
+ }
+
+ test("SortApplications-3") {
+ assert((i20 :: h22 :: Nil) === sortApplicationsByStartTime(List(i20, h22)))
+ }
+
+ test("findLatest") {
+ assert(Some(h22) === findLatestApplication(List(h12, h22, i20)))
+ }
+
+ test("findLatest-2") {
+ assert(Some(h22) === findLatestApplication(List(h22, i20)))
+ }
+
+ test("findLatest-3") {
+ assert(Some(i20) === findLatestApplication(List(h12, i20)))
+ }
+
+ test("buildEntityIds") {
+ val sparkAppId = "spark_app_id_2"
+ val attempt = "attempt_id"
+ val yarnAppStr = yarnAppId.toString
+ val attemptId = Some(yarnAttemptId)
+ val attemptIdStr = yarnAttemptId.toString
+ assert(attemptIdStr === buildEntityId(yarnAppId, attemptId),
+ "all fields")
+ assert(attemptIdStr === buildEntityId(yarnAppId, attemptId),
+ "no attempt ID")
+ assert(yarnAppStr === buildEntityId(yarnAppId, None), "yarnAppId only")
+ }
+
+ test("buildApplicationAttemptIdField") {
+ val sparkAppId = "spark_app_id_2"
+ val attempt = "attempt_id"
+ assert(attempt === buildApplicationAttemptIdField(Some(attempt)),
+ "all fields")
+ assert(SINGLE_ATTEMPT === buildApplicationAttemptIdField(None),
+ "attempt = None")
+ }
+
+ test("EntityAndBack") {
+ val sparkAppId = Some("spark-app-id-1")
+ val yarnAppStr = yarnAppId.toString
+ val sparkAttemptId = Some("spark-attempt-id")
+ val yarnAttemptIdStr = yarnAttemptId.toString
+
+ val entity = createTimelineEntity(yarnAppId,
+ Some(yarnAttemptId),
+ sparkAppId,
+ sparkAttemptId,
+ "app",
+ "user",
+ 1000, 0, 1000)
+ val entityDescription = describeEntity(entity)
+ val ev1 = entity.getOtherInfo.get(FIELD_ENTITY_VERSION)
+ val version = numberField(entity, FIELD_ENTITY_VERSION, -1).longValue()
+ assert (0 < version, s"wrong version in $entityDescription")
+
+ // build an TimelineApplicationHistoryInfo instance
+ val info = toApplicationHistoryInfo(entity)
+ assert(yarnAppStr === info.id, "info.id")
+ val attempt = info.attempts.head
+ assert(sparkAttemptId === attempt.attemptId, s"attempt.attemptId in $attempt")
+ assert(yarnAttemptIdStr === attempt.entityId, s"attempt.entityId in $attempt")
+ assert(version === attempt.version, s"version in $attempt")
+ }
+
+ test("EntityWithoutAttempt") {
+ val sparkAppId = Some("spark-app-id-1")
+ val yarnAppStr = yarnAppId.toString
+ val yarnAttemptIdStr = yarnAttemptId.toString
+
+ val entity = createTimelineEntity(yarnAppId,
+ None,
+ sparkAppId,
+ None,
+ "app",
+ "user",
+ 1000, 0, 1000)
+ val info = toApplicationHistoryInfo(entity)
+ assert(yarnAppStr === info.id)
+
+ val attempt = info.attempts.head
+ assert("1" === attempt.attemptId.get, "attempt.attemptId")
+ assert(yarnAppStr === attempt.entityId, "attempt.entityId")
+ }
+
+ test("MergeHistoryEvents") {
+ val one_1 = new TimelineApplicationHistoryInfo("app1", "one", attempt_1_1 :: Nil)
+ val one_2 = new TimelineApplicationHistoryInfo("app1", "one", attempt_1_2 :: Nil)
+ val one_0 = new TimelineApplicationHistoryInfo("app1", "one", Nil)
+
+ val merge_12 = mergeAttempts(one_1, one_2)
+ assertListSize( merge_12.attempts, 2, "merged attempt list")
+ assert(List(attempt_1_2, attempt_1_1) === merge_12.attempts)
+
+ val merge_10 = mergeAttempts(one_1, one_0)
+ assert(1 === merge_10.attempts.size)
+ assert(one_1 === mergeAttempts(one_1, one_1))
+ }
+
+ test("MergeHistoryEventsIdNone") {
+ val one_1 = new TimelineApplicationHistoryInfo("1", "one", none_incomplete :: Nil)
+ val one_2 = new TimelineApplicationHistoryInfo("1", "one", none_completed :: Nil)
+ val one_0 = new TimelineApplicationHistoryInfo("1", "one", Nil)
+ }
+
+ test("merge-results-None-attemptId-incomplete-first") {
+ assert(List(none_completed) === mergeAttemptInfoLists(List(none_incomplete),
+ List(none_completed)))
+ }
+
+ test("merge-results-None-attemptId-incomplete-second") {
+ // and in the other order
+ assert(List(none_completed) === mergeAttemptInfoLists(List(none_completed),
+ List(none_incomplete)))
+ }
+
+ test("MergeAttemptOrdering-1") {
+ assert(none_completed === mostRecentAttempt(none_completed, none_incomplete))
+ }
+
+ test("MergeAttemptOrdering-2") {
+ assert(none_completed === mostRecentAttempt(none_incomplete, none_completed))
+ }
+
+ test("MergeAttemptOrdering-3") {
+ assert(none_completed === mostRecentAttempt(none_incomplete, none_completed))
+ }
+
+ test("MergeAttemptOrdering-4") {
+ assert(attempt_1_1_updated === mostRecentAttempt(attempt_1_1, attempt_1_1_updated))
+ }
+
+ test("MergeAttemptOrdering-5") {
+ assert(attempt_1_1_updated === mostRecentAttempt(attempt_1_1_updated, attempt_1_1))
+ }
+
+ test("MergeAttemptOrdering-6") {
+ assert(none_completed_orig_time ===
+ mostRecentAttempt(none_incomplete, none_completed_orig_time))
+ }
+
+ test("MergeAttemptOrdering-7") {
+ assert(none_completed_orig_time ===
+ mostRecentAttempt(none_completed_orig_time, none_incomplete))
+ }
+
+ test("MergeAttemptOrdering-8") {
+ assert(attempt_1_1_updated_version === mostRecentAttempt(attempt_1_1,
+ attempt_1_1_updated_version))
+ }
+
+ test("MergeAttemptOrdering-9") {
+ assert(attempt_1_1_updated_version === mostRecentAttempt(attempt_1_1_updated_version,
+ attempt_1_1))
+ }
+
+ test("MergeAttemptOrdering-10") {
+ assert(attempt_1_1_completed_v4 === mostRecentAttempt(attempt_1_1_updated_version,
+ attempt_1_1_completed_v4))
+ }
+
+ test("MergeAttemptOrdering-11") {
+ assert(attempt_1_1_completed_v4 === mostRecentAttempt(attempt_1_1_completed_v4,
+ attempt_1_1_updated_version))
+ }
+
+ test("SortAttempts-by-version") {
+ assert((i20 :: h22 :: Nil) === sortApplicationsByStartTime(List(i20, h22)))
+ }
+
+ test("merge-results-updated-first") {
+ // and in the other order
+ assert(List(attempt_1_1_updated_version) ===
+ mergeAttemptInfoLists(List(attempt_1_1_updated_version), List(attempt_1_1)))
+ }
+
+ test("merge-results-updated-second") {
+ // and in the other order
+ assert(List(attempt_1_1_updated_version) ===
+ mergeAttemptInfoLists(List(attempt_1_1), List(attempt_1_1_updated_version)))
+ }
+
+ test("merge-results-v2_v3") {
+ // and in the other order
+ assert(List(attempt_1_2, attempt_1_1_updated_version) ===
+ mergeAttemptInfoLists(List(attempt_1_2), List(attempt_1_1_updated_version)))
+ }
+
+ test("merge-results-v1_v2_v3") {
+ // and in the other order
+ assert(List(attempt_1_2, attempt_1_1_updated_version) ===
+ mergeAttemptInfoLists(List(attempt_1_2, attempt_1_1),
+ List(attempt_1_1_updated_version)))
+ }
+
+
+ test("merge-multiple-attempts-by-version") {
+
+ val app2 = new TimelineApplicationHistoryInfo("app2", "app2", List(attempt_1_1))
+
+ val histories1 = combineResults(Nil, List(app2))
+ val app2_updated = new TimelineApplicationHistoryInfo("app2", "app2",
+ List(attempt_1_1_completed_v4))
+
+ val histories2 = combineResults(Nil, List(app2_updated))
+ val merged = combineResults(histories1, histories2)
+ merged should have size 1
+ val finalApp = merged.head
+ val finalAppDescription = describeApplicationHistoryInfo(finalApp)
+ val finalAttempt = finalApp.attempts.head
+ assert(finalAttempt.completed, s"not completed $finalAppDescription")
+ assert(List(historyInfo(a1_attempt_1, a1_attempt_2.attempts ++ a1_attempt_1.attempts))
+ === combineResults(List(a1_attempt_1), List(a1_attempt_2)))
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/publish/ContextEventPublishingSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/publish/ContextEventPublishingSuite.scala
new file mode 100644
index 000000000000..4cdf4085b2ce
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/publish/ContextEventPublishingSuite.scala
@@ -0,0 +1,67 @@
+/*
+ * 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.yarn.publish
+
+import java.io.File
+
+import org.apache.spark.deploy.history.yarn.integration.AbstractHistoryIntegrationTests
+import org.apache.spark.deploy.history.yarn.testtools.{HistoryServiceListeningToSparkContext, TimelineSingleEntryBatchSize}
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+/**
+ * hookup from spark context to timeline.
+ */
+class ContextEventPublishingSuite
+ extends AbstractHistoryIntegrationTests
+ with HistoryServiceListeningToSparkContext
+ with TimelineSingleEntryBatchSize {
+
+ test ("Publish events via Context") {
+ describe("Publish events via Context")
+ // hook up to spark context
+ historyService = startHistoryService(sc)
+ assert(historyService.listening, s"listening $historyService")
+ assert(1 === historyService.batchSize, s"batch size in $historyService")
+ assert(historyService.bondedToATS, s"not bonded to ATS: $historyService")
+ // post in an app start
+ var flushes = 0
+ logDebug("posting app start")
+ enqueue(appStartEvent())
+ flushes += 1
+ awaitEmptyQueue(historyService, TEST_STARTUP_DELAY)
+
+ // add a local file to the context, so generate another event
+ val tempFile = File.createTempFile("test", ".txt")
+ sc.addFile(tempFile.toURI.toString)
+ tempFile.delete()
+ flushes += 1
+
+ // closing context generates an application stop
+ logDebug("stopping context")
+ sc.stop()
+ awaitEmptyQueue(historyService, TEST_STARTUP_DELAY)
+
+ flushes += 1
+ logDebug(s"status: $historyService")
+
+ // seeing intermittent failure
+ awaitFlushCount(historyService, flushes, TEST_STARTUP_DELAY)
+ flushHistoryServiceToSuccess()
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/publish/TimelinePostSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/publish/TimelinePostSuite.scala
new file mode 100644
index 000000000000..a9e6c4e1d6bc
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/publish/TimelinePostSuite.scala
@@ -0,0 +1,40 @@
+/*
+ * 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.yarn.publish
+
+import org.apache.spark.deploy.history.yarn.YarnEventListener
+import org.apache.spark.deploy.history.yarn.integration.AbstractHistoryIntegrationTests
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+class TimelinePostSuite extends AbstractHistoryIntegrationTests {
+
+ test("Round Trip App Stop") {
+ historyService = startHistoryService(sc)
+ val outcome = postEvent(appStartEvent(), now())
+ historyService.stop()
+ awaitEmptyQueue(historyService, TEST_STARTUP_DELAY)
+ }
+
+ test("App Start Via Event Listener") {
+ historyService = startHistoryService(sc)
+ val listener = new YarnEventListener(sc, historyService)
+ val sparkEvt = appStartEvent()
+ listener.onApplicationStart(sparkEvt)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/AbstractYarnHistoryTests.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/AbstractYarnHistoryTests.scala
new file mode 100644
index 000000000000..d0fc8048c07e
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/AbstractYarnHistoryTests.scala
@@ -0,0 +1,133 @@
+/*
+ * 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.yarn.testtools
+
+import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
+import org.scalatest.{BeforeAndAfter, Matchers}
+
+import org.apache.spark.{LocalSparkContext, Logging, SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.deploy.history.yarn.{YarnEventListener, YarnHistoryService}
+import org.apache.spark.scheduler.cluster.SchedulerExtensionServiceBinding
+
+/**
+ * This is a base class for the YARN history test suites, both mock and integration
+ *
+ * Subclasses are expected to use traits and/or overriding of
+ * [[ContextSetup.setupConfiguration()]]
+ * to tune the configuration of the instantiated context.
+ *
+ * To avoid any ambiguity about the ordering/invocation of
+ * any before/after code, the operations are passed to
+ * overriddeable `setup()` and `teardown()`
+ * invocations. Subclasses must relay the method calls to their
+ * superclasses to ensure correct setup and teardown.
+ */
+abstract class AbstractYarnHistoryTests
+ extends SparkFunSuite
+ with TimelineOptionsInContext
+ with TimelineServiceDisabled
+ with HistoryServiceNotListeningToSparkContext
+ with LocalSparkContext
+ with BeforeAndAfter
+ with Logging
+ with ExtraAssertions
+ with Matchers {
+
+ /*
+ * Setup phase creates the spark context, and anything else which tests require
+ */
+ before {
+ setup()
+ }
+
+ /*
+ * Setup creates the spark context
+ */
+ protected def setup(): Unit = {
+ val sparkConf = new SparkConf()
+
+ if (sc != null) {
+ fail("Spark Context is not null -a previous test did not clean up properly")
+ }
+
+ sc = createSparkContext(sparkConf)
+ logDebug(s"Created context $sc")
+ }
+
+ /**
+ * Create the spark context
+ * @param sparkConf configuration to extend
+ */
+ protected def createSparkContext(sparkConf: SparkConf): SparkContext = {
+ sparkConf.setMaster("local").setAppName("AbstractYarnHistoryTests")
+ logInfo("Creating a new spark context")
+ new SparkContext(setupConfiguration(sparkConf))
+ }
+
+ /**
+ * Create and start a history service.
+ *
+ * @param sc context
+ * @param id application ID
+ * @param appAttemptId optional attempt ID
+ * @return the instantiated service
+ */
+ protected def startHistoryService(
+ sc: SparkContext,
+ id: ApplicationId = applicationId,
+ appAttemptId: Option[ApplicationAttemptId] = Some(attemptId)): YarnHistoryService = {
+ assertNotNull(sc, "Spark context")
+ val service = new YarnHistoryService()
+ service.start(SchedulerExtensionServiceBinding(sc, id, appAttemptId))
+ assert(YarnHistoryService.StartedState === service.serviceState, s"wrong state: $service")
+ service
+ }
+
+ /**
+ * Create a history service, post an event sequence, then stop the service.
+ * The @ of attempts posted is returned in the response to stop tests being brittle
+ * against the numbe of events posted automatically. This is the code which posts the events
+ * -it's the code that can reliably announce this.
+ * @param sc context
+ * @return (the (now closed) history service, the count of events posted for use in assertions)
+ */
+ def postEvents(sc: SparkContext): (YarnHistoryService, Int) = {
+ val service: YarnHistoryService = startHistoryService(sc)
+ val listener = new YarnEventListener(sc, service)
+ var eventsPosted = 0
+ try {
+ listener.onApplicationStart(applicationStart)
+ listener.onEnvironmentUpdate(environmentUpdate)
+ listener.onApplicationEnd(applicationEnd)
+ eventsPosted += 3
+ // wait for them all to be processed
+ awaitEventsProcessed(service, eventsPosted, TEST_STARTUP_DELAY)
+ // only 2 post events, as the update does not trigger a post unless batch size == 1
+ awaitAtLeast(2, TEST_STARTUP_DELAY,
+ () => service.postAttempts,
+ () => s"Post count in $service")
+ } finally {
+ // because the events have been processed and waited for,
+ // a duplicate end event will not be posted
+ service.stop()
+ }
+ (service, eventsPosted)
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/ContextSetup.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/ContextSetup.scala
new file mode 100644
index 000000000000..cbfc208b7e77
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/ContextSetup.scala
@@ -0,0 +1,91 @@
+/*
+ * 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.yarn.testtools
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.YarnHistoryService
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+/**
+ * Trait implemented by everything setting up a context; the model is that
+ * the traits can be chained, with the final state determined by the order
+ *
+ * 1. base implementation does nothing.
+ * 2. subclass traits are expected to call the superclass first, then
+ * apply their own options.
+ */
+trait ContextSetup {
+
+ def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ sparkConf
+ }
+}
+
+trait TimelineServiceEnabled extends ContextSetup {
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ hadoopOpt(super.setupConfiguration(sparkConf),
+ YarnConfiguration.TIMELINE_SERVICE_ENABLED, "true")
+ }
+}
+
+trait TimelineServiceDisabled extends ContextSetup {
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ hadoopOpt(super.setupConfiguration(sparkConf),
+ YarnConfiguration.TIMELINE_SERVICE_ENABLED, "false")
+ }
+}
+
+/**
+ * Add the timeline options
+ */
+trait TimelineOptionsInContext extends ContextSetup {
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ YarnTestUtils.addBasicTimelineOptions(super.setupConfiguration(sparkConf))
+ }
+}
+
+/**
+ * request that created history services register with the spark context for lifecycle events
+ */
+trait HistoryServiceListeningToSparkContext extends ContextSetup {
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ super.setupConfiguration(sparkConf).set(YarnHistoryService.REGISTER_LISTENER, "true")
+ }
+}
+
+/**
+ * request that created history services are not registered with the spark context for
+ * lifecycle events
+ */
+trait HistoryServiceNotListeningToSparkContext extends ContextSetup {
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ super.setupConfiguration(sparkConf).set(YarnHistoryService.REGISTER_LISTENER, "false")
+ }
+}
+
+/**
+ * Switch to single entry batch sizes
+ */
+trait TimelineSingleEntryBatchSize extends ContextSetup {
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ super.setupConfiguration(sparkConf).set(YarnHistoryService.BATCH_SIZE, "1")
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/ExtraAssertions.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/ExtraAssertions.scala
new file mode 100644
index 000000000000..e012694a0733
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/ExtraAssertions.scala
@@ -0,0 +1,247 @@
+/*
+ * 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.yarn.testtools
+
+import java.util.{Collection => JCollection}
+
+import org.apache.hadoop.service.Service
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity
+import org.scalatest.Assertions
+
+import org.apache.spark.Logging
+import org.apache.spark.deploy.history.yarn.YarnHistoryService
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+
+/**
+ * Miscellaneous extra assertions
+ */
+trait ExtraAssertions extends Logging with Assertions {
+
+ /**
+ * Assert that an exception's toString value contains the supplied text.
+ * If not, an error is logged and the exception is rethrown
+ *
+ * @param ex exception
+ * @param text text
+ */
+ def assertExceptionMessageContains(ex: Exception, text: String): Unit = {
+ if (!ex.toString.contains(text)) {
+ logError(s"Did not find text $text in $ex", ex)
+ throw ex
+ }
+ }
+
+ /**
+ * Check the exception message text and toString check
+ *
+ * @param ex exception to examine
+ * @param messageCheck string to check in the `Exception.getMessage()` string
+ * @param stringCheck string to look for in the `Exception.toString()` string;
+ * if empty the check is skipped.
+ */
+ def assertExceptionDetails(ex: Throwable, messageCheck: String, stringCheck: String): Unit = {
+ assertNotNull(ex.getMessage, s"Exception message in $ex")
+ if (!ex.getMessage.contains(messageCheck)) {
+ throw ex
+ }
+ if (!stringCheck.isEmpty && !ex.toString.contains(stringCheck)) {
+ throw ex
+ }
+ }
+
+ /**
+ * Assert that a value is not null
+ *
+ * @param value value
+ * @param text text for assertion
+ */
+ def assertNotNull(value: Any, text: String): Unit = {
+ assert(value !== null, s"Null value; $text")
+ }
+
+ /**
+ * Assert that an optional value is not `None`.
+ *
+ * @param value value
+ * @param text text for assertion
+ */
+ def assertSome(value: Option[Any], text: String): Unit = {
+ assert(value.nonEmpty, s"optional value is None; $text")
+ }
+
+ /**
+ * Assert that an optional value is `None`
+ *
+ * @param value value
+ * @param text text for assertion
+ */
+ def assertNone(value: Option[Any], text: String): Unit = {
+ assert(value.isEmpty, s"Optional value is $value.get; $text")
+ }
+
+ /**
+ * Assert that a Spark traversable instance is not empty
+ *
+ * @param traversable the traversable to check
+ * @param text text for assertion
+ * @tparam T traversable type
+ */
+ def assertNotEmpty[T](traversable: Traversable[T], text: String): Unit = {
+ assert(traversable.nonEmpty, s"Empty traversable; $text")
+ }
+
+ /**
+ * Assert that a java collection is not empty
+ *
+ * @param collection the collection to check
+ * @param text text for assertion
+ * @tparam T collection type
+ */
+ def assertNotEmpty[T](collection: JCollection[T], text: String): Unit = {
+ assert (!collection.isEmpty, s"Empty collection; $text")
+ }
+
+ /**
+ * Assert the list is of the given size. if not all elements are logged @ error,
+ * then the method raises a failure.
+ *
+ * @param list list to examine
+ * @param expectedSize expected size
+ * @param message error message
+ * @tparam T list type
+ */
+ def assertListSize[T](list: Seq[T], expectedSize: Int, message: String): Unit = {
+ assertNotNull(list, message)
+ if (list.size != expectedSize) {
+ // no match
+ val errorText = s"Wrong list size: expected=$expectedSize actual=${list.size}: $message"
+ logError(errorText)
+ list.foreach { e => logError(e.toString) }
+ fail(errorText)
+ }
+ }
+
+ /**
+ * Assert that a list is Nil (and implicitly, not null)
+ *
+ * If not, an assertion is raised that contains the message and the list
+ * @param list list to check
+ * @param message message to raise
+ * @tparam T list type
+ */
+ def assertNil[T](list: Seq[T], message: String): Unit = {
+ assertNotNull(list, message)
+ if (list != Nil) {
+ fail(message + " " + list)
+ }
+ }
+
+ /**
+ * Assert that a service is not listening
+ *
+ * @param historyService history service
+ */
+ def assertNotListening(historyService: YarnHistoryService): Unit = {
+ assert(!historyService.listening, s"history service is listening for events: $historyService")
+ }
+
+ /**
+ * Assert that the number of events processed matches the number expected
+ *
+ * @param historyService history service
+ * @param expected expected number
+ * @param details text to include in error messages
+ */
+ def assertEventsProcessed(historyService: YarnHistoryService,
+ expected: Int, details: String): Unit = {
+ assertResult(expected, "wrong number of events processed " + details) {
+ historyService.eventsProcessed
+ }
+ }
+
+ /**
+ * Assert that two timeline entities are non-null and equal
+ *
+ * @param expected expected entry
+ * @param actual actual
+ */
+ def assertEntitiesEqual(expected: TimelineEntity, actual: TimelineEntity): Unit = {
+ require(expected != null)
+ require(actual != null)
+ assert(expected === actual,
+ s"Expected ${describeEntity(expected)}; got ${describeEntity(actual)}}")
+ }
+
+ /**
+ * Assert that a service is in a specific state
+ *
+ * @param service service
+ * @param state required state
+ */
+ def assertInState(service: Service, state: Service.STATE): Unit = {
+ assertNotNull(service, "null service")
+ assert(service.isInState(state), s"not in state $state: $service")
+ }
+
+ /**
+ * Assert that a source string contains the `contained` substring.
+ * (This is not a check for a proper subset; equality is also acceptable)
+ * @param source source string
+ * @param contained string to look for
+ */
+ def assertContains(source: String, contained: String, text: String = ""): Unit = {
+ assertNotNull(source, s"$text null source")
+ assertNotNull(contained, s"$text null `contained`")
+ if (!source.contains(contained)) {
+ fail(s"$text -Did not find '$contained' in '$source'")
+ }
+ }
+
+ /**
+ * Assert that a source string does contains the `contained` substring.
+ * @param source source string
+ * @param contained string to look for
+ */
+ def assertDoesNotContain(source: String, contained: String, text: String = ""): Unit = {
+ assertNotNull(source, s"$text null source")
+ assertNotNull(contained, s"$text null `contained`")
+ if (source.contains(contained)) {
+ fail(s"$text -Found '$contained' in '$source'")
+ }
+ }
+
+ /**
+ * Assert that a `[String, String]` map contains a `key:value` mapping,
+ * and that the value contains the specified text.
+ * @param map map to query
+ * @param key key to retrieve
+ * @param text text to look for in the resolved value
+ */
+ protected def assertMapValueContains(map: Map[String, String],
+ key: String, text: String): Unit = {
+ map.get(key) match {
+ case Some(s) =>
+ if (!text.isEmpty && !s.contains(text)) {
+ fail(s"Did not find '$text' in key[$key] = '$s'")
+ }
+
+ case None =>
+ fail(s"No entry for key $key")
+ }
+ }
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/FreePortFinder.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/FreePortFinder.scala
new file mode 100644
index 000000000000..7174f4bbcbc5
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/FreePortFinder.scala
@@ -0,0 +1,101 @@
+/*
+ * 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.yarn.testtools
+
+import java.net.{InetAddress, ServerSocket}
+
+import scala.collection.mutable.ListBuffer
+
+/**
+ * Trait to find free ports on localhost
+ */
+trait FreePortFinder {
+
+ /**
+ * Find a free port by listening on port 0
+ * @return
+ */
+ def findPort(): Int = {
+ tryToListen(0)._2
+ }
+
+ /**
+ * Simple function to see if a port is free; if it is return the address and the port allocated.
+ *
+ * This function can be passed to `Util.startServiceOnPort`
+ * @param port port to try. If 0, the OS chooses the port
+ * @return an (address, port) tuple
+ */
+ def tryToListen(port: Int): (InetAddress, Int) = {
+ val socket = new ServerSocket(port)
+ val address = socket.getInetAddress
+ val localPort = socket.getLocalPort
+ socket.close()
+ (address, localPort)
+ }
+
+ /**
+ * Return the value of the local host address -defaults to 127.0.0.1
+ * @return the address to use for local/loopback addresses.
+ */
+ def localIPv4Address(): String = {
+ "127.0.0.1"
+ }
+
+ /**
+ * Get a local address as an address:port string and an integer port value
+ * @return a free port to bind to
+ */
+ def findIPv4AddressAsPortPair(): (String, Int) = {
+ val port = findPort()
+ (localhostAndPort(port), port)
+ }
+
+ /**
+ * Given a port, return a localhost:port pair
+ * @param port port
+ * @return the name for the localhost for test runs.
+ */
+ def localhostAndPort(port: Int): String = {
+ localIPv4Address() + ":" + port
+ }
+
+ /**
+ * Find the specified number of unique ports. This is done in parallel, so the
+ * ports are guaranteed to be different. The ports are all closed afterwards,
+ * so other network services started may grab those same ports.
+ *
+ * @param count number of ports to find.
+ * @return a list of ports to use.
+ */
+ def findUniquePorts(count: Integer): Seq[Integer] = {
+ val sockets = new ListBuffer[ServerSocket]()
+ val ports = new ListBuffer[Integer]()
+ for (i <- 1 to count) {
+ val socket = new ServerSocket(0)
+ sockets += socket
+ ports += socket.getLocalPort
+ }
+ sockets.foreach(_.close())
+ // Uniqueness: foreach port, there only exists one port in the list which has that value
+ require(ports.map( p => ports.count(_ == p) == 1).count(_ == true) == count,
+ s"Duplicate port allocation in " + ports.mkString(","))
+
+ ports
+ }
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/YarnTestUtils.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/YarnTestUtils.scala
new file mode 100644
index 000000000000..8ee3c3a0bf5f
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/testtools/YarnTestUtils.scala
@@ -0,0 +1,594 @@
+/*
+ * 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.yarn.testtools
+
+import java.io.IOException
+import java.net.URL
+
+import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore
+
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.{YarnHistoryService, YarnTimelineUtils}
+import org.apache.spark.scheduler.cluster.{StubApplicationAttemptId, StubApplicationId}
+import org.apache.spark.scheduler.{JobFailed, JobSucceeded, SparkListenerApplicationEnd, SparkListenerApplicationStart, SparkListenerEnvironmentUpdate, SparkListenerEvent, SparkListenerJobEnd, SparkListenerJobStart}
+import org.apache.spark.util.Utils
+
+object YarnTestUtils extends ExtraAssertions with FreePortFinder {
+
+ val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]](
+ "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")),
+ "Spark Properties" -> Seq(("Job throughput", "80000 jobs/s, regardless of job type")),
+ "System Properties" -> Seq(("Username", "guest"), ("Password", "guest")),
+ "Classpath Entries" -> Seq(("Super library", "/tmp/super_library"))))
+
+ /**
+ * Application name used in the app start event and tests
+ */
+ val APP_NAME = "spark-demo"
+
+ /**
+ * User submitting the job
+ */
+ val APP_USER = "data-scientist"
+
+ /**
+ * application ID
+ */
+ val APP_ID = "application_id_0001"
+
+ /**
+ * Spark option to set for the history provider
+ */
+ val SPARK_HISTORY_PROVIDER = "spark.history.provider"
+
+ /**
+ * Constant used to define history port in Spark `HistoryServer` class
+ */
+ val SPARK_HISTORY_UI_PORT = "spark.history.ui.port"
+
+ val completedJobsMarker = "Completed Jobs (1)"
+ val activeJobsMarker = "Active Jobs (1)"
+
+
+ /**
+ * Time to wait for anything to start/state to be reached
+ */
+ val TEST_STARTUP_DELAY = 5000
+
+ /** probes during service shutdown need to handle delayed posting */
+ val SERVICE_SHUTDOWN_DELAY = 10000
+
+ /**
+ * Cancel a test if the network isn't there.
+ *
+ * If called during setup, this will abort the test
+ */
+ def cancelIfOffline(): Unit = {
+
+ try {
+ val hostname = Utils.localHostName()
+ log.debug(s"local hostname is $hostname")
+ } catch {
+ case ex: IOException =>
+ cancel(s"Localhost name not known: $ex", ex)
+ }
+ }
+
+ /**
+ * Return a time value
+ *
+ * @return the current time in milliseconds
+ */
+ def now(): Long = {
+ System.currentTimeMillis()
+ }
+
+ /**
+ * Get a time in the future
+ *
+ * @param millis future time in millis
+ * @return now + the time offset
+ */
+ def future(millis: Long): Long = {
+ now() + millis
+ }
+
+ /**
+ * Log an entry with a line either side. This aids splitting up tests from the noisy logs
+ *
+ * @param text text to log
+ */
+ def describe(text: String): Unit = {
+ logInfo(s"\nTest:\n $text\n\n")
+ }
+
+ /**
+ * Set a hadoop opt in the config.
+ *
+ * This adds the `"spark.hadoop."` prefix to all entries which do not already have it
+ *
+ * @param sparkConfig target configuration
+ * @param key hadoop option key
+ * @param value value
+ */
+ def hadoopOpt(sparkConfig: SparkConf, key: String, value: String): SparkConf = {
+ if (key.startsWith("spark.hadoop.")) {
+ sparkConfig.set(key, value)
+ } else {
+ sparkConfig.set("spark.hadoop." + key, value)
+ }
+ }
+
+ /**
+ * Bulk set of an entire map of Hadoop options
+ *
+ * @param sparkConfig target configuration
+ * @param options option map
+ */
+ def applyHadoopOptions(sparkConfig: SparkConf, options: Map[String, String]): SparkConf = {
+ options.foreach( e => hadoopOpt(sparkConfig, e._1, e._2))
+ sparkConfig
+ }
+
+ /**
+ * Apply the basic timeline options to the hadoop config
+ *
+ * @return the modified config
+ */
+ def addBasicTimelineOptions(sparkConf: SparkConf): SparkConf = {
+ val ports = findUniquePorts(3)
+ applyHadoopOptions(sparkConf,
+ Map(YarnConfiguration.TIMELINE_SERVICE_ENABLED -> "true",
+ YarnConfiguration.TIMELINE_SERVICE_ADDRESS -> localhostAndPort(ports.head),
+ YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS -> localhostAndPort(ports(1)),
+ YarnConfiguration.TIMELINE_SERVICE_STORE -> classOf[MemoryTimelineStore].getName,
+ YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES -> "1",
+ YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS -> "200"))
+
+ // final port in the set
+ sparkConf.set(SPARK_HISTORY_UI_PORT, ports(2).toString)
+
+ // turn off the minimum refresh interval.
+ // uses a string to it can be set from code that doesn't refer to any provider-side
+ // classes
+ sparkConf.set("spark.history.yarn.min-refresh-interval", "0")
+
+ // shorter reset interval and shutdown time
+ sparkConf.set(POST_RETRY_INTERVAL, "10ms")
+ sparkConf.set(SHUTDOWN_WAIT_TIME, "5s")
+ }
+
+ /**
+ * Convert the single timeline event in a timeline entity to a spark event
+ *
+ * @param entity entity to convert, which must contain exactly one event.
+ * @return the converted event
+ */
+ def convertToSparkEvent(entity: TimelineEntity): SparkListenerEvent = {
+ assertResult(1, "-wrong # of events in the timeline entry") {
+ entity.getEvents().size()
+ }
+ YarnTimelineUtils.toSparkEvent(entity.getEvents().get(0))
+ }
+
+ /**
+ * Create an app start event, using the fixed [[APP_NAME]] and [[APP_USER]] values
+ * for appname and user; no attempt ID
+ *
+ * @param time application start time
+ * @param appId event ID; default is [[APP_ID]]
+ * @return the event
+ */
+ def appStartEventWithAttempt(time: Long = 1,
+ appId: String,
+ user: String,
+ attemptId: ApplicationAttemptId): SparkListenerApplicationStart = {
+ appStartEvent(time, appId, user, Some(attemptId.toString))
+ }
+
+ /**
+ * Create an app start event
+ *
+ * @param time application start time
+ * @param appId event ID; default is [[APP_ID]]
+ * @param user the user; defaults is [[APP_USER]]
+ * @param attempt attempt ID; default is `None`
+ * @return the event
+ */
+ def appStartEvent(time: Long = 1434920400000L,
+ appId: String = APP_ID,
+ user: String = APP_USER,
+ attempt: Option[String] = None,
+ name: String = APP_NAME): SparkListenerApplicationStart = {
+ require(name != null)
+ require(appId != null)
+ require(user != null)
+ SparkListenerApplicationStart(name, Some(appId), time, user, attempt)
+ }
+
+ def appStartEvent(time: Long, ctx: SparkContext): SparkListenerApplicationStart = {
+ appStartEvent(time, ctx.applicationId, ctx.sparkUser, ctx.applicationAttemptId)
+ }
+
+ def appStopEvent(time: Long = 1): SparkListenerApplicationEnd = {
+ new SparkListenerApplicationEnd(time)
+ }
+
+ def jobStartEvent(time: Long, id: Int) : SparkListenerJobStart = {
+ new SparkListenerJobStart(id, time, Nil, null)
+ }
+
+ def jobSuccessEvent(time: Long, id: Int) : SparkListenerJobEnd = {
+ new SparkListenerJobEnd(id, time, JobSucceeded)
+ }
+
+ def jobFailureEvent(time: Long, id: Int, ex: Exception) : SparkListenerJobEnd = {
+ new SparkListenerJobEnd(id, time, JobFailed(ex))
+ }
+
+ def newEntity(time: Long): TimelineEntity = {
+ val entity = new TimelineEntity
+ entity.setStartTime(time)
+ entity.setEntityId("post")
+ entity.setEntityType(YarnHistoryService.SPARK_EVENT_ENTITY_TYPE)
+ entity
+ }
+
+ val applicationId: ApplicationId = new StubApplicationId(0, 1111L)
+ val attemptId: ApplicationAttemptId = new StubApplicationAttemptId(applicationId, 1)
+ val applicationStart = appStartEventWithAttempt(now(), applicationId.toString, "bob", attemptId)
+ val applicationEnd = SparkListenerApplicationEnd(now() + 60000)
+
+ /**
+ * Outcomes of probes
+ */
+ sealed abstract class Outcome
+ case class Fail() extends Outcome
+ case class Retry() extends Outcome
+ case class Success() extends Outcome
+ case class TimedOut() extends Outcome
+
+ /**
+ * Spin and sleep awaiting an observable state.
+ *
+ * The scalatest `eventually` operator is similar, and even adds exponential backoff.
+ * What this offers is:
+ *
+ * 1. The ability of the probe to offer more than just success/fail, but a "fail fast"
+ * operation which stops spinning early.
+ * 2. A detailed operation to invoke on failure, so provide more diagnostics than
+ * just the assertion.
+ *
+ * @param interval sleep interval
+ * @param timeout time to wait
+ * @param probe probe to execute
+ * @param failure closure invoked on timeout/probe failure
+ */
+ def spinForState(description: String,
+ interval: Long,
+ timeout: Long,
+ probe: () => Outcome,
+ failure: (Outcome, Int, Boolean) => Unit): Unit = {
+ logInfo(description)
+ val timelimit = now() + timeout
+ var result: Outcome = Retry()
+ var current = 0L
+ var iterations = 0
+ do {
+ iterations += 1
+ result = probe()
+ if (result == Retry()) {
+ // probe says retry
+ current = now()
+ if (current> timelimit) {
+ // timeout, uprate to fail
+ result = TimedOut()
+ } else {
+ Thread.sleep(interval)
+ }
+ }
+ } while (result == Retry())
+ result match {
+ case Fail() => failure(result, iterations, false)
+ case TimedOut() => failure(result, iterations, true)
+ case _ =>
+ }
+ }
+
+ /**
+ * Convert a boolean into a success or retry outcome, that is:
+ * false is considered "retry", not a failure
+ *
+ * @param value value to probe
+ * @return
+ */
+ def outcomeFromBool(value: Boolean): Outcome = {
+ if (value) Success() else Retry()
+ }
+
+ /**
+ * From an increasing counter, compare the results and decide whether to succeed, fail or try
+ * again. Requires that if actual is greater than expected, it is a failed state.
+ *
+ * @param expected expected outcome
+ * @param actual actual value
+ */
+ def outcomeFromCounter(expected: Long, actual: Long): Outcome = {
+ if (expected == actual) {
+ Success()
+ } else if (actual < expected) {
+ Retry()
+ } else {
+ Fail()
+ }
+ }
+
+ /**
+ * From an increasing counter, compare the results and decide whether to succeed or try
+ * again. Any value equal to or greater than expected is a success. Ideal for waiting for
+ * asynchronous operations to complete
+ * @param expected expected outcome
+ * @param actual actual value
+ */
+ def outcomeAtLeast(expected: Long, actual: Long): Outcome = {
+ if (actual >= expected) Success() else Retry()
+ }
+
+ /**
+ * Curryable function to use for timeouts if something more specific is not needed
+ *
+ * @param text text mesage on timeouts
+ * @param iterations number of iterations performed
+ * @param timeout true if the event was a timeout (i.e. not a failure)
+ */
+ def timeout(text: String, iterations: Int, timeout: Boolean): Unit = {
+ fail(text)
+ }
+
+ /**
+ * a No-op on failure
+ *
+ * @param outcome outcome of the last operation
+ * @param iterations number of iterations performed
+ * @param timeout did the wait result in a timeout
+ */
+ def failure_noop(outcome: Outcome, iterations: Int, timeout: Boolean): Unit = {
+ }
+
+ /**
+ * Spin for the number of processed events to exactly match the supplied value.
+ *
+ * Fails if the timeout is exceeded
+ *
+ * @param historyService history
+ * @param expected exact number to await
+ * @param timeout timeout in millis
+ */
+ def awaitEventsProcessed(historyService: YarnHistoryService,
+ expected: Int, timeout: Long): Unit = {
+
+ def eventsProcessedCheck(): Outcome = {
+ outcomeFromCounter(expected, historyService.eventsProcessed)
+ }
+
+ def eventProcessFailure(outcome: Outcome, iterations: Int, timeout: Boolean): Unit = {
+ val eventsCount = historyService.eventsProcessed
+ val details = s"Expected $expected events" + s" actual=$eventsCount" +
+ s" after $iterations iterations; in $historyService"
+ if (timeout) {
+ val message = s"Timeout: $details"
+ logError(message)
+ fail(message)
+ } else if (expected != eventsCount) {
+ val message = s"event count mismatch; $details;"
+ logError(message)
+ fail(message)
+ fail(s"Expected $details")
+ }
+ }
+
+ spinForState("awaitEventsProcessed",
+ interval = 50,
+ timeout = timeout,
+ probe = eventsProcessedCheck,
+ failure = eventProcessFailure)
+ }
+
+ /**
+ * Spin awaiting a URL to be accessible. Useful to await a web application
+ * going live before running the tests against it
+ *
+ * @param url URL to probe
+ * @param timeout timeout in mils
+ */
+ def awaitURL(url: URL, timeout: Long): Unit = {
+ def probe(): Outcome = {
+ try {
+ url.openStream().close()
+ Success()
+ } catch {
+ case ioe: IOException => Retry()
+ }
+ }
+
+ /*
+ failure action is simply to attempt the connection without
+ catching the exception raised
+ */
+ def failure(outcome: Outcome, iterations: Int, timeout: Boolean): Unit = {
+ url.openStream().close()
+ }
+
+ spinForState(s"Awaiting a response from URL $url",
+ interval = 50, timeout = timeout, probe = probe, failure = failure)
+ }
+
+
+ /**
+ * Wait for a history service's queue to become empty
+ *
+ * @param historyService service
+ * @param timeout timeout
+ */
+ def awaitEmptyQueue(historyService: YarnHistoryService, timeout: Long): Unit = {
+
+ spinForState("awaiting empty queue",
+ interval = 50,
+ timeout = timeout,
+ probe = () => outcomeFromBool(historyService.postingQueueSize == 0),
+ failure = (_, _, _) => fail(s"queue never cleared after $timeout mS for $historyService"))
+ }
+
+ /**
+ * Await for the count of flushes in the history service to match the expected value
+ *
+ * @param historyService service
+ * @param count min number of flushes
+ * @param timeout timeout
+ */
+ def awaitFlushCount(historyService: YarnHistoryService, count: Long, timeout: Long): Unit = {
+ spinForState(s"awaiting flush count of $count",
+ interval = 50,
+ timeout = timeout,
+ probe = () => outcomeFromBool(historyService.getFlushCount >= count),
+ failure = (_, _, _) => fail(s"flush count not $count after $timeout mS in $historyService"))
+ }
+
+ /**
+ * Await the number of post events
+ * @param service service
+ * @param posts attempt count.
+ */
+ def awaitPostAttemptCount(service: YarnHistoryService, posts: Long): Unit = {
+ awaitCount(posts, TEST_STARTUP_DELAY,
+ () => service.postAttempts,
+ () => s"Post count in $service")
+ }
+
+ /**
+ * Await the number of post events
+ *
+ * @param service service
+ * @param posts attempt count.
+ */
+ def awaitPostSuccessCount(service: YarnHistoryService, posts: Long): Unit = {
+ awaitCount(posts, TEST_STARTUP_DELAY,
+ () => service.postSuccesses,
+ () => s"Post count in $service")
+ }
+
+ /**
+ * Await for the counter function to match the expected value
+ *
+ * @param expected desired count
+ * @param timeout timeout
+ * @param counter function to return an integer
+ * @param diagnostics diagnostics string evaluated on timeout
+ */
+ def awaitCount(expected: Long, timeout: Long,
+ counter: () => Long, diagnostics: () => String): Unit = {
+ spinForState(s"awaiting probe count of $expected",
+ 50, timeout,
+ () => outcomeFromCounter(expected, counter()),
+ (_, _, _) =>
+ fail(s"Expected $expected equalled ${counter()} after $timeout mS: ${diagnostics()}"))
+ }
+
+ /**
+ * Await for the counter function to match the expected value
+ *
+ * @param expected desired count
+ * @param timeout timeout
+ * @param counter function to return an integer
+ * @param diagnostics diagnostics string evaluated on timeout
+ */
+ def awaitAtLeast(expected: Long, timeout: Long,
+ counter: () => Long, diagnostics: () => String): Unit = {
+ spinForState(s"awaiting probe count of at least $expected",
+ 50, timeout,
+ () => outcomeAtLeast(expected, counter()),
+ (_, _, _) =>
+ fail(s"Expected >= $expected got ${counter()} after $timeout mS: ${diagnostics()}"))
+ }
+
+
+ /**
+ * Probe operation to wait for an empty queue
+ *
+ * @param historyService history service
+ * @param timeout timeout in milliseconds
+ * @param failOnTimeout flag -fail vs warn on timeout. Default: true
+ */
+ def awaitServiceThreadStopped(historyService: YarnHistoryService, timeout: Long,
+ failOnTimeout: Boolean = true): Unit = {
+ assertNotNull(historyService, "null historyService")
+ spinForState("awaitServiceThreadStopped",
+ interval = 50,
+ timeout = timeout,
+ probe = () => outcomeFromBool(!historyService.isPostThreadActive),
+ failure = (_, _, _) => if (failOnTimeout) {
+ fail(s"After $timeout mS, history service post thread did not finish:" +
+ s" $historyService")
+ } else {
+ logWarning(s"After $timeout mS, history service post thread did not finish:" +
+ s" $historyService")
+ })
+ }
+
+ /**
+ * Wait for a specified operation to return a list of the desired size
+ *
+ * @param expectedSize expected size of list
+ * @param message message on failure
+ * @param timeout timeout
+ * @param operation operation to create the list
+ * @tparam T list type
+ * @return the list created in the last successful operation
+ */
+ def awaitListSize[T](expectedSize: Int, message: String, timeout: Long,
+ operation: () => List[T]): List[T] = {
+ // last list fetched
+ var list: List[T] = Nil
+ def probeOperation(): Outcome = {
+ list = operation()
+ outcomeFromBool(list.size == expectedSize)
+ }
+ def failOperation(o: Outcome, i: Int, b: Boolean) = {
+ assertListSize(list, expectedSize, message)
+ }
+ spinForState(message, 50, timeout, probeOperation, failOperation)
+ list
+ }
+
+ /**
+ * Show a Spark context in a string form
+ * @param ctx context
+ * @return a string value for assertions and other messages
+ */
+ def asString(ctx: SparkContext): String = {
+ s"Spark Context ${ctx.appName} ID ${ctx.applicationId} attempts ${ctx.applicationAttemptId}"
+ }
+
+}
+
+
+
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/AbstractMockHistorySuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/AbstractMockHistorySuite.scala
new file mode 100644
index 000000000000..69a90865abc3
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/AbstractMockHistorySuite.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.yarn.unit
+
+import org.apache.hadoop.service.ServiceOperations
+import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
+import org.apache.hadoop.yarn.api.records.timeline.{TimelineEntity, TimelinePutResponse}
+import org.apache.hadoop.yarn.client.api.TimelineClient
+import org.mockito.Matchers._
+import org.mockito.Mockito._
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.SparkContext
+import org.apache.spark.deploy.history.yarn.YarnHistoryService
+import org.apache.spark.deploy.history.yarn.testtools.{AbstractYarnHistoryTests, TimelineServiceEnabled}
+import org.apache.spark.scheduler.cluster.SchedulerExtensionServiceBinding
+
+/**
+ * Mock histories have the timeline service enabled by default -it is
+ * a mock one though.
+ */
+class AbstractMockHistorySuite() extends AbstractYarnHistoryTests
+ with TimelineServiceEnabled with MockitoSugar {
+
+ protected var timelineClient: TimelineClient = _
+
+ protected var response: TimelinePutResponse = _
+
+ /**
+ * Set up the mock timeline client and response instances.
+ */
+ override protected def setup(): Unit = {
+ super.setup()
+ timelineClient = mock[TimelineClient]
+ response = mock[TimelinePutResponse]
+ when(timelineClient.putEntities(any(classOf[TimelineEntity]))).thenReturn(response)
+ }
+
+ override def afterEach(): Unit = {
+ ServiceOperations.stopQuietly(timelineClient)
+ super.afterEach()
+ }
+
+ /**
+ * Create and start a history service.
+ * @param sc context
+ * @param id application ID
+ * @param attemptId
+ * @return the instantiated service
+ */
+ override protected def startHistoryService(
+ sc: SparkContext,
+ id: ApplicationId,
+ attemptId: Option[ApplicationAttemptId] = None): YarnHistoryService = {
+ val service = spy(new YarnHistoryService())
+ assertNotNull(timelineClient, "timeline client")
+ doReturn(timelineClient).when(service).createTimelineClient()
+ service.start(SchedulerExtensionServiceBinding(sc, id, attemptId))
+ service
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/EventMarshallingSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/EventMarshallingSuite.scala
new file mode 100644
index 000000000000..a3dad56edd08
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/EventMarshallingSuite.scala
@@ -0,0 +1,171 @@
+/*
+ * 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.yarn.unit
+
+import java.io.IOException
+
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.{Logging, SparkFunSuite}
+import org.apache.spark.deploy.history.yarn.YarnHistoryService
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.testtools.ExtraAssertions
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.scheduler.{AccumulableInfo, JobSucceeded, SparkListenerBlockUpdated, SparkListenerEvent, SparkListenerJobEnd, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskGettingResult, SparkListenerTaskStart, StageInfo, TaskInfo, TaskLocality}
+
+/**
+ * Test low-level marshalling, robustness and quality of exception messages.
+ */
+class EventMarshallingSuite extends SparkFunSuite
+ with BeforeAndAfter with Logging with ExtraAssertions {
+
+ val stageInfo = new StageInfo(12, 13, "stageinfo-1", 4, Nil, Nil, "staged info")
+
+ val taskInfo = new TaskInfo(100, 101, 102, 103, "executor", "host", TaskLocality.ANY, true)
+
+ before {
+ stageInfo.submissionTime = Some(100000)
+ stageInfo.completionTime = Some(200000)
+ stageInfo.failureReason = Some("network problems")
+ val ai = new AccumulableInfo(1, "accumulator", Some("update"), "value", false)
+ stageInfo.accumulables.put(1, ai)
+ }
+
+ test("unmarshall empty event") {
+ val event = new TimelineEvent
+ val ex = intercept[IOException] {
+ toSparkEvent(event)
+ }
+ assertExceptionMessageContains(ex, E_EMPTY_EVENTINFO)
+ }
+
+ test("unmarshall entity type") {
+ val event = new TimelineEvent
+ event.setEventType(YarnHistoryService.SPARK_EVENT_ENTITY_TYPE)
+ val ex = intercept[IOException] {
+ toSparkEvent(event)
+ }
+ assertExceptionMessageContains(ex, E_EMPTY_EVENTINFO)
+ }
+
+ test("round trip app start") {
+ val startEvent = appStartEvent(1)
+ assert(APP_USER === startEvent.sparkUser)
+ assert(APP_NAME === startEvent.appName)
+ val dest = validateRoundTrip(startEvent)
+ assert(startEvent.time === dest.time )
+ assert(startEvent.sparkUser === dest.sparkUser )
+ assert(APP_NAME === dest.appName)
+ }
+
+ test("round trip app end") {
+ validateRoundTrip(appStopEvent(1))
+ }
+
+ test("SparkListenerStageSubmitted") {
+ val src = new SparkListenerStageSubmitted(stageInfo)
+ val dest = roundTrip(src)
+ assert(isEqual(stageInfo, dest.stageInfo))
+ }
+
+ test("SparkListenerStageCompleted") {
+ val src = new SparkListenerStageCompleted(stageInfo)
+ val dest = roundTrip(src)
+ assert(isEqual(stageInfo, dest.stageInfo))
+ }
+
+ test("SparkListenerTaskStart") {
+ val src = new SparkListenerTaskStart(1, 2, taskInfo)
+ val dest = roundTrip(src)
+ assert(isEqual(taskInfo, dest.taskInfo))
+ }
+
+ test("SparkListenerTaskGettingResult") {
+ val src = new SparkListenerTaskGettingResult(taskInfo)
+ val dest = roundTrip(src)
+ assert(isEqual(taskInfo, dest.taskInfo))
+ }
+
+ test("SparkListenerJobEnd") {
+ val endTime = 3000L
+ val id = 3
+ val result = JobSucceeded
+ val src = new SparkListenerJobEnd(id, endTime, result)
+ val dest = roundTrip(src)
+ assert(endTime === dest.time)
+ assert(id === dest.jobId)
+ assert(result === dest.jobResult)
+ }
+
+ test("SparkListenerBlockUpdated is ignored") {
+ assert(toTimelineEvent(new SparkListenerBlockUpdated(null), 0).isEmpty)
+ }
+
+ def validateRoundTrip[T <: SparkListenerEvent](sparkEvt: T): T = {
+ val trip = roundTrip(sparkEvt)
+ assertResult(sparkEvt) {
+ trip
+ }
+ trip
+ }
+
+ /**
+ * Marshall then unmarshall a spark event.
+ *
+ * @param src source
+ * @return a new spark event built from the marshalled JSON value.
+ */
+ private def roundTrip[T <: SparkListenerEvent ](src: T): T = {
+ val event = toSparkEvent(toTimelineEvent(src, 100).get)
+ event.asInstanceOf[T]
+ }
+
+ /**
+ * Task info equality; does not check accumulables.
+ *
+ * @param l left item
+ * @param r right item
+ * @return true if the values are equal
+ */
+ def isEqual(l: TaskInfo, r: TaskInfo) : Boolean = {
+ l.taskId == r.taskId &&
+ l.index == r.index &&
+ l.attemptNumber == r.attemptNumber &&
+ l.executorId == r.executorId &&
+ l.host == r.host &&
+ l.speculative == r.speculative &&
+ l.taskLocality == r.taskLocality &&
+ l.gettingResultTime == r.gettingResultTime &&
+ l.finishTime == r.finishTime &&
+ l.failed == r.failed &&
+ l.accumulables.size == r.accumulables.size
+ }
+
+ def isEqual(l: StageInfo, r: StageInfo): Boolean = {
+ l.stageId == r.stageId &&
+ l.name == r.name &&
+ l.attemptId == r.attemptId &&
+ l.numTasks == r.numTasks &&
+ l.details == r.details &&
+ l.submissionTime == r.submissionTime &&
+ l.completionTime == r.completionTime &&
+ l.failureReason == r.failureReason &&
+ l.accumulables.size == r.accumulables.size
+ }
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/HistoryServiceInstantiationSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/HistoryServiceInstantiationSuite.scala
new file mode 100644
index 000000000000..7ef1a2970541
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/HistoryServiceInstantiationSuite.scala
@@ -0,0 +1,53 @@
+/*
+ * 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.yarn.unit
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.YarnHistoryService
+import org.apache.spark.deploy.history.yarn.testtools.AbstractYarnHistoryTests
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.scheduler.cluster.{SchedulerExtensionServiceBinding, SchedulerExtensionServices}
+
+/**
+ * Test the integration with [[SchedulerExtensionServices]].
+ */
+class HistoryServiceInstantiationSuite extends AbstractYarnHistoryTests {
+
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ super.setupConfiguration(sparkConf)
+ sparkConf.set(SchedulerExtensionServices.SPARK_YARN_SERVICES, YarnHistoryService.CLASSNAME)
+ }
+
+ test("Contains History Service") {
+ val services = new SchedulerExtensionServices
+ try {
+ services.start(SchedulerExtensionServiceBinding(sc, applicationId))
+ val serviceList = services.getServices
+ assert(serviceList.nonEmpty, "empty service list")
+ val Seq(history) = serviceList
+ val historyService = history.asInstanceOf[YarnHistoryService]
+ assert(historyService.serviceState === YarnHistoryService.StartedState)
+ services.stop()
+ assert(historyService.serviceState === YarnHistoryService.StoppedState)
+ } finally {
+ services.stop()
+ }
+ }
+
+}
+
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/HistoryWithDisabledTimelineSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/HistoryWithDisabledTimelineSuite.scala
new file mode 100644
index 000000000000..b14be6efb6a4
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/HistoryWithDisabledTimelineSuite.scala
@@ -0,0 +1,75 @@
+/*
+ * 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.yarn.unit
+
+import org.apache.spark.deploy.history.yarn.YarnHistoryService
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.testtools.AbstractYarnHistoryTests
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.scheduler.cluster.SchedulerExtensionServiceBinding
+
+/**
+ * Test that with the timeline service disabled, public operations degrade gracefully.
+ */
+class HistoryWithDisabledTimelineSuite extends AbstractYarnHistoryTests {
+
+ test("BasicLifecycle") {
+ val service = new YarnHistoryService()
+ // verify that the string operator does not fail
+ service.toString()
+
+ service.start(SchedulerExtensionServiceBinding(sc, applicationId, Some(attemptId)))
+ assert(StartedState === service.serviceState, "not stopped : $service")
+ assert(!service.bondedToATS, s"service is bonded to ats $service")
+ assert(!service.listening, s"service is listening $service")
+ assertResult(null, s"service address : $service") {
+ service.timelineWebappAddress
+ }
+ intercept[Exception] {
+ service.timelineClient
+ }
+ assert(!service.isPostThreadActive, s"service post thread active: $service")
+
+ // verify that the string operator does not fail
+ service.toString()
+ service.stop()
+ assert(StoppedState === service.serviceState, "not stopped : $service")
+ // verify that the string operator does not fail
+ service.toString()
+ }
+
+ test("QueueAndFlush") {
+ val service = new YarnHistoryService()
+ try {
+ service.start(SchedulerExtensionServiceBinding(sc, applicationId, Some(attemptId)))
+ service.enqueue(appStartEvent())
+ service.enqueue(appStopEvent())
+
+ assert(0 === service.eventsQueued, "queue")
+
+ service.asyncFlush()
+ assert(0 === service.getFlushCount, "flush count")
+
+ service.stop()
+ assert(0 === service.getFlushCount, "flush count")
+ } finally {
+ service.stop()
+ }
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockBatchingTimelinePostSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockBatchingTimelinePostSuite.scala
new file mode 100644
index 000000000000..94dd20faa6dd
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockBatchingTimelinePostSuite.scala
@@ -0,0 +1,52 @@
+/*
+ * 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.yarn.unit
+
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity
+import org.mockito.Matchers._
+import org.mockito.Mockito._
+
+import org.apache.spark.deploy.history.yarn.testtools.TimelineSingleEntryBatchSize
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+/**
+ * Mock tests with Batch size 1.
+ */
+class MockBatchingTimelinePostSuite extends AbstractMockHistorySuite
+ with TimelineSingleEntryBatchSize {
+
+ test("retry upload on failure") {
+ describe("mock failures, verify retry count incremented")
+ // timeline client to throw an RTE on the first put
+ when(timelineClient.putEntities(any(classOf[TimelineEntity])))
+ .thenThrow(new RuntimeException("triggered"))
+ .thenReturn(response)
+
+ val (service, eventsPosted) = postEvents(sc)
+ // now await some retries asynchronously
+ awaitAtLeast(2, TEST_STARTUP_DELAY,
+ () => service.postAttempts,
+ () => s"Post count in $service")
+ service.stop()
+ awaitServiceThreadStopped(service, TEST_STARTUP_DELAY)
+ // there should have been three flushed
+ assert(eventsPosted === service.getFlushCount, s"expected $eventsPosted flushed for $service" )
+ verify(timelineClient, times(service.postAttempts.toInt))
+ .putEntities(any(classOf[TimelineEntity]))
+ }
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryBulkPostingSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryBulkPostingSuite.scala
new file mode 100644
index 000000000000..28f33fba2373
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryBulkPostingSuite.scala
@@ -0,0 +1,84 @@
+/*
+ * 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.yarn.unit
+
+import java.io.IOException
+
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity
+import org.mockito.Matchers._
+import org.mockito.Mockito._
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.YarnEventListener
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.scheduler.SparkListenerJobStart
+
+/**
+ * Mock event posting
+ */
+class MockHistoryBulkPostingSuite extends AbstractMockHistorySuite {
+
+ val batchSize = 5
+ val queueLimit = 5
+
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ val conf = super.setupConfiguration(sparkConf)
+ conf.set(BATCH_SIZE, batchSize.toString())
+ conf.set(POST_EVENT_LIMIT, queueLimit.toString)
+ conf.set(POST_RETRY_INTERVAL, "0ms")
+ }
+
+ test("Massive Event Posting") {
+ describe("Post many events to a failing")
+ // timeline client to throw an exception on every POST
+
+ when(timelineClient.putEntities(any(classOf[TimelineEntity])))
+ .thenThrow(new IOException("triggered"))
+
+ val service = startHistoryService(sc)
+ try {
+ val listener = new YarnEventListener(sc, service)
+ // start
+ listener.onApplicationStart(applicationStart)
+ // post many more events
+ 1 to (batchSize * 2 * queueLimit ) foreach { t =>
+ listener.onJobStart(new SparkListenerJobStart(1, t, Nil))
+ }
+ // events dropped
+ awaitAtLeast(batchSize, TEST_STARTUP_DELAY,
+ () => service.eventsDropped,
+ () => service.toString())
+
+ // posts failed
+ awaitAtLeast(10, SERVICE_SHUTDOWN_DELAY,
+ () => service.postFailures,
+ () => service.toString())
+
+ // now trigger a service shutdown with the blocking queue
+ describe("Service stop")
+ service.stop()
+ awaitServiceThreadStopped(service, SERVICE_SHUTDOWN_DELAY)
+ logDebug(s"$service")
+ } finally {
+ logDebug("stopping service in finally() clause")
+ service.stop()
+ }
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryEventPostingSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryEventPostingSuite.scala
new file mode 100644
index 000000000000..18f35284e449
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryEventPostingSuite.scala
@@ -0,0 +1,170 @@
+/*
+ * 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.yarn.unit
+
+import java.util.concurrent.atomic.AtomicBoolean
+
+import org.apache.hadoop.yarn.api.records.timeline.{TimelineEntity, TimelinePutResponse}
+import org.mockito.Matchers._
+import org.mockito.Mockito._
+import org.mockito.invocation.InvocationOnMock
+import org.mockito.stubbing.Answer
+
+import org.apache.spark.deploy.history.yarn.YarnEventListener
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+
+/**
+ * Mock event posting.
+ */
+class MockHistoryEventPostingSuite extends AbstractMockHistorySuite {
+
+ /*
+ * Make sure the low-level stuff the other tests depend on is there
+ */
+ test("Timeline client") {
+ describe("low-level timeline client test")
+ assert(response === timelineClient.putEntities(new TimelineEntity))
+ verify(timelineClient).putEntities(any(classOf[TimelineEntity]))
+ }
+
+ test("Event Queueing") {
+ describe("event queueing")
+ val (history, eventsPosted) = postEvents(sc)
+ awaitEventsProcessed(history, eventsPosted, TEST_STARTUP_DELAY)
+ }
+
+ test("batch processing of Spark listener events") {
+ val (historyService, _) = postEvents(sc)
+ verify(timelineClient, times(historyService.postAttempts.toInt))
+ .putEntities(any(classOf[TimelineEntity]))
+ }
+
+ test("PostEventsNoServiceStop") {
+ describe("verify that events are pushed on any triggered flush," +
+ " even before a service is stopped")
+ val service = startHistoryService(sc)
+ try {
+ val listener = new YarnEventListener(sc, service)
+ listener.onApplicationStart(applicationStart)
+ service.asyncFlush()
+ awaitEventsProcessed(service, 1, TEST_STARTUP_DELAY)
+ awaitFlushCount(service, 1, TEST_STARTUP_DELAY)
+ awaitPostAttemptCount(service, 1)
+ logDebug(s"$service")
+ verify(timelineClient, times(1)).putEntities(any(classOf[TimelineEntity]))
+ } finally {
+ logDebug("stopping service in finally() clause")
+ service.stop()
+ }
+ }
+
+ /**
+ * This is a convoluted little test designed to verify something: stopping
+ * the service while it is waiting for something to happen in the thread posting
+ * service will cause the thread to complete successfully.
+ *
+ * That is: even though the YARN code may potentially swallow interrupted exceptions,
+ * once it returns, the exit flag is picked up and the post thread switches
+ * into fast shutdown mode, attempting to post any remaining threads.
+ */
+ test("PostEventsBlockingOperation") {
+ describe("verify that events are pushed on any triggered flush," +
+ " even before a service is stopped")
+ val entered = new AtomicBoolean(false)
+ val exit = new AtomicBoolean(false)
+
+ /**
+ * Set a flag ang then notify the listeners
+ * @param b atomic bool flag to set
+ */
+ def setAndNotify(b: AtomicBoolean): Unit = {
+ b.synchronized {
+ b.set(true)
+ b.notify()
+ }
+ }
+
+ // wait for a boolean to be set; interrupts are discarded.
+ def waitForSet(b: AtomicBoolean, timeout: Long): Unit = {
+ b.synchronized {
+ while (!b.get()) {
+ try {
+ b.wait(timeout)
+ if (!b.get()) {
+ fail("post operation never started")
+ }
+ } catch {
+ case irq: InterruptedException =>
+ case ex: RuntimeException => throw ex
+ }
+ }
+ }
+ }
+
+ // Mockito answer which doesn't return until the `exit` flag is set,
+ // and which sets the `entered` flag on entry.
+ // this is designed to synchronize the posting thread with the test runner thread
+ class delayedAnswer extends Answer[TimelinePutResponse]() {
+
+
+ override def answer(invocation: InvocationOnMock): TimelinePutResponse = {
+ // flag the operation has started
+ if (!exit.get()) {
+ logDebug("setting `entered` flag")
+ setAndNotify(entered)
+ logDebug("waiting for `exit` flag")
+ exit.synchronized {
+ exit.wait(TEST_STARTUP_DELAY)
+ }
+ }
+ new TimelinePutResponse
+ }
+ }
+ when(timelineClient.putEntities(any(classOf[TimelineEntity])))
+ .thenAnswer(new delayedAnswer())
+
+ val service = startHistoryService(sc)
+ try {
+ val listener = new YarnEventListener(sc, service)
+ listener.onApplicationStart(applicationStart)
+ awaitPostAttemptCount(service, 1)
+ setAndNotify(entered)
+ entered.synchronized {
+ if (!entered.get()) {
+ logDebug("waiting for `entered` flag")
+ entered.wait(TEST_STARTUP_DELAY)
+ if (!entered.get()) {
+ fail("post operation never started")
+ }
+ }
+ }
+ // trigger the stop process. The interrupt will be probably be lost
+ logDebug("stopping service")
+ service.stop()
+ logDebug("setting `exit` flag")
+ setAndNotify(exit)
+ awaitServiceThreadStopped(service, SERVICE_SHUTDOWN_DELAY)
+ logDebug(s"$service")
+ } finally {
+ setAndNotify(exit)
+ logDebug("stopping service in finally() clause")
+ service.stop()
+ }
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryFlushingSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryFlushingSuite.scala
new file mode 100644
index 000000000000..a893e8d2acc0
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryFlushingSuite.scala
@@ -0,0 +1,80 @@
+/*
+ * 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.yarn.unit
+
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity
+import org.mockito.Matchers._
+import org.mockito.Mockito._
+import org.scalatest.{BeforeAndAfter, Matchers}
+
+import org.apache.spark.Logging
+import org.apache.spark.deploy.history.yarn.YarnEventListener
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.scheduler.SparkListenerUnpersistRDD
+
+/**
+ * Tests to verify that timeline operations happen even before services are closed.
+ *
+ * There's an async queue involved here, so the tests spin until a state is met or not.
+ */
+class MockHistoryFlushingSuite extends AbstractMockHistorySuite
+ with BeforeAndAfter with Matchers with Logging {
+
+ test("PostEventsNoServiceStop") {
+ describe("verify that events are pushed on any triggered flush," +
+ " even before a service is stopped")
+ val service = startHistoryService(sc)
+ try {
+ assert(service.timelineServiceEnabled, s"no timeline service in $service")
+ service.timelineClient
+ service.createTimelineClient()
+ val listener = new YarnEventListener(sc, service)
+ listener.onApplicationStart(applicationStart)
+ service.asyncFlush()
+ awaitPostAttemptCount(service, 1)
+ verify(timelineClient, times(1)).putEntities(any(classOf[TimelineEntity]))
+ } finally {
+ service.stop()
+ }
+ }
+
+ test("PostEventsWithServiceStop") {
+ describe("verify that events are pushed on service stop")
+ val service = startHistoryService(sc)
+ try {
+ service.timelineClient
+ service.createTimelineClient()
+ val listener = new YarnEventListener(sc, service)
+ listener.onApplicationStart(applicationStart)
+ awaitPostAttemptCount(service, 1)
+ verify(timelineClient, times(1)).putEntities(any(classOf[TimelineEntity]))
+ listener.onUnpersistRDD(SparkListenerUnpersistRDD(1))
+ // expecting two events
+ awaitPostAttemptCount(service, 1)
+
+ // now stop the service and await the final post
+ service.stop()
+ awaitServiceThreadStopped(service, TEST_STARTUP_DELAY)
+ verify(timelineClient, times(2)).putEntities(any(classOf[TimelineEntity]))
+ } finally {
+ logDebug(s"teardown of $service")
+ service.stop()
+ }
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryServiceLifecycleSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryServiceLifecycleSuite.scala
new file mode 100644
index 000000000000..885791b1ef7f
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/MockHistoryServiceLifecycleSuite.scala
@@ -0,0 +1,92 @@
+/*
+ * 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.yarn.unit
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.history.yarn.YarnHistoryService
+import org.apache.spark.deploy.history.yarn.YarnHistoryService._
+import org.apache.spark.deploy.history.yarn.testtools.{ContextSetup, HistoryServiceListeningToSparkContext}
+import org.apache.spark.deploy.history.yarn.testtools.YarnTestUtils._
+import org.apache.spark.scheduler.cluster.SchedulerExtensionServiceBinding
+
+class MockHistoryServiceLifecycleSuite
+ extends AbstractMockHistorySuite
+ with ContextSetup
+ with HistoryServiceListeningToSparkContext {
+
+ /**
+ * Set the batch size to 2, purely so that we can trace its path through
+ * the configuration system.
+ */
+ override def setupConfiguration(sparkConf: SparkConf): SparkConf = {
+ super.setupConfiguration(sparkConf).set(BATCH_SIZE, "2")
+ }
+
+ /*
+ * Test service lifecycle ops and that stop() is re-entrant
+ */
+ test("Service Lifecycle") {
+ describe("service lifecycle operations")
+
+ assertResult("2", s"batch size in context") {
+ sc.conf.get(BATCH_SIZE)
+ }
+
+ assertResult("true", s"listening flag") {
+ sc.conf.get(REGISTER_LISTENER)
+ }
+
+ val service = startHistoryService(sc)
+ assertResult(StartedState, "not started") {
+ service.serviceState
+ }
+ assertResult(2, s"batch size in $service") {
+ service.batchSize
+ }
+ assertResult(true, s"listen flag in $service") {
+ service.listening
+ }
+
+ service.stop()
+ assertResult(StoppedState, s"not stopped: $service") {
+ service.serviceState
+ }
+
+ // and expect an attempt to start again to fail
+ intercept[IllegalArgumentException] {
+ service.start(SchedulerExtensionServiceBinding(sc, applicationId, None))
+ }
+ // repeated stop() is harmless
+ service.stop()
+ }
+
+ test("ServiceStartArguments1") {
+ val service = new YarnHistoryService()
+ intercept[IllegalArgumentException] {
+ service.start(SchedulerExtensionServiceBinding(null, applicationId, None))
+ }
+ }
+
+ test("ServiceStartArguments2") {
+ val service = new YarnHistoryService()
+ intercept[IllegalArgumentException] {
+ service.start(SchedulerExtensionServiceBinding(sc, null, None))
+ }
+ }
+
+}
diff --git a/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/YarnTimelineUtilsSuite.scala b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/YarnTimelineUtilsSuite.scala
new file mode 100644
index 000000000000..0f4ab0ff04d2
--- /dev/null
+++ b/yarn/src/history/test/scala/org/apache/spark/deploy/history/yarn/unit/YarnTimelineUtilsSuite.scala
@@ -0,0 +1,50 @@
+/*
+ * 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.yarn.unit
+
+import java.net.{NoRouteToHostException, URI}
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.{Logging, SparkFunSuite}
+import org.apache.spark.deploy.history.yarn.YarnTimelineUtils._
+import org.apache.spark.deploy.history.yarn.testtools.ExtraAssertions
+
+/**
+ * Tests of methods in [[org.apache.spark.deploy.history.yarn.YarnTimelineUtils]].
+ */
+class YarnTimelineUtilsSuite extends SparkFunSuite
+ with BeforeAndAfter with Logging with ExtraAssertions {
+
+ test("verifyNoHost") {
+ intercept[NoRouteToHostException] {
+ validateEndpoint(new URI("http://0.0.0.0:8080/ws"))
+ }
+ }
+
+ test("verifyNoPort") {
+ intercept[NoRouteToHostException] {
+ validateEndpoint(new URI("http://127.0.1.1:0/ws"))
+ }
+ }
+
+ test("verifyValid") {
+ validateEndpoint(new URI("http://127.0.1.1:8080/ws"))
+ }
+
+}