Skip to content

Commit 7b362b2

Browse files
Almost working.
1 parent 3d18ebc commit 7b362b2

File tree

3 files changed

+76
-28
lines changed

3 files changed

+76
-28
lines changed
Lines changed: 44 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,44 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.spark.status.api.v1
18+
19+
import java.io.OutputStream
20+
import java.lang.annotation.Annotation
21+
import java.lang.reflect.Type
22+
import javax.ws.rs.Produces
23+
import javax.ws.rs.core.{MultivaluedMap, MediaType}
24+
import javax.ws.rs.ext.{Provider, MessageBodyWriter}
25+
26+
class DownloadMessageWriter extends MessageBodyWriter[Object] {
27+
28+
override def writeTo(t: Object, `type`: Class[_], genericType: Type,
29+
annotations: Array[Annotation], mediaType: MediaType,
30+
httpHeaders: MultivaluedMap[String, AnyRef], entityStream: OutputStream): Unit = {
31+
t match {
32+
case downloader @ EventLogDownloadResource(_) =>
33+
downloader.getEventLogs()
34+
}
35+
}
36+
37+
override def getSize(t: Object, `type`: Class[_], genericType: Type,
38+
annotations: Array[Annotation], mediaType: MediaType): Long = {
39+
-1L
40+
}
41+
42+
override def isWriteable(`type`: Class[_], genericType: Type, annotations: Array[Annotation],
43+
mediaType: MediaType): Boolean = true
44+
}

core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala

Lines changed: 31 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -16,48 +16,54 @@
1616
*/
1717
package org.apache.spark.status.api.v1
1818

19-
import java.io.{FileInputStream, OutputStream, File, InputStream}
19+
import java.io.{BufferedInputStream, FileInputStream, OutputStream, File, InputStream}
20+
import javax.ws.rs.ext.Provider
2021
import javax.ws.rs.{GET, Produces}
21-
import javax.ws.rs.core.{MultivaluedMap, MediaType}
22+
import javax.ws.rs.core.{StreamingOutput, MultivaluedMap, MediaType}
2223

2324
import org.apache.spark.deploy.history.HistoryServer
2425
import org.apache.spark.util.Utils
2526

26-
@Produces(Array(MediaType.APPLICATION_OCTET_STREAM))
27+
2728
private[v1] class EventLogDownloadResource(val uIRoot: UIRoot, val appId: String) {
2829

30+
private def getErrorOutput(err: String): StreamingOutput = {
31+
new StreamingOutput {
32+
override def write(outputStream: OutputStream): Unit = {
33+
outputStream.write(
34+
s"File download not available for application : $appId due to $err".getBytes("utf-8"))
35+
}
36+
}
37+
}
38+
2939
@GET
30-
def getEventLogs(headers: MultivaluedMap[String, AnyRef], outputStream: OutputStream): Unit = {
40+
@Produces(Array(MediaType.APPLICATION_OCTET_STREAM))
41+
def getEventLogs(): StreamingOutput = {
3142
uIRoot match {
3243
case hs: HistoryServer =>
3344
val dir = Utils.createTempDir()
3445
Utils.chmod700(dir)
3546
hs.copyEventLogsToDirectory(appId, dir)
36-
dir.listFiles().headOption.foreach { zipFile =>
37-
headers.add("Content-Length", zipFile.length().toString)
38-
headers.add("Content-Type", MediaType.APPLICATION_OCTET_STREAM)
39-
headers.add("Content-Disposition", s"attachment; filename=${zipFile.getName}")
40-
var inputStream: InputStream = null
41-
try {
42-
inputStream = new FileInputStream(zipFile)
43-
val buffer = new Array[Byte](1024 * 1024)
44-
var remaining = true
45-
while (remaining) {
46-
val read = inputStream.read(buffer)
47-
if (read != -1) {
48-
outputStream.write(buffer, 0, read)
49-
} else {
50-
remaining = false
47+
dir.listFiles().headOption.foreach { file =>
48+
return new StreamingOutput {
49+
override def write(output: OutputStream): Unit = {
50+
val inStream = new BufferedInputStream(new FileInputStream(file))
51+
val buffer = new Array[Byte](1024 * 1024)
52+
var dataRemains = true
53+
while (dataRemains) {
54+
val read = inStream.read(buffer)
55+
if (read > 0) {
56+
output.write(buffer, 0, read)
57+
} else {
58+
dataRemains = false
59+
}
5160
}
61+
output.flush()
5262
}
53-
outputStream.flush()
54-
} finally {
55-
inputStream.close()
56-
Utils.deleteRecursively(dir)
5763
}
5864
}
59-
case _ => outputStream.write(
60-
s"File download not available for application : $appId".getBytes("utf-8"))
65+
getErrorOutput("No files in dir.")
66+
case _ => getErrorOutput("hs not history server")
6167
}
6268
}
6369
}

core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,7 @@ import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature}
3838
* Note that jersey automatically discovers this class based on its package and its annotations.
3939
*/
4040
@Provider
41-
@Produces(Array(MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM))
41+
@Produces(Array(MediaType.APPLICATION_JSON))
4242
private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{
4343

4444
val mapper = new ObjectMapper() {
@@ -69,8 +69,6 @@ private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{
6969
outputStream: OutputStream): Unit = {
7070
t match {
7171
case ErrorWrapper(err) => outputStream.write(err.getBytes())
72-
case downloader @ EventLogDownloadResource(_) =>
73-
downloader.getEventLogs(multivaluedMap, outputStream)
7472
case _ => mapper.writeValue(outputStream, t)
7573
}
7674
}

0 commit comments

Comments
 (0)