aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark
diff options
context:
space:
mode:
Diffstat (limited to 'core/src/main/scala/org/apache/spark')
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala28
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala107
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/io/CompressionCodec.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/package.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala17
15 files changed, 308 insertions, 30 deletions
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index faf0c2362a..f2641851cb 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -631,20 +631,26 @@ class SparkContext(
* filesystems), or an HTTP, HTTPS or FTP URI.
*/
def addJar(path: String) {
- if (null == path) {
+ if (path == null) {
logWarning("null specified as parameter to addJar",
new SparkException("null specified as parameter to addJar"))
} else {
- val env = SparkEnv.get
- val uri = new URI(path)
- val key = uri.getScheme match {
- case null | "file" =>
- if (env.hadoop.isYarnMode()) {
- logWarning("local jar specified as parameter to addJar under Yarn mode")
- return
- }
- env.httpFileServer.addJar(new File(uri.getPath))
- case _ => path
+ var key = ""
+ if (path.contains("\\")) {
+ // For local paths with backslashes on Windows, URI throws an exception
+ key = env.httpFileServer.addJar(new File(path))
+ } else {
+ val uri = new URI(path)
+ key = uri.getScheme match {
+ case null | "file" =>
+ if (env.hadoop.isYarnMode()) {
+ logWarning("local jar specified as parameter to addJar under Yarn mode")
+ return
+ }
+ env.httpFileServer.addJar(new File(uri.getPath))
+ case _ =>
+ path
+ }
}
addedJars(key) = System.currentTimeMillis
logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key))
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
index 08e3f670f5..67d45723ba 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
@@ -17,8 +17,8 @@
package org.apache.spark.api.python
-import java.io.{File, DataInputStream, IOException}
-import java.net.{Socket, SocketException, InetAddress}
+import java.io.{OutputStreamWriter, File, DataInputStream, IOException}
+import java.net.{ServerSocket, Socket, SocketException, InetAddress}
import scala.collection.JavaConversions._
@@ -26,11 +26,30 @@ import org.apache.spark._
private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String])
extends Logging {
+
+ // Because forking processes from Java is expensive, we prefer to launch a single Python daemon
+ // (pyspark/daemon.py) and tell it to fork new workers for our tasks. This daemon currently
+ // only works on UNIX-based systems now because it uses signals for child management, so we can
+ // also fall back to launching workers (pyspark/worker.py) directly.
+ val useDaemon = !System.getProperty("os.name").startsWith("Windows")
+
var daemon: Process = null
val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1))
var daemonPort: Int = 0
def create(): Socket = {
+ if (useDaemon) {
+ createThroughDaemon()
+ } else {
+ createSimpleWorker()
+ }
+ }
+
+ /**
+ * Connect to a worker launched through pyspark/daemon.py, which forks python processes itself
+ * to avoid the high cost of forking from Java. This currently only works on UNIX-based systems.
+ */
+ private def createThroughDaemon(): Socket = {
synchronized {
// Start the daemon if it hasn't been started
startDaemon()
@@ -50,6 +69,78 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
}
}
+ /**
+ * Launch a worker by executing worker.py directly and telling it to connect to us.
+ */
+ private def createSimpleWorker(): Socket = {
+ var serverSocket: ServerSocket = null
+ try {
+ serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1)))
+
+ // Create and start the worker
+ val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME")
+ val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/worker.py"))
+ val workerEnv = pb.environment()
+ workerEnv.putAll(envVars)
+ val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH")
+ workerEnv.put("PYTHONPATH", pythonPath)
+ val worker = pb.start()
+
+ // Redirect the worker's stderr to ours
+ new Thread("stderr reader for " + pythonExec) {
+ setDaemon(true)
+ override def run() {
+ scala.util.control.Exception.ignoring(classOf[IOException]) {
+ // FIXME: We copy the stream on the level of bytes to avoid encoding problems.
+ val in = worker.getErrorStream
+ val buf = new Array[Byte](1024)
+ var len = in.read(buf)
+ while (len != -1) {
+ System.err.write(buf, 0, len)
+ len = in.read(buf)
+ }
+ }
+ }
+ }.start()
+
+ // Redirect worker's stdout to our stderr
+ new Thread("stdout reader for " + pythonExec) {
+ setDaemon(true)
+ override def run() {
+ scala.util.control.Exception.ignoring(classOf[IOException]) {
+ // FIXME: We copy the stream on the level of bytes to avoid encoding problems.
+ val in = worker.getInputStream
+ val buf = new Array[Byte](1024)
+ var len = in.read(buf)
+ while (len != -1) {
+ System.err.write(buf, 0, len)
+ len = in.read(buf)
+ }
+ }
+ }
+ }.start()
+
+ // Tell the worker our port
+ val out = new OutputStreamWriter(worker.getOutputStream)
+ out.write(serverSocket.getLocalPort + "\n")
+ out.flush()
+
+ // Wait for it to connect to our socket
+ serverSocket.setSoTimeout(10000)
+ try {
+ return serverSocket.accept()
+ } catch {
+ case e: Exception =>
+ throw new SparkException("Python worker did not connect back in time", e)
+ }
+ } finally {
+ if (serverSocket != null) {
+ serverSocket.close()
+ }
+ }
+ null
+ }
+
def stop() {
stopDaemon()
}
@@ -73,12 +164,12 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
// Redirect the stderr to ours
new Thread("stderr reader for " + pythonExec) {
+ setDaemon(true)
override def run() {
scala.util.control.Exception.ignoring(classOf[IOException]) {
- // FIXME HACK: We copy the stream on the level of bytes to
- // attempt to dodge encoding problems.
+ // FIXME: We copy the stream on the level of bytes to avoid encoding problems.
val in = daemon.getErrorStream
- var buf = new Array[Byte](1024)
+ val buf = new Array[Byte](1024)
var len = in.read(buf)
while (len != -1) {
System.err.write(buf, 0, len)
@@ -93,11 +184,11 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
// Redirect further stdout output to our stderr
new Thread("stdout reader for " + pythonExec) {
+ setDaemon(true)
override def run() {
scala.util.control.Exception.ignoring(classOf[IOException]) {
- // FIXME HACK: We copy the stream on the level of bytes to
- // attempt to dodge encoding problems.
- var buf = new Array[Byte](1024)
+ // FIXME: We copy the stream on the level of bytes to avoid encoding problems.
+ val buf = new Array[Byte](1024)
var len = in.read(buf)
while (len != -1) {
System.err.write(buf, 0, len)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala
index 2d75ad5a2c..5a24042e14 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.master
import com.codahale.metrics.{Gauge, MetricRegistry}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
index 8dd0a42f71..23d1cb77da 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.master
import com.codahale.metrics.{Gauge, MetricRegistry}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala
index 6427c0178f..df269fd047 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.worker
import com.codahale.metrics.{Gauge, MetricRegistry}
diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
index 17653cd560..bf8fb4fd21 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.executor
import com.codahale.metrics.{Gauge, MetricRegistry}
diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index 90a0420caf..570a979b56 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -39,17 +39,13 @@ trait CompressionCodec {
private[spark] object CompressionCodec {
def createCodec(): CompressionCodec = {
- // Set the default codec to Snappy since the LZF implementation initializes a pretty large
- // buffer for every stream, which results in a lot of memory overhead when the number of
- // shuffle reduce buckets are large.
- createCodec(classOf[SnappyCompressionCodec].getName)
+ createCodec(System.getProperty(
+ "spark.io.compression.codec", classOf[LZFCompressionCodec].getName))
}
def createCodec(codecName: String): CompressionCodec = {
- Class.forName(
- System.getProperty("spark.io.compression.codec", codecName),
- true,
- Thread.currentThread.getContextClassLoader).newInstance().asInstanceOf[CompressionCodec]
+ Class.forName(codecName, true, Thread.currentThread.getContextClassLoader)
+ .newInstance().asInstanceOf[CompressionCodec]
}
}
diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala
index c0ec527339..f132e2b735 100644
--- a/core/src/main/scala/org/apache/spark/package.scala
+++ b/core/src/main/scala/org/apache/spark/package.scala
@@ -1,5 +1,3 @@
-import org.apache.spark.rdd.{SequenceFileRDDFunctions, DoubleRDDFunctions, PairRDDFunctions}
-
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@@ -30,6 +28,6 @@ import org.apache.spark.rdd.{SequenceFileRDDFunctions, DoubleRDDFunctions, PairR
* type (e.g. RDD[(Int, Int)] through implicit conversions when you
* `import org.apache.spark.SparkContext._`.
*/
-package object spark {
+package object spark {
// For package docs only
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
index ce0dc9093d..22e3723ac8 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.scheduler
import com.codahale.metrics.{Gauge,MetricRegistry}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala
index 24190cdd67..3d709cfde4 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.storage
import com.codahale.metrics.{Gauge,MetricRegistry}
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
index 6e56c22d04..d1868dcf78 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.ui.exec
import javax.servlet.http.HttpServletRequest
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index 86e0af0399..e2bcd98545 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.ui.jobs
import scala.Seq
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
index ce92b6932b..89fffcb80d 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.ui.jobs
import javax.servlet.http.HttpServletRequest
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
index f31465e59d..5670c933bd 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.ui.jobs
import scala.collection.mutable.HashMap
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
index 71e58a977e..a8911e46ae 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
@@ -1,3 +1,20 @@
+/*
+ * 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.ui.jobs
import java.util.Date