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 NameDefaultMeaning
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.interval1s + 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.size3 + How many events to batch up before submitting them to the timeline service. + This is a performance optimization. +
spark.hadoop.yarn.timeline.post.limit1000 + 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.waittime30s + Maximum time in to wait for event posting to complete when the service stops. +
spark.hadoop.yarn.timeline.listentrue + 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 NameDefaultMeaning
spark.history.yarn.window.limit24h + 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.interval60s + 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.interval30s + 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-limit1000 + Maximum number of application histories to fetch + from the timeline server in a single GET request. +
spark.history.yarn.diagnosticsfalse + 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.applicationstrue + 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 @@ 64m 512m 512m + 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 @@ libfb303 test + + 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}/classes target/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")) + } + +}