diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
index 24a0b5220695c78a36579d0e22b2ca979818a496..31b4dd7c0f427fc9d72a0fb27facafc32a2142f3 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
@@ -17,8 +17,8 @@
 package org.apache.spark.status.api.v1
 
 import java.util.{Arrays, Date, List => JList}
-import javax.ws.rs.{GET, PathParam, Produces, QueryParam}
 import javax.ws.rs.core.MediaType
+import javax.ws.rs.{GET, Produces, QueryParam}
 
 import org.apache.spark.executor.{InputMetrics => InternalInputMetrics, OutputMetrics => InternalOutputMetrics, ShuffleReadMetrics => InternalShuffleReadMetrics, ShuffleWriteMetrics => InternalShuffleWriteMetrics, TaskMetrics => InternalTaskMetrics}
 import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo}
@@ -59,6 +59,15 @@ private[v1] object AllStagesResource {
       stageUiData: StageUIData,
       includeDetails: Boolean): StageData = {
 
+    val taskLaunchTimes = stageUiData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0)
+
+    val firstTaskLaunchedTime: Option[Date] =
+      if (taskLaunchTimes.nonEmpty) {
+        Some(new Date(taskLaunchTimes.min))
+      } else {
+        None
+      }
+
     val taskData = if (includeDetails) {
       Some(stageUiData.taskData.map { case (k, v) => k -> convertTaskData(v) } )
     } else {
@@ -92,6 +101,9 @@ private[v1] object AllStagesResource {
       numCompleteTasks = stageUiData.numCompleteTasks,
       numFailedTasks = stageUiData.numFailedTasks,
       executorRunTime = stageUiData.executorRunTime,
+      submissionTime = stageInfo.submissionTime.map(new Date(_)),
+      firstTaskLaunchedTime,
+      completionTime = stageInfo.completionTime.map(new Date(_)),
       inputBytes = stageUiData.inputBytes,
       inputRecords = stageUiData.inputRecords,
       outputBytes = stageUiData.outputBytes,
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
index baddfc50c1a40289d90498c5f221290547ae6609..5feb1dc2e5b74cbf9205bca6ab1d8217ca915d5f 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
@@ -120,6 +120,9 @@ class StageData private[spark](
     val numFailedTasks: Int,
 
     val executorRunTime: Long,
+    val submissionTime: Option[Date],
+    val firstTaskLaunchedTime: Option[Date],
+    val completionTime: Option[Date],
 
     val inputBytes: Long,
     val inputRecords: Long,
diff --git a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json
index 31ac9beea87886f56f33cf48f0381f77105d5f18..8f8067f86d57f4d9a2744959faa09028a2a20486 100644
--- a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json
@@ -6,6 +6,9 @@
   "numCompleteTasks" : 8,
   "numFailedTasks" : 0,
   "executorRunTime" : 162,
+  "submissionTime" : "2015-02-03T16:43:07.191GMT",
+  "firstTaskLaunchedTime" : "2015-02-03T16:43:07.191GMT",
+  "completionTime" : "2015-02-03T16:43:07.226GMT",
   "inputBytes" : 160,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -28,6 +31,9 @@
   "numCompleteTasks" : 8,
   "numFailedTasks" : 0,
   "executorRunTime" : 3476,
+  "submissionTime" : "2015-02-03T16:43:05.829GMT",
+  "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT",
+  "completionTime" : "2015-02-03T16:43:06.286GMT",
   "inputBytes" : 28000128,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -50,6 +56,9 @@
   "numCompleteTasks" : 8,
   "numFailedTasks" : 0,
   "executorRunTime" : 4338,
+  "submissionTime" : "2015-02-03T16:43:04.228GMT",
+  "firstTaskLaunchedTime" : "2015-02-03T16:43:04.234GMT",
+  "completionTime" : "2015-02-03T16:43:04.819GMT",
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -64,4 +73,4 @@
   "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
   "schedulingPool" : "default",
   "accumulatorUpdates" : [ ]
-} ]
\ No newline at end of file
+} ]
diff --git a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json
index bff6a4f69d07767f5f98299cd300b30718faeff3..08b692eda80286fa31293034ef0e588292236a8e 100644
--- a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json
@@ -6,6 +6,9 @@
   "numCompleteTasks" : 7,
   "numFailedTasks" : 1,
   "executorRunTime" : 278,
