This is an automated email from the ASF dual-hosted git repository.

joemcdonnell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 57964c83d37bdd61ec08b738c56725940138ab7d
Author: Eyizoha <18770832...@163.com>
AuthorDate: Mon Feb 27 15:17:56 2023 +0800

    IMPALA-11823: Add more items to Impala web UI queries page
    
    When operating and maintaining an Impala cluster or analyzing historical
    query performance, it will be helpful if we show the memory consumed,
    the amount of data read, and other information of the query from the
    historical query page of the web UI. The current historical query page
    does not display these information, so we should count this information
    when the query is executed and display it on the web page.
    
    This patch modifies the query list page (/queries) and query detail
    pages (/query_plan, etc.).
    On the list page, some metrics are added for each query record,
    including queuing time, memory usage, memory estimation, bytes read, and
    bytes sent. In addition, the Details column now shows the query ID and
    the position is adjusted to make them at the top of the record for easy
    clicking.
    On the query detail page, a similar record table is added to display the
    key information of the current query. In addition, a timeline display is
    added to the summary page (which is exactly the same as the timeline in
    the profile, just for quick viewing). For queries that are running, the
    above information will be automatically refreshed (only for the plan and
    summary tabs).
    To make it clear what each metric means, tooltips are added to all list
    headers.
    
    Change-Id: I19c75461a6405025fa433ae84d2c94d013fcaacb
    Reviewed-on: http://gerrit.cloudera.org:8080/19417
    Reviewed-by: Impala Public Jenkins <impala-public-jenk...@cloudera.com>
    Tested-by: Impala Public Jenkins <impala-public-jenk...@cloudera.com>
---
 be/src/runtime/coordinator.h                       |   5 +
 be/src/scheduling/admission-control-client.h       |   3 +-
 be/src/scheduling/admission-control-service.cc     |   6 +-
 be/src/scheduling/admission-controller.cc          |  11 +-
 be/src/scheduling/admission-controller.h           |   3 +-
 .../scheduling/local-admission-control-client.cc   |   6 +-
 be/src/scheduling/local-admission-control-client.h |   3 +-
 .../scheduling/remote-admission-control-client.cc  |  10 +-
 .../scheduling/remote-admission-control-client.h   |   3 +-
 be/src/scheduling/schedule-state.cc                |   1 +
 be/src/service/client-request-state.cc             |   2 +-
 be/src/service/client-request-state.h              |  17 ++
 be/src/service/impala-http-handler.cc              | 129 +++++++++++++--
 be/src/service/impala-http-handler.h               |   5 +-
 be/src/service/impala-server.cc                    |  16 +-
 be/src/service/impala-server.h                     |  20 +++
 be/src/util/runtime-profile.cc                     |  71 +++++----
 be/src/util/runtime-profile.h                      |   5 +
 common/protobuf/admission_control_service.proto    |   9 ++
 www/common-header.tmpl                             |   2 +-
 www/queries.tmpl                                   | 148 +++++++++++------
 www/query_detail_tabs.tmpl                         | 177 +++++++++++++++++++++
 www/query_plan.tmpl                                |   1 +
 www/query_summary.tmpl                             |   4 +
 24 files changed, 550 insertions(+), 107 deletions(-)

