From 3d18ebcb7145dbd888590650a693e1f42605497a Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 19 May 2015 13:54:20 -0700 Subject: [PATCH 01/15] [WIP] Try getting the event log download to work. --- .../history/ApplicationHistoryProvider.scala | 8 ++ .../deploy/history/FsHistoryProvider.scala | 45 +++++++++- .../spark/deploy/history/HistoryServer.scala | 10 ++- .../spark/deploy/master/ui/MasterWebUI.scala | 4 +- ...otResource.scala => ApiRootResource.scala} | 17 +++- .../api/v1/EventLogDownloadResource.scala | 70 ++++++++++++++++ .../status/api/v1/JacksonMessageWriter.scala | 6 +- .../scala/org/apache/spark/ui/SparkUI.scala | 4 +- .../history/FsHistoryProviderSuite.scala | 84 ++++++++++++++++++- 9 files changed, 234 insertions(+), 14 deletions(-) rename core/src/main/scala/org/apache/spark/status/api/v1/{JsonRootResource.scala => ApiRootResource.scala} (94%) create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 298a8201960d1..4ed91fb8885cc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy.history +import java.io.File + import org.apache.spark.ui.SparkUI private[spark] case class ApplicationAttemptInfo( @@ -62,4 +64,10 @@ private[history] abstract class ApplicationHistoryProvider { */ def getConfig(): Map[String, String] = Map() + /** + * Get the event logs for the given application. The event logs are compressed into a zip file + * and copied into the directory passed in. + */ + def copyApplicationEventLogs(appId: String, directory: File) = { } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 45c2be34c8680..837ddb632b54a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,8 +17,10 @@ package org.apache.spark.deploy.history -import java.io.{BufferedInputStream, FileNotFoundException, IOException, InputStream} +import java.io.{FileOutputStream, File, BufferedInputStream, FileNotFoundException, + IOException, InputStream} import java.util.concurrent.{ExecutorService, Executors, TimeUnit} +import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.mutable @@ -219,6 +221,47 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + override def copyApplicationEventLogs(appId: String, directory: File): Unit = { + val buffer = new Array[Byte](64 * 1024) + /** + * Copy the data from the path specified into a new [[ZipEntry]] with the remotePath's name. + */ + def copyToZipStream(remotePath: Path, zipStream: ZipOutputStream): Unit = { + val inputStream = fs.open(remotePath, 1 * 1024 * 1024) // 1MB Buffer + zipStream.putNextEntry(new ZipEntry(remotePath.getName)) + var dataRemaining = true + while (dataRemaining) { + val length = inputStream.read(buffer) + if (length != -1) { + zipStream.write(buffer, 0, length) + } else { + dataRemaining = false + } + } + zipStream.closeEntry() + inputStream.close() + } + + applications.get(appId).foreach { appInfo => + val outFile = new File(directory, s"eventLogs-$appId.zip") + val zipStream = new ZipOutputStream(new FileOutputStream(outFile)) + appInfo.attempts.foreach { attempt => + val remotePath = new Path(logDir, attempt.logPath) + if (isLegacyLogDirectory(fs.getFileStatus(remotePath))) { + val filesIter = fs.listFiles(remotePath, true) + while (filesIter.hasNext) { + copyToZipStream(filesIter.next().getPath, zipStream) + } + } else { + copyToZipStream(remotePath, zipStream) + } + } + zipStream.finish() + zipStream.close() + } + } + + /** * Replay the log files in the list and merge the list of old applications with new ones */ 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 517cbe5176241..b302a790f3eae 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 @@ -17,6 +17,7 @@ package org.apache.spark.deploy.history +import java.io.File import java.util.NoSuchElementException import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} @@ -25,7 +26,7 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.status.api.v1.{ApplicationInfo, ApplicationsListResource, JsonRootResource, UIRoot} +import org.apache.spark.status.api.v1.{ApplicationInfo, ApplicationsListResource, ApiRootResource, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{SignalLogger, Utils} @@ -125,7 +126,7 @@ class HistoryServer( def initialize() { attachPage(new HistoryPage(this)) - attachHandler(JsonRootResource.getJsonServlet(this)) + attachHandler(ApiRootResource.getApiServlet(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) @@ -172,6 +173,11 @@ class HistoryServer( getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) } + def copyEventLogsToDirectory(appId: String, destDir: File): Unit = { + provider.copyApplicationEventLogs(appId, destDir) + } + + /** * Returns the provider configuration to show in the listing page. * diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index eb26e9f99c70b..10b250e93eaec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.Logging import org.apache.spark.deploy.master.Master -import org.apache.spark.status.api.v1.{ApplicationsListResource, ApplicationInfo, JsonRootResource, UIRoot} +import org.apache.spark.status.api.v1.{ApplicationsListResource, ApplicationInfo, ApiRootResource, UIRoot} import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.RpcUtils @@ -47,7 +47,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(JsonRootResource.getJsonServlet(this)) + attachHandler(ApiRootResource.getApiServlet(this)) attachHandler(createRedirectHandler( "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST"))) attachHandler(createRedirectHandler( diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala similarity index 94% rename from core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala rename to core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index c3ec45f54681b..472becd7ebb08 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -39,7 +39,7 @@ import org.apache.spark.ui.SparkUI * HistoryServerSuite. */ @Path("/v1") -private[v1] class JsonRootResource extends UIRootFromServletContext { +private[v1] class ApiRootResource extends UIRootFromServletContext { @Path("applications") def getApplicationList(): ApplicationListResource = { @@ -164,13 +164,22 @@ private[v1] class JsonRootResource extends UIRootFromServletContext { } } + @Path("applications/{appId}/{attemptId}/download") + def getEventLogs( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): EventLogDownloadResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new EventLogDownloadResource(ui, appId) + } + } + } -private[spark] object JsonRootResource { +private[spark] object ApiRootResource { - def getJsonServlet(uiRoot: UIRoot): ServletContextHandler = { + def getApiServlet(uiRoot: UIRoot): ServletContextHandler = { val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) - jerseyContext.setContextPath("/json") + jerseyContext.setContextPath("/api") val holder:ServletHolder = new ServletHolder(classOf[ServletContainer]) holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass", "com.sun.jersey.api.core.PackagesResourceConfig") diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala new file mode 100644 index 0000000000000..d7ccac0f8666a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -0,0 +1,70 @@ +/* + * 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.status.api.v1 + +import java.io.{FileInputStream, OutputStream, File, InputStream} +import javax.ws.rs.{GET, Produces} +import javax.ws.rs.core.{MultivaluedMap, MediaType} + +import org.apache.spark.deploy.history.HistoryServer +import org.apache.spark.util.Utils + +@Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) +private[v1] class EventLogDownloadResource(val uIRoot: UIRoot, val appId: String) { + + @GET + def getEventLogs(headers: MultivaluedMap[String, AnyRef], outputStream: OutputStream): Unit = { + uIRoot match { + case hs: HistoryServer => + val dir = Utils.createTempDir() + Utils.chmod700(dir) + hs.copyEventLogsToDirectory(appId, dir) + dir.listFiles().headOption.foreach { zipFile => + headers.add("Content-Length", zipFile.length().toString) + headers.add("Content-Type", MediaType.APPLICATION_OCTET_STREAM) + headers.add("Content-Disposition", s"attachment; filename=${zipFile.getName}") + var inputStream: InputStream = null + try { + inputStream = new FileInputStream(zipFile) + val buffer = new Array[Byte](1024 * 1024) + var remaining = true + while (remaining) { + val read = inputStream.read(buffer) + if (read != -1) { + outputStream.write(buffer, 0, read) + } else { + remaining = false + } + } + outputStream.flush() + } finally { + inputStream.close() + Utils.deleteRecursively(dir) + } + } + case _ => outputStream.write( + s"File download not available for application : $appId".getBytes("utf-8")) + } + } +} + +private[v1] object EventLogDownloadResource { + + def unapply(resource: EventLogDownloadResource): Option[(UIRoot, String)] = { + Some((resource.uIRoot, resource.appId)) + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala index 202a5191ad57d..16073e306169d 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala @@ -38,7 +38,7 @@ import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature} * Note that jersey automatically discovers this class based on its package and its annotations. */ @Provider -@Produces(Array(MediaType.APPLICATION_JSON)) +@Produces(Array(MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM)) private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ val mapper = new ObjectMapper() { @@ -68,7 +68,9 @@ private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ multivaluedMap: MultivaluedMap[String, AnyRef], outputStream: OutputStream): Unit = { t match { - case ErrorWrapper(err) => outputStream.write(err.getBytes("utf-8")) + case ErrorWrapper(err) => outputStream.write(err.getBytes()) + case downloader @ EventLogDownloadResource(_) => + downloader.getEventLogs(multivaluedMap, outputStream) case _ => mapper.writeValue(outputStream, t) } } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index bfe4a180e8a6f..c99b92a2f57b2 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -19,7 +19,7 @@ package org.apache.spark.ui import java.util.Date -import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo, JsonRootResource, UIRoot} +import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo, ApiRootResource, UIRoot} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener @@ -64,7 +64,7 @@ private[spark] class SparkUI private ( attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) - attachHandler(JsonRootResource.getJsonServlet(this)) + attachHandler(ApiRootResource.getApiServlet(this)) // This should be POST only, but, the YARN AM proxy won't proxy POSTs attachHandler(createRedirectHandler( "/stages/stage/kill", "/stages", stagesTab.handleKillRequest, diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index a0a0afa48833e..691bf100f0085 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -17,11 +17,13 @@ package org.apache.spark.deploy.history -import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStreamWriter} +import java.io.{FileInputStream, BufferedOutputStream, File, FileOutputStream, OutputStreamWriter} import java.net.URI import java.util.concurrent.TimeUnit +import java.util.zip.ZipInputStream import scala.io.Source +import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ @@ -335,6 +337,86 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers assert(!log2.exists()) } + test("Event log copy") { + + def getFileContent(file: File): Array[Byte] = { + val buff = new Array[Byte](file.length().toInt) + val in = new FileInputStream(file) + try { + in.read(buff) + } finally { + in.close() + } + buff + } + + def unzipToDir(zipFile: File, outputDir: File): Unit = { + val zipStream = new ZipInputStream(new FileInputStream(zipFile)) + try { + val buffer = new Array[Byte](128) + var entry = zipStream.getNextEntry + while (entry != null) { + val unzippedFile = new File(outputDir, entry.getName) + val ostream = new BufferedOutputStream(new FileOutputStream(unzippedFile)) + try { + var dataRemains = true + while (dataRemains) { + val read = zipStream.read(buffer) + if (read > 0) ostream.write(buffer, 0, read) else dataRemains = false + } + } finally { + ostream.close() + } + zipStream.closeEntry() + entry = zipStream.getNextEntry + } + } finally { + zipStream.close() + } + } + + val provider = new FsHistoryProvider(createTestConf()) + val log1 = newLogFile("downloadApp1", Some("attempt1"), inProgress = false) + writeFile(log1, true, None, + SparkListenerApplicationStart("downloadApp1", Some("downloadApp1"), System + .currentTimeMillis() - 400, "test", Some("attempt1")), + SparkListenerApplicationEnd(System.currentTimeMillis() - 200) + ) + val log1Buffer = getFileContent(log1) + val log2 = newLogFile("downloadApp1", Some("attempt2"), inProgress = false) + writeFile(log2, true, None, + SparkListenerApplicationStart("downloadApp1", Some("downloadApp1"), System + .currentTimeMillis() - 100, "test", Some("attempt2")), + SparkListenerApplicationEnd(System.currentTimeMillis()) + ) + val log2Buffer = getFileContent(log2) + provider.checkForLogs() + var inputDir: File = null + var outputDir: File = null + try { + inputDir = Utils.createTempDir() + Utils.chmod700(inputDir) + outputDir = Utils.createTempDir() + Utils.chmod700(outputDir) + provider.copyApplicationEventLogs("downloadApp1", inputDir) + val zipFile = inputDir.listFiles.headOption + zipFile.foreach { file => + unzipToDir(file, outputDir) + } + var filesCompared = 0 + outputDir.listFiles().foreach { outputFile => + val bufferToCompare = if (outputFile.getName == log1.getName) log1Buffer else log2Buffer + val result = getFileContent(outputFile) + result should equal (bufferToCompare) + filesCompared += 1 + } + assert(filesCompared === 2) + } finally { + if (inputDir != null) Utils.deleteRecursively(inputDir) + if (outputDir != null) Utils.deleteRecursively(outputDir) + } + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: From 7b362b257eb22c0a1de7d6603613d9c88a9e92d4 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 19 May 2015 17:32:34 -0700 Subject: [PATCH 02/15] Almost working. --- .../status/api/v1/DownloadMessageWriter.scala | 44 +++++++++++++++ .../api/v1/EventLogDownloadResource.scala | 56 ++++++++++--------- .../status/api/v1/JacksonMessageWriter.scala | 4 +- 3 files changed, 76 insertions(+), 28 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/DownloadMessageWriter.scala diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/DownloadMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/DownloadMessageWriter.scala new file mode 100644 index 0000000000000..f185e6a420f7a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/DownloadMessageWriter.scala @@ -0,0 +1,44 @@ +/* + * 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.status.api.v1 + +import java.io.OutputStream +import java.lang.annotation.Annotation +import java.lang.reflect.Type +import javax.ws.rs.Produces +import javax.ws.rs.core.{MultivaluedMap, MediaType} +import javax.ws.rs.ext.{Provider, MessageBodyWriter} + +class DownloadMessageWriter extends MessageBodyWriter[Object] { + + override def writeTo(t: Object, `type`: Class[_], genericType: Type, + annotations: Array[Annotation], mediaType: MediaType, + httpHeaders: MultivaluedMap[String, AnyRef], entityStream: OutputStream): Unit = { + t match { + case downloader @ EventLogDownloadResource(_) => + downloader.getEventLogs() + } + } + + override def getSize(t: Object, `type`: Class[_], genericType: Type, + annotations: Array[Annotation], mediaType: MediaType): Long = { + -1L + } + + override def isWriteable(`type`: Class[_], genericType: Type, annotations: Array[Annotation], + mediaType: MediaType): Boolean = true +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala index d7ccac0f8666a..fa6261f86f761 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -16,48 +16,54 @@ */ package org.apache.spark.status.api.v1 -import java.io.{FileInputStream, OutputStream, File, InputStream} +import java.io.{BufferedInputStream, FileInputStream, OutputStream, File, InputStream} +import javax.ws.rs.ext.Provider import javax.ws.rs.{GET, Produces} -import javax.ws.rs.core.{MultivaluedMap, MediaType} +import javax.ws.rs.core.{StreamingOutput, MultivaluedMap, MediaType} import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.util.Utils -@Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) + private[v1] class EventLogDownloadResource(val uIRoot: UIRoot, val appId: String) { + private def getErrorOutput(err: String): StreamingOutput = { + new StreamingOutput { + override def write(outputStream: OutputStream): Unit = { + outputStream.write( + s"File download not available for application : $appId due to $err".getBytes("utf-8")) + } + } + } + @GET - def getEventLogs(headers: MultivaluedMap[String, AnyRef], outputStream: OutputStream): Unit = { + @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) + def getEventLogs(): StreamingOutput = { uIRoot match { case hs: HistoryServer => val dir = Utils.createTempDir() Utils.chmod700(dir) hs.copyEventLogsToDirectory(appId, dir) - dir.listFiles().headOption.foreach { zipFile => - headers.add("Content-Length", zipFile.length().toString) - headers.add("Content-Type", MediaType.APPLICATION_OCTET_STREAM) - headers.add("Content-Disposition", s"attachment; filename=${zipFile.getName}") - var inputStream: InputStream = null - try { - inputStream = new FileInputStream(zipFile) - val buffer = new Array[Byte](1024 * 1024) - var remaining = true - while (remaining) { - val read = inputStream.read(buffer) - if (read != -1) { - outputStream.write(buffer, 0, read) - } else { - remaining = false + dir.listFiles().headOption.foreach { file => + return new StreamingOutput { + override def write(output: OutputStream): Unit = { + val inStream = new BufferedInputStream(new FileInputStream(file)) + val buffer = new Array[Byte](1024 * 1024) + var dataRemains = true + while (dataRemains) { + val read = inStream.read(buffer) + if (read > 0) { + output.write(buffer, 0, read) + } else { + dataRemains = false + } } + output.flush() } - outputStream.flush() - } finally { - inputStream.close() - Utils.deleteRecursively(dir) } } - case _ => outputStream.write( - s"File download not available for application : $appId".getBytes("utf-8")) + getErrorOutput("No files in dir.") + case _ => getErrorOutput("hs not history server") } } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala index 16073e306169d..3d2a75bb526af 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala @@ -38,7 +38,7 @@ import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature} * Note that jersey automatically discovers this class based on its package and its annotations. */ @Provider -@Produces(Array(MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM)) +@Produces(Array(MediaType.APPLICATION_JSON)) private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ val mapper = new ObjectMapper() { @@ -69,8 +69,6 @@ private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ outputStream: OutputStream): Unit = { t match { case ErrorWrapper(err) => outputStream.write(err.getBytes()) - case downloader @ EventLogDownloadResource(_) => - downloader.getEventLogs(multivaluedMap, outputStream) case _ => mapper.writeValue(outputStream, t) } } From 32b7662f0910a6f0d71098bd6db7fb31e1ebf6d6 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 20 May 2015 20:49:09 -0700 Subject: [PATCH 03/15] Use UIRoot directly in ApiRootResource. Also, use `Response` class to set headers. --- .../spark/deploy/history/HistoryServer.scala | 3 +- .../spark/status/api/v1/ApiRootResource.scala | 6 +- .../status/api/v1/DownloadMessageWriter.scala | 44 ------------- .../api/v1/EventLogDownloadResource.scala | 61 ++++++++++--------- 4 files changed, 37 insertions(+), 77 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/DownloadMessageWriter.scala 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 b302a790f3eae..ec104bde45e31 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 @@ -173,8 +173,9 @@ class HistoryServer( getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) } - def copyEventLogsToDirectory(appId: String, destDir: File): Unit = { + def copyEventLogsToDirectory(appId: String, destDir: File): Boolean = { provider.copyApplicationEventLogs(appId, destDir) + true } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index 472becd7ebb08..41d39f4a9aa5d 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.status.api.v1 +import java.io.File import javax.servlet.ServletContext import javax.ws.rs._ import javax.ws.rs.core.{Context, Response} @@ -168,11 +169,8 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { def getEventLogs( @PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): EventLogDownloadResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => - new EventLogDownloadResource(ui, appId) - } + new EventLogDownloadResource(uiRoot, appId) } - } private[spark] object ApiRootResource { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/DownloadMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/DownloadMessageWriter.scala deleted file mode 100644 index f185e6a420f7a..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/DownloadMessageWriter.scala +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.status.api.v1 - -import java.io.OutputStream -import java.lang.annotation.Annotation -import java.lang.reflect.Type -import javax.ws.rs.Produces -import javax.ws.rs.core.{MultivaluedMap, MediaType} -import javax.ws.rs.ext.{Provider, MessageBodyWriter} - -class DownloadMessageWriter extends MessageBodyWriter[Object] { - - override def writeTo(t: Object, `type`: Class[_], genericType: Type, - annotations: Array[Annotation], mediaType: MediaType, - httpHeaders: MultivaluedMap[String, AnyRef], entityStream: OutputStream): Unit = { - t match { - case downloader @ EventLogDownloadResource(_) => - downloader.getEventLogs() - } - } - - override def getSize(t: Object, `type`: Class[_], genericType: Type, - annotations: Array[Annotation], mediaType: MediaType): Long = { - -1L - } - - override def isWriteable(`type`: Class[_], genericType: Type, annotations: Array[Annotation], - mediaType: MediaType): Boolean = true -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala index fa6261f86f761..7bdbf3ded9082 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -16,54 +16,59 @@ */ package org.apache.spark.status.api.v1 -import java.io.{BufferedInputStream, FileInputStream, OutputStream, File, InputStream} -import javax.ws.rs.ext.Provider +import java.io.{BufferedInputStream, FileInputStream, OutputStream} import javax.ws.rs.{GET, Produces} -import javax.ws.rs.core.{StreamingOutput, MultivaluedMap, MediaType} +import javax.ws.rs.core.{Response, StreamingOutput, MediaType} import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.util.Utils - +@Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) private[v1] class EventLogDownloadResource(val uIRoot: UIRoot, val appId: String) { - private def getErrorOutput(err: String): StreamingOutput = { - new StreamingOutput { - override def write(outputStream: OutputStream): Unit = { - outputStream.write( - s"File download not available for application : $appId due to $err".getBytes("utf-8")) - } - } - } - @GET - @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) - def getEventLogs(): StreamingOutput = { + def getEventLogs(): Response = { uIRoot match { case hs: HistoryServer => val dir = Utils.createTempDir() Utils.chmod700(dir) hs.copyEventLogsToDirectory(appId, dir) dir.listFiles().headOption.foreach { file => - return new StreamingOutput { + val stream = new StreamingOutput { override def write(output: OutputStream): Unit = { - val inStream = new BufferedInputStream(new FileInputStream(file)) - val buffer = new Array[Byte](1024 * 1024) - var dataRemains = true - while (dataRemains) { - val read = inStream.read(buffer) - if (read > 0) { - output.write(buffer, 0, read) - } else { - dataRemains = false + try { + val inStream = new BufferedInputStream(new FileInputStream(file)) + val buffer = new Array[Byte](1024 * 1024) + var dataRemains = true + while (dataRemains) { + val read = inStream.read(buffer) + if (read > 0) { + output.write(buffer, 0, read) + } else { + dataRemains = false + } } + output.flush() + } finally { + Utils.deleteRecursively(dir) } - output.flush() } } + return Response.ok(stream) + .header("Content-Length", file.length().toString) + .header("Content-Disposition", s"attachment; filename=${file.getName}") + .header("Content-Type", MediaType.APPLICATION_OCTET_STREAM) + .build() } - getErrorOutput("No files in dir.") - case _ => getErrorOutput("hs not history server") + Response.serverError() + .entity(s"Event logs for $appId not found.") + .status(Response.Status.NOT_FOUND) + .build() + case _ => + Response.serverError() + .entity("History Server is not running - cannot return event logs.") + .status(Response.Status.SERVICE_UNAVAILABLE) + .build() } } } From fd6ab0028a2d6cddc392f0ee48fdd157d2d22b66 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 20 May 2015 20:57:11 -0700 Subject: [PATCH 04/15] Fix style issues --- .../spark/deploy/history/ApplicationHistoryProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 4ed91fb8885cc..34ecc1957d638 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -68,6 +68,6 @@ private[history] abstract class ApplicationHistoryProvider { * Get the event logs for the given application. The event logs are compressed into a zip file * and copied into the directory passed in. */ - def copyApplicationEventLogs(appId: String, directory: File) = { } + def copyApplicationEventLogs(appId: String, directory: File): Unit = { } } From 0fc14249c1394ddc043b06cb0c11d4b119eed0da Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 22 May 2015 12:24:18 -0700 Subject: [PATCH 05/15] File download now works for individual attempts and the entire application. --- .../history/ApplicationHistoryProvider.scala | 9 ++- .../deploy/history/FsHistoryProvider.scala | 45 ++++------- .../spark/deploy/history/HistoryServer.scala | 9 ++- .../spark/status/api/v1/ApiRootResource.scala | 12 +++ .../api/v1/EventLogDownloadResource.scala | 78 ++++++++++++------- .../scala/org/apache/spark/util/Utils.scala | 37 +++++++++ .../history/FsHistoryProviderSuite.scala | 30 +------ .../deploy/history/HistoryServerSuite.scala | 48 ++++++++++-- 8 files changed, 168 insertions(+), 100 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 34ecc1957d638..4b18369316c14 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -17,7 +17,9 @@ package org.apache.spark.deploy.history -import java.io.File +import java.io.{OutputStream, File} + +import org.apache.hadoop.fs.Path import org.apache.spark.ui.SparkUI @@ -65,9 +67,8 @@ private[history] abstract class ApplicationHistoryProvider { def getConfig(): Map[String, String] = Map() /** - * Get the event logs for the given application. The event logs are compressed into a zip file - * and copied into the directory passed in. + * Get the [[Path]]s to the Event log directories. */ - def copyApplicationEventLogs(appId: String, directory: File): Unit = { } + def getEventLogPaths(appId: String, attemptId: String): Seq[Path] = Seq.empty } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 837ddb632b54a..27819b7a93cb0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,15 +17,15 @@ package org.apache.spark.deploy.history -import java.io.{FileOutputStream, File, BufferedInputStream, FileNotFoundException, - IOException, InputStream} +import java.io.{OutputStream, FileOutputStream, File, BufferedInputStream, FileNotFoundException, IOException, InputStream} import java.util.concurrent.{ExecutorService, Executors, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.{FSDataInputStream, FileStatus, Path} import org.apache.hadoop.fs.permission.AccessControlException import org.apache.spark.{Logging, SecurityManager, SparkConf} @@ -221,44 +221,29 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - override def copyApplicationEventLogs(appId: String, directory: File): Unit = { - val buffer = new Array[Byte](64 * 1024) - /** - * Copy the data from the path specified into a new [[ZipEntry]] with the remotePath's name. - */ - def copyToZipStream(remotePath: Path, zipStream: ZipOutputStream): Unit = { - val inputStream = fs.open(remotePath, 1 * 1024 * 1024) // 1MB Buffer - zipStream.putNextEntry(new ZipEntry(remotePath.getName)) - var dataRemaining = true - while (dataRemaining) { - val length = inputStream.read(buffer) - if (length != -1) { - zipStream.write(buffer, 0, length) - } else { - dataRemaining = false - } - } - zipStream.closeEntry() - inputStream.close() - } + override def getEventLogPaths( + appId: String, + attemptId: String): Seq[Path] = { + + var filePaths = new ArrayBuffer[Path]() applications.get(appId).foreach { appInfo => - val outFile = new File(directory, s"eventLogs-$appId.zip") - val zipStream = new ZipOutputStream(new FileOutputStream(outFile)) - appInfo.attempts.foreach { attempt => + appInfo.attempts.find { attempt => + if (attempt.attemptId.isDefined && attempt.attemptId.get == attemptId) true + else false + }.foreach { attempt => val remotePath = new Path(logDir, attempt.logPath) if (isLegacyLogDirectory(fs.getFileStatus(remotePath))) { val filesIter = fs.listFiles(remotePath, true) while (filesIter.hasNext) { - copyToZipStream(filesIter.next().getPath, zipStream) + filePaths += filesIter.next().getPath } } else { - copyToZipStream(remotePath, zipStream) + filePaths += remotePath } } - zipStream.finish() - zipStream.close() } + filePaths } 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 e30e213ec080e..4d672e66d075f 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 @@ -17,11 +17,11 @@ package org.apache.spark.deploy.history -import java.io.File import java.util.NoSuchElementException import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import com.google.common.cache._ +import org.apache.hadoop.fs.Path import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} @@ -174,9 +174,10 @@ class HistoryServer( getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) } - def copyEventLogsToDirectory(appId: String, destDir: File): Boolean = { - provider.copyApplicationEventLogs(appId, destDir) - true + def getEventLogPaths( + appId: String, + attemptId: String): Seq[Path] = { + provider.getEventLogPaths(appId, attemptId) } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index bf2cc2e72f1fe..39e7adfb37072 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -164,6 +164,18 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { } } + @Path("applications/{appId}/download") + def getEventLogs( + @PathParam("appId") appId: String): EventLogDownloadResource = { + new EventLogDownloadResource(uiRoot, appId, None) + } + + @Path("applications/{appId}/{attemptId}/download") + def getEventLogs( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): EventLogDownloadResource = { + new EventLogDownloadResource(uiRoot, appId, Some(attemptId)) + } } private[spark] object ApiRootResource { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala index 7bdbf3ded9082..80115fca8e546 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -20,50 +20,70 @@ import java.io.{BufferedInputStream, FileInputStream, OutputStream} import javax.ws.rs.{GET, Produces} import javax.ws.rs.core.{Response, StreamingOutput, MediaType} +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.fs.Path + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.util.Utils @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) -private[v1] class EventLogDownloadResource(val uIRoot: UIRoot, val appId: String) { +private[v1] class EventLogDownloadResource( + val uIRoot: UIRoot, + val appId: String, + val attemptId: Option[String]) extends Logging { + val conf = SparkHadoopUtil.get.newConfiguration(new SparkConf) @GET def getEventLogs(): Response = { uIRoot match { case hs: HistoryServer => - val dir = Utils.createTempDir() - Utils.chmod700(dir) - hs.copyEventLogsToDirectory(appId, dir) - dir.listFiles().headOption.foreach { file => - val stream = new StreamingOutput { - override def write(output: OutputStream): Unit = { - try { - val inStream = new BufferedInputStream(new FileInputStream(file)) - val buffer = new Array[Byte](1024 * 1024) - var dataRemains = true - while (dataRemains) { - val read = inStream.read(buffer) - if (read > 0) { - output.write(buffer, 0, read) - } else { - dataRemains = false - } + var logsNotFound = false + val fileName: String = { + attemptId match { + case Some(id) => s"eventLogs-$appId-$id.zip" + case None => s"eventLogs-$appId.zip" + } + } + val stream = new StreamingOutput { + override def write(output: OutputStream): Unit = { + attemptId match { + case Some(id) => + Utils.zipFilesToStream(hs.getEventLogPaths(appId, id), conf, output) + case None => + val appInfo = hs.getApplicationInfoList.find(_.id == appId) + appInfo match { + case Some(info) => + val attempts = info.attempts + val files = new ArrayBuffer[Path] + attempts.foreach { attempt => + attempt.attemptId.foreach { attemptId => + logInfo(s"Attempt found: ${attemptId}") + files ++= hs.getEventLogPaths(appId, attemptId) + } + } + if (files.nonEmpty) { + Utils.zipFilesToStream(files, conf, output) + } + case None => logsNotFound = true } - output.flush() - } finally { - Utils.deleteRecursively(dir) - } } + output.flush() } - return Response.ok(stream) - .header("Content-Length", file.length().toString) - .header("Content-Disposition", s"attachment; filename=${file.getName}") + } + if (logsNotFound) { + Response.serverError() + .entity(s"Event logs are not available for app: $appId.") + .status(Response.Status.SERVICE_UNAVAILABLE) + .build() + } else { + Response.ok(stream) + .header("Content-Disposition", s"attachment; filename=$fileName") .header("Content-Type", MediaType.APPLICATION_OCTET_STREAM) .build() } - Response.serverError() - .entity(s"Event logs for $appId not found.") - .status(Response.Status.NOT_FOUND) - .build() case _ => Response.serverError() .entity("History Server is not running - cannot return event logs.") diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 6a7d1fae3320e..d1dce0eb5f036 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,6 +21,7 @@ import java.io._ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer +import java.util.zip.{ZipOutputStream, ZipEntry} import java.util.{PriorityQueue, Properties, Locale, Random, UUID} import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection @@ -776,6 +777,42 @@ private[spark] object Utils extends Logging { localRootDirs = null } + /** + * This method compresses the files passed in, and writes the compressed data out into the + * [[OutputStream]] passed in. Each file is written as a new [[ZipEntry]] with its name being + * the name of the file being compressed. + */ + private[spark] def zipFilesToStream( + files: Seq[Path], + hadoopConf: Configuration, + outputStream: OutputStream): Unit = { + + // Passing in an output stream actually makes this more efficient since we don't have to + // create an additional file to which the compressed data is written which has to be read + // again by the reader, especially if the data needs to be sent over the wire via an + // OutputStream - in which case the destination output stream can be directly passed in here. + + val fs = FileSystem.get(hadoopConf) + val buffer = new Array[Byte](64 * 1024) + val zipStream = new ZipOutputStream(outputStream) + files.foreach { remotePath => + val inputStream = fs.open(remotePath, 1 * 1024 * 1024) // 1MB Buffer + zipStream.putNextEntry(new ZipEntry(remotePath.getName)) + var dataRemaining = true + while (dataRemaining) { + val length = inputStream.read(buffer) + if (length != -1) { + zipStream.write(buffer, 0, length) + } else { + dataRemaining = false + } + } + zipStream.closeEntry() + inputStream.close() + } + zipStream.close() + } + /** * Shuffle the elements of a collection into a random order, returning the * result in a new collection. Unlike scala.util.Random.shuffle, this method diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 691bf100f0085..22ac358ce0439 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.history -import java.io.{FileInputStream, BufferedOutputStream, File, FileOutputStream, OutputStreamWriter} +import java.io.{ByteArrayOutputStream, FileInputStream, BufferedOutputStream, File, FileOutputStream, OutputStreamWriter} import java.net.URI import java.util.concurrent.TimeUnit import java.util.zip.ZipInputStream @@ -382,39 +382,15 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers .currentTimeMillis() - 400, "test", Some("attempt1")), SparkListenerApplicationEnd(System.currentTimeMillis() - 200) ) - val log1Buffer = getFileContent(log1) val log2 = newLogFile("downloadApp1", Some("attempt2"), inProgress = false) writeFile(log2, true, None, SparkListenerApplicationStart("downloadApp1", Some("downloadApp1"), System .currentTimeMillis() - 100, "test", Some("attempt2")), SparkListenerApplicationEnd(System.currentTimeMillis()) ) - val log2Buffer = getFileContent(log2) provider.checkForLogs() - var inputDir: File = null - var outputDir: File = null - try { - inputDir = Utils.createTempDir() - Utils.chmod700(inputDir) - outputDir = Utils.createTempDir() - Utils.chmod700(outputDir) - provider.copyApplicationEventLogs("downloadApp1", inputDir) - val zipFile = inputDir.listFiles.headOption - zipFile.foreach { file => - unzipToDir(file, outputDir) - } - var filesCompared = 0 - outputDir.listFiles().foreach { outputFile => - val bufferToCompare = if (outputFile.getName == log1.getName) log1Buffer else log2Buffer - val result = getFileContent(outputFile) - result should equal (bufferToCompare) - filesCompared += 1 - } - assert(filesCompared === 2) - } finally { - if (inputDir != null) Utils.deleteRecursively(inputDir) - if (outputDir != null) Utils.deleteRecursively(outputDir) - } + provider.getEventLogPaths("downloadApp1", "attempt1").head.getName should be (log1.getName) + provider.getEventLogPaths("downloadApp1", "attempt2").head.getName should be (log2.getName) } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 4adb5122bcf1a..e3851c15fa574 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -16,8 +16,10 @@ */ package org.apache.spark.deploy.history -import java.io.{File, FileInputStream, FileWriter, IOException} +import java.io.{FileOutputStream, BufferedOutputStream, InputStream, File, FileInputStream, FileWriter, IOException} import java.net.{HttpURLConnection, URL} +import java.util.zip.ZipInputStream +import javax.servlet.http import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import org.apache.commons.io.{FileUtils, IOUtils} @@ -147,6 +149,14 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with } } + test("download all logs for app with multiple attempts") { + val (code, inputStream, error) = + HistoryServerSuite.connectAndGetInputStream(generateURL("applications/local-1426533911241")) + code should be (HttpServletResponse.SC_OK) + inputStream should not be None + error should be (None) + } + test("response codes on bad paths") { val badAppId = getContentAndCode("applications/foobar") badAppId._1 should be (HttpServletResponse.SC_NOT_FOUND) @@ -202,7 +212,11 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with } def getUrl(path: String): String = { - HistoryServerSuite.getUrl(new URL(s"http://localhost:$port/api/v1/$path")) + HistoryServerSuite.getUrl(generateURL(path)) + } + + def generateURL(path: String): URL = { + new URL(s"http://localhost:$port/api/v1/$path") } def generateExpectation(name: String, path: String): Unit = { @@ -212,6 +226,23 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with out.write(json) out.close() } + + def unzipToDir(inputStream: InputStream, dir: File): Unit = { + val unzipStream = new ZipInputStream(inputStream) + val buffer = new Array[Byte](64 * 1024 * 1024) + var nextEntry = unzipStream.getNextEntry + while(nextEntry != null) { + val file = new File(dir, nextEntry.getName) + val outputStream = new BufferedOutputStream(new FileOutputStream(file)) + var read = Integer.MAX_VALUE + while (read != -1) { + read = unzipStream.read(buffer) + outputStream.write(buffer, 0, read) + } + outputStream.close() + nextEntry = unzipStream.getNextEntry + } + } } object HistoryServerSuite { @@ -233,13 +264,18 @@ object HistoryServerSuite { } def getContentAndCode(url: URL): (Int, Option[String], Option[String]) = { + val (code, in, errString) = connectAndGetInputStream(url) + val inString = in.map(IOUtils.toString) + (code, inString, errString) + } + + def connectAndGetInputStream(url: URL): (Int, Option[InputStream], Option[String]) = { val connection = url.openConnection().asInstanceOf[HttpURLConnection] connection.setRequestMethod("GET") connection.connect() val code = connection.getResponseCode() - val inString = try { - val in = Option(connection.getInputStream()) - in.map(IOUtils.toString) + val inStream = try { + Option(connection.getInputStream()) } catch { case io: IOException => None } @@ -249,7 +285,7 @@ object HistoryServerSuite { } catch { case io: IOException => None } - (code, inString, errString) + (code, inStream, errString) } From a48b91fd858214d1c0fcf1206bd97c72cf6d07e8 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 27 May 2015 01:06:37 -0700 Subject: [PATCH 06/15] Refactor to make attemptId optional in the API. Also added tests. --- .../history/ApplicationHistoryProvider.scala | 9 +-- .../deploy/history/FsHistoryProvider.scala | 30 +++----- .../spark/deploy/history/HistoryServer.scala | 5 +- .../spark/status/api/v1/ApiRootResource.scala | 2 + .../api/v1/EventLogDownloadResource.scala | 69 +++++++++++-------- .../scala/org/apache/spark/util/Utils.scala | 36 ++++++---- .../application_list_json_expectation.json | 16 +++++ .../completed_app_list_json_expectation.json | 16 +++++ .../minDate_app_list_json_expectation.json | 24 ++++++- .../spark-events/local-1430917381535_1 | 5 ++ .../spark-events/local-1430917381535_2 | 5 ++ .../history/FsHistoryProviderSuite.scala | 6 +- .../deploy/history/HistoryServerSuite.scala | 51 ++++++++++++-- 13 files changed, 188 insertions(+), 86 deletions(-) create mode 100644 core/src/test/resources/spark-events/local-1430917381535_1 create mode 100644 core/src/test/resources/spark-events/local-1430917381535_2 diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 4b18369316c14..1fe828ac3324b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy.history -import java.io.{OutputStream, File} - import org.apache.hadoop.fs.Path import org.apache.spark.ui.SparkUI @@ -67,8 +65,11 @@ private[history] abstract class ApplicationHistoryProvider { def getConfig(): Map[String, String] = Map() /** - * Get the [[Path]]s to the Event log directories. + * Get the [[Path]]s to the Event log files. For legacy event log directories, directory path + * itself is returned. The caller is responsible for listing the files and using them as needed. + * If the attemptId is [[None]], event logs corresponding to all attempts for the given + * application are downloaded as a single zip file. */ - def getEventLogPaths(appId: String, attemptId: String): Seq[Path] = Seq.empty + def getEventLogPaths(appId: String, attemptId: Option[String]): Seq[Path] = Seq.empty } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 27819b7a93cb0..8621b42094f1c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,15 +17,14 @@ package org.apache.spark.deploy.history -import java.io.{OutputStream, FileOutputStream, File, BufferedInputStream, FileNotFoundException, IOException, InputStream} +import java.io.{File, FileOutputStream, BufferedInputStream, FileNotFoundException, IOException, InputStream} import java.util.concurrent.{ExecutorService, Executors, TimeUnit} -import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} -import org.apache.hadoop.fs.{FSDataInputStream, FileStatus, Path} +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.fs.permission.AccessControlException import org.apache.spark.{Logging, SecurityManager, SparkConf} @@ -221,27 +220,14 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - override def getEventLogPaths( - appId: String, - attemptId: String): Seq[Path] = { - - var filePaths = new ArrayBuffer[Path]() + override def getEventLogPaths(appId: String, attemptId: Option[String]): Seq[Path] = { + val filePaths = new ArrayBuffer[Path]() applications.get(appId).foreach { appInfo => - appInfo.attempts.find { attempt => - if (attempt.attemptId.isDefined && attempt.attemptId.get == attemptId) true - else false - }.foreach { attempt => - val remotePath = new Path(logDir, attempt.logPath) - if (isLegacyLogDirectory(fs.getFileStatus(remotePath))) { - val filesIter = fs.listFiles(remotePath, true) - while (filesIter.hasNext) { - filePaths += filesIter.next().getPath - } - } else { - filePaths += remotePath - } - } + // If no attempt is specified, or there is no attemptId for attempts, return all attempts + appInfo.attempts.filter { attempt => + attempt.attemptId.isEmpty || attemptId.isEmpty || attempt.attemptId.get == attemptId.get + }.foreach { attempt => filePaths += new Path(logDir, attempt.logPath) } } filePaths } 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 4d672e66d075f..9d1ac2d107d98 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 @@ -174,13 +174,10 @@ class HistoryServer( getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) } - def getEventLogPaths( - appId: String, - attemptId: String): Seq[Path] = { + override def getEventLogPaths(appId: String, attemptId: Option[String]): Seq[Path] = { provider.getEventLogPaths(appId, attemptId) } - /** * Returns the provider configuration to show in the listing page. * diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index 39e7adfb37072..6436bd3b169a4 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -22,6 +22,7 @@ import javax.ws.rs.core.{Context, Response} import com.sun.jersey.api.core.ResourceConfig import com.sun.jersey.spi.container.servlet.ServletContainer +import org.apache.hadoop.fs import org.eclipse.jetty.server.handler.ContextHandler import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} @@ -204,6 +205,7 @@ private[spark] object ApiRootResource { private[spark] trait UIRoot { def getSparkUI(appKey: String): Option[SparkUI] def getApplicationInfoList: Iterator[ApplicationInfo] + def getEventLogPaths(appId: String, attemptId: Option[String]): Seq[fs.Path] = Seq.empty /** * Get the spark UI with the given appID, and apply a function diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala index 80115fca8e546..e4b81f750356e 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.status.api.v1 -import java.io.{BufferedInputStream, FileInputStream, OutputStream} +import java.io.{File, FileOutputStream, BufferedInputStream, FileInputStream, OutputStream} import javax.ws.rs.{GET, Produces} import javax.ws.rs.core.{Response, StreamingOutput, MediaType} @@ -41,7 +41,7 @@ private[v1] class EventLogDownloadResource( uIRoot match { case hs: HistoryServer => var logsNotFound = false - val fileName: String = { + val fileName = { attemptId match { case Some(id) => s"eventLogs-$appId-$id.zip" case None => s"eventLogs-$appId.zip" @@ -49,28 +49,9 @@ private[v1] class EventLogDownloadResource( } val stream = new StreamingOutput { override def write(output: OutputStream): Unit = { - attemptId match { - case Some(id) => - Utils.zipFilesToStream(hs.getEventLogPaths(appId, id), conf, output) - case None => - val appInfo = hs.getApplicationInfoList.find(_.id == appId) - appInfo match { - case Some(info) => - val attempts = info.attempts - val files = new ArrayBuffer[Path] - attempts.foreach { attempt => - attempt.attemptId.foreach { attemptId => - logInfo(s"Attempt found: ${attemptId}") - files ++= hs.getEventLogPaths(appId, attemptId) - } - } - if (files.nonEmpty) { - Utils.zipFilesToStream(files, conf, output) - } - case None => logsNotFound = true - } - } - output.flush() + val eventLogs = hs.getEventLogPaths(appId, attemptId) + if (eventLogs.isEmpty) logsNotFound = true + else zipLogFiles(eventLogs, output) } } if (logsNotFound) { @@ -86,16 +67,44 @@ private[v1] class EventLogDownloadResource( } case _ => Response.serverError() - .entity("History Server is not running - cannot return event logs.") + .entity("Event logs are only available through the history server.") .status(Response.Status.SERVICE_UNAVAILABLE) .build() } } -} - -private[v1] object EventLogDownloadResource { - def unapply(resource: EventLogDownloadResource): Option[(UIRoot, String)] = { - Some((resource.uIRoot, resource.appId)) + private def zipLogFiles(eventLogs: Seq[Path], output: OutputStream): Unit = { + val areLegacyLogs = eventLogs.headOption.exists { path => + path.getFileSystem(conf).isDirectory(path) + } + val pathsToZip = if (areLegacyLogs) { + new ArrayBuffer[Path]() + } else { + eventLogs + } + var tempDir: File = null + try { + if (areLegacyLogs) { + tempDir = Utils.createTempDir() + Utils.chmod700(tempDir) + eventLogs.foreach { logPath => + // If the event logs are directories (legacy), then create a zip file for each + // one and write each of these files to the eventual output. + val fs = logPath.getFileSystem(conf) + val logFiles = fs.listFiles(logPath, true) + val zipFile = new File(tempDir, logPath.getName + ".zip") + pathsToZip.asInstanceOf[ArrayBuffer[Path]] += new Path(zipFile.toURI) + val outputStream = new FileOutputStream(zipFile) + val paths = new ArrayBuffer[Path]() + while (logFiles.hasNext) { + paths += logFiles.next().getPath + } + Utils.zipFilesToStream(paths, conf, outputStream) + } + } + Utils.zipFilesToStream(pathsToZip, conf, output) + } finally { + if (tempDir != null) Utils.deleteRecursively(tempDir) + } } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d1dce0eb5f036..30cc66e9e07b0 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,7 +21,7 @@ import java.io._ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer -import java.util.zip.{ZipOutputStream, ZipEntry} +import java.util.zip.{ZipEntry, ZipOutputStream} import java.util.{PriorityQueue, Properties, Locale, Random, UUID} import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection @@ -794,23 +794,29 @@ private[spark] object Utils extends Logging { val fs = FileSystem.get(hadoopConf) val buffer = new Array[Byte](64 * 1024) - val zipStream = new ZipOutputStream(outputStream) - files.foreach { remotePath => - val inputStream = fs.open(remotePath, 1 * 1024 * 1024) // 1MB Buffer - zipStream.putNextEntry(new ZipEntry(remotePath.getName)) - var dataRemaining = true - while (dataRemaining) { - val length = inputStream.read(buffer) - if (length != -1) { - zipStream.write(buffer, 0, length) - } else { - dataRemaining = false + val zipStream = Some(new ZipOutputStream(outputStream)) + try { + files.foreach { remotePath => + val inputStream = Some(fs.open(remotePath, 1 * 1024 * 1024)) // 1MB Buffer + try { + zipStream.get.putNextEntry(new ZipEntry(remotePath.getName)) + var dataRemaining = true + while (dataRemaining) { + val length = inputStream.get.read(buffer) + if (length != -1) { + zipStream.get.write(buffer, 0, length) + } else { + dataRemaining = false + } + } + zipStream.get.closeEntry() + } finally { + inputStream.foreach(_.close()) } } - zipStream.closeEntry() - inputStream.close() + } finally { + zipStream.foreach(_.close()) } - zipStream.close() } /** diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index ce4fe80b66aa5..26ac45f4d1b4b 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -41,6 +41,22 @@ "sparkUser" : "irashid", "completed" : true } ] +}, { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] }, { "id" : "local-1422981759269", "name" : "Spark shell", diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index ce4fe80b66aa5..26ac45f4d1b4b 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -41,6 +41,22 @@ "sparkUser" : "irashid", "completed" : true } ] +}, { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] }, { "id" : "local-1422981759269", "name" : "Spark shell", diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index dca86fe5f7e6a..7c8bfbf8de96d 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -24,11 +24,29 @@ "completed" : true } ] }, { - "id" : "local-1425081759269", + "id": "local-1425081759269", + "name": "Spark shell", + "attempts": [ + { + "startTime": "2015-02-28T00:02:38.277GMT", + "endTime": "2015-02-28T00:02:46.912GMT", + "sparkUser": "irashid", + "completed": true + } + ] +}, { + "id" : "local-1430917381535", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2015-02-28T00:02:38.277GMT", - "endTime" : "2015-02-28T00:02:46.912GMT", + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", "sparkUser" : "irashid", "completed" : true } ] diff --git a/core/src/test/resources/spark-events/local-1430917381535_1 b/core/src/test/resources/spark-events/local-1430917381535_1 new file mode 100644 index 0000000000000..ea27c67f4977a --- /dev/null +++ b/core/src/test/resources/spark-events/local-1430917381535_1 @@ -0,0 +1,5 @@ +{"Event":"SparkListenerLogStart","Spark Version":"1.4.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":61103},"Maximum Memory":278019440,"Timestamp":1430917381651} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.102","spark.eventLog.enabled":"true","spark.driver.port":"61101","spark.repl.class.uri":"http://192.168.1.102:61100","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.eventLog.dir":"/Users/irashid/github/kraps/core/src/test/resources/spark-events","spark.fileserver.uri":"http://192.168.1.102:61102","spark.tachyonStore.folderName":"spark-aaaf41b3-d1dd-447f-8951-acf51490758b","spark.app.id":"local-1430917381534"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/github/spark","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.eventLog.enabled=true --conf spark.eventLog.dir=/Users/irashid/github/kraps/core/src/test/resources/spark-events --class org.apache.spark.repl.Main spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/etc/hadoop":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/github/spark/conf/":"System Classpath","/Users/irashid/github/spark/assembly/target/scala-2.10/spark-assembly-1.4.0-SNAPSHOT-hadoop2.5.0.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1430917381535","Timestamp":1430917380893,"User":"irashid","App Attempt ID":"1"} +{"Event":"SparkListenerApplicationEnd","Timestamp":1422981766912} \ No newline at end of file diff --git a/core/src/test/resources/spark-events/local-1430917381535_2 b/core/src/test/resources/spark-events/local-1430917381535_2 new file mode 100644 index 0000000000000..35cce42fb70de --- /dev/null +++ b/core/src/test/resources/spark-events/local-1430917381535_2 @@ -0,0 +1,5 @@ +{"Event":"SparkListenerLogStart","Spark Version":"1.4.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":61103},"Maximum Memory":278019440,"Timestamp":1430917381651} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.102","spark.eventLog.enabled":"true","spark.driver.port":"61101","spark.repl.class.uri":"http://192.168.1.102:61100","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.eventLog.dir":"/Users/irashid/github/kraps/core/src/test/resources/spark-events","spark.fileserver.uri":"http://192.168.1.102:61102","spark.tachyonStore.folderName":"spark-aaaf41b3-d1dd-447f-8951-acf51490758b","spark.app.id":"local-1430917381534"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/github/spark","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.eventLog.enabled=true --conf spark.eventLog.dir=/Users/irashid/github/kraps/core/src/test/resources/spark-events --class org.apache.spark.repl.Main spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/etc/hadoop":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/github/spark/conf/":"System Classpath","/Users/irashid/github/spark/assembly/target/scala-2.10/spark-assembly-1.4.0-SNAPSHOT-hadoop2.5.0.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1430917381535","Timestamp":1430917380893,"User":"irashid","App Attempt ID":"2"} +{"Event":"SparkListenerApplicationEnd","Timestamp":1422981766912} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 22ac358ce0439..df3af6b04e52c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -389,8 +389,10 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers SparkListenerApplicationEnd(System.currentTimeMillis()) ) provider.checkForLogs() - provider.getEventLogPaths("downloadApp1", "attempt1").head.getName should be (log1.getName) - provider.getEventLogPaths("downloadApp1", "attempt2").head.getName should be (log2.getName) + provider.getEventLogPaths("downloadApp1", Some("attempt1")) + .head.getName should be (log1.getName) + provider.getEventLogPaths("downloadApp1", Some("attempt2")) + .head.getName should be (log2.getName) } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index e3851c15fa574..cfeb6bff274dc 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -27,6 +27,7 @@ import org.mockito.Mockito.when import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import org.scalatest.mock.MockitoSugar +import org.apache.spark.util.Utils import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf} import org.apache.spark.ui.SparkUI @@ -150,11 +151,47 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with } test("download all logs for app with multiple attempts") { - val (code, inputStream, error) = - HistoryServerSuite.connectAndGetInputStream(generateURL("applications/local-1426533911241")) + doDownloadTest(None) + } + + test("download one log for app with multiple attempts") { + (1 to 2).foreach{ attemptId => doDownloadTest(Some(attemptId)) } + } + + // Test that the files are downloaded correctly, and validate them. + def doDownloadTest(attemptId: Option[Int]): Unit = { + + val url = attemptId match { + case Some(id) => + new URL(s"${generateURL("applications/local-1430917381535")}/$id/download") + case None => + new URL(s"${generateURL("applications/local-1430917381535")}/download") + } + + val (code, inputStream, error) = HistoryServerSuite.connectAndGetInputStream(url) code should be (HttpServletResponse.SC_OK) inputStream should not be None error should be (None) + + def validateFile(fileName: String, tempDir: File): Unit = { + val exp = IOUtils.toString(new FileInputStream(new File(logDir, fileName))) + val input = IOUtils.toString(new FileInputStream(new File(tempDir, fileName))) + input should be(exp) + } + + val dir = Utils.createTempDir() + try { + Utils.chmod700(dir) + unzipToDir(inputStream.get, dir) + val files = dir.listFiles() + attemptId match { + case Some(_) => files.length should be (1) + case None => files.length should be (2) + } + validateFile(files.head.getName, dir) + } finally { + Utils.deleteRecursively(dir) + } } test("response codes on bad paths") { @@ -234,14 +271,16 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with while(nextEntry != null) { val file = new File(dir, nextEntry.getName) val outputStream = new BufferedOutputStream(new FileOutputStream(file)) - var read = Integer.MAX_VALUE - while (read != -1) { - read = unzipStream.read(buffer) - outputStream.write(buffer, 0, read) + var dataRemains = true + while (dataRemains) { + val read = unzipStream.read(buffer) + if (read != -1) outputStream.write(buffer, 0, read) + else dataRemains = false } outputStream.close() nextEntry = unzipStream.getNextEntry } + unzipStream.close() } } From 4fc518c98e14a7e958c6afe074350e6068d47a20 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 27 May 2015 01:20:20 -0700 Subject: [PATCH 07/15] Fix rat failures. --- .rat-excludes | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.rat-excludes b/.rat-excludes index c0f81b57fe09d..936f8b7576162 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -80,5 +80,7 @@ local-1425081759269/* local-1426533911241/* local-1426633911242/* local-1430917381534/* +local-1430917381535_1 +local-1430917381535_2 DESCRIPTION NAMESPACE From 0b66948d41003cdd894f1268d32dd01585476269 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 27 May 2015 02:00:31 -0700 Subject: [PATCH 08/15] Minor formatting/import fixes. --- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 2 +- .../spark/status/api/v1/EventLogDownloadResource.scala | 4 ++-- .../apache/spark/status/api/v1/JacksonMessageWriter.scala | 2 +- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- .../spark/deploy/history/FsHistoryProviderSuite.scala | 3 +-- .../apache/spark/deploy/history/HistoryServerSuite.scala | 6 +++--- 6 files changed, 9 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 8621b42094f1c..3836e3ab8bda7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.history -import java.io.{File, FileOutputStream, BufferedInputStream, FileNotFoundException, IOException, InputStream} +import java.io.{BufferedInputStream, FileNotFoundException, IOException, InputStream} import java.util.concurrent.{ExecutorService, Executors, TimeUnit} import scala.collection.mutable diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala index e4b81f750356e..9d87d59f50885 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -16,9 +16,9 @@ */ package org.apache.spark.status.api.v1 -import java.io.{File, FileOutputStream, BufferedInputStream, FileInputStream, OutputStream} +import java.io.{File, FileOutputStream, OutputStream} import javax.ws.rs.{GET, Produces} -import javax.ws.rs.core.{Response, StreamingOutput, MediaType} +import javax.ws.rs.core.{MediaType, Response, StreamingOutput} import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala index 3d2a75bb526af..202a5191ad57d 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala @@ -68,7 +68,7 @@ private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ multivaluedMap: MultivaluedMap[String, AnyRef], outputStream: OutputStream): Unit = { t match { - case ErrorWrapper(err) => outputStream.write(err.getBytes()) + case ErrorWrapper(err) => outputStream.write(err.getBytes("utf-8")) case _ => mapper.writeValue(outputStream, t) } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 30cc66e9e07b0..560a544bb4685 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,8 +21,8 @@ import java.io._ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer -import java.util.zip.{ZipEntry, ZipOutputStream} import java.util.{PriorityQueue, Properties, Locale, Random, UUID} +import java.util.zip.{ZipEntry, ZipOutputStream} import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index df3af6b04e52c..5e2d0e3e3ff04 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -17,13 +17,12 @@ package org.apache.spark.deploy.history -import java.io.{ByteArrayOutputStream, FileInputStream, BufferedOutputStream, File, FileOutputStream, OutputStreamWriter} +import java.io.{BufferedOutputStream, FileInputStream, File, FileOutputStream, OutputStreamWriter} import java.net.URI import java.util.concurrent.TimeUnit import java.util.zip.ZipInputStream import scala.io.Source -import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index cfeb6bff274dc..0be81058a8c15 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.deploy.history -import java.io.{FileOutputStream, BufferedOutputStream, InputStream, File, FileInputStream, FileWriter, IOException} +import java.io.{ BufferedOutputStream, FileOutputStream, File, FileInputStream, + FileWriter, InputStream, IOException} import java.net.{HttpURLConnection, URL} import java.util.zip.ZipInputStream -import javax.servlet.http import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import org.apache.commons.io.{FileUtils, IOUtils} @@ -27,9 +27,9 @@ import org.mockito.Mockito.when import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import org.scalatest.mock.MockitoSugar -import org.apache.spark.util.Utils import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf} import org.apache.spark.ui.SparkUI +import org.apache.spark.util.Utils /** * A collection of tests against the historyserver, including comparing responses from the json From 5a5f3e210563070f27297d2bff2f64cfb51c960e Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 27 May 2015 09:21:26 -0700 Subject: [PATCH 09/15] Fix test ordering issue. --- .../application_list_json_expectation.json | 32 +++++++++---------- .../completed_app_list_json_expectation.json | 32 +++++++++---------- .../minDate_app_list_json_expectation.json | 32 +++++++++---------- .../spark-events/local-1430917381535_1 | 6 ++-- .../spark-events/local-1430917381535_2 | 4 +-- 5 files changed, 53 insertions(+), 53 deletions(-) diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index 26ac45f4d1b4b..d575bf2f284b9 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -7,6 +7,22 @@ "sparkUser" : "irashid", "completed" : true } ] +}, { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "sparkUser" : "irashid", + "completed" : true + } ] }, { "id" : "local-1426533911241", "name" : "Spark shell", @@ -41,22 +57,6 @@ "sparkUser" : "irashid", "completed" : true } ] -}, { - "id" : "local-1430917381535", - "name" : "Spark shell", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2015-05-06T13:03:00.893GMT", - "endTime" : "2015-02-03T16:42:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - }, { - "attemptId" : "2", - "startTime" : "2015-05-06T13:03:00.893GMT", - "endTime" : "2015-02-03T16:42:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - } ] }, { "id" : "local-1422981759269", "name" : "Spark shell", diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index 26ac45f4d1b4b..d575bf2f284b9 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -7,6 +7,22 @@ "sparkUser" : "irashid", "completed" : true } ] +}, { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "sparkUser" : "irashid", + "completed" : true + } ] }, { "id" : "local-1426533911241", "name" : "Spark shell", @@ -41,22 +57,6 @@ "sparkUser" : "irashid", "completed" : true } ] -}, { - "id" : "local-1430917381535", - "name" : "Spark shell", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2015-05-06T13:03:00.893GMT", - "endTime" : "2015-02-03T16:42:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - }, { - "attemptId" : "2", - "startTime" : "2015-05-06T13:03:00.893GMT", - "endTime" : "2015-02-03T16:42:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - } ] }, { "id" : "local-1422981759269", "name" : "Spark shell", diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index 7c8bfbf8de96d..15c2de8ef99ea 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -7,6 +7,22 @@ "sparkUser" : "irashid", "completed" : true } ] +}, { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "sparkUser" : "irashid", + "completed" : true + } ] }, { "id" : "local-1426533911241", "name" : "Spark shell", @@ -34,20 +50,4 @@ "completed": true } ] -}, { - "id" : "local-1430917381535", - "name" : "Spark shell", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2015-05-06T13:03:00.893GMT", - "endTime" : "2015-02-03T16:42:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - }, { - "attemptId" : "2", - "startTime" : "2015-05-06T13:03:00.893GMT", - "endTime" : "2015-02-03T16:42:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - } ] } ] \ No newline at end of file diff --git a/core/src/test/resources/spark-events/local-1430917381535_1 b/core/src/test/resources/spark-events/local-1430917381535_1 index ea27c67f4977a..d5a1303344825 100644 --- a/core/src/test/resources/spark-events/local-1430917381535_1 +++ b/core/src/test/resources/spark-events/local-1430917381535_1 @@ -1,5 +1,5 @@ {"Event":"SparkListenerLogStart","Spark Version":"1.4.0-SNAPSHOT"} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":61103},"Maximum Memory":278019440,"Timestamp":1430917381651} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":61103},"Maximum Memory":278019440,"Timestamp":1430917380880} {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.102","spark.eventLog.enabled":"true","spark.driver.port":"61101","spark.repl.class.uri":"http://192.168.1.102:61100","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.eventLog.dir":"/Users/irashid/github/kraps/core/src/test/resources/spark-events","spark.fileserver.uri":"http://192.168.1.102:61102","spark.tachyonStore.folderName":"spark-aaaf41b3-d1dd-447f-8951-acf51490758b","spark.app.id":"local-1430917381534"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/github/spark","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.eventLog.enabled=true --conf spark.eventLog.dir=/Users/irashid/github/kraps/core/src/test/resources/spark-events --class org.apache.spark.repl.Main spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/etc/hadoop":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/github/spark/conf/":"System Classpath","/Users/irashid/github/spark/assembly/target/scala-2.10/spark-assembly-1.4.0-SNAPSHOT-hadoop2.5.0.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1430917381535","Timestamp":1430917380893,"User":"irashid","App Attempt ID":"1"} -{"Event":"SparkListenerApplicationEnd","Timestamp":1422981766912} \ No newline at end of file +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1430917381535","Timestamp":1430917380880,"User":"irashid","App Attempt ID":"1"} +{"Event":"SparkListenerApplicationEnd","Timestamp":1430917380890} \ No newline at end of file diff --git a/core/src/test/resources/spark-events/local-1430917381535_2 b/core/src/test/resources/spark-events/local-1430917381535_2 index 35cce42fb70de..abb637a22e1e3 100644 --- a/core/src/test/resources/spark-events/local-1430917381535_2 +++ b/core/src/test/resources/spark-events/local-1430917381535_2 @@ -1,5 +1,5 @@ {"Event":"SparkListenerLogStart","Spark Version":"1.4.0-SNAPSHOT"} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":61103},"Maximum Memory":278019440,"Timestamp":1430917381651} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":61103},"Maximum Memory":278019440,"Timestamp":1430917380893} {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.102","spark.eventLog.enabled":"true","spark.driver.port":"61101","spark.repl.class.uri":"http://192.168.1.102:61100","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.eventLog.dir":"/Users/irashid/github/kraps/core/src/test/resources/spark-events","spark.fileserver.uri":"http://192.168.1.102:61102","spark.tachyonStore.folderName":"spark-aaaf41b3-d1dd-447f-8951-acf51490758b","spark.app.id":"local-1430917381534"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/github/spark","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.eventLog.enabled=true --conf spark.eventLog.dir=/Users/irashid/github/kraps/core/src/test/resources/spark-events --class org.apache.spark.repl.Main spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/etc/hadoop":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/github/spark/conf/":"System Classpath","/Users/irashid/github/spark/assembly/target/scala-2.10/spark-assembly-1.4.0-SNAPSHOT-hadoop2.5.0.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath"}} {"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1430917381535","Timestamp":1430917380893,"User":"irashid","App Attempt ID":"2"} -{"Event":"SparkListenerApplicationEnd","Timestamp":1422981766912} \ No newline at end of file +{"Event":"SparkListenerApplicationEnd","Timestamp":1430917380950} \ No newline at end of file From 1100b40bdc6d22cda525a1c6f701a465c232c8f3 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 27 May 2015 22:28:08 -0700 Subject: [PATCH 10/15] Ensure that `Path` does not appear in interfaces, by rafactoring interfaces. --- .../history/ApplicationHistoryProvider.scala | 13 +-- .../deploy/history/FsHistoryProvider.scala | 59 +++++++++++--- .../spark/deploy/history/HistoryServer.scala | 9 ++- .../spark/status/api/v1/ApiRootResource.scala | 11 ++- .../api/v1/EventLogDownloadResource.scala | 77 +++++------------- .../scala/org/apache/spark/util/Utils.scala | 6 -- .../history/FsHistoryProviderSuite.scala | 79 ++++++++----------- .../deploy/history/HistoryServerSuite.scala | 43 ++++------ .../deploy/history/HistoryTestUtils.scala | 42 ++++++++++ 9 files changed, 178 insertions(+), 161 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 1fe828ac3324b..0896f495db9c0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -17,8 +17,9 @@ package org.apache.spark.deploy.history -import org.apache.hadoop.fs.Path +import java.io.OutputStream +import org.apache.spark.SparkException import org.apache.spark.ui.SparkUI private[spark] case class ApplicationAttemptInfo( @@ -65,11 +66,11 @@ private[history] abstract class ApplicationHistoryProvider { def getConfig(): Map[String, String] = Map() /** - * Get the [[Path]]s to the Event log files. For legacy event log directories, directory path - * itself is returned. The caller is responsible for listing the files and using them as needed. - * If the attemptId is [[None]], event logs corresponding to all attempts for the given - * application are downloaded as a single zip file. + * Writes out the event logs to the output stream provided. The logs will be compressed into a + * single zip file and written out. + * @throws SparkException if the logs for the app id cannot be found. */ - def getEventLogPaths(appId: String, attemptId: Option[String]): Seq[Path] = Seq.empty + @throws(classOf[SparkException]) + def writeEventLogs(appId: String, attemptId: Option[String], outputStream: OutputStream): Unit } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 3836e3ab8bda7..12b2cf1374c50 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,17 +17,18 @@ package org.apache.spark.deploy.history -import java.io.{BufferedInputStream, FileNotFoundException, IOException, InputStream} +import java.io.{OutputStream, FileOutputStream, File, BufferedInputStream, + FileNotFoundException, IOException, InputStream} import java.util.concurrent.{ExecutorService, Executors, TimeUnit} import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer +import scala.util.control.NonFatal import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.fs.permission.AccessControlException -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ @@ -60,7 +61,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) .map { d => Utils.resolveURI(d).toString } .getOrElse(DEFAULT_LOG_DIR) - private val fs = Utils.getHadoopFileSystem(logDir, SparkHadoopUtil.get.newConfiguration(conf)) + private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + private val fs = Utils.getHadoopFileSystem(logDir, hadoopConf) // Used by check event thread and clean log thread. // Scheduled thread pool size must be one, otherwise it will have concurrent issues about fs @@ -220,16 +222,49 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - override def getEventLogPaths(appId: String, attemptId: Option[String]): Seq[Path] = { + override def writeEventLogs( + appId: String, + attemptId: Option[String], + outputStream: OutputStream): Unit = { - val filePaths = new ArrayBuffer[Path]() - applications.get(appId).foreach { appInfo => - // If no attempt is specified, or there is no attemptId for attempts, return all attempts - appInfo.attempts.filter { attempt => - attempt.attemptId.isEmpty || attemptId.isEmpty || attempt.attemptId.get == attemptId.get - }.foreach { attempt => filePaths += new Path(logDir, attempt.logPath) } + applications.get(appId) match { + case Some(appInfo) => + val dirsToClear = new mutable.ArrayBuffer[File]() + try { + // If no attempt is specified, or there is no attemptId for attempts, return all attempts + val pathsToZip = appInfo.attempts.filter { attempt => + attempt.attemptId.isEmpty || attemptId.isEmpty || attempt.attemptId.get == attemptId.get + }.map { attempt => + val logPath = new Path(logDir, attempt.logPath) + if (isLegacyLogDirectory(fs.getFileStatus(logPath))) { + val localDir = Utils.createTempDir() + Utils.chmod700(localDir) + dirsToClear += localDir + val outputFile = new File(localDir, logPath.getName) + val outputStream = new FileOutputStream(outputFile) + val files = fs.listFiles(logPath, false) + val paths = new mutable.ArrayBuffer[Path]() + while (files.hasNext) { + paths += files.next().getPath + } + Utils.zipFilesToStream(paths, hadoopConf, outputStream) + new Path(outputFile.toURI) + } else { + new Path(logDir, attempt.logPath) + } + } + Utils.zipFilesToStream(pathsToZip, hadoopConf, outputStream) + } finally { + dirsToClear.foreach { dir => + try { + Utils.deleteRecursively(dir) + } catch { + case NonFatal(e) => logWarning(s"Error while attempting to delete $dir.") + } + } + } + case None => throw new SparkException(s"Logs for $appId not found.") } - filePaths } 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 9d1ac2d107d98..6ef09751da8b7 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 @@ -17,11 +17,11 @@ package org.apache.spark.deploy.history +import java.io.OutputStream import java.util.NoSuchElementException import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import com.google.common.cache._ -import org.apache.hadoop.fs.Path import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} @@ -174,8 +174,11 @@ class HistoryServer( getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) } - override def getEventLogPaths(appId: String, attemptId: Option[String]): Seq[Path] = { - provider.getEventLogPaths(appId, attemptId) + override def writeEventLogs( + appId: String, + attemptId: Option[String], + outputStream: OutputStream): Unit = { + provider.writeEventLogs(appId, attemptId, outputStream) } /** diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index 6436bd3b169a4..731119c5549b2 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -16,13 +16,13 @@ */ package org.apache.spark.status.api.v1 +import java.io.OutputStream import javax.servlet.ServletContext import javax.ws.rs._ import javax.ws.rs.core.{Context, Response} import com.sun.jersey.api.core.ResourceConfig import com.sun.jersey.spi.container.servlet.ServletContainer -import org.apache.hadoop.fs import org.eclipse.jetty.server.handler.ContextHandler import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} @@ -165,13 +165,13 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { } } - @Path("applications/{appId}/download") + @Path("applications/{appId}/logs") def getEventLogs( @PathParam("appId") appId: String): EventLogDownloadResource = { new EventLogDownloadResource(uiRoot, appId, None) } - @Path("applications/{appId}/{attemptId}/download") + @Path("applications/{appId}/{attemptId}/logs") def getEventLogs( @PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): EventLogDownloadResource = { @@ -205,7 +205,10 @@ private[spark] object ApiRootResource { private[spark] trait UIRoot { def getSparkUI(appKey: String): Option[SparkUI] def getApplicationInfoList: Iterator[ApplicationInfo] - def getEventLogPaths(appId: String, attemptId: Option[String]): Seq[fs.Path] = Seq.empty + def writeEventLogs( + appId: String, + attemptId: Option[String], + outputStream: OutputStream): Unit = { } /** * Get the spark UI with the given appID, and apply a function diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala index 9d87d59f50885..72f43f887c2fb 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -16,18 +16,15 @@ */ package org.apache.spark.status.api.v1 -import java.io.{File, FileOutputStream, OutputStream} +import java.io.OutputStream import javax.ws.rs.{GET, Produces} import javax.ws.rs.core.{MediaType, Response, StreamingOutput} -import scala.collection.mutable.ArrayBuffer - -import org.apache.hadoop.fs.Path +import scala.util.control.NonFatal import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer -import org.apache.spark.util.Utils @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) private[v1] class EventLogDownloadResource( @@ -40,30 +37,29 @@ private[v1] class EventLogDownloadResource( def getEventLogs(): Response = { uIRoot match { case hs: HistoryServer => - var logsNotFound = false - val fileName = { - attemptId match { - case Some(id) => s"eventLogs-$appId-$id.zip" - case None => s"eventLogs-$appId.zip" + try { + val fileName = { + attemptId match { + case Some(id) => s"eventLogs-$appId-$id.zip" + case None => s"eventLogs-$appId.zip" + } } - } - val stream = new StreamingOutput { - override def write(output: OutputStream): Unit = { - val eventLogs = hs.getEventLogPaths(appId, attemptId) - if (eventLogs.isEmpty) logsNotFound = true - else zipLogFiles(eventLogs, output) + + val stream = new StreamingOutput { + override def write(output: OutputStream) = hs.writeEventLogs(appId, attemptId, output) } - } - if (logsNotFound) { - Response.serverError() - .entity(s"Event logs are not available for app: $appId.") - .status(Response.Status.SERVICE_UNAVAILABLE) - .build() - } else { + Response.ok(stream) .header("Content-Disposition", s"attachment; filename=$fileName") .header("Content-Type", MediaType.APPLICATION_OCTET_STREAM) .build() + + } catch { + case NonFatal(e) => + Response.serverError() + .entity(s"Event logs are not available for app: $appId.") + .status(Response.Status.SERVICE_UNAVAILABLE) + .build() } case _ => Response.serverError() @@ -72,39 +68,4 @@ private[v1] class EventLogDownloadResource( .build() } } - - private def zipLogFiles(eventLogs: Seq[Path], output: OutputStream): Unit = { - val areLegacyLogs = eventLogs.headOption.exists { path => - path.getFileSystem(conf).isDirectory(path) - } - val pathsToZip = if (areLegacyLogs) { - new ArrayBuffer[Path]() - } else { - eventLogs - } - var tempDir: File = null - try { - if (areLegacyLogs) { - tempDir = Utils.createTempDir() - Utils.chmod700(tempDir) - eventLogs.foreach { logPath => - // If the event logs are directories (legacy), then create a zip file for each - // one and write each of these files to the eventual output. - val fs = logPath.getFileSystem(conf) - val logFiles = fs.listFiles(logPath, true) - val zipFile = new File(tempDir, logPath.getName + ".zip") - pathsToZip.asInstanceOf[ArrayBuffer[Path]] += new Path(zipFile.toURI) - val outputStream = new FileOutputStream(zipFile) - val paths = new ArrayBuffer[Path]() - while (logFiles.hasNext) { - paths += logFiles.next().getPath - } - Utils.zipFilesToStream(paths, conf, outputStream) - } - } - Utils.zipFilesToStream(pathsToZip, conf, output) - } finally { - if (tempDir != null) Utils.deleteRecursively(tempDir) - } - } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 560a544bb4685..59f0ce73933c6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -786,12 +786,6 @@ private[spark] object Utils extends Logging { files: Seq[Path], hadoopConf: Configuration, outputStream: OutputStream): Unit = { - - // Passing in an output stream actually makes this more efficient since we don't have to - // create an additional file to which the compressed data is written which has to be read - // again by the reader, especially if the data needs to be sent over the wire via an - // OutputStream - in which case the destination output stream can be directly passed in here. - val fs = FileSystem.get(hadoopConf) val buffer = new Array[Byte](64 * 1024) val zipStream = Some(new ZipOutputStream(outputStream)) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 5e2d0e3e3ff04..cc3820f95b378 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -20,10 +20,10 @@ package org.apache.spark.deploy.history import java.io.{BufferedOutputStream, FileInputStream, File, FileOutputStream, OutputStreamWriter} import java.net.URI import java.util.concurrent.TimeUnit -import java.util.zip.ZipInputStream import scala.io.Source +import org.apache.commons.io.IOUtils import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} @@ -337,61 +337,48 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("Event log copy") { - - def getFileContent(file: File): Array[Byte] = { - val buff = new Array[Byte](file.length().toInt) - val in = new FileInputStream(file) - try { - in.read(buff) - } finally { - in.close() - } - buff + val provider = new FsHistoryProvider(createTestConf()) + val logs = (1 to 2).map { i => + val log = newLogFile("downloadApp1", Some(s"attempt$i"), inProgress = false) + writeFile(log, true, None, + SparkListenerApplicationStart( + "downloadApp1", Some("downloadApp1"), 5000 * i, "test", Some(s"attempt$i")), + SparkListenerApplicationEnd(5001 * i) + ) + log } + provider.checkForLogs() - def unzipToDir(zipFile: File, outputDir: File): Unit = { - val zipStream = new ZipInputStream(new FileInputStream(zipFile)) + (1 to 2).foreach { i => + val outDir = Utils.createTempDir() + val unzipDir = Utils.createTempDir() try { - val buffer = new Array[Byte](128) - var entry = zipStream.getNextEntry - while (entry != null) { - val unzippedFile = new File(outputDir, entry.getName) - val ostream = new BufferedOutputStream(new FileOutputStream(unzippedFile)) + Utils.chmod700(outDir) + Utils.chmod700(unzipDir) + val outFile = new File(outDir, s"file$i.zip") + val outputStream = new FileOutputStream(outFile) + provider.writeEventLogs("downloadApp1", Some(s"attempt$i"), outputStream) + HistoryTestUtils.unzipToDir(new FileInputStream(outFile), unzipDir) + unzipDir.listFiles().foreach { log => + val inFile = logs.find(_.getName == log.getName).get + val expStream = new FileInputStream(inFile) + val resultStream = new FileInputStream(log) try { - var dataRemains = true - while (dataRemains) { - val read = zipStream.read(buffer) - if (read > 0) ostream.write(buffer, 0, read) else dataRemains = false - } + val input = IOUtils.toString(expStream) + val out = IOUtils.toString(resultStream) + out should be(input) } finally { - ostream.close() + Seq(expStream, resultStream).foreach { s => + Utils.tryWithSafeFinally(s.close())() + } } - zipStream.closeEntry() - entry = zipStream.getNextEntry } } finally { - zipStream.close() + Seq(outDir, unzipDir).foreach { f => + Utils.tryWithSafeFinally(Utils.deleteRecursively(f))() + } } } - - val provider = new FsHistoryProvider(createTestConf()) - val log1 = newLogFile("downloadApp1", Some("attempt1"), inProgress = false) - writeFile(log1, true, None, - SparkListenerApplicationStart("downloadApp1", Some("downloadApp1"), System - .currentTimeMillis() - 400, "test", Some("attempt1")), - SparkListenerApplicationEnd(System.currentTimeMillis() - 200) - ) - val log2 = newLogFile("downloadApp1", Some("attempt2"), inProgress = false) - writeFile(log2, true, None, - SparkListenerApplicationStart("downloadApp1", Some("downloadApp1"), System - .currentTimeMillis() - 100, "test", Some("attempt2")), - SparkListenerApplicationEnd(System.currentTimeMillis()) - ) - provider.checkForLogs() - provider.getEventLogPaths("downloadApp1", Some("attempt1")) - .head.getName should be (log1.getName) - provider.getEventLogPaths("downloadApp1", Some("attempt2")) - .head.getName should be (log2.getName) } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 0be81058a8c15..d862d209534e4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -22,6 +22,8 @@ import java.net.{HttpURLConnection, URL} import java.util.zip.ZipInputStream import javax.servlet.http.{HttpServletRequest, HttpServletResponse} +import scala.util.control.NonFatal + import org.apache.commons.io.{FileUtils, IOUtils} import org.mockito.Mockito.when import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} @@ -155,7 +157,7 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with } test("download one log for app with multiple attempts") { - (1 to 2).foreach{ attemptId => doDownloadTest(Some(attemptId)) } + (1 to 2).foreach { attemptId => doDownloadTest(Some(attemptId)) } } // Test that the files are downloaded correctly, and validate them. @@ -163,9 +165,9 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with val url = attemptId match { case Some(id) => - new URL(s"${generateURL("applications/local-1430917381535")}/$id/download") + new URL(s"${generateURL("applications/local-1430917381535")}/$id/logs") case None => - new URL(s"${generateURL("applications/local-1430917381535")}/download") + new URL(s"${generateURL("applications/local-1430917381535")}/logs") } val (code, inputStream, error) = HistoryServerSuite.connectAndGetInputStream(url) @@ -174,15 +176,23 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with error should be (None) def validateFile(fileName: String, tempDir: File): Unit = { - val exp = IOUtils.toString(new FileInputStream(new File(logDir, fileName))) - val input = IOUtils.toString(new FileInputStream(new File(tempDir, fileName))) - input should be(exp) + val inStream = new FileInputStream(new File(logDir, fileName)) + val outStream = new FileInputStream(new File(tempDir, fileName)) + try { + val exp = IOUtils.toString(inStream) + val input = IOUtils.toString(outStream) + input should be(exp) + } finally { + Seq(inStream, outStream).foreach { s => + Utils.tryWithSafeFinally(s.close())() + } + } } val dir = Utils.createTempDir() try { Utils.chmod700(dir) - unzipToDir(inputStream.get, dir) + HistoryTestUtils.unzipToDir(inputStream.get, dir) val files = dir.listFiles() attemptId match { case Some(_) => files.length should be (1) @@ -263,25 +273,6 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with out.write(json) out.close() } - - def unzipToDir(inputStream: InputStream, dir: File): Unit = { - val unzipStream = new ZipInputStream(inputStream) - val buffer = new Array[Byte](64 * 1024 * 1024) - var nextEntry = unzipStream.getNextEntry - while(nextEntry != null) { - val file = new File(dir, nextEntry.getName) - val outputStream = new BufferedOutputStream(new FileOutputStream(file)) - var dataRemains = true - while (dataRemains) { - val read = unzipStream.read(buffer) - if (read != -1) outputStream.write(buffer, 0, read) - else dataRemains = false - } - outputStream.close() - nextEntry = unzipStream.getNextEntry - } - unzipStream.close() - } } object HistoryServerSuite { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala new file mode 100644 index 0000000000000..7697a29a646a3 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.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 + +import java.io.{BufferedOutputStream, File, FileOutputStream, InputStream} +import java.util.zip.ZipInputStream + +object HistoryTestUtils { + + def unzipToDir(inputStream: InputStream, dir: File): Unit = { + val unzipStream = new ZipInputStream(inputStream) + val buffer = new Array[Byte](64 * 1024 * 1024) + var nextEntry = unzipStream.getNextEntry + while(nextEntry != null) { + val file = new File(dir, nextEntry.getName) + val outputStream = new BufferedOutputStream(new FileOutputStream(file)) + var dataRemains = true + while (dataRemains) { + val read = unzipStream.read(buffer) + if (read != -1) outputStream.write(buffer, 0, read) + else dataRemains = false + } + outputStream.close() + nextEntry = unzipStream.getNextEntry + } + unzipStream.close() + } +} From ffffb537a965d9c346bd2d53ba9a9db5922abf4f Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sat, 30 May 2015 12:33:33 -0700 Subject: [PATCH 11/15] Changed interface to use zip stream. Added more tests. --- .../history/ApplicationHistoryProvider.scala | 4 +- .../deploy/history/FsHistoryProvider.scala | 66 +++++++++------ .../spark/deploy/history/HistoryServer.scala | 6 +- .../spark/status/api/v1/ApiRootResource.scala | 4 +- .../api/v1/EventLogDownloadResource.scala | 11 ++- .../scala/org/apache/spark/util/Utils.scala | 37 --------- .../history/FsHistoryProviderSuite.scala | 13 +-- .../deploy/history/HistoryServerSuite.scala | 80 ++++++++++++------- .../deploy/history/HistoryTestUtils.scala | 40 +++++++--- 9 files changed, 144 insertions(+), 117 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 0896f495db9c0..5f5e0fe1c34d7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.history -import java.io.OutputStream +import java.util.zip.ZipOutputStream import org.apache.spark.SparkException import org.apache.spark.ui.SparkUI @@ -71,6 +71,6 @@ private[history] abstract class ApplicationHistoryProvider { * @throws SparkException if the logs for the app id cannot be found. */ @throws(classOf[SparkException]) - def writeEventLogs(appId: String, attemptId: Option[String], outputStream: OutputStream): Unit + def writeEventLogs(appId: String, attemptId: Option[String], zipStream: ZipOutputStream): Unit } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 12b2cf1374c50..e468c228b612b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,15 +17,14 @@ package org.apache.spark.deploy.history -import java.io.{OutputStream, FileOutputStream, File, BufferedInputStream, - FileNotFoundException, IOException, InputStream} +import java.io.{BufferedInputStream, FileNotFoundException, InputStream, IOException, OutputStream} import java.util.concurrent.{ExecutorService, Executors, TimeUnit} +import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.mutable -import scala.util.control.NonFatal import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.fs.permission.AccessControlException import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} @@ -225,43 +224,58 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def writeEventLogs( appId: String, attemptId: Option[String], - outputStream: OutputStream): Unit = { + zipStream: ZipOutputStream): Unit = { + + /** + * This method compresses the files passed in, and writes the compressed data out into the + * [[OutputStream]] passed in. Each file is written as a new [[ZipEntry]] with its name being + * the name of the file being compressed. + */ + def zipFileToStream(file: Path, entryName: String, outputStream: ZipOutputStream): Unit = { + val fs = FileSystem.get(hadoopConf) + val buffer = new Array[Byte](64 * 1024) + val inputStream = fs.open(file, 1 * 1024 * 1024) // 1MB Buffer + try { + outputStream.putNextEntry(new ZipEntry(entryName)) + var dataRemaining = true + while (dataRemaining) { + val length = inputStream.read(buffer) + if (length != -1) { + outputStream.write(buffer, 0, length) + } else { + dataRemaining = false + } + } + outputStream.closeEntry() + } finally { + inputStream.close() + } + } applications.get(appId) match { case Some(appInfo) => - val dirsToClear = new mutable.ArrayBuffer[File]() try { // If no attempt is specified, or there is no attemptId for attempts, return all attempts - val pathsToZip = appInfo.attempts.filter { attempt => + appInfo.attempts.filter { attempt => attempt.attemptId.isEmpty || attemptId.isEmpty || attempt.attemptId.get == attemptId.get - }.map { attempt => + }.foreach { attempt => val logPath = new Path(logDir, attempt.logPath) + // If this is a legacy directory, then add the directory to the zipStream and add + // each file to that directory. if (isLegacyLogDirectory(fs.getFileStatus(logPath))) { - val localDir = Utils.createTempDir() - Utils.chmod700(localDir) - dirsToClear += localDir - val outputFile = new File(localDir, logPath.getName) - val outputStream = new FileOutputStream(outputFile) val files = fs.listFiles(logPath, false) - val paths = new mutable.ArrayBuffer[Path]() + zipStream.putNextEntry(new ZipEntry(attempt.logPath + "/")) + zipStream.closeEntry() while (files.hasNext) { - paths += files.next().getPath + val file = files.next().getPath + zipFileToStream(file, attempt.logPath + Path.SEPARATOR + file.getName, zipStream) } - Utils.zipFilesToStream(paths, hadoopConf, outputStream) - new Path(outputFile.toURI) } else { - new Path(logDir, attempt.logPath) + zipFileToStream(new Path(logDir, attempt.logPath), attempt.logPath, zipStream) } } - Utils.zipFilesToStream(pathsToZip, hadoopConf, outputStream) } finally { - dirsToClear.foreach { dir => - try { - Utils.deleteRecursively(dir) - } catch { - case NonFatal(e) => logWarning(s"Error while attempting to delete $dir.") - } - } + zipStream.close() } case None => throw new SparkException(s"Logs for $appId not found.") } 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 6ef09751da8b7..10638afb74900 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 @@ -17,8 +17,8 @@ package org.apache.spark.deploy.history -import java.io.OutputStream import java.util.NoSuchElementException +import java.util.zip.ZipOutputStream import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import com.google.common.cache._ @@ -177,8 +177,8 @@ class HistoryServer( override def writeEventLogs( appId: String, attemptId: Option[String], - outputStream: OutputStream): Unit = { - provider.writeEventLogs(appId, attemptId, outputStream) + zipStream: ZipOutputStream): Unit = { + provider.writeEventLogs(appId, attemptId, zipStream) } /** diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index 731119c5549b2..b5735a66dc8f0 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.status.api.v1 -import java.io.OutputStream +import java.util.zip.ZipOutputStream import javax.servlet.ServletContext import javax.ws.rs._ import javax.ws.rs.core.{Context, Response} @@ -208,7 +208,7 @@ private[spark] trait UIRoot { def writeEventLogs( appId: String, attemptId: Option[String], - outputStream: OutputStream): Unit = { } + zipStream: ZipOutputStream): Unit = { } /** * Get the spark UI with the given appID, and apply a function diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala index 72f43f887c2fb..77b6d07a0f86f 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -17,6 +17,7 @@ package org.apache.spark.status.api.v1 import java.io.OutputStream +import java.util.zip.ZipOutputStream import javax.ws.rs.{GET, Produces} import javax.ws.rs.core.{MediaType, Response, StreamingOutput} @@ -46,7 +47,15 @@ private[v1] class EventLogDownloadResource( } val stream = new StreamingOutput { - override def write(output: OutputStream) = hs.writeEventLogs(appId, attemptId, output) + override def write(output: OutputStream) = { + val zipStream = new ZipOutputStream(output) + try { + hs.writeEventLogs(appId, attemptId, zipStream) + } finally { + zipStream.close() + } + + } } Response.ok(stream) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 59f0ce73933c6..6a7d1fae3320e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -22,7 +22,6 @@ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer import java.util.{PriorityQueue, Properties, Locale, Random, UUID} -import java.util.zip.{ZipEntry, ZipOutputStream} import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection @@ -777,42 +776,6 @@ private[spark] object Utils extends Logging { localRootDirs = null } - /** - * This method compresses the files passed in, and writes the compressed data out into the - * [[OutputStream]] passed in. Each file is written as a new [[ZipEntry]] with its name being - * the name of the file being compressed. - */ - private[spark] def zipFilesToStream( - files: Seq[Path], - hadoopConf: Configuration, - outputStream: OutputStream): Unit = { - val fs = FileSystem.get(hadoopConf) - val buffer = new Array[Byte](64 * 1024) - val zipStream = Some(new ZipOutputStream(outputStream)) - try { - files.foreach { remotePath => - val inputStream = Some(fs.open(remotePath, 1 * 1024 * 1024)) // 1MB Buffer - try { - zipStream.get.putNextEntry(new ZipEntry(remotePath.getName)) - var dataRemaining = true - while (dataRemaining) { - val length = inputStream.get.read(buffer) - if (length != -1) { - zipStream.get.write(buffer, 0, length) - } else { - dataRemaining = false - } - } - zipStream.get.closeEntry() - } finally { - inputStream.foreach(_.close()) - } - } - } finally { - zipStream.foreach(_.close()) - } - } - /** * Shuffle the elements of a collection into a random order, returning the * result in a new collection. Unlike scala.util.Random.shuffle, this method diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index cc3820f95b378..c92a72c6bd975 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.history import java.io.{BufferedOutputStream, FileInputStream, File, FileOutputStream, OutputStreamWriter} import java.net.URI import java.util.concurrent.TimeUnit +import java.util.zip.ZipOutputStream import scala.io.Source @@ -356,13 +357,15 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers Utils.chmod700(outDir) Utils.chmod700(unzipDir) val outFile = new File(outDir, s"file$i.zip") - val outputStream = new FileOutputStream(outFile) + val outputStream = new ZipOutputStream(new FileOutputStream(outFile)) provider.writeEventLogs("downloadApp1", Some(s"attempt$i"), outputStream) HistoryTestUtils.unzipToDir(new FileInputStream(outFile), unzipDir) - unzipDir.listFiles().foreach { log => - val inFile = logs.find(_.getName == log.getName).get - val expStream = new FileInputStream(inFile) - val resultStream = new FileInputStream(log) + val actualFiles = unzipDir.listFiles() + assert(actualFiles.length == 1) + actualFiles.foreach { actualFile => + val expFile = logs.find(_.getName == actualFile.getName).get + val expStream = new FileInputStream(expFile) + val resultStream = new FileInputStream(actualFile) try { val input = IOUtils.toString(expStream) val out = IOUtils.toString(resultStream) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index d862d209534e4..ff3cf6765e3dd 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -16,14 +16,10 @@ */ package org.apache.spark.deploy.history -import java.io.{ BufferedOutputStream, FileOutputStream, File, FileInputStream, - FileWriter, InputStream, IOException} +import java.io.{File, FileInputStream, FileWriter, InputStream, IOException} import java.net.{HttpURLConnection, URL} -import java.util.zip.ZipInputStream import javax.servlet.http.{HttpServletRequest, HttpServletResponse} -import scala.util.control.NonFatal - import org.apache.commons.io.{FileUtils, IOUtils} import org.mockito.Mockito.when import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} @@ -153,21 +149,43 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with } test("download all logs for app with multiple attempts") { - doDownloadTest(None) + doDownloadTest("local-1430917381535", None) } test("download one log for app with multiple attempts") { - (1 to 2).foreach { attemptId => doDownloadTest(Some(attemptId)) } + (1 to 2).foreach { attemptId => doDownloadTest("local-1430917381535", Some(attemptId)) } + } + + test("download legacy logs - all attempts") { + doDownloadTest("local-1426533911241", None, legacy = true) + } + + test("download legacy logs - single attempts") { + (1 to 2). foreach { + attemptId => doDownloadTest("local-1426533911241", Some(attemptId), legacy = true) + } } // Test that the files are downloaded correctly, and validate them. - def doDownloadTest(attemptId: Option[Int]): Unit = { + def doDownloadTest(appId: String, attemptId: Option[Int], legacy: Boolean = false): Unit = { + + def validateFile(expStream: FileInputStream, actualStream: FileInputStream): Unit = { + try { + val expected = IOUtils.toString(expStream) + val actual = IOUtils.toString(actualStream) + actual should be(expected) + } finally { + Seq(expStream, actualStream).foreach { s => + Utils.tryWithSafeFinally(s.close())() + } + } + } val url = attemptId match { case Some(id) => - new URL(s"${generateURL("applications/local-1430917381535")}/$id/logs") + new URL(s"${generateURL(s"applications/$appId")}/$id/logs") case None => - new URL(s"${generateURL("applications/local-1430917381535")}/logs") + new URL(s"${generateURL(s"applications/$appId")}/logs") } val (code, inputStream, error) = HistoryServerSuite.connectAndGetInputStream(url) @@ -175,30 +193,36 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with inputStream should not be None error should be (None) - def validateFile(fileName: String, tempDir: File): Unit = { - val inStream = new FileInputStream(new File(logDir, fileName)) - val outStream = new FileInputStream(new File(tempDir, fileName)) - try { - val exp = IOUtils.toString(inStream) - val input = IOUtils.toString(outStream) - input should be(exp) - } finally { - Seq(inStream, outStream).foreach { s => - Utils.tryWithSafeFinally(s.close())() - } - } - } - val dir = Utils.createTempDir() try { Utils.chmod700(dir) HistoryTestUtils.unzipToDir(inputStream.get, dir) - val files = dir.listFiles() + val unzippedContent = dir.listFiles() attemptId match { - case Some(_) => files.length should be (1) - case None => files.length should be (2) + case Some(_) => unzippedContent.length should be (1) + case None => unzippedContent.length should be (2) + } + + // If these are legacy files, then each of the unzipped contents is actually a legacy log dir. + if (legacy) { + unzippedContent.foreach { legacyDir => + assert(legacyDir.isDirectory) + val logFiles = legacyDir.listFiles() + logFiles.length should be (3) + logFiles.foreach { f => + val actualStream = new FileInputStream(f) + val expectedStream = + new FileInputStream(new File(new File(logDir, legacyDir.getName), f.getName)) + validateFile(expectedStream, actualStream) + } + } + } else { + unzippedContent.foreach { f => + val actualStream = new FileInputStream(f) + val expectedStream = new FileInputStream(new File(logDir, f.getName)) + validateFile(expectedStream, actualStream) + } } - validateFile(files.head.getName, dir) } finally { Utils.deleteRecursively(dir) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala index 7697a29a646a3..cfa048c3d6001 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala @@ -23,20 +23,34 @@ object HistoryTestUtils { def unzipToDir(inputStream: InputStream, dir: File): Unit = { val unzipStream = new ZipInputStream(inputStream) - val buffer = new Array[Byte](64 * 1024 * 1024) - var nextEntry = unzipStream.getNextEntry - while(nextEntry != null) { - val file = new File(dir, nextEntry.getName) - val outputStream = new BufferedOutputStream(new FileOutputStream(file)) - var dataRemains = true - while (dataRemains) { - val read = unzipStream.read(buffer) - if (read != -1) outputStream.write(buffer, 0, read) - else dataRemains = false + try { + val buffer = new Array[Byte](64 * 1024 * 1024) + var nextEntry = unzipStream.getNextEntry + while (nextEntry != null) { + var file: File = null + if (nextEntry.isDirectory) { + val newDir = new File(dir, nextEntry.getName) + newDir.mkdirs() + } else { + val splits = nextEntry.getName.split("/") + if (splits.length == 2) { + file = new File(new File(dir, splits(0)), splits(1)) + } else { + file = new File(dir, nextEntry.getName) + } + val outputStream = new BufferedOutputStream(new FileOutputStream(file)) + var dataRemains = true + while (dataRemains) { + val read = unzipStream.read(buffer) + if (read != -1) outputStream.write(buffer, 0, read) + else dataRemains = false + } + outputStream.close() + } + nextEntry = unzipStream.getNextEntry } - outputStream.close() - nextEntry = unzipStream.getNextEntry + } finally { + unzipStream.close() } - unzipStream.close() } } From d8ddedec49dd147c6eb2b89bc0542651309bf5a4 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sat, 30 May 2015 12:40:55 -0700 Subject: [PATCH 12/15] Remove unnecessary case in EventLogDownloadResource. --- .../deploy/history/FsHistoryProvider.scala | 2 +- .../spark/status/api/v1/ApiRootResource.scala | 11 ++-- .../api/v1/EventLogDownloadResource.scala | 56 ++++++++----------- 3 files changed, 31 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index e468c228b612b..de6b0bcd5c4a6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -27,7 +27,7 @@ import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.fs.permission.AccessControlException -import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index b5735a66dc8f0..6fe6988e1c15b 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -205,10 +205,13 @@ private[spark] object ApiRootResource { private[spark] trait UIRoot { def getSparkUI(appKey: String): Option[SparkUI] def getApplicationInfoList: Iterator[ApplicationInfo] - def writeEventLogs( - appId: String, - attemptId: Option[String], - zipStream: ZipOutputStream): Unit = { } + + def writeEventLogs(appId: String, attemptId: Option[String], zipStream: ZipOutputStream): Unit = { + Response.serverError() + .entity("Event logs are only available through the history server.") + .status(Response.Status.SERVICE_UNAVAILABLE) + .build() + } /** * Get the spark UI with the given appID, and apply a function diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala index 77b6d07a0f86f..d416dba8324d8 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -25,7 +25,6 @@ import scala.util.control.NonFatal import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.history.HistoryServer @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) private[v1] class EventLogDownloadResource( @@ -36,43 +35,34 @@ private[v1] class EventLogDownloadResource( @GET def getEventLogs(): Response = { - uIRoot match { - case hs: HistoryServer => - try { - val fileName = { - attemptId match { - case Some(id) => s"eventLogs-$appId-$id.zip" - case None => s"eventLogs-$appId.zip" - } - } - - val stream = new StreamingOutput { - override def write(output: OutputStream) = { - val zipStream = new ZipOutputStream(output) - try { - hs.writeEventLogs(appId, attemptId, zipStream) - } finally { - zipStream.close() - } + try { + val fileName = { + attemptId match { + case Some(id) => s"eventLogs-$appId-$id.zip" + case None => s"eventLogs-$appId.zip" + } + } - } + val stream = new StreamingOutput { + override def write(output: OutputStream) = { + val zipStream = new ZipOutputStream(output) + try { + uIRoot.writeEventLogs(appId, attemptId, zipStream) + } finally { + zipStream.close() } - Response.ok(stream) - .header("Content-Disposition", s"attachment; filename=$fileName") - .header("Content-Type", MediaType.APPLICATION_OCTET_STREAM) - .build() - - } catch { - case NonFatal(e) => - Response.serverError() - .entity(s"Event logs are not available for app: $appId.") - .status(Response.Status.SERVICE_UNAVAILABLE) - .build() } - case _ => + } + + Response.ok(stream) + .header("Content-Disposition", s"attachment; filename=$fileName") + .header("Content-Type", MediaType.APPLICATION_OCTET_STREAM) + .build() + } catch { + case NonFatal(e) => Response.serverError() - .entity("Event logs are only available through the history server.") + .entity(s"Event logs are not available for app: $appId.") .status(Response.Status.SERVICE_UNAVAILABLE) .build() } From 6e8156e7629d156a703cb14596156c0e150e591e Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 2 Jun 2015 18:39:01 -0700 Subject: [PATCH 13/15] Simplify tests, use Guava stream copy methods. --- .../deploy/history/FsHistoryProvider.scala | 12 +--- .../history/FsHistoryProviderSuite.scala | 53 ++++++--------- .../deploy/history/HistoryServerSuite.scala | 68 ++++++++----------- 3 files changed, 50 insertions(+), 83 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index de6b0bcd5c4a6..52b149b273e4b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -23,6 +23,7 @@ import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.mutable +import com.google.common.io.ByteStreams import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.fs.permission.AccessControlException @@ -233,19 +234,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) */ def zipFileToStream(file: Path, entryName: String, outputStream: ZipOutputStream): Unit = { val fs = FileSystem.get(hadoopConf) - val buffer = new Array[Byte](64 * 1024) val inputStream = fs.open(file, 1 * 1024 * 1024) // 1MB Buffer try { outputStream.putNextEntry(new ZipEntry(entryName)) - var dataRemaining = true - while (dataRemaining) { - val length = inputStream.read(buffer) - if (length != -1) { - outputStream.write(buffer, 0, length) - } else { - dataRemaining = false - } - } + ByteStreams.copy(inputStream, outputStream) outputStream.closeEntry() } finally { inputStream.close() diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index c92a72c6bd975..4b66c2d652087 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -17,14 +17,16 @@ package org.apache.spark.deploy.history -import java.io.{BufferedOutputStream, FileInputStream, File, FileOutputStream, OutputStreamWriter} +import java.io.{BufferedOutputStream, ByteArrayInputStream, ByteArrayOutputStream, File, + FileOutputStream, OutputStreamWriter} import java.net.URI import java.util.concurrent.TimeUnit -import java.util.zip.ZipOutputStream +import java.util.zip.{ZipInputStream, ZipOutputStream} import scala.io.Source -import org.apache.commons.io.IOUtils +import com.google.common.base.Charsets +import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} @@ -351,36 +353,23 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers provider.checkForLogs() (1 to 2).foreach { i => - val outDir = Utils.createTempDir() - val unzipDir = Utils.createTempDir() - try { - Utils.chmod700(outDir) - Utils.chmod700(unzipDir) - val outFile = new File(outDir, s"file$i.zip") - val outputStream = new ZipOutputStream(new FileOutputStream(outFile)) - provider.writeEventLogs("downloadApp1", Some(s"attempt$i"), outputStream) - HistoryTestUtils.unzipToDir(new FileInputStream(outFile), unzipDir) - val actualFiles = unzipDir.listFiles() - assert(actualFiles.length == 1) - actualFiles.foreach { actualFile => - val expFile = logs.find(_.getName == actualFile.getName).get - val expStream = new FileInputStream(expFile) - val resultStream = new FileInputStream(actualFile) - try { - val input = IOUtils.toString(expStream) - val out = IOUtils.toString(resultStream) - out should be(input) - } finally { - Seq(expStream, resultStream).foreach { s => - Utils.tryWithSafeFinally(s.close())() - } - } - } - } finally { - Seq(outDir, unzipDir).foreach { f => - Utils.tryWithSafeFinally(Utils.deleteRecursively(f))() - } + val underlyingStream = new ByteArrayOutputStream() + val outputStream = new ZipOutputStream(underlyingStream) + provider.writeEventLogs("downloadApp1", Some(s"attempt$i"), outputStream) + outputStream.close() + val inputStream = new ZipInputStream(new ByteArrayInputStream(underlyingStream.toByteArray)) + var totalEntries = 0 + var entry = inputStream.getNextEntry + entry should not be null + while (entry != null) { + val actual = new String(ByteStreams.toByteArray(inputStream), Charsets.UTF_8) + val expected = Files.toString(logs.find(_.getName == entry.getName).get, Charsets.UTF_8) + actual should be (expected) + totalEntries += 1 + entry = inputStream.getNextEntry } + totalEntries should be (1) + inputStream.close() } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index ff3cf6765e3dd..1e87aedd8e907 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -18,8 +18,11 @@ package org.apache.spark.deploy.history import java.io.{File, FileInputStream, FileWriter, InputStream, IOException} import java.net.{HttpURLConnection, URL} +import java.util.zip.ZipInputStream import javax.servlet.http.{HttpServletRequest, HttpServletResponse} +import com.google.common.base.Charsets +import com.google.common.io.{ByteStreams, Files} import org.apache.commons.io.{FileUtils, IOUtils} import org.mockito.Mockito.when import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} @@ -27,7 +30,6 @@ import org.scalatest.mock.MockitoSugar import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.Utils /** * A collection of tests against the historyserver, including comparing responses from the json @@ -169,18 +171,6 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with // Test that the files are downloaded correctly, and validate them. def doDownloadTest(appId: String, attemptId: Option[Int], legacy: Boolean = false): Unit = { - def validateFile(expStream: FileInputStream, actualStream: FileInputStream): Unit = { - try { - val expected = IOUtils.toString(expStream) - val actual = IOUtils.toString(actualStream) - actual should be(expected) - } finally { - Seq(expStream, actualStream).foreach { s => - Utils.tryWithSafeFinally(s.close())() - } - } - } - val url = attemptId match { case Some(id) => new URL(s"${generateURL(s"applications/$appId")}/$id/logs") @@ -193,39 +183,35 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with inputStream should not be None error should be (None) - val dir = Utils.createTempDir() - try { - Utils.chmod700(dir) - HistoryTestUtils.unzipToDir(inputStream.get, dir) - val unzippedContent = dir.listFiles() - attemptId match { - case Some(_) => unzippedContent.length should be (1) - case None => unzippedContent.length should be (2) - } - - // If these are legacy files, then each of the unzipped contents is actually a legacy log dir. + val zipStream = new ZipInputStream(inputStream.get) + var entry = zipStream.getNextEntry + entry should not be null + val totalFiles = { if (legacy) { - unzippedContent.foreach { legacyDir => - assert(legacyDir.isDirectory) - val logFiles = legacyDir.listFiles() - logFiles.length should be (3) - logFiles.foreach { f => - val actualStream = new FileInputStream(f) - val expectedStream = - new FileInputStream(new File(new File(logDir, legacyDir.getName), f.getName)) - validateFile(expectedStream, actualStream) - } - } + attemptId.map { x => 3 }.getOrElse(6) } else { - unzippedContent.foreach { f => - val actualStream = new FileInputStream(f) - val expectedStream = new FileInputStream(new File(logDir, f.getName)) - validateFile(expectedStream, actualStream) + attemptId.map { x => 1 }.getOrElse(2) + } + } + var filesCompared = 0 + while(entry != null) { + if (!entry.isDirectory) { + val expectedFile = { + if (legacy) { + val splits = entry.getName.split("/") + new File(new File(logDir, splits(0)), splits(1)) + } else { + new File(logDir, entry.getName) + } } + val expected = Files.toString(expectedFile, Charsets.UTF_8) + val actual = new String(ByteStreams.toByteArray(zipStream), Charsets.UTF_8) + actual should be (expected) + filesCompared += 1 } - } finally { - Utils.deleteRecursively(dir) + entry = zipStream.getNextEntry } + filesCompared should be (totalFiles) } test("response codes on bad paths") { From a131be6a6abd88bf1b84c18e2d1e625190f4fc54 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 2 Jun 2015 20:38:24 -0700 Subject: [PATCH 14/15] Fix style issues. --- .../history/FsHistoryProviderSuite.scala | 2 +- .../deploy/history/HistoryServerSuite.scala | 4 +- .../deploy/history/HistoryTestUtils.scala | 56 ------------------- 3 files changed, 3 insertions(+), 59 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 353cb1bc5a5a1..09075eeb539aa 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -21,7 +21,7 @@ import java.io.{BufferedOutputStream, ByteArrayInputStream, ByteArrayOutputStrea FileOutputStream, OutputStreamWriter} import java.net.URI import java.util.concurrent.TimeUnit -import java.util.zip.{ZipInputStream, ZipOutputStream} +import java.util.zip.{ZipInputStream, ZipOutputStream} import scala.io.Source diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 99f0a45df9867..e5b5e1bb65337 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -194,7 +194,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } } var filesCompared = 0 - while(entry != null) { + while (entry != null) { if (!entry.isDirectory) { val expectedFile = { if (legacy) { @@ -205,7 +205,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } } val expected = Files.toString(expectedFile, Charsets.UTF_8) - val actual = new String(ByteStreams.toByteArray(zipStream), Charsets.UTF_8) + val actual = new String(ByteStreams.toByteArray(zipStream), Charsets.UTF_8) actual should be (expected) filesCompared += 1 } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala deleted file mode 100644 index cfa048c3d6001..0000000000000 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryTestUtils.scala +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.history - -import java.io.{BufferedOutputStream, File, FileOutputStream, InputStream} -import java.util.zip.ZipInputStream - -object HistoryTestUtils { - - def unzipToDir(inputStream: InputStream, dir: File): Unit = { - val unzipStream = new ZipInputStream(inputStream) - try { - val buffer = new Array[Byte](64 * 1024 * 1024) - var nextEntry = unzipStream.getNextEntry - while (nextEntry != null) { - var file: File = null - if (nextEntry.isDirectory) { - val newDir = new File(dir, nextEntry.getName) - newDir.mkdirs() - } else { - val splits = nextEntry.getName.split("/") - if (splits.length == 2) { - file = new File(new File(dir, splits(0)), splits(1)) - } else { - file = new File(dir, nextEntry.getName) - } - val outputStream = new BufferedOutputStream(new FileOutputStream(file)) - var dataRemains = true - while (dataRemains) { - val read = unzipStream.read(buffer) - if (read != -1) outputStream.write(buffer, 0, read) - else dataRemains = false - } - outputStream.close() - } - nextEntry = unzipStream.getNextEntry - } - } finally { - unzipStream.close() - } - } -} From 221cc26784c4af8c210367e70d97961a0d8cb819 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 3 Jun 2015 11:22:26 -0700 Subject: [PATCH 15/15] Update docs with new API information. --- docs/monitoring.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/monitoring.md b/docs/monitoring.md index e75018499003a..31ecddc6dbbb9 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -228,6 +228,14 @@ for a running application, at `http://localhost:4040/api/v1`. /applications/[app-id]/storage/rdd/[rdd-id] Details for the storage status of a given RDD + + /applications/[app-id]/logs + Download the event logs for all attempts of the given application as a zip file + + + /applications/[app-id]/[attempt-id/logs + Download the event logs for the specified attempt of the given application as a zip file + When running on Yarn, each application has multiple attempts, so `[app-id]` is actually