+  "submissionTime" : "2015-02-03T16:43:06.296GMT",
+  "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT",
+  "completionTime" : "2015-02-03T16:43:06.347GMT",
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -20,4 +23,4 @@
   "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:25)\n$line11.$read$$iwC$$iwC.<init>(<console>:27)\n$line11.$read$$iwC.<init>(<console>:29)\n$line11.$read.<init>(<console>:31)\n$line11.$read$.<init>(<console>:35)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
   "schedulingPool" : "default",
   "accumulatorUpdates" : [ ]
-} ]
\ No newline at end of file
+} ]
diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
index 111cb8163eb3df0673d08ca0c6280bc5a95da46f..b07011d4f113f53a2bd9b61e9d3b22bfbe61eb44 100644
--- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
@@ -6,6 +6,9 @@
   "numCompleteTasks" : 8,
   "numFailedTasks" : 0,
   "executorRunTime" : 3476,
+  "submissionTime" : "2015-02-03T16:43:05.829GMT",
+  "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT",
+  "completionTime" : "2015-02-03T16:43:06.286GMT",
   "inputBytes" : 28000128,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -267,4 +270,4 @@
       "diskBytesSpilled" : 0
     }
   }
-}
\ No newline at end of file
+}
diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
index ef339f89afa45f56c7ee6cf174951daf01e7e440..2f71520549e1f86557b09b0f015cebdbab32073f 100644
--- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
@@ -6,6 +6,9 @@
   "numCompleteTasks" : 8,
   "numFailedTasks" : 0,
   "executorRunTime" : 3476,
+  "submissionTime" : "2015-02-03T16:43:05.829GMT",
+  "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT",
+  "completionTime" : "2015-02-03T16:43:06.286GMT",
   "inputBytes" : 28000128,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -267,4 +270,4 @@
       "diskBytesSpilled" : 0
     }
   }
-} ]
\ No newline at end of file
+} ]
diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json
index 056fac7088594bc5cd427c47147f7458a1a446c5..5b957ed54955644f1a59533a22c48bad8a9e6e8a 100644
--- a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json
@@ -6,6 +6,9 @@
   "numCompleteTasks" : 8,
   "numFailedTasks" : 0,
   "executorRunTime" : 162,
+  "submissionTime" : "2015-02-03T16:43:07.191GMT",
+  "firstTaskLaunchedTime" : "2015-02-03T16:43:07.191GMT",
+  "completionTime" : "2015-02-03T16:43:07.226GMT",
   "inputBytes" : 160,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -28,6 +31,9 @@
   "numCompleteTasks" : 8,
   "numFailedTasks" : 0,
   "executorRunTime" : 3476,
+  "submissionTime" : "2015-02-03T16:43:05.829GMT",
+  "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT",
+  "completionTime" : "2015-02-03T16:43:06.286GMT",
   "inputBytes" : 28000128,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -50,6 +56,9 @@
   "numCompleteTasks" : 8,
   "numFailedTasks" : 0,
   "executorRunTime" : 4338,
+  "submissionTime" : "2015-02-03T16:43:04.228GMT",
+  "firstTaskLaunchedTime" : "2015-02-03T16:43:04.234GMT",
+  "completionTime" : "2015-02-03T16:43:04.819GMT",
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -72,6 +81,9 @@
   "numCompleteTasks" : 7,
   "numFailedTasks" : 1,
   "executorRunTime" : 278,