diff --git a/be/src/runtime/coordinator.h b/be/src/runtime/coordinator.h
index c982b46ae..aaeaa89fb 100644
--- a/be/src/runtime/coordinator.h
+++ b/be/src/runtime/coordinator.h
@@ -212,6 +212,10 @@ class Coordinator { // NOLINT: The member variables could 
be re-ordered to save
     /// peak value from any backend.
     int64_t peak_per_host_mem_consumption = 0;
 
+    /// Total peak memory usage for this query at all backend.
+    /// Note that it is the sum of peaks, not the peak of the sum.
+    int64_t total_peak_mem_usage = 0;
+
     /// Total bytes read across all scan nodes.
     int64_t bytes_read = 0;
 
@@ -234,6 +238,7 @@ class Coordinator { // NOLINT: The member variables could 
be re-ordered to save
     void Merge(const ResourceUtilization& other) {
       peak_per_host_mem_consumption =
           std::max(peak_per_host_mem_consumption, 
other.peak_per_host_mem_consumption);
+      total_peak_mem_usage += other.peak_per_host_mem_consumption;
       bytes_read += other.bytes_read;
       exchange_bytes_sent += other.exchange_bytes_sent;
       scan_bytes_sent += other.scan_bytes_sent;
diff --git a/be/src/scheduling/admission-control-client.h 
b/be/src/scheduling/admission-control-client.h
index 2622b4112..9c077dd2b 100644
--- a/be/src/scheduling/admission-control-client.h
+++ b/be/src/scheduling/admission-control-client.h
@@ -45,7 +45,8 @@ class AdmissionControlClient {
   // Called to schedule and admit the query. Blocks until an admission 
decision is made.
   virtual Status SubmitForAdmission(const 
AdmissionController::AdmissionRequest& request,
       RuntimeProfile::EventSequence* query_events,
-      std::unique_ptr<QuerySchedulePB>* schedule_result) = 0;
+      std::unique_ptr<QuerySchedulePB>* schedule_result,
+      int64_t* wait_start_time_ms, int64_t* wait_end_time_ms) = 0;
 
   // Called when the query has completed to release all of its resources.
   virtual void ReleaseQuery(int64_t peak_mem_consumption) = 0;
diff --git a/be/src/scheduling/admission-control-service.cc 
b/be/src/scheduling/admission-control-service.cc
index 3cf4682ab..979143149 100644
--- a/be/src/scheduling/admission-control-service.cc
+++ b/be/src/scheduling/admission-control-service.cc
@@ -167,10 +167,14 @@ void AdmissionControlService::GetQueryStatus(const 
GetQueryStatusRequestPB* req,
     if (admission_state->submitted) {
       if (!admission_state->admission_done) {
         bool timed_out;
+        int64_t wait_start_time_ms, wait_end_time_ms;
         admission_state->admit_status =
             AdmissiondEnv::GetInstance()->admission_controller()->WaitOnQueued(
                 req->query_id(), &admission_state->schedule,
-                FLAGS_admission_status_wait_time_ms, &timed_out);
+                FLAGS_admission_status_wait_time_ms, &timed_out,
+                &wait_start_time_ms, &wait_end_time_ms);
+        resp->set_wait_start_time_ms(wait_start_time_ms);
+        resp->set_wait_end_time_ms(wait_end_time_ms);
         if (!timed_out) {
           admission_state->admission_done = true;
           if (admission_state->admit_status.ok()) {
diff --git a/be/src/scheduling/admission-controller.cc 
b/be/src/scheduling/admission-controller.cc
index 7fd062dff..76bdd774c 100644
--- a/be/src/scheduling/admission-controller.cc
+++ b/be/src/scheduling/admission-controller.cc
@@ -1344,8 +1344,10 @@ Status AdmissionController::SubmitForAdmission(const 
AdmissionRequest& request,
 
 Status AdmissionController::WaitOnQueued(const UniqueIdPB& query_id,
     unique_ptr<QuerySchedulePB>* schedule_result, int64_t timeout_ms,
-    bool* wait_timed_out) {
+    bool* wait_timed_out, int64_t* wait_start_time_ms, int64_t* 
wait_end_time_ms) {
   if (wait_timed_out != nullptr) *wait_timed_out = false;
+  if (wait_start_time_ms != nullptr) *wait_start_time_ms = 0;
+  if (wait_end_time_ms != nullptr) *wait_end_time_ms = 0;
 
   QueueNode* queue_node;
   {
@@ -1358,6 +1360,8 @@ Status AdmissionController::WaitOnQueued(const 
UniqueIdPB& query_id,
     queue_node = &it->second;
   }
 
+  if (wait_start_time_ms != nullptr) *wait_start_time_ms = 
queue_node->wait_start_ms;
+
   int64_t queue_wait_timeout_ms = 
GetQueueTimeoutForPoolMs(queue_node->pool_cfg);
 
   // Block in Get() up to the time out, waiting for the promise to be set when 
the query
@@ -1366,7 +1370,8 @@ Status AdmissionController::WaitOnQueued(const 
UniqueIdPB& query_id,
   queue_node->admit_outcome->Get(
       (timeout_ms > 0 ? min(queue_wait_timeout_ms, timeout_ms) : 
queue_wait_timeout_ms),
       &get_timed_out);
-  int64_t wait_time_ms = MonotonicMillis() - queue_node->wait_start_ms;
+  int64_t wait_end_ms = MonotonicMillis();
+  int64_t wait_time_ms = wait_end_ms - queue_node->wait_start_ms;
 
   queue_node->profile->AddInfoString(PROFILE_INFO_KEY_INITIAL_QUEUE_REASON,
       Substitute(PROFILE_INFO_VAL_INITIAL_QUEUE_REASON, wait_time_ms,
@@ -1378,6 +1383,8 @@ Status AdmissionController::WaitOnQueued(const 
UniqueIdPB& query_id,
     return Status::OK();
   }
 
+  if (wait_end_time_ms != nullptr) *wait_end_time_ms = wait_end_ms;
+
   const auto queue_node_deleter = MakeScopeExitTrigger([&]() {
     lock_guard<mutex> lock(queue_nodes_lock_);
     queue_nodes_.erase(query_id);
diff --git a/be/src/scheduling/admission-controller.h 
b/be/src/scheduling/admission-controller.h
index 01b470fd7..64f0e139a 100644
--- a/be/src/scheduling/admission-controller.h
+++ b/be/src/scheduling/admission-controller.h
@@ -378,7 +378,8 @@ class AdmissionController {
   /// function returns due to the timeout 'wait_timed_out' will be true.
   Status WaitOnQueued(const UniqueIdPB& query_id,
       std::unique_ptr<QuerySchedulePB>* schedule_result, int64_t timeout_ms = 
0,
-      bool* wait_timed_out = nullptr);
+      bool* wait_timed_out = nullptr, int64_t* wait_start_time_ms = nullptr,
+      int64_t* wait_end_time_ms = nullptr);
 
   /// Updates the pool statistics when a query completes (either successfully,
   /// is cancelled or failed). This should be called for all requests that have
diff --git a/be/src/scheduling/local-admission-control-client.cc 
b/be/src/scheduling/local-admission-control-client.cc
index e85f34b28..94f9b84f0 100644
--- a/be/src/scheduling/local-admission-control-client.cc
+++ b/be/src/scheduling/local-admission-control-client.cc
@@ -32,7 +32,8 @@ 
LocalAdmissionControlClient::LocalAdmissionControlClient(const TUniqueId& query_
 Status LocalAdmissionControlClient::SubmitForAdmission(
     const AdmissionController::AdmissionRequest& request,
     RuntimeProfile::EventSequence* query_events,
-    std::unique_ptr<QuerySchedulePB>* schedule_result) {
+    std::unique_ptr<QuerySchedulePB>* schedule_result,
+    int64_t* wait_start_time_ms, int64_t* wait_end_time_ms) {
   ScopedEvent completedEvent(
       query_events, AdmissionControlClient::QUERY_EVENT_COMPLETED_ADMISSION);
   query_events->MarkEvent(QUERY_EVENT_SUBMIT_FOR_ADMISSION);
@@ -43,7 +44,8 @@ Status LocalAdmissionControlClient::SubmitForAdmission(
     query_events->MarkEvent(QUERY_EVENT_QUEUED);
     DCHECK(status.ok());
     status = ExecEnv::GetInstance()->admission_controller()->WaitOnQueued(
-        request.query_id, schedule_result);
+        request.query_id, schedule_result, /*timeout_ms*/ 0, 
/*wait_timed_out*/ nullptr,
+        wait_start_time_ms, wait_end_time_ms);
   }
   return status;
 }
diff --git a/be/src/scheduling/local-admission-control-client.h 
b/be/src/scheduling/local-admission-control-client.h
index f8b7a73a8..7d675272e 100644
--- a/be/src/scheduling/local-admission-control-client.h
+++ b/be/src/scheduling/local-admission-control-client.h
@@ -37,7 +37,8 @@ class LocalAdmissionControlClient : public 
AdmissionControlClient {
 
   virtual Status SubmitForAdmission(const 
AdmissionController::AdmissionRequest& request,
       RuntimeProfile::EventSequence* query_events,
-      std::unique_ptr<QuerySchedulePB>* schedule_result) override;
+      std::unique_ptr<QuerySchedulePB>* schedule_result,
+      int64_t* wait_start_time_ms, int64_t* wait_end_time_ms) override;
   virtual void ReleaseQuery(int64_t peak_mem_consumption) override;
   virtual void ReleaseQueryBackends(
       const std::vector<NetworkAddressPB>& host_addr) override;
diff --git a/be/src/scheduling/remote-admission-control-client.cc 
b/be/src/scheduling/remote-admission-control-client.cc
index 9e877f6e4..accc06ca1 100644
--- a/be/src/scheduling/remote-admission-control-client.cc
+++ b/be/src/scheduling/remote-admission-control-client.cc
@@ -97,7 +97,8 @@ Status 
RemoteAdmissionControlClient::TryAdmitQuery(AdmissionControlServiceProxy*
 Status RemoteAdmissionControlClient::SubmitForAdmission(
     const AdmissionController::AdmissionRequest& request,
     RuntimeProfile::EventSequence* query_events,
-    std::unique_ptr<QuerySchedulePB>* schedule_result) {
+    std::unique_ptr<QuerySchedulePB>* schedule_result,
+    int64_t* wait_start_time_ms, int64_t* wait_end_time_ms) {
   ScopedEvent completedEvent(
       query_events, AdmissionControlClient::QUERY_EVENT_COMPLETED_ADMISSION);
 
@@ -161,6 +162,13 @@ Status RemoteAdmissionControlClient::SubmitForAdmission(
       request.summary_profile->Update(tree);
     }
 
+    if (wait_start_time_ms != nullptr && 
get_status_resp.has_wait_start_time_ms()) {
+      *wait_start_time_ms = get_status_resp.wait_start_time_ms();
+    }
+    if (wait_end_time_ms != nullptr && get_status_resp.has_wait_end_time_ms()) 
{
+      *wait_end_time_ms = get_status_resp.wait_end_time_ms();
+    }
+
     if (get_status_resp.has_query_schedule()) {
       schedule_result->reset(new QuerySchedulePB());
       schedule_result->get()->Swap(get_status_resp.mutable_query_schedule());
diff --git a/be/src/scheduling/remote-admission-control-client.h 
b/be/src/scheduling/remote-admission-control-client.h
index 1cc3b5fdf..48881a5f3 100644
--- a/be/src/scheduling/remote-admission-control-client.h
+++ b/be/src/scheduling/remote-admission-control-client.h
@@ -53,7 +53,8 @@ class RemoteAdmissionControlClient : public 
AdmissionControlClient {
 
   virtual Status SubmitForAdmission(const 
AdmissionController::AdmissionRequest& request,
       RuntimeProfile::EventSequence* query_events,
-      std::unique_ptr<QuerySchedulePB>* schedule_result) override;
+      std::unique_ptr<QuerySchedulePB>* schedule_result,
+      int64_t* wait_start_time_ms, int64_t* wait_end_time_ms) override;
   virtual void ReleaseQuery(int64_t peak_mem_consumption) override;
   virtual void ReleaseQueryBackends(
       const std::vector<NetworkAddressPB>& host_addr) override;
diff --git a/be/src/scheduling/schedule-state.cc 
b/be/src/scheduling/schedule-state.cc
index aa69c91fd..59e8c78c8 100644
--- a/be/src/scheduling/schedule-state.cc
+++ b/be/src/scheduling/schedule-state.cc
@@ -347,6 +347,7 @@ void ScheduleState::UpdateMemoryRequirements(const 
TPoolConfig& pool_cfg,
   
query_schedule_pb_->set_coord_backend_mem_to_admit(coord_backend_mem_to_admit);
   query_schedule_pb_->set_per_backend_mem_limit(per_backend_mem_limit);
   query_schedule_pb_->set_per_backend_mem_to_admit(per_backend_mem_to_admit);
+  query_schedule_pb_->set_cluster_mem_est(GetClusterMemoryToAdmit());
 }
 
 void ScheduleState::set_executor_group(string executor_group) {
diff --git a/be/src/service/client-request-state.cc 
b/be/src/service/client-request-state.cc
index a27248db5..b7d9ab535 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -587,7 +587,7 @@ void ClientRequestState::FinishExecQueryOrDmlRequest() {
       {query_id_pb, ExecEnv::GetInstance()->backend_id(),
           exec_request_->query_exec_request, exec_request_->query_options,
           summary_profile_, blacklisted_executor_addresses_},
-      query_events_, &schedule_);
+      query_events_, &schedule_, &wait_start_time_ms_, &wait_end_time_ms_);
   {
     lock_guard<mutex> l(lock_);
     if (!UpdateQueryStatus(admit_status).ok()) return;
diff --git a/be/src/service/client-request-state.h 
b/be/src/service/client-request-state.h
index a6f580082..9f6f4ad91 100644
--- a/be/src/service/client-request-state.h
+++ b/be/src/service/client-request-state.h
@@ -352,6 +352,18 @@ class ClientRequestState {
   /// Returns the max size of the result_cache_ in number of rows.
   int64_t result_cache_max_size() const { return result_cache_max_size_; }
 
+  /// Returns duration of the query is waiting in the queue.
+  int64_t wait_time_ms() const {
+    // wait_start_time_ms_ is zero iff the query is not require queuing.
+    if (wait_start_time_ms_ == 0) return 0;
+    // wait_end_time_ms_ might still be zero if the query is not yet dequeue.
+    // Use the current Unix time in that case. Note that the duration can be
+    // negative if a system clock reset happened after the query was initiated.
+    int64_t wait_end_time_ms = wait_end_time_ms_ > 0LL ?
+        wait_end_time_ms_ : MonotonicMillis();
+    return wait_end_time_ms - wait_start_time_ms_;
+  }
+
   /// Sets the RetryState to RETRYING. Updates the runtime profile with the 
retry status
   /// and cause. Must be called while 'lock_' is held. Sets the query_status_. 
Future
   /// calls to UpdateQueryStatus will not have any effect. This is necessary 
to prevent
@@ -660,6 +672,11 @@ class ClientRequestState {
   /// the query option FETCH_ROWS_TIMEOUT_MS.
   const int64_t fetch_rows_timeout_us_;
 
+  /// Start/end time of the query queuing, in Unix milliseconds.
+  /// Initialized to 0, indicating that queries do not need to be queued.
+  int64_t wait_start_time_ms_ = 0;
+  int64_t wait_end_time_ms_ = 0;
+
   /// If this ClientRequestState was created as a retry of a previously failed 
query, the
   /// original_id_ is set to the query id of the original query that failed. 
The
   /// "original" query is the query that was submitted by the user that failed 
and had to
diff --git a/be/src/service/impala-http-handler.cc 
b/be/src/service/impala-http-handler.cc
index 2545cd3ae..96349450c 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -422,8 +422,75 @@ void ImpalaHttpHandler::QueryMemoryHandler(const 
Webserver::WebRequest& req,
   document->AddMember("query_id", query_id, document->GetAllocator());
 }
 
+void ImpalaHttpHandler::AddQueryRecordTips(Document* document) {
+  document->AddMember("tips_query_id", "Unique ID of the query, click ID to 
jump to the "
+      "query details page to get more detailed information.", 
document->GetAllocator());
+
+  document->AddMember("tips_action_cancel", "Cancel the query, abort the 
execution and "
+      "release resources.", document->GetAllocator());
+
+  document->AddMember("tips_action_close", "Close the query and release 
resources.",
+      document->GetAllocator());
+
+  document->AddMember("tips_user", "The user who submitted the query.",
+      document->GetAllocator());
+
+  document->AddMember("tips_default_db", "The default database for query.",
+      document->GetAllocator());
+
+  document->AddMember("tips_query_type", "The type of query.",
+      document->GetAllocator());
+
+  document->AddMember("tips_start_time", "The start time of the query, i.e. 
the time when"
+      " the query was submitted.", document->GetAllocator());
+
+  document->AddMember("tips_waiting_time", "The duration of the query in the 
waiting to "
+      "close state.", document->GetAllocator());
+
+  document->AddMember("tips_end_time", "The end time of the query, i.e. the 
time when the"
+      " query is completed, canceled or failed.", document->GetAllocator());
+
+  document->AddMember("tips_duration", "The total duration of the query, 
including queued"
+      " time.", document->GetAllocator());
+
+  document->AddMember("tips_queued_duration", "The duration of the query being 
queued due"
+      " to admission control.", document->GetAllocator());
+
+  document->AddMember("tips_mem_usage", "The sum of memory usage peaks on each 
executor "
+      "during the query execution.", document->GetAllocator());
+
+  document->AddMember("tips_mem_estimate", "The cluster wide estimated memory 
usage of "
+      "this query.", document->GetAllocator());
+
+  document->AddMember("tips_scan_progress", "The scan progress of the query, 
i.e. the "
+      "completed number of scan ranges / the total number of scan ranges.",
+      document->GetAllocator());
+
+  document->AddMember("tips_bytes_read", "The total number of bytes read from 
the data "
+      "source during the query execution.", document->GetAllocator());
+
+  document->AddMember("tips_bytes_sent", "The total number of bytes sent by 
this query in"
+      " exchange nodes. Does not include remote reads, data written to disk, 
or data sent"
+      " to the client.", document->GetAllocator());
+
+  document->AddMember("tips_state", "The current state of the query.",
+      document->GetAllocator());
+
+  document->AddMember("tips_last_event", "The latest event reported during the 
execution "
+      "of the query.", document->GetAllocator());
+
+  document->AddMember("tips_rows_fetched", "The number of rows already fetched 
from the "
+      "query by the client.", document->GetAllocator());
+
+  document->AddMember("tips_resource_pool", "The resource pool that the query 
is "
+      "submitted to.", document->GetAllocator());
+
+  document->AddMember("tips_statement", "The statement submitted for the 
query.",
+      document->GetAllocator());
+}
+
 void ImpalaHttpHandler::QueryStateToJson(const ImpalaServer::QueryStateRecord& 
record,
-    Value* value, Document* document) {
+    Value* value, Document* document, bool inflight) {
   Value user(record.effective_user.c_str(), document->GetAllocator());
   value->AddMember("effective_user", user, document->GetAllocator());
 
@@ -443,11 +510,11 @@ void ImpalaHttpHandler::QueryStateToJson(const 
ImpalaServer::QueryStateRecord& r
   value->AddMember("stmt_type", stmt_type, document->GetAllocator());
 
   Value start_time(ToStringFromUnixMicros(record.start_time_us,
-      TimePrecision::Nanosecond).c_str(), document->GetAllocator());
+      TimePrecision::Millisecond).c_str(), document->GetAllocator());
   value->AddMember("start_time", start_time, document->GetAllocator());
 
   Value end_time(ToStringFromUnixMicros(record.end_time_us,
-      TimePrecision::Nanosecond).c_str(), document->GetAllocator());
+      TimePrecision::Millisecond).c_str(), document->GetAllocator());
   value->AddMember("end_time", end_time, document->GetAllocator());
 
   // record.end_time_us might still be zero if the query is not yet done
@@ -460,6 +527,21 @@ void ImpalaHttpHandler::QueryStateToJson(const 
ImpalaServer::QueryStateRecord& r
   Value val_duration(printed_duration.c_str(), document->GetAllocator());
   value->AddMember("duration", val_duration, document->GetAllocator());
 
+  const string& printed_queued_duration = 
PrettyPrinter::Print(record.wait_time_ms,
+      TUnit::TIME_MS);
+  Value queued_duration(printed_queued_duration.c_str(), 
document->GetAllocator());
+  value->AddMember("queued_duration", queued_duration, 
document->GetAllocator());
+
+  const string& printed_mem_usage = 
PrettyPrinter::Print(record.total_peak_mem_usage,
+      TUnit::BYTES);
+  Value mem_usage(printed_mem_usage.c_str(), document->GetAllocator());
+  value->AddMember("mem_usage", mem_usage, document->GetAllocator());
+
+  const string& printed_mem_est = PrettyPrinter::Print(record.cluster_mem_est,
+      TUnit::BYTES);
+  Value mem_est(printed_mem_est.c_str(), document->GetAllocator());
+  value->AddMember("mem_est", mem_est, document->GetAllocator());
+
   string progress = "N/A";
   if (record.has_coord) {
     stringstream ss;
@@ -476,6 +558,16 @@ void ImpalaHttpHandler::QueryStateToJson(const 
ImpalaServer::QueryStateRecord& r
   Value progress_json(progress.c_str(), document->GetAllocator());
   value->AddMember("progress", progress_json, document->GetAllocator());
 
+  const string& printed_bytes_read = PrettyPrinter::Print(record.bytes_read,
+      TUnit::BYTES);
+  Value bytes_read(printed_bytes_read.c_str(), document->GetAllocator());
+  value->AddMember("bytes_read", bytes_read, document->GetAllocator());
+
+  const string& printed_bytes_sent = PrettyPrinter::Print(record.bytes_sent,
+      TUnit::BYTES);
+  Value bytes_sent(printed_bytes_sent.c_str(), document->GetAllocator());
+  value->AddMember("bytes_sent", bytes_sent, document->GetAllocator());
+
   Value state(record.query_state.c_str(), document->GetAllocator());
   value->AddMember("state", state, document->GetAllocator());
 
@@ -493,8 +585,8 @@ void ImpalaHttpHandler::QueryStateToJson(const 
ImpalaServer::QueryStateRecord& r
   // Waiting to be closed.
   bool waiting = record.beeswax_query_state == beeswax::QueryState::EXCEPTION 
||
       record.all_rows_returned;
-  value->AddMember("waiting", waiting, document->GetAllocator());
-  value->AddMember("executing", !waiting, document->GetAllocator());
+  value->AddMember("waiting", inflight && waiting, document->GetAllocator());
+  value->AddMember("executing", inflight && !waiting, 
document->GetAllocator());
 
   int64_t waiting_time = impala::UnixMillis() - record.last_active_time_ms;
   string waiting_time_str = "";
@@ -510,6 +602,8 @@ void ImpalaHttpHandler::QueryStateToJson(const 
ImpalaServer::QueryStateRecord& r
 
 void ImpalaHttpHandler::QueryStateHandler(const Webserver::WebRequest& req,
     Document* document) {
+  AddQueryRecordTips(document);
+
   set<ImpalaServer::QueryStateRecord, ImpalaServer::QueryStateRecordLessThan>
       sorted_query_records;
 
@@ -524,7 +618,7 @@ void ImpalaHttpHandler::QueryStateHandler(const 
Webserver::WebRequest& req,
   int64_t num_waiting_queries = 0;
   for (const ImpalaServer::QueryStateRecord& record: sorted_query_records) {
     Value record_json(kObjectType);
-    QueryStateToJson(record, &record_json, document);
+    QueryStateToJson(record, &record_json, document, true);
 
     if (record_json["waiting"].GetBool()) ++num_waiting_queries;
 
@@ -554,7 +648,7 @@ void ImpalaHttpHandler::QueryStateHandler(const 
Webserver::WebRequest& req,
       // Don't show duplicated entries between in-flight and completed queries.
       if (in_flight_query_ids.find(log_entry->id) != 
in_flight_query_ids.end()) continue;
       Value record_json(kObjectType);
-      QueryStateToJson(*log_entry, &record_json, document);
+      QueryStateToJson(*log_entry, &record_json, document, false);
       completed_queries.PushBack(record_json, document->GetAllocator());
     }
   }
@@ -915,11 +1009,12 @@ void ImpalaHttpHandler::QuerySummaryHandler(bool 
include_json_plan, bool include
     return;
   }
 
+  shared_ptr<ImpalaServer::QueryStateRecord> query_record = nullptr;
   TExecSummary summary;
   string stmt;
   string plan;
   Status query_status;
-  bool found = false;
+  bool inflight = false;
   vector<TPlanFragment> fragments;
 
   // Search the in-flight queries first, followed by the archived ones.
@@ -927,7 +1022,8 @@ void ImpalaHttpHandler::QuerySummaryHandler(bool 
include_json_plan, bool include
     QueryHandle query_handle;
     status = server_->GetQueryHandle(query_id, &query_handle);
     if (status.ok()) {
-      found = true;
+      inflight = true;
+      query_record = 
make_shared<ImpalaServer::QueryStateRecord>(*query_handle);
       // If the query plan isn't generated, avoid waiting for the request
       // state lock to be acquired, since it could potentially be an expensive
       // call, if the table Catalog metadata loading is in progress. Instead
@@ -956,8 +1052,7 @@ void ImpalaHttpHandler::QuerySummaryHandler(bool 
include_json_plan, bool include
     }
   }
 
-  if (!found) {
-    shared_ptr<ImpalaServer::QueryStateRecord> query_record;
+  if (!inflight) {
     if (!server_->GetQueryRecord(query_id, &query_record).ok()) {
       const string& err = Substitute("Unknown query id: $0", 
PrintId(query_id));
       Value json_error(err.c_str(), document->GetAllocator());
@@ -984,18 +1079,28 @@ void ImpalaHttpHandler::QuerySummaryHandler(bool 
include_json_plan, bool include
     const string& printed_summary = PrintExecSummary(summary);
     Value json_summary(printed_summary.c_str(), document->GetAllocator());
     document->AddMember("summary", json_summary, document->GetAllocator());
+    Value json_timeline(query_record->timeline.c_str(), 
document->GetAllocator());
+    document->AddMember("timeline", json_timeline, document->GetAllocator());
   }
   Value json_stmt(RedactCopy(stmt).c_str(), document->GetAllocator());
   document->AddMember("stmt", json_stmt, document->GetAllocator());
   Value json_plan_text(RedactCopy(plan).c_str(), document->GetAllocator());
   document->AddMember("plan", json_plan_text, document->GetAllocator());
-  Value json_inflight(found);
+  Value json_inflight(inflight);
   document->AddMember("inflight", json_inflight, document->GetAllocator());
+  Value json_not_inflight(!inflight);
+  document->AddMember("not_inflight", json_not_inflight, 
document->GetAllocator());
 
   // Redact the error in case the query is contained in the error message.
   Value json_status(query_status.ok() ? "OK" :
       RedactCopy(query_status.GetDetail()).c_str(), document->GetAllocator());
   document->AddMember("status", json_status, document->GetAllocator());
+
+  AddQueryRecordTips(document);
+
+  Value record_json(kObjectType);
+  QueryStateToJson(*query_record, &record_json, document, inflight);
+  document->AddMember("record_json", record_json, document->GetAllocator());
 }
 
 void ImpalaHttpHandler::BackendsHandler(const Webserver::WebRequest& req,
diff --git a/be/src/service/impala-http-handler.h 
b/be/src/service/impala-http-handler.h
index 09b039a03..118102fa7 100644
--- a/be/src/service/impala-http-handler.h
+++ b/be/src/service/impala-http-handler.h
@@ -75,6 +75,9 @@ class ImpalaHttpHandler {
   void HadoopVarzHandler(const Webserver::WebRequest& req,
       rapidjson::Document* document);
 
+  /// Add description strings for the query record table header tooltips.
+  void AddQueryRecordTips(rapidjson::Document* document);
+
   /// Returns two sorted lists of queries, one in-flight and one completed, as 
well as a
   /// list of active backends and their plan-fragment count.
   //
@@ -202,7 +205,7 @@ class ImpalaHttpHandler {
   /// Helper method to render a single QueryStateRecord as a Json object Used 
by
   /// QueryStateHandler().
   void QueryStateToJson(const ImpalaServer::QueryStateRecord& record,
-      rapidjson::Value* value, rapidjson::Document* document);
+      rapidjson::Value* value, rapidjson::Document* document, bool inflight);
 
   /// Json callback for /backends, which prints a table of known backends.
   /// "backends" : [
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 93ca35585..f18c52bcb 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -2369,13 +2369,27 @@ void ImpalaServer::QueryStateRecord::Init(const 
ClientRequestState& query_handle
   default_db = query_handle.default_db();
   start_time_us = query_handle.start_time_us();
   end_time_us = query_handle.end_time_us();
-  has_coord = false;
+  wait_time_ms = query_handle.wait_time_ms();
+  query_handle.summary_profile()->GetTimeline(&timeline);
 
   Coordinator* coord = query_handle.GetCoordinator();
   if (coord != nullptr) {
     num_complete_fragments = coord->progress().num_complete();
     total_fragments = coord->progress().total();
+    auto utilization = coord->ComputeQueryResourceUtilization();
+    total_peak_mem_usage = utilization.total_peak_mem_usage;
+    cluster_mem_est = query_handle.schedule()->cluster_mem_est();
+    bytes_read = utilization.bytes_read;
+    bytes_sent = utilization.exchange_bytes_sent + utilization.scan_bytes_sent;
     has_coord = true;
+  } else {
+    num_complete_fragments = 0;
+    total_fragments = 0;
+    total_peak_mem_usage = 0;
+    cluster_mem_est = 0;
+    bytes_read = 0;
+    bytes_sent = 0;
+    has_coord = false;
   }
   beeswax_query_state = query_handle.BeeswaxQueryState();
   ClientRequestState::RetryState retry_state = query_handle.retry_state();
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index 87c318de6..f8764cb90 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -952,6 +952,26 @@ class ImpalaServer : public ImpalaServiceIf,
     /// ClientRequestState.
     int64_t start_time_us, end_time_us;
 
+    /// The request waited time in ms for queued.
+    int64_t wait_time_ms;
+
+    /// Total peak memory usage by this query at all backends.
+    int64_t total_peak_mem_usage;
+
+    /// The cluster wide estimated memory usage of this query.
+    int64_t cluster_mem_est;
+
+    /// Total bytes read by this query at all backends.
+    int64_t bytes_read;
+
+    /// The total number of bytes sent (across the network) by this query in 
exchange
+    /// nodes. Does not include remote reads, data written to disk, or data 
sent to the
+    /// client.
+    int64_t bytes_sent;
+
+    // Query timeline from summary profile.
+    std::string timeline;
+
     /// Summary of execution for this query.
     TExecSummary exec_summary;
 
diff --git a/be/src/util/runtime-profile.cc b/be/src/util/runtime-profile.cc
index 09b1292e2..75068bc68 100644
--- a/be/src/util/runtime-profile.cc
+++ b/be/src/util/runtime-profile.cc
@@ -1538,41 +1538,43 @@ static void PrettyPrintTimeSeries(const string& label, 
const int64_t* samples, i
   stream << endl;
 }
 
-void RuntimeProfile::PrettyPrintSubclassCounters(
-    ostream* s, const string& prefix, Verbosity verbosity) const {
+void RuntimeProfile::PrettyPrintTimeline(ostream* s, const string& prefix) 
const {
   ostream& stream = *s;
-  {
-    // Print all the event timers as the following:
-    // <EventKey> Timeline: 2s719ms
-    //     - Event 1: 6.522us (6.522us)
-    //     - Event 2: 2s288ms (2s288ms)
-    //     - Event 3: 2s410ms (121.138ms)
-    // The times in parentheses are the time elapsed since the last event.
-    vector<EventSequence::Event> events;
-    lock_guard<SpinLock> l(event_sequence_lock_);
-    for (const auto& event_sequence : event_sequence_map_) {
-      // If the stopwatch has never been started (e.g. because this sequence 
came from
-      // Thrift), look for the last element to tell us the total runtime. For
-      // currently-updating sequences, it's better to use the stopwatch value 
because that
-      // updates continuously.
-      int64_t last = event_sequence.second->ElapsedTime();
-      event_sequence.second->GetEvents(&events);
-      if (last == 0 && events.size() > 0) last = events.back().second;
-      stream << prefix << "  " << event_sequence.first << ": "
-             << PrettyPrinter::Print(last, TUnit::TIME_NS)
+  // Print all the event timers as the following:
+  // <EventKey> Timeline: 2s719ms
+  //     - Event 1: 6.522us (6.522us)
+  //     - Event 2: 2s288ms (2s288ms)
+  //     - Event 3: 2s410ms (121.138ms)
+  // The times in parentheses are the time elapsed since the last event.
+  vector<EventSequence::Event> events;
+  lock_guard<SpinLock> l(event_sequence_lock_);
+  for (const auto& event_sequence : event_sequence_map_) {
+    // If the stopwatch has never been started (e.g. because this sequence 
came from
+    // Thrift), look for the last element to tell us the total runtime. For
+    // currently-updating sequences, it's better to use the stopwatch value 
because that
+    // updates continuously.
+    int64_t last = event_sequence.second->ElapsedTime();
+    event_sequence.second->GetEvents(&events);
+    if (last == 0 && events.size() > 0) last = events.back().second;
+    stream << prefix << event_sequence.first << ": "
+           << PrettyPrinter::Print(last, TUnit::TIME_NS)
+           << endl;
+
+    int64_t prev = 0L;
+    event_sequence.second->GetEvents(&events);
+    for (const EventSequence::Event& event: events) {
+      stream << prefix << "   - " << event.first << ": "
+             << PrettyPrinter::Print(event.second, TUnit::TIME_NS) << " ("
+             << PrettyPrinter::Print(event.second - prev, TUnit::TIME_NS) << 
")"
              << endl;
-
-      int64_t prev = 0L;
-      event_sequence.second->GetEvents(&events);
-      for (const EventSequence::Event& event: events) {
-        stream << prefix << "     - " << event.first << ": "
-               << PrettyPrinter::Print(event.second, TUnit::TIME_NS) << " ("
-               << PrettyPrinter::Print(event.second - prev, TUnit::TIME_NS) << 
")"
-               << endl;
-        prev = event.second;
-      }
+      prev = event.second;
     }
   }
+}
+
+void RuntimeProfile::PrettyPrintSubclassCounters(
+    ostream* s, const string& prefix, Verbosity verbosity) const {
+  PrettyPrintTimeline(s, prefix + "  ");
 
   {
     // Print time series counters.
@@ -1846,6 +1848,13 @@ void RuntimeProfile::GetExecSummary(TExecSummary* 
t_exec_summary) const {
   *t_exec_summary = t_exec_summary_;
 }
 
+void RuntimeProfile::GetTimeline(string* output) const {
+  DCHECK(output != nullptr);
+  stringstream stream;
+  PrettyPrintTimeline(&stream, "");
+  *output = stream.str();
+}
+
 void RuntimeProfile::SetPlanNodeId(int node_id) {
   DCHECK(!metadata_.__isset.data_sink_id) << "Don't set conflicting metadata";
   metadata_.__set_plan_node_id(node_id);
diff --git a/be/src/util/runtime-profile.h b/be/src/util/runtime-profile.h
index 8e1a1fb2e..c6b8fece6 100644
--- a/be/src/util/runtime-profile.h
+++ b/be/src/util/runtime-profile.h
@@ -727,6 +727,9 @@ class RuntimeProfile : public RuntimeProfileBase {
   /// Get a copy of exec_summary to t_exec_summary
   void GetExecSummary(TExecSummary* t_exec_summary) const;
 
+  /// Print all the event timers as timeline into output.
+  void GetTimeline(std::string* output) const;
+
  protected:
   virtual int GetNumInputProfiles() const override { return 1; }
 
@@ -746,6 +749,8 @@ class RuntimeProfile : public RuntimeProfileBase {
   void PrettyPrintInfoStrings(
       std::ostream* s, const std::string& prefix, Verbosity verbosity) const 
override;
 
+  void PrettyPrintTimeline(std::ostream* s, const string& prefix) const;
+
   /// Print any additional counters from this subclass.
   void PrettyPrintSubclassCounters(
       std::ostream* s, const std::string& prefix, Verbosity verbosity) const 
override;
diff --git a/common/protobuf/admission_control_service.proto 
b/common/protobuf/admission_control_service.proto
index f844b3091..811b0f4ad 100644
--- a/common/protobuf/admission_control_service.proto
+++ b/common/protobuf/admission_control_service.proto
@@ -151,6 +151,9 @@ message QuerySchedulePB {
   // Set by the admission controller with a value that is only valid if it was 
admitted
   // successfully.
   optional int64 coord_backend_mem_to_admit = 8;
+
+  /// The cluster wide estimated memory usage of this query.
+  optional int64 cluster_mem_est = 9;
 }
 
 message AdmitQueryRequestPB {
@@ -186,6 +189,12 @@ message GetQueryStatusResponsePB {
 
   // Idx of the TRuntimeProfileTree sidecar.
   optional int32 summary_profile_sidecar_idx = 3;
+
+  // Start time of the query queuing, in Unix milliseconds.
+  optional int64 wait_start_time_ms = 4;
+
+  // End time of the query queuing, in Unix milliseconds.
+  optional int64 wait_end_time_ms = 5;
 }
 
 message ReleaseQueryRequestPB {
diff --git a/www/common-header.tmpl b/www/common-header.tmpl
index baa07ad8c..e51885686 100644
--- a/www/common-header.tmpl
+++ b/www/common-header.tmpl
@@ -66,7 +66,7 @@ common-footer.tmpl) }}
         </nav>
       </div>
     </header>
-<div class='container-fluid' style='width:80%'>
+<div class='container-fluid' style='width:90%'>
 {{#error}}
 <div class="alert alert-danger">
   <strong>Error:</strong>
diff --git a/www/queries.tmpl b/www/queries.tmpl
index 893b424b7..59513c4c2 100644
--- a/www/queries.tmpl
+++ b/www/queries.tmpl
@@ -28,37 +28,53 @@ command line parameter.</p>
 
 <table class='table table-hover table-border'>
   <tr>
-    <th>User</th>
-    <th>Default Db</th>
-    <th>Statement</th>
-    <th>Query Type</th>
-    <th>Start Time</th>
-    <th>Duration</th>
-    <th>Scan Progress</th>
-    <th>State</th>
-    <th>Last Event</th>
-    <th># rows fetched</th>
-    <th>Resource Pool</th>
-    <th>Details</th>
-    <th>Action</th>
+    <th title="{{tips_query_id}}">Query ID</th>
+    <th title="{{tips_action_cancel}}">Action</th>
+    <th title="{{tips_user}}">User</th>
+    <th title="{{tips_default_db}}">Default Db</th>
+    <th title="{{tips_query_type}}">Query Type</th>
+    <th title="{{tips_start_time}}">Start Time</th>
+    <th>
+      <span title="{{tips_duration}}">Duration</span>
+      <hr style="margin-top:0px;margin-bottom:0px;"/>
+      <span title="{{tips_queued_duration}}">Queued&nbsp;Duration</span>
+    </th>
+    <th>
+      <span title="{{tips_mem_usage}}">Mem&nbsp;Usage</span>
+      <hr style="margin-top:0px;margin-bottom:0px;"/>
+      <span title="{{tips_mem_estimate}}">Mem&nbsp;Estimate</span>
+    </th>
+    <th title="{{tips_scan_progress}}">Scan Progress</th>
+    <th>
+      <span title="{{tips_bytes_read}}">Bytes&nbsp;Read</span>
+      <hr style="margin-top:0px;margin-bottom:0px;"/>
+      <span title="{{tips_bytes_sent}}">Bytes&nbsp;Sent</span>
+    </th>
+    <th title="{{tips_state}}">State</th>
+    <th title="{{tips_last_event}}">Last Event</th>
+    <th title="{{tips_rows_fetched}}"># rows fetched</th>
+    <th title="{{tips_resource_pool}}">Resource Pool</th>
+    <th title="{{tips_statement}}">Statement</th>
   </tr>
 {{! filter to get just executing queries from in_flight_queries}}
 {{#in_flight_queries}}
 {{?executing}}
   <tr>
+    <td style="min-width:150px;word-break:break-all;"><a href='{{ 
__common__.host-url }}/query_plan?query_id={{query_id}}'>{{query_id}}</a></td>
+    <td><a href='{{ __common__.host-url 
}}/cancel_query?query_id={{query_id}}'>Cancel</a></td>
     <td>{{effective_user}}</td>
     <td>{{default_db}}</td>
-    <td><samp>{{stmt}}</samp></td>
     <td><samp>{{stmt_type}}</samp></td>
     <td>{{start_time}}</td>
-    <td>{{duration}}</td>
+    <td>{{duration}}<hr 
style="margin-top:0px;margin-bottom:0px;"/>{{queued_duration}}</td>
+    <td>{{mem_usage}}<hr 
style="margin-top:0px;margin-bottom:0px;"/>{{mem_est}}</td>
     <td>{{progress}}</td>
+    <td>{{bytes_read}}<hr 
style="margin-top:0px;margin-bottom:0px;"/>{{bytes_sent}}</td>
     <td><samp>{{state}}</samp></td>
     <td><samp>{{last_event}}</samp></td>
     <td>{{rows_fetched}}</td>
     <td>{{resource_pool}}</td>
-    <td><a href='{{ __common__.host-url 
}}/query_plan?query_id={{query_id}}'>Details</a></td>
-    <td><a href='{{ __common__.host-url 
}}/cancel_query?query_id={{query_id}}'>Cancel</a></td>
+    <td><samp>{{stmt}}</samp></td>
   </tr>
 {{/executing}}
 {{/in_flight_queries}}
@@ -71,39 +87,55 @@ command line parameter.</p>
 
 <table class='table table-hover table-border'>
   <tr>
-    <th>User</th>
-    <th>Default Db</th>
-    <th>Statement</th>
-    <th>Query Type</th>
-    <th>Start Time</th>
-    <th>Waiting Time</th>
-    <th>Duration</th>
-    <th>Scan Progress</th>
-    <th>State</th>
-    <th>Last Event</th>
-    <th># rows fetched</th>
-    <th>Resource Pool</th>
-    <th>Details</th>
-    <th>Action</th>
+    <th title="{{tips_query_id}}">Query ID</th>
+    <th title="{{tips_action_close}}">Action</th>
+    <th title="{{tips_user}}">User</th>
+    <th title="{{tips_default_db}}">Default Db</th>
+    <th title="{{tips_query_type}}">Query Type</th>
+    <th title="{{tips_start_time}}">Start Time</th>
+    <th title="{{tips_waiting_time}}">Waiting Time</th>
+    <th>
+      <span title="{{tips_duration}}">Duration</span>
+      <hr style="margin-top:0px;margin-bottom:0px;"/>
+      <span title="{{tips_queued_duration}}">Queued&nbsp;Duration</span>
+    </th>
+    <th>
+      <span title="{{tips_mem_usage}}">Mem&nbsp;Usage</span>
+      <hr style="margin-top:0px;margin-bottom:0px;"/>
+      <span title="{{tips_mem_estimate}}">Mem&nbsp;Estimate</span>
+    </th>
+    <th title="{{tips_scan_progress}}">Scan Progress</th>
+    <th>
+      <span title="{{tips_bytes_read}}">Bytes&nbsp;Read</span>
+      <hr style="margin-top:0px;margin-bottom:0px;"/>
+      <span title="{{tips_bytes_sent}}">Bytes&nbsp;Sent</span>
+    </th>
+    <th title="{{tips_state}}">State</th>
+    <th title="{{tips_last_event}}">Last Event</th>
+    <th title="{{tips_rows_fetched}}"># rows fetched</th>
+    <th title="{{tips_resource_pool}}">Resource Pool</th>
+    <th title="{{tips_statement}}">Statement</th>
   </tr>
 {{! filter to get just waiting queries from in_flight_queries}}
 {{#in_flight_queries}}
 {{?waiting}}
   <tr>
+    <td style="min-width:150px;word-break:break-all;"><a href='{{ 
__common__.host-url }}/query_plan?query_id={{query_id}}'>{{query_id}}</a></td>
+    <td><a href='{{ __common__.host-url 
}}/cancel_query?query_id={{query_id}}'>Close</a></td>
     <td>{{effective_user}}</td>
     <td>{{default_db}}</td>
-    <td><samp>{{stmt}}</samp></td>
     <td><samp>{{stmt_type}}</samp></td>
     <td>{{start_time}}</td>
     <td>{{waiting_time}}</td>
-    <td>{{duration}}</td>
+    <td>{{duration}}<hr 
style="margin-top:0px;margin-bottom:0px;"/>{{queued_duration}}</td>
+    <td>{{mem_usage}}<hr 
style="margin-top:0px;margin-bottom:0px;"/>{{mem_est}}</td>
     <td>{{progress}}</td>
+    <td>{{bytes_read}}<hr 
style="margin-top:0px;margin-bottom:0px;"/>{{bytes_sent}}</td>
     <td><samp>{{state}}</samp></td>
     <td><samp>{{last_event}}</samp></td>
     <td>{{rows_fetched}}</td>
     <td>{{resource_pool}}</td>
-    <td><a href='{{ __common__.host-url 
}}/query_plan?query_id={{query_id}}'>Details</a></td>
-    <td><a href='{{ __common__.host-url 
}}/cancel_query?query_id={{query_id}}'>Close</a></td>
+    <td><samp>{{stmt}}</samp></td>
   </tr>
 {{/waiting}}
 {{/in_flight_queries}}
@@ -113,33 +145,49 @@ command line parameter.</p>
 
 <table class='table table-hover table-border'>
   <tr>
-    <th>User</th>
-    <th>Default Db</th>
-    <th>Statement</th>
-    <th>Query Type</th>
-    <th>Start Time</th>
-    <th>End Time</th>
-    <th>Duration</th>
-    <th>Scan Progress</th>
-    <th>State</th>
-    <th># rows fetched</th>
-    <th>Resource Pool</th>
-    <th>Details</th>
+    <th title="{{tips_query_id}}">Query ID</th>
+    <th title="{{tips_user}}">User</th>
+    <th title="{{tips_default_db}}">Default Db</th>
+    <th title="{{tips_query_type}}">Query Type</th>
+    <th title="{{tips_start_time}}">Start Time</th>
+    <th title="{{tips_end_time}}">End Time</th>
+    <th>
+      <span title="{{tips_duration}}">Duration</span>
+      <hr style="margin-top:0px;margin-bottom:0px;"/>
+      <span title="{{tips_queued_duration}}">Queued&nbsp;Duration</span>
+    </th>
+    <th>
+      <span title="{{tips_mem_usage}}">Mem&nbsp;Usage</span>
+      <hr style="margin-top:0px;margin-bottom:0px;"/>
+      <span title="{{tips_mem_estimate}}">Mem&nbsp;Estimate</span>
+    </th>
+    <th title="{{tips_scan_progress}}">Scan Progress</th>
+    <th>
+      <span title="{{tips_bytes_read}}">Bytes&nbsp;Read</span>
+      <hr style="margin-top:0px;margin-bottom:0px;"/>
+      <span title="{{tips_bytes_sent}}">Bytes&nbsp;Sent</span>
+    </th>
+    <th title="{{tips_state}}">State</th>
+    <th title="{{tips_rows_fetched}}"># rows fetched</th>
+    <th title="{{tips_resource_pool}}">Resource Pool</th>
+    <th title="{{tips_statement}}">Statement</th>
   </tr>
 {{#completed_queries}}
   <tr>
+    <td style="min-width:150px;word-break:break-all;"><a href='{{ 
__common__.host-url }}/query_plan?query_id={{query_id}}'>{{query_id}}</a></td>
     <td>{{effective_user}}</td>
     <td>{{default_db}}</td>
-    <td><samp>{{stmt}}</samp></td>
     <td><samp>{{stmt_type}}</samp></td>
     <td>{{start_time}}</td>
     <td>{{end_time}}</td>
-    <td>{{duration}}</td>
+    <td>{{duration}}<hr 
style="margin-top:0px;margin-bottom:0px;"/>{{queued_duration}}</td>
+    <td>{{mem_usage}}<hr 
style="margin-top:0px;margin-bottom:0px;"/>{{mem_est}}</td>
     <td>{{progress}}</td>
+    <td>{{bytes_read}}<hr 
style="margin-top:0px;margin-bottom:0px;"/>{{bytes_sent}}</td>
     <td><samp>{{state}}</samp></td>
     <td>{{rows_fetched}}</td>
     <td>{{resource_pool}}</td>
-    <td><a href='{{ __common__.host-url 
}}/query_plan?query_id={{query_id}}'>Details</a></td>
+    <td><samp>{{stmt}}</samp></td>
   </tr>
 {{/completed_queries}}
 </table>
diff --git a/www/query_detail_tabs.tmpl b/www/query_detail_tabs.tmpl
index c1b945b8f..61d9926d9 100644
--- a/www/query_detail_tabs.tmpl
+++ b/www/query_detail_tabs.tmpl
@@ -18,8 +18,76 @@ under the License.
 -->
 
 <h3>Query <tt>{{query_id}}</tt></h3>
+{{#record_json}}
+<h4><em>User:</em> <span id="user">{{effective_user}}</span></h4>
+<h4><em>State:</em> <span id="state">{{state}}</span></h4>
+{{/record_json}}
 {{?status}}<h4><em>Status:</em> <span 
id="status">{{status}}</span></h4>{{/status}}
 
+{{#record_json}}
+<table id='record' class='table table-hover table-border'>
+  <tr>
+    {{#executing}}
+    <th title="{{tips_action_cancel}}">Action</th>
+    {{/executing}}
+    {{#waiting}}
+    <th title="{{tips_action_close}}">Action</th>
+    {{/waiting}}
+    <th title="{{tips_default_db}}">Default Db</th>
+    <th title="{{tips_query_type}}">Query Type</th>
+    <th title="{{tips_start_time}}">Start Time</th>
+    {{#waiting}}
+    <th title="{{tips_waiting_time}}">Waiting Time</th>
+    {{/waiting}}
+    {{#not_inflight}}
+    <th title="{{tips_end_time}}">End Time</th>
+    {{/not_inflight}}
+    <th title="{{tips_duration}}">Duration</th>
+    <th title="{{tips_queued_duration}}">Queued Duration</th>
+    <th title="{{tips_mem_usage}}">Mem Usage</th>
+    <th title="{{tips_mem_estimate}}">Mem Estimate</th>
+    <th title="{{tips_scan_progress}}">Scan Progress</th>
+    <th title="{{tips_bytes_read}}">Bytes Read</th>
+    <th title="{{tips_bytes_sent}}">Bytes Sent</th>
+    {{#inflight}}
+    <th title="{{tips_last_event}}">Last Event</th>
+    {{/inflight}}
+    <th title="{{tips_rows_fetched}}"># rows fetched</th>
+    <th title="{{tips_resource_pool}}">Resource Pool</th>
+  </tr>
+
+  <tr>
+    {{#executing}}
+    <td><a href='{{ __common__.host-url 
}}/cancel_query?query_id={{query_id}}'>Cancel</a></td>
+    {{/executing}}
+    {{#waiting}}
+    <td><a href='{{ __common__.host-url 
}}/cancel_query?query_id={{query_id}}'>Close</a></td>
+    {{/waiting}}
+    <td>{{default_db}}</td>
+    <td><samp>{{stmt_type}}</samp></td>
+    <td>{{start_time}}</td>
+    {{#waiting}}
+    <td>{{waiting_time}}</td>
+    {{/waiting}}
+    {{#not_inflight}}
+    <td>{{end_time}}</td>
+    {{/not_inflight}}
+    <td>{{duration}}</td>
+    <td>{{queued_duration}}</td>
+    <td>{{mem_usage}}</td>
+    <td>{{mem_est}}</td>
+    <td>{{progress}}</td>
+    <td>{{bytes_read}}</td>
+    <td>{{bytes_sent}}</td>
+    {{#inflight}}
+    <td><samp>{{last_event}}</samp></td>
+    {{/inflight}}
+    <td>{{rows_fetched}}</td>
+    <td>{{resource_pool}}</td>
+  </tr>
+</table>
+{{/record_json}}
+
 <ul class="nav nav-tabs">
   <li class="nav-item" role="presentation"><a class="nav-link" id="plan-tab" 
href="{{ __common__.host-url }}/query_plan?query_id={{query_id}}">Plan</a></li>
   <li class="nav-item" role="presentation"><a class="nav-link" id="stmt-tab" 
href="{{ __common__.host-url }}/query_stmt?query_id={{query_id}}">Query</a></li>
@@ -31,3 +99,112 @@ under the License.
   <li class="nav-item" role="presentation"><a class="nav-link" 
id="backends-tab" href="{{ __common__.host-url 
}}/query_backends?query_id={{query_id}}">Backends</a></li>
   <li class="nav-item" role="presentation"><a class="nav-link" 
id="finstances-tab" href="{{ __common__.host-url 
}}/query_finstances?query_id={{query_id}}">Fragment Instances</a></li>
 </ul>
+
+<script>
+if (typeof index == "undefined") {
+  var index = {};
+
+  // For inflight (executing or waiting) query
+  index.action = 0;
+  index.default_db = 1;
+  index.stmt_type = 2;
+  index.start_time = 3;
+
+  // For not inflight query
+  index.end_time = 3;
+
+  // For executing query
+  index.executing = {
+    duration        : 4,
+    queued_duration : 5,
+    mem_usage       : 6,
+    mem_est         : 7,
+    progress        : 8,
+    bytes_read      : 9,
+    bytes_sent      : 10,
+    last_event      : 11,
+    rows_fetched    : 12
+  };
+
+  // For waiting query
+  index.waiting = {
+    waiting_time    : 4,
+    duration        : 5,
+    queued_duration : 6,
+    mem_usage       : 7,
+    mem_est         : 8,
+    progress        : 9,
+    bytes_read      : 10,
+    bytes_sent      : 11,
+    last_event      : 12,
+    rows_fetched    : 13
+  };
+}
+
+function update_executing_record(data_row, record_json) {
+  for (name in index.executing) {
+    data_row.cells[index.executing[name]].textContent = record_json[name];
+  }
+}
+
+function refresh_record(record_json) {
+  document.getElementById("state").textContent = record_json["state"];
+  record = document.getElementById("record");
+  header_row = record.rows[0];
+  data_row = record.rows[1];
+
+  // Query is finished
+  if (header_row.cells[index.action].textContent != "Action") {
+    return;
+  }
+
+  // Query is executing
+  if (record_json["executing"]) {
+    update_executing_record(data_row, record_json);
+    return;
+  }
+
+  // Query is waiting
+  if (record_json["waiting"]) {
+    // Executing -> Waiting
+    if (header_row.cells[index.waiting.waiting_time].textContent != "Waiting 
Time") {
+      update_executing_record(data_row, record_json);
+      // Add column 'Waiting Time' before 'Duration'.
+      header_row.insertCell(index.executing.duration).outerHTML = "<th 
title=\"{{tips_waiting_time}}\">Waiting Time</th>";
+      data_row.insertCell(index.executing.duration);
+      // Update action tooltips.
+      header_row.cells[index.action].outerHTML = "<th 
title=\"{{tips_action_close}}\">Action</th>";
+      // Rename action 'Cancel' to 'Close'.
+      data_row.cells[index.action].childNodes[0].textContent = "Close";
+    }
+    // Update 'Waiting Time'.
+    data_row.cells[index.waiting.waiting_time].textContent = 
record_json["waiting_time"];
+    return;
+  }
+
+  // Query is going to finished
+  // Executing -> Finished
+  if (header_row.cells[index.waiting.waiting_time].textContent != "Waiting 
Time") {
+    update_executing_record(data_row, record_json);
+    // Delete column 'Last Event'.
+    header_row.deleteCell(index.executing.last_event);
+    data_row.deleteCell(index.executing.last_event);
+    // Add column 'End Time' before 'Duration'.
+    header_row.insertCell(index.executing.duration).outerHTML = "<th 
title=\"{{tips_end_time}}\">End Time</th>";
+    data_row.insertCell(index.executing.duration);
+  // Waiting -> Finished
+  } else {
+    // Delete column 'Last Event'.
+    header_row.deleteCell(index.waiting.last_event);
+    data_row.deleteCell(index.waiting.last_event);
+    // Rename column 'Waiting Time' to 'End Time'.
+    header_row.cells[index.waiting.waiting_time].outerHTML = "<th 
title=\"{{tips_end_time}}\">End Time</th>";
+  }
+  // Delete column 'Action'.
+  header_row.deleteCell(index.action);
+  data_row.deleteCell(index.action);
+  // Update 'End Time'.
+  data_row.cells[index.end_time].textContent = record_json["end_time"];
+  return;
+}
+</script>
diff --git a/www/query_plan.tmpl b/www/query_plan.tmpl
index ed2373fcc..341437680 100644
--- a/www/query_plan.tmpl
+++ b/www/query_plan.tmpl
@@ -143,6 +143,7 @@ var is_first = true;
 function renderGraph(ignored_arg) {
   if (req.status != 200) return;
   var json = JSON.parse(req.responseText);
+  refresh_record(json.record_json);
   var plan = json["plan_json"];
   var inflight = json["inflight"];
   if (!inflight) {
diff --git a/www/query_summary.tmpl b/www/query_summary.tmpl
index adfbeff6b..a1c6c3192 100644
--- a/www/query_summary.tmpl
+++ b/www/query_summary.tmpl
@@ -28,6 +28,8 @@ under the License.
   </label>  Last updated: <span id="last-updated"></span>
 </div>
 
+<h3>Timeline</h3>
+<pre id="timeline">{{timeline}}</pre>
 <h3>Exec Summary</h3>
 <pre id="summary">{{summary}}</pre>
 
@@ -45,6 +47,8 @@ function refresh() {
     }
     var blob = xhr.response;
     json = JSON.parse(blob);
+    refresh_record(json.record_json);
+    document.getElementById("timeline").textContent = json["timeline"].trim();
     document.getElementById("summary").textContent = json["summary"].trim();
     document.getElementById("status").textContent = json["status"];
     document.getElementById("last-updated").textContent = new Date();

Reply via email to