Skip to content

Commit

Permalink
Changed interface to use zip stream. Added more tests.
Browse files Browse the repository at this point in the history
  • Loading branch information
harishreedharan committed May 30, 2015
1 parent 1100b40 commit ffffb53
Show file tree
Hide file tree
Showing 9 changed files with 144 additions and 117 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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

}
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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.")
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}

Expand Down Expand Up @@ -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)
Expand Down
37 changes: 0 additions & 37 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -153,52 +149,80 @@ 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)
code should be (HttpServletResponse.SC_OK)
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)
}
Expand Down
Loading

0 comments on commit ffffb53

Please sign in to comment.