+  "submissionTime" : "2015-02-03T16:43:06.296GMT",
+  "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT",
+  "completionTime" : "2015-02-03T16:43:06.347GMT",
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -86,4 +98,4 @@
   "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:25)\n$line11.$read$$iwC$$iwC.<init>(<console>:27)\n$line11.$read$$iwC.<init>(<console>:29)\n$line11.$read.<init>(<console>:31)\n$line11.$read$.<init>(<console>:35)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
   "schedulingPool" : "default",
   "accumulatorUpdates" : [ ]
-} ]
\ No newline at end of file
+} ]
diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json
index 79ccacd3096933fbc290e3f13082a403a7d47605..afa425f8c27bbf77ffa24769ad1654a39ee73453 100644
--- a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json
@@ -6,6 +6,9 @@
   "numCompleteTasks" : 8,
   "numFailedTasks" : 0,
   "executorRunTime" : 120,
+  "submissionTime" : "2015-03-16T19:25:36.103GMT",
+  "firstTaskLaunchedTime" : "2015-03-16T19:25:36.515GMT",
+  "completionTime" : "2015-03-16T19:25:36.579GMT",
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -24,4 +27,4 @@
     "name" : "my counter",
     "value" : "5050"
   } ]
-} ]
\ No newline at end of file
+} ]
diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
index 32d5731676ad5126b17d53aef2d07d8118d49976..12665a152c9ec9b047dbc841ad7d34b7d557a8cc 100644
--- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
@@ -6,6 +6,9 @@
   "numCompleteTasks" : 8,
   "numFailedTasks" : 0,
   "executorRunTime" : 120,
+  "submissionTime" : "2015-03-16T19:25:36.103GMT",
+  "firstTaskLaunchedTime" : "2015-03-16T19:25:36.515GMT",
+  "completionTime" : "2015-03-16T19:25:36.579GMT",
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
@@ -239,4 +242,4 @@
       "diskBytesSpilled" : 0
     }
   }
-}
\ No newline at end of file
+}
diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..88817dccf349711f7139bce993366c244925943a
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.status.api.v1
+
+import java.util.Date
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.scheduler.{StageInfo, TaskInfo, TaskLocality}
+import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
+
+class AllStagesResourceSuite extends SparkFunSuite {
+
+  def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = {
+    val tasks = new HashMap[Long, TaskUIData]
+    taskLaunchTimes.zipWithIndex.foreach { case (time, idx) =>
+      tasks(idx.toLong) = new TaskUIData(
+        new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None, None)
+    }
+
+    val stageUiData = new StageUIData()
+    stageUiData.taskData = tasks
+    val status = StageStatus.ACTIVE
+    val stageInfo = new StageInfo(
+      1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty)
+    val stageData = AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, false)
+
+    stageData.firstTaskLaunchedTime
+  }
+
+  test("firstTaskLaunchedTime when there are no tasks") {
+    val result = getFirstTaskLaunchTime(Seq())
+    assert(result == None)
+  }
+
+  test("firstTaskLaunchedTime when there are tasks but none launched") {
+    val result = getFirstTaskLaunchTime(Seq(-100L, -200L, -300L))
+    assert(result == None)
+  }
+
+  test("firstTaskLaunchedTime when there are tasks and some launched") {
+    val result = getFirstTaskLaunchTime(Seq(-100L, 1449255596000L, 1449255597000L))
+    assert(result == Some(new Date(1449255596000L)))
+  }
+
+}
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index b4aa6adc3c620cf6a9174cfb194c7746b0723b91..685cb419ca8a7b3e17c5b189f7baf2c2ae93d169 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -132,7 +132,9 @@ object MimaExcludes {
         ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.jdbc.NoopDialect$")
       ) ++ Seq (
         ProblemFilters.exclude[MissingMethodProblem](
-          "org.apache.spark.status.api.v1.ApplicationInfo.this")
+          "org.apache.spark.status.api.v1.ApplicationInfo.this"),
+        ProblemFilters.exclude[MissingMethodProblem](
+          "org.apache.spark.status.api.v1.StageData.this")
       ) ++ Seq(
         // SPARK-11766 add toJson to Vector
         ProblemFilters.exclude[MissingMethodProblem](