diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala
index b969e41e4e55c..b98fdf16eef31 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala
@@ -152,7 +152,7 @@ class StreamingQueryPagedTable(
val headerAndCss: Seq[(String, Boolean, Option[String])] = {
Seq(
("Name", true, None),
- ("Status", false, None),
+ ("Status", true, None),
("ID", true, None),
("Run ID", true, None),
("Start Time", true, None),
@@ -197,7 +197,7 @@ class StreamingQueryPagedTable(
{streamingQuery.id} |
{streamingQuery.runId} |
{SparkUIUtils.formatDate(streamingQuery.startTimestamp)} |
- {query.duration} |
+ {SparkUIUtils.formatDurationVerbose(query.duration)} |
{withNoProgress(streamingQuery, {query.avgInput.formatted("%.2f")}, "NaN")} |
{withNoProgress(streamingQuery, {query.avgProcess.formatted("%.2f")}, "NaN")} |
{withNoProgress(streamingQuery, {streamingQuery.lastProgress.batchId}, "NaN")} |
@@ -207,7 +207,7 @@ class StreamingQueryPagedTable(
}
case class StructuredStreamingRow(
- duration: String,
+ duration: Long,
avgInput: Double,
avgProcess: Double,
streamingUIData: StreamingQueryUIData)
@@ -224,12 +224,12 @@ class StreamingQueryDataSource(uiData: Seq[StreamingQueryUIData], sortColumn: St
private def streamingRow(query: StreamingQueryUIData): StructuredStreamingRow = {
val duration = if (isActive) {
- SparkUIUtils.formatDurationVerbose(System.currentTimeMillis() - query.startTimestamp)
+ System.currentTimeMillis() - query.startTimestamp
} else {
withNoProgress(query, {
val endTimeMs = query.lastProgress.timestamp
- SparkUIUtils.formatDurationVerbose(parseProgressTimestamp(endTimeMs) - query.startTimestamp)
- }, "-")
+ parseProgressTimestamp(endTimeMs) - query.startTimestamp
+ }, 0)
}
val avgInput = (query.recentProgress.map(p => withNumberInvalid(p.inputRowsPerSecond)).sum /
@@ -244,6 +244,7 @@ class StreamingQueryDataSource(uiData: Seq[StreamingQueryUIData], sortColumn: St
private def ordering(sortColumn: String, desc: Boolean): Ordering[StructuredStreamingRow] = {
val ordering: Ordering[StructuredStreamingRow] = sortColumn match {
case "Name" => Ordering.by(q => UIUtils.getQueryName(q.streamingUIData))
+ case "Status" => Ordering.by(q => UIUtils.getQueryStatus(q.streamingUIData))
case "ID" => Ordering.by(_.streamingUIData.id)
case "Run ID" => Ordering.by(_.streamingUIData.runId)
case "Start Time" => Ordering.by(_.streamingUIData.startTimestamp)