You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2019/08/08 01:06:34 UTC

[impala] 02/03: IMPALA-5149: Provide query profile in JSON format

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

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

commit fb3b1079d21f814c05ef429aeaca4fbdf9915a7b
Author: Jiawei Wang <ji...@cloudera.com>
AuthorDate: Wed Jul 3 14:21:38 2019 -0700

    IMPALA-5149: Provide query profile in JSON format
    
    Description:
    Today there is a text and Thrift version of the query profile, but
    it would be useful to have a JSON version for portability and machine
    consumption. The ultimate goal is to have a Download JSON format
    profile link along with the other two formats.
    
    Modification:
    1.Add Json format download option in impalad profile page
    2.Add ToJson() function for RuntimeProfile, Counters, EventSequence
    3.Add JSON format into QueryStateRecord
    4.Add tests for E2E test, unit test, hs2 test
    5.Modify query profile page to different download option
    6.Modify HS2 server to support get JSON format profile
    
    Future compatibility:
    The schema of the JSON format can be changed in the future with
    the standardization of Profile and Counter structure.
    
    Tests:
    E2E tests:
    tests/webserver/test_web_pages.py - test_download_profile
     * merge text and json format download together
    HS2 tests:
    tests/hs2/test_hs2.py - test_get_profile
     * add json format test
    BE Unit tests:
    be/src/util/runtime-profile-test.cc
     * ToJson.RuntimeProfileToJsonTest
     * ToJson.EmptyTest
     * ToJson.EventSequenceToJsonTest
     * ToJson.TimeSeriesCounterToJsonTest
    
    Change-Id: I8181ac818bf22207ca1deabd9220c397ae723ec1
    Reviewed-on: http://gerrit.cloudera.org:8080/13801
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/service/impala-beeswax-server.cc |   3 +-
 be/src/service/impala-hs2-server.cc     |  13 +-
 be/src/service/impala-http-handler.cc   |  27 +++-
 be/src/service/impala-http-handler.h    |   5 +
 be/src/service/impala-server.cc         |  29 +++-
 be/src/service/impala-server.h          |   9 +-
 be/src/util/CMakeLists.txt              |   2 +-
 be/src/util/runtime-profile-counters.h  |  78 +++++++++--
 be/src/util/runtime-profile-test.cc     | 170 ++++++++++++++++++++++-
 be/src/util/runtime-profile.cc          | 238 ++++++++++++++++++++++++++++++++
 be/src/util/runtime-profile.h           |  27 +++-
 common/thrift/ImpalaService.thrift      |   3 +-
 common/thrift/RuntimeProfile.thrift     |   3 +
 tests/hs2/test_hs2.py                   |  23 +++
 tests/webserver/test_web_pages.py       |  50 +++++--
 www/query_profile.tmpl                  |  26 ++--
 16 files changed, 646 insertions(+), 60 deletions(-)

diff --git a/be/src/service/impala-beeswax-server.cc b/be/src/service/impala-beeswax-server.cc
index 9e3198b..76b6a75 100644
--- a/be/src/service/impala-beeswax-server.cc
+++ b/be/src/service/impala-beeswax-server.cc
@@ -418,7 +418,8 @@ void ImpalaServer::GetRuntimeProfile(string& profile_output, const QueryHandle&
   // GetRuntimeProfile() will validate that the user has access to 'query_id'.
   VLOG_RPC << "GetRuntimeProfile(): query_id=" << PrintId(query_id);
   Status status = GetRuntimeProfileOutput(
-      query_id, GetEffectiveUser(*session), TRuntimeProfileFormat::STRING, &ss, nullptr);
+      query_id, GetEffectiveUser(*session), TRuntimeProfileFormat::STRING,
+      &ss, nullptr, nullptr);
   if (!status.ok()) {
     ss << "GetRuntimeProfile error: " << status.GetDetail();
     RaiseBeeswaxException(ss.str(), SQLSTATE_GENERAL_ERROR);
diff --git a/be/src/service/impala-hs2-server.cc b/be/src/service/impala-hs2-server.cc
index b50a213..697ef67 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -32,6 +32,8 @@
 #include <gperftools/malloc_extension.h>
 #include <gtest/gtest.h>
 #include <gutil/strings/substitute.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/prettywriter.h>
 #include <thrift/protocol/TDebugProtocol.h>
 
 #include "common/logging.h"
@@ -975,12 +977,19 @@ void ImpalaServer::GetRuntimeProfile(
 
   stringstream ss;
   TRuntimeProfileTree thrift_profile;
+  rapidjson::Document json_profile(rapidjson::kObjectType);
   HS2_RETURN_IF_ERROR(return_val,
-      GetRuntimeProfileOutput(
-          query_id, GetEffectiveUser(*session), request.format, &ss, &thrift_profile),
+      GetRuntimeProfileOutput(query_id, GetEffectiveUser(*session), request.format, &ss,
+          &thrift_profile, &json_profile),
       SQLSTATE_GENERAL_ERROR);
   if (request.format == TRuntimeProfileFormat::THRIFT) {
     return_val.__set_thrift_profile(thrift_profile);
+  } else if (request.format == TRuntimeProfileFormat::JSON) {
+    rapidjson::StringBuffer sb;
+    rapidjson::PrettyWriter<rapidjson::StringBuffer> writer(sb);
+    json_profile.Accept(writer);
+    ss << sb.GetString();
+    return_val.__set_profile(ss.str());
   } else {
     DCHECK(request.format == TRuntimeProfileFormat::STRING
         || request.format == TRuntimeProfileFormat::BASE64);
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index c24a8fb..20e0716 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -131,6 +131,9 @@ void ImpalaHttpHandler::RegisterHandlers(Webserver* webserver) {
   webserver->RegisterUrlCallback("/query_profile_plain_text", "raw_text.tmpl",
         MakeCallback(this, &ImpalaHttpHandler::QueryProfileTextHandler), false);
 
+  webserver->RegisterUrlCallback("/query_profile_json", "raw_text.tmpl",
+      MakeCallback(this, &ImpalaHttpHandler::QueryProfileJsonHandler), false);
+
   webserver->RegisterUrlCallback("/inflight_query_ids", "raw_text.tmpl",
       MakeCallback(this, &ImpalaHttpHandler::InflightQueryIdsHandler), false);
 
@@ -237,7 +240,7 @@ void ImpalaHttpHandler::QueryProfileHandler(const Webserver::WebRequest& req,
 
   stringstream ss;
   Status status = server_->GetRuntimeProfileOutput(
-      unique_id, "", TRuntimeProfileFormat::STRING, &ss, nullptr);
+      unique_id, "", TRuntimeProfileFormat::STRING, &ss, nullptr, nullptr);
   if (!status.ok()) {
     Value error(status.GetDetail().c_str(), document->GetAllocator());
     document->AddMember("error", error, document->GetAllocator());
@@ -260,25 +263,37 @@ void ImpalaHttpHandler::QueryProfileHelper(const Webserver::WebRequest& req,
     ss << status.GetDetail();
   } else {
     Status status = server_->GetRuntimeProfileOutput(
-      unique_id, "", format, &ss, nullptr);
+      unique_id, "", format, &ss, nullptr, document);
     if (!status.ok()) {
       ss.str(Substitute("Could not obtain runtime profile: $0", status.GetDetail()));
     }
   }
-  document->AddMember(rapidjson::StringRef(Webserver::ENABLE_RAW_HTML_KEY), true,
-      document->GetAllocator());
-  Value profile(ss.str().c_str(), document->GetAllocator());
-  document->AddMember("contents", profile, document->GetAllocator());
+  // JSON format contents already been added inside document in GetRuntimeProfileOutput()
+  if (format != TRuntimeProfileFormat::JSON){
+    Value profile(ss.str().c_str(), document->GetAllocator());
+    document->AddMember("contents", profile, document->GetAllocator());
+  }
 }
 
 void ImpalaHttpHandler::QueryProfileEncodedHandler(const Webserver::WebRequest& req,
     Document* document) {
   QueryProfileHelper(req, document, TRuntimeProfileFormat::BASE64);
+  document->AddMember(rapidjson::StringRef(Webserver::ENABLE_RAW_HTML_KEY), true,
+      document->GetAllocator());
 }
 
 void ImpalaHttpHandler::QueryProfileTextHandler(const Webserver::WebRequest& req,
     Document* document) {
   QueryProfileHelper(req, document, TRuntimeProfileFormat::STRING);
+  document->AddMember(rapidjson::StringRef(Webserver::ENABLE_RAW_HTML_KEY), true,
+      document->GetAllocator());
+}
+
+void ImpalaHttpHandler::QueryProfileJsonHandler(const Webserver::WebRequest& req,
+    Document* document) {
+  QueryProfileHelper(req, document, TRuntimeProfileFormat::JSON);
+  document->AddMember(rapidjson::StringRef(Webserver::ENABLE_PLAIN_JSON_KEY), true,
+      document->GetAllocator());
 }
 
 void ImpalaHttpHandler::InflightQueryIdsHandler(const Webserver::WebRequest& req,
diff --git a/be/src/service/impala-http-handler.h b/be/src/service/impala-http-handler.h
index ce742a4..e34ad68 100644
--- a/be/src/service/impala-http-handler.h
+++ b/be/src/service/impala-http-handler.h
@@ -123,6 +123,11 @@ class ImpalaHttpHandler {
   void QueryProfileTextHandler(const Webserver::WebRequest& req,
       rapidjson::Document* document);
 
+  /// Upon return, 'document' will contain the query profile as a JSON object in
+  /// 'contents'.
+  void QueryProfileJsonHandler(const Webserver::WebRequest& req,
+      rapidjson::Document* document);
+
   /// Produces a list of inflight query IDs printed as text in 'contents'.
   void InflightQueryIdsHandler(const Webserver::WebRequest& req,
       rapidjson::Document* document);
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 511e3c5..d0966ba 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -40,6 +40,7 @@
 #include <rapidjson/rapidjson.h>
 #include <rapidjson/stringbuffer.h>
 #include <rapidjson/writer.h>
+#include <rapidjson/error/en.h>
 #include <sys/socket.h>
 #include <sys/types.h>
 
@@ -794,7 +795,7 @@ Status ImpalaServer::InitProfileLogging() {
 
 Status ImpalaServer::GetRuntimeProfileOutput(const TUniqueId& query_id,
     const string& user, TRuntimeProfileFormat::type format, stringstream* output,
-    TRuntimeProfileTree* thrift_output) {
+    TRuntimeProfileTree* thrift_output, Document* json_output) {
   DCHECK(output != nullptr);
   // Search for the query id in the active query map
   {
@@ -814,8 +815,10 @@ Status ImpalaServer::GetRuntimeProfileOutput(const TUniqueId& query_id,
         RETURN_IF_ERROR(request_state->profile()->SerializeToArchiveString(output));
       } else if (format == TRuntimeProfileFormat::THRIFT) {
         request_state->profile()->ToThrift(thrift_output);
+      } else if (format == TRuntimeProfileFormat::JSON) {
+        request_state->profile()->ToJson(json_output);
       } else {
-        DCHECK(format == TRuntimeProfileFormat::STRING);
+        DCHECK_EQ(format, TRuntimeProfileFormat::STRING);
         request_state->profile()->PrettyPrint(output);
       }
       return Status::OK();
@@ -840,8 +843,19 @@ Status ImpalaServer::GetRuntimeProfileOutput(const TUniqueId& query_id,
     } else if (format == TRuntimeProfileFormat::THRIFT) {
       RETURN_IF_ERROR(RuntimeProfile::DeserializeFromArchiveString(
           query_record->second->encoded_profile_str, thrift_output));
+    } else if (format == TRuntimeProfileFormat::JSON) {
+      ParseResult parse_ok = json_output->Parse(
+          query_record->second->json_profile_str.c_str());
+      // When there is an error, the json_output will stay unchanged
+      // based on rapidjson parse API
+      if (!parse_ok){
+        string err = strings::Substitute("JSON parse error: $0 (Offset: $1)",
+            GetParseError_En(parse_ok.Code()), parse_ok.Offset());
+        VLOG(1) << err;
+        return Status::Expected(err);
+      }
     } else {
-      DCHECK(format == TRuntimeProfileFormat::STRING);
+      DCHECK_EQ(format, TRuntimeProfileFormat::STRING);
       (*output) << query_record->second->profile_str;
     }
   }
@@ -2012,6 +2026,15 @@ ImpalaServer::QueryStateRecord::QueryStateRecord(const ClientRequestState& reque
     stringstream ss;
     request_state.profile()->PrettyPrint(&ss);
     profile_str = ss.str();
+
+    Document json_profile(rapidjson::kObjectType);
+    request_state.profile()->ToJson(&json_profile);
+
+    StringBuffer buffer;
+    Writer<StringBuffer> writer(buffer);
+    json_profile.Accept(writer);
+    json_profile_str = buffer.GetString();
+
     if (encoded_profile.empty()) {
       Status status =
           request_state.profile()->SerializeToArchiveString(&encoded_profile_str);
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index a052b5a..4717225 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -706,14 +706,16 @@ class ImpalaServer : public ImpalaServiceIf,
   /// found, otherwise a Status object with an error message will be returned. The
   /// output stream will not be modified on error.
   /// On success, if 'format' is BASE64 or STRING then 'output' will be set, or if
-  /// 'format' is THRIFT then 'thrift_output' will be set.
+  /// 'format' is THRIFT then 'thrift_output' will be set. If 'format' is JSON
+  /// then 'json_output' will be set.
   /// If the user asking for this profile is the same user that runs the query
   /// and that user has access to the runtime profile, the profile is written to
   /// the output. Otherwise, nothing is written to output and an error code is
   /// returned to indicate an authorization error.
   Status GetRuntimeProfileOutput(const TUniqueId& query_id, const std::string& user,
       TRuntimeProfileFormat::type format, std::stringstream* output,
-      TRuntimeProfileTree* thrift_output) WARN_UNUSED_RESULT;
+      TRuntimeProfileTree* thrift_output,
+      rapidjson::Document* json_output) WARN_UNUSED_RESULT;
 
   /// Returns the exec summary for this query if the user asking for the exec
   /// summary is the same user that run the query and that user has access to the full
@@ -803,6 +805,9 @@ class ImpalaServer : public ImpalaServiceIf,
     /// Base64 encoded runtime profile
     std::string encoded_profile_str;
 
+    /// JSON based runtime profile
+    std::string json_profile_str;
+
     /// Query id
     TUniqueId id;
 
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index 1e3c0bb..6ad6059 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -190,7 +190,7 @@ ADD_UNIFIED_BE_LSAN_TEST(redactor-config-parser-test ParserTest.*)
 ADD_UNIFIED_BE_LSAN_TEST(redactor-test "RedactorTest.*")
 ADD_UNIFIED_BE_LSAN_TEST(redactor-unconfigured-test "RedactorUnconfigTest.*")
 ADD_UNIFIED_BE_LSAN_TEST(rle-test "BitArray.*:RleTest.*")
-ADD_UNIFIED_BE_LSAN_TEST(runtime-profile-test "CountersTest.*:TimerCounterTest.*:TimeSeriesCounterTest.*:VariousNumbers/TimeSeriesCounterResampleTest.*:ToThrift.*")
+ADD_UNIFIED_BE_LSAN_TEST(runtime-profile-test "CountersTest.*:TimerCounterTest.*:TimeSeriesCounterTest.*:VariousNumbers/TimeSeriesCounterResampleTest.*:ToThrift.*:ToJson.*")
 ADD_UNIFIED_BE_LSAN_TEST(string-parser-test "StringToInt.*:StringToIntWithBase.*:StringToFloat.*:StringToBool.*:StringToDate.*")
 ADD_UNIFIED_BE_LSAN_TEST(string-util-test "TruncateDownTest.*:TruncateUpTest.*:CommaSeparatedContainsTest.*")
 ADD_UNIFIED_BE_LSAN_TEST(symbols-util-test "SymbolsUtil.*")
diff --git a/be/src/util/runtime-profile-counters.h b/be/src/util/runtime-profile-counters.h
index cb557ce..fe612ce 100644
--- a/be/src/util/runtime-profile-counters.h
+++ b/be/src/util/runtime-profile-counters.h
@@ -99,7 +99,7 @@ class RuntimeProfile::HighWaterMarkCounter : public RuntimeProfile::Counter {
  public:
   HighWaterMarkCounter(TUnit::type unit) : Counter(unit) {}
 
-  virtual void Add(int64_t delta) {
+  void Add(int64_t delta) override {
     int64_t new_val = current_value_.Add(delta);
     UpdateMax(new_val);
   }
@@ -118,11 +118,15 @@ class RuntimeProfile::HighWaterMarkCounter : public RuntimeProfile::Counter {
     }
   }
 
-  virtual void Set(int64_t v) {
+  void Set(int64_t v) override {
     current_value_.Store(v);
     UpdateMax(v);
   }
 
+  string CounterType() const override {
+    return "HighWaterMarkCounter";
+  }
+
   int64_t current_value() const { return current_value_.Load(); }
 
  private:
@@ -150,10 +154,14 @@ class RuntimeProfile::DerivedCounter : public RuntimeProfile::Counter {
     : Counter(unit),
       counter_fn_(counter_fn) {}
 
-  virtual int64_t value() const {
+  int64_t value() const override {
     return counter_fn_();
   }
 
+  string CounterType() const override {
+    return "DerivedCounter";
+  }
+
  private:
   SampleFunction counter_fn_;
 };
@@ -198,11 +206,15 @@ class RuntimeProfile::AveragedCounter : public RuntimeProfile::Counter {
     }
   }
 
+  string CounterType() const override {
+    return "AveragedCounter";
+  }
+
   /// The value for this counter should be updated through UpdateCounter().
   /// Set() and Add() should not be used.
-  virtual void Set(double value) { DCHECK(false); }
-  virtual void Set(int64_t value) { DCHECK(false); }
-  virtual void Add(int64_t delta) { DCHECK(false); }
+  void Set(double value) override { DCHECK(false); }
+  void Set(int64_t value) override { DCHECK(false); }
+  void Add(int64_t delta) override { DCHECK(false); }
 
  private:
   /// Map from counters to their existing values. Modified via UpdateCounter().
@@ -250,15 +262,27 @@ class RuntimeProfile::SummaryStatsCounter : public RuntimeProfile::Counter {
 
   /// The value for this counter should be updated through UpdateCounter() or SetStats().
   /// Set() and Add() should not be used.
-  virtual void Set(double value) { DCHECK(false); }
-  virtual void Set(int64_t value) { DCHECK(false); }
-  virtual void Add(int64_t delta) { DCHECK(false); }
+  void Set(double value) override { DCHECK(false); }
+  void Set(int64_t value) override { DCHECK(false); }
+  void Add(int64_t delta) override { DCHECK(false); }
 
   /// Overwrites the existing counter with 'counter'
   void SetStats(const TSummaryStatsCounter& counter);
 
   void ToThrift(TSummaryStatsCounter* counter, const std::string& name);
 
+  void ToJson(rapidjson::Document& document, rapidjson::Value* val) const override {
+    Counter::ToJson(document, val);
+    val->AddMember("min", min_, document.GetAllocator());
+    val->AddMember("max", max_, document.GetAllocator());
+    val->AddMember("avg", value(), document.GetAllocator());
+    val->AddMember("num_of_samples", total_num_values_, document.GetAllocator());
+  }
+
+  string CounterType() const override {
+    return "SummaryStatsCounter";
+  }
+
  private:
   /// The total number of values seen so far.
   int32_t total_num_values_;
@@ -366,6 +390,17 @@ class RuntimeProfile::EventSequence {
 
   void ToThrift(TEventSequence* seq);
 
+  /// Builds a new Value into 'value', using (if required) the allocator from
+  /// 'document'. Should set the following fields where appropriate:
+  /// {
+  ///   “offset” : xxx,
+  ///   “events”: [{
+  ///       “label”: xxx,
+  ///       “timestamp”: xxx
+  ///   },{...}]
+  /// }
+  void ToJson(rapidjson::Document& document, rapidjson::Value* value);
+
  private:
   /// Sorts events by their timestamp. Caller must hold lock_.
   void SortEvents() {
@@ -410,6 +445,17 @@ class RuntimeProfile::TimeSeriesCounter {
 
   void ToThrift(TTimeSeriesCounter* counter);
 
+  /// Builds a new Value into 'value', using (if required) the allocator from
+  /// 'document'. Should set the following fields where appropriate:
+  /// {
+  ///   “counter_name” : xxx,
+  ///   “unit” : xxx,
+  ///   “num” : xxx,
+  ///   “period” : xxx,
+  ///   “data”: “x,x,x,x”
+  /// }
+  virtual void ToJson(rapidjson::Document& document, rapidjson::Value* val);
+
   /// Adds a sample to the counter. Caller must hold lock_.
   virtual void AddSampleLocked(int64_t value, int ms_elapsed) = 0;
 
@@ -512,7 +558,7 @@ class RuntimeProfile::ConcurrentTimerCounter : public Counter {
  public:
   ConcurrentTimerCounter(TUnit::type unit) : Counter(unit) {}
 
-  virtual int64_t value() const { return csw_.TotalRunningTime(); }
+  int64_t value() const override { return csw_.TotalRunningTime(); }
 
   void Start() { csw_.Start(); }
 
@@ -523,22 +569,26 @@ class RuntimeProfile::ConcurrentTimerCounter : public Counter {
 
   /// The value for this counter should come from internal ConcurrentStopWatch.
   /// Set() and Add() should not be used.
-  virtual void Set(double value) {
+  void Set(double value) override {
     DCHECK(false);
   }
 
-  virtual void Set(int64_t value) {
+  void Set(int64_t value) override {
     DCHECK(false);
   }
 
-  virtual void Set(int value) {
+  void Set(int value) override {
     DCHECK(false);
   }
 
-  virtual void Add(int64_t delta) {
+  void Add(int64_t delta) override {
     DCHECK(false);
   }
 
+  string CounterType() const override {
+    return "ConcurrentTimerCounter";
+  }
+
  private:
   ConcurrentStopWatch csw_;
 };
diff --git a/be/src/util/runtime-profile-test.cc b/be/src/util/runtime-profile-test.cc
index 18e77ab..70d69dc 100644
--- a/be/src/util/runtime-profile-test.cc
+++ b/be/src/util/runtime-profile-test.cc
@@ -84,7 +84,7 @@ TEST(CountersTest, Basic) {
   from_thrift->GetExecSummary(&exec_summary_result);
   EXPECT_EQ(exec_summary_result.status, status);
 
-  // Seralize/deserialize to archive string
+  // Serialize/deserialize to archive string
   string archive_str;
   EXPECT_OK(profile_a->SerializeToArchiveString(&archive_str));
   TRuntimeProfileTree deserialized_thrift_profile;
@@ -1159,5 +1159,173 @@ TEST(ToThrift, NodeMetadataIsSetCorrectly) {
   EXPECT_TRUE(thrift_profile.nodes[0].__isset.node_metadata);
 }
 
+TEST(ToJson, RuntimeProfileToJsonTest) {
+  ObjectPool pool;
+  RuntimeProfile* profile_a = RuntimeProfile::Create(&pool, "ProfileA");
+  RuntimeProfile* profile_a1 = RuntimeProfile::Create(&pool, "ProfileA1");
+  RuntimeProfile* profile_ab = RuntimeProfile::Create(&pool, "ProfileAb");
+  RuntimeProfile::Counter* counter_a;
+
+  // Initialize for further validation
+  profile_a->AddChild(profile_a1);
+  profile_a->AddChild(profile_ab);
+  profile_a->AddInfoString("Key", "Value");
+
+  counter_a = profile_a->AddCounter("A", TUnit::UNIT);
+  counter_a->Set(1);
+  RuntimeProfile::HighWaterMarkCounter* high_water_counter =
+      profile_a->AddHighWaterMarkCounter("high_water_counter", TUnit::BYTES);
+  high_water_counter->Set(10);
+  high_water_counter->Add(10);
+  high_water_counter->Set(10);
+
+  RuntimeProfile::SummaryStatsCounter* summary_stats_counter =
+      profile_a->AddSummaryStatsCounter("summary_stats_counter", TUnit::TIME_NS);
+  summary_stats_counter->UpdateCounter(10);
+  summary_stats_counter->UpdateCounter(20);
+
+  // Serialize to json
+  rapidjson::Document doc(rapidjson::kObjectType);
+  profile_a->ToJson(&doc);
+  rapidjson::Value& content = doc["contents"];
+
+  // Check profile correct
+  EXPECT_EQ("ProfileA", content["profile_name"]);
+  EXPECT_EQ("ProfileA1", content["child_profiles"][0]["profile_name"]);
+  EXPECT_EQ("ProfileAb", content["child_profiles"][1]["profile_name"]);
+
+  // Check Info String correct
+  EXPECT_EQ(1, content["info_strings"].Size());
+  EXPECT_EQ("Key", content["info_strings"][0]["key"]);
+  EXPECT_EQ("Value", content["info_strings"][0]["value"]);
+
+  // Check counter value matches
+  EXPECT_EQ(2, content["counters"].Size());
+  for (auto& itr : content["counters"].GetArray()) {
+    // check normal Counter
+    if (itr["counter_name"] == "A") {
+      EXPECT_EQ(1, itr["value"].GetInt());
+      EXPECT_EQ("UNIT", itr["unit"]);
+      EXPECT_EQ("Counter", itr["kind"]);
+    }// check HighWaterMarkCounter
+    else if (itr["counter_name"] == "high_water_counter") {
+      EXPECT_EQ(20, itr["value"].GetInt());
+      EXPECT_EQ("BYTES", itr["unit"]);
+      EXPECT_EQ("HighWaterMarkCounter", itr["kind"]);
+    } else {
+      DCHECK(false);
+    }
+  }
+
+  // Check SummaryStatsCounter
+  EXPECT_EQ(1, content["summary_stats_counters"].Size());
+  for (auto& itr : content["summary_stats_counters"].GetArray()) {
+    if (itr["counter_name"] == "summary_stats_counter") {
+      EXPECT_EQ(10, itr["min"].GetInt());
+      EXPECT_EQ(20, itr["max"].GetInt());
+      EXPECT_EQ(15, itr["avg"].GetInt());
+      EXPECT_EQ(2, itr["num_of_samples"].GetInt());
+      EXPECT_EQ("TIME_NS", itr["unit"]);
+      EXPECT_TRUE(!itr.HasMember("kind"));
+    }
+  }
+}
+
+// Test when some fields are not set. ToJson will not add them as a member
+TEST(ToJson, EmptyTest) {
+  ObjectPool pool;
+  RuntimeProfile* profile = RuntimeProfile::Create(&pool, "Profile");
+
+  // Serialize to json
+  rapidjson::Document doc(rapidjson::kObjectType);
+  profile->ToJson(&doc);
+  rapidjson::Value& content = doc["contents"];
+
+  EXPECT_EQ("Profile", content["profile_name"]);
+  EXPECT_TRUE(content.HasMember("num_children"));
+
+  // Empty profile should not have following members
+  EXPECT_TRUE(!content.HasMember("info_strings"));
+  EXPECT_TRUE(!content.HasMember("event_sequences"));
+  EXPECT_TRUE(!content.HasMember("counters"));
+  EXPECT_TRUE(!content.HasMember("summary_stats_counters"));
+  EXPECT_TRUE(!content.HasMember("time_series_counters"));
+  EXPECT_TRUE(!content.HasMember("child_profiles"));
+
+}
+
+TEST(ToJson, EventSequenceToJsonTest) {
+  ObjectPool pool;
+  RuntimeProfile* profile = RuntimeProfile::Create(&pool, "Profile");
+  RuntimeProfile::EventSequence* seq = profile->AddEventSequence("event sequence");
+  seq->MarkEvent("aaaa");
+  seq->MarkEvent("bbbb");
+  seq->MarkEvent("cccc");
+
+  // Serialize to json
+  rapidjson::Document doc(rapidjson::kObjectType);
+  rapidjson::Value event_sequence_json(rapidjson::kObjectType);
+  seq->ToJson(doc, &event_sequence_json);
+
+  EXPECT_EQ(0, event_sequence_json["offset"].GetInt());
+
+  uint64_t last_timestamp = 0;
+  string last_string = "";
+  for (auto& itr : event_sequence_json["events"].GetArray()) {
+    EXPECT_TRUE(itr["timestamp"].GetInt() >= last_timestamp);
+    last_timestamp = itr["timestamp"].GetInt();
+    string label = string(itr["label"].GetString());
+    EXPECT_TRUE(label > last_string);
+    last_string = label;
+  }
+}
+
+TEST(ToJson, TimeSeriesCounterToJsonTest) {
+  ObjectPool pool;
+  RuntimeProfile* profile = RuntimeProfile::Create(&pool, "Profile");
+
+  // 1. TimeSeriesCounter should be empty
+  rapidjson::Document doc(rapidjson::kObjectType);
+  profile->ToJson(&doc);
+  EXPECT_TRUE(!doc["contents"].HasMember("time_series_counters"));
+
+  // 2. Check Serialize to json
+  const int test_period = FLAGS_periodic_counter_update_period_ms;
+
+  // Add a counter with a sample function that counts up, starting from 0.
+  int value = 0;
+  auto sample_fn = [&value]() { return value++; };
+
+  // We increase the value of this flag to allow the counter to store enough samples.
+  FLAGS_status_report_interval_ms = 50000;
+  RuntimeProfile::TimeSeriesCounter* counter =
+      profile->AddChunkedTimeSeriesCounter("TimeSeriesCounter", TUnit::UNIT, sample_fn);
+  RuntimeProfile::TimeSeriesCounter* counter2 =
+      profile->AddSamplingTimeSeriesCounter("SamplingCounter", TUnit::UNIT, sample_fn);
+
+  // Stop counter updates from interfering with the rest of the test.
+  StopAndClearCounter(profile, counter);
+
+  // Reset value after previous values have been retrieved.
+  value = 0;
+  for (int i = 0; i < 64; ++i) counter->AddSample(test_period);
+
+  value = 0;
+  for (int i = 0; i < 80; ++i) counter2->AddSample(test_period);
+
+  profile->ToJson(&doc);
+  EXPECT_STR_CONTAINS(
+      doc["contents"]["time_series_counters"][1]["data"].GetString(), "0,1,2,3,4");
+
+  EXPECT_STR_CONTAINS(
+      doc["contents"]["time_series_counters"][1]["data"].GetString(), "60,61,62,63");
+
+  EXPECT_STR_CONTAINS(
+      doc["contents"]["time_series_counters"][0]["data"].GetString(), "0,2,4,6");
+
+  EXPECT_STR_CONTAINS(
+      doc["contents"]["time_series_counters"][0]["data"].GetString(), "72,74,76,78");
+}
+
 } // namespace impala
 
diff --git a/be/src/util/runtime-profile.cc b/be/src/util/runtime-profile.cc
index 78ae8d1..fd68cfe 100644
--- a/be/src/util/runtime-profile.cc
+++ b/be/src/util/runtime-profile.cc
@@ -46,6 +46,8 @@
 DECLARE_int32(status_report_interval_ms);
 DECLARE_int32(periodic_counter_update_period_ms);
 
+using namespace rapidjson;
+
 namespace impala {
 
 // Thread counters name
@@ -698,6 +700,179 @@ RuntimeProfile::EventSequence* RuntimeProfile::GetEventSequence(const string& na
   return it->second;
 }
 
+void RuntimeProfile::ToJson(Document* d) const{
+  // queryObj that stores all JSON format profile information
+  Value queryObj(kObjectType);
+  RuntimeProfile::ToJsonHelper(&queryObj, d);
+  d->RemoveMember("contents");
+  d->AddMember("contents", queryObj, d->GetAllocator());
+}
+
+void RuntimeProfile::ToJsonCounters(Value* parent, Document* d,
+    const string& counter_name, const CounterMap& counter_map,
+    const ChildCounterMap& child_counter_map) const{
+  auto& allocator = d->GetAllocator();
+  ChildCounterMap::const_iterator itr = child_counter_map.find(counter_name);
+  if (itr != child_counter_map.end()) {
+    const set<string>& child_counters = itr->second;
+    for (const string& child_counter: child_counters) {
+      CounterMap::const_iterator iter = counter_map.find(child_counter);
+      if (iter == counter_map.end()) continue;
+
+      Value counter(kObjectType);
+      iter->second->ToJson(*d, &counter);
+      counter.AddMember("counter_name", StringRef(child_counter.c_str()), allocator);
+
+      Value child_counters_json(kArrayType);
+      RuntimeProfile::ToJsonCounters(&child_counters_json, d,
+          child_counter, counter_map,child_counter_map);
+      if (!child_counters_json.Empty()){
+        counter.AddMember("child_counters", child_counters_json, allocator);
+      }
+      parent->PushBack(counter, allocator);
+    }
+  }
+}
+
+void RuntimeProfile::ToJsonHelper(Value* parent, Document* d) const{
+  Document::AllocatorType& allocator = d->GetAllocator();
+  // Create copy of counter_map_ and child_counter_map_ so we don't need to hold lock
+  // while we call value() on the counters (some of those might be DerivedCounters).
+  CounterMap counter_map;
+  ChildCounterMap child_counter_map;
+  {
+    lock_guard<SpinLock> l(counter_map_lock_);
+    counter_map = counter_map_;
+    child_counter_map = child_counter_map_;
+  }
+
+  // 1. Name
+  Value name(name_.c_str(), allocator);
+  parent->AddMember("profile_name", name, allocator);
+
+  // 2. Num_children
+  parent->AddMember("num_children", children_.size(), allocator);
+
+  // 3. Metadata
+  // Set the required metadata field to the plan node ID for compatibility with any tools
+  // that rely on the plan node id being set there.
+  // Legacy field. May contain the node ID for plan nodes.
+  // Replaced by node_metadata, which contains richer metadata.
+  parent->AddMember("metadata",
+      metadata_.__isset.plan_node_id ? metadata_.plan_node_id : -1, allocator);
+  // requires exactly one field of a union to be set so we only mark node_metadata
+  // as set if that is the case.
+  if (metadata_.__isset.plan_node_id || metadata_.__isset.data_sink_id){
+    Value node_metadata_json(kObjectType);
+    if (metadata_.__isset.plan_node_id){
+      node_metadata_json.AddMember("plan_node_id", metadata_.plan_node_id, allocator);
+    }
+    if (metadata_.__isset.data_sink_id){
+      node_metadata_json.AddMember("data_sink_id", metadata_.data_sink_id, allocator);
+    }
+    parent->AddMember("node_metadata", node_metadata_json, allocator);
+  }
+
+  // 4. Info_strings
+  {
+    lock_guard<SpinLock> l(info_strings_lock_);
+    if (!info_strings_.empty()) {
+      Value info_strings_json(kArrayType);
+      for (const string& key : info_strings_display_order_) {
+        Value info_string_json(kObjectType);
+        Value key_json(key.c_str(), allocator);
+        auto value_itr = info_strings_.find(key);
+        DCHECK(value_itr != info_strings_.end());
+        Value value_json(value_itr->second.c_str(), allocator);
+        info_string_json.AddMember("key", key_json, allocator);
+        info_string_json.AddMember("value", value_json, allocator);
+        info_strings_json.PushBack(info_string_json, allocator);
+      }
+      parent->AddMember("info_strings", info_strings_json, allocator);
+    }
+  }
+
+  // 5. Events
+  {
+    lock_guard<SpinLock> l(event_sequence_lock_);
+    if (!event_sequence_map_.empty()) {
+      Value event_sequences_json(kArrayType);
+      for (EventSequenceMap::const_iterator it = event_sequence_map_.begin();
+           it != event_sequence_map_.end(); ++it) {
+        Value event_sequence_json(kObjectType);
+        it->second->ToJson(*d, &event_sequence_json);
+        event_sequences_json.PushBack(event_sequence_json, allocator);
+      }
+      parent->AddMember("event_sequences", event_sequences_json, allocator);
+    }
+  }
+
+
+  // 6. Counters
+  Value counters(kArrayType);
+  RuntimeProfile::ToJsonCounters(&counters , d, "", counter_map, child_counter_map);
+  if (!counters.Empty()) {
+    parent->AddMember("counters", counters, allocator);
+  }
+
+  // 7. SummaryStatsCounter
+  {
+    lock_guard<SpinLock> l(summary_stats_map_lock_);
+    if (!summary_stats_map_.empty()) {
+      Value summary_stats_counters_json(kArrayType);
+      for (const SummaryStatsCounterMap::value_type& v : summary_stats_map_) {
+        Value summary_stats_counter(kObjectType);
+        Value summary_name(v.first.c_str(), allocator);
+        v.second->ToJson(*d, &summary_stats_counter);
+        // Remove Kind here because it would be redundant information for users
+        summary_stats_counter.RemoveMember("kind");
+        summary_stats_counter.AddMember("counter_name", summary_name, allocator);
+        summary_stats_counters_json.PushBack(summary_stats_counter, allocator);
+      }
+      parent->AddMember(
+          "summary_stats_counters", summary_stats_counters_json, allocator);
+    }
+  }
+
+  // 8. Time_series_counter_map
+  {
+    // Print all time series counters as following:
+    //    - <Name> (<period>): <val1>, <val2>, <etc>
+    lock_guard<SpinLock> l(counter_map_lock_);
+    if (!time_series_counter_map_.empty()) {
+      Value time_series_counters_json(kArrayType);
+      for (const TimeSeriesCounterMap::value_type& v : time_series_counter_map_) {
+        TimeSeriesCounter* counter = v.second;
+        Value time_series_json(kObjectType);
+        counter->ToJson(*d, &time_series_json);
+        time_series_counters_json.PushBack(time_series_json, allocator);
+      }
+      parent->AddMember("time_series_counters", time_series_counters_json, allocator);
+    }
+  }
+
+  // 9. Children Runtime Profiles
+  //
+  // Create copy of children_ so we don't need to hold lock while we call
+  // ToJsonHelper() on the children.
+  ChildVector children;
+  {
+    lock_guard<SpinLock> l(children_lock_);
+    children = children_;
+  }
+
+  if (!children.empty()) {
+    Value child_profiles(kArrayType);
+    for (int i = 0; i < children.size(); ++i) {
+      RuntimeProfile* profile = children[i].first;
+      Value child_profile(kObjectType);
+      profile->ToJsonHelper(&child_profile, d);
+      child_profiles.PushBack(child_profile, allocator);
+    }
+    parent->AddMember("child_profiles", child_profiles, allocator);
+  }
+}
+
 // Print the profile:
 //  1. Profile Name
 //  2. Info Strings
@@ -1403,4 +1578,67 @@ int32_t RuntimeProfile::SummaryStatsCounter::TotalNumValues() {
   return total_num_values_;
 }
 
+void RuntimeProfile::Counter::ToJson(Document& document, Value* val) const {
+  Value counter_json(kObjectType);
+  counter_json.AddMember("value", value(), document.GetAllocator());
+  auto unit_itr = _TUnit_VALUES_TO_NAMES.find(unit_);
+  DCHECK(unit_itr != _TUnit_VALUES_TO_NAMES.end());
+  Value unit_json(unit_itr->second, document.GetAllocator());
+  counter_json.AddMember("unit", unit_json, document.GetAllocator());
+  Value kind_json(CounterType().c_str(), document.GetAllocator());
+  counter_json.AddMember("kind", kind_json, document.GetAllocator());
+  *val = counter_json;
+}
+
+void RuntimeProfile::TimeSeriesCounter::ToJson(Document& document, Value* val) {
+  lock_guard<SpinLock> lock(lock_);
+  Value counter_json(kObjectType);
+  counter_json.AddMember("counter_name",
+      StringRef(name_.c_str()), document.GetAllocator());
+  auto unit_itr = _TUnit_VALUES_TO_NAMES.find(unit_);
+  DCHECK(unit_itr != _TUnit_VALUES_TO_NAMES.end());
+  Value unit_json(unit_itr->second, document.GetAllocator());
+  counter_json.AddMember("unit", unit_json, document.GetAllocator());
+
+  int num, period;
+  const int64_t* samples = GetSamplesLocked(&num, &period);
+
+  counter_json.AddMember("num", num, document.GetAllocator());
+  counter_json.AddMember("period", period, document.GetAllocator());
+  stringstream stream;
+  // Clamp number of printed values at 64, the maximum number of values in the
+  // SamplingTimeSeriesCounter.
+  int step = 1 + (num - 1) / 64;
+  period *= step;
+
+  for (int i = 0; i < num; i += step) {
+    stream << samples[i];
+    if (i + step < num) stream << ",";
+  }
+
+  Value samples_data_json(stream.str().c_str(), document.GetAllocator());
+  counter_json.AddMember("data", samples_data_json, document.GetAllocator());
+  *val = counter_json;
+}
+
+void RuntimeProfile::EventSequence::ToJson(Document& document, Value* value) {
+  boost::lock_guard<SpinLock> event_lock(lock_);
+  SortEvents();
+
+  Value event_sequence_json(kObjectType);
+  event_sequence_json.AddMember("offset", offset_, document.GetAllocator());
+
+  Value events_json(kArrayType);
+
+  for (const Event& ev: events_) {
+    Value event_json(kObjectType);
+    event_json.AddMember("label", StringRef(ev.first.c_str()), document.GetAllocator());
+    event_json.AddMember("timestamp", ev.second, document.GetAllocator());
+    events_json.PushBack(event_json, document.GetAllocator());
+  }
+
+  event_sequence_json.AddMember("events", events_json, document.GetAllocator());
+  *value = event_sequence_json;
+}
+
 }
diff --git a/be/src/util/runtime-profile.h b/be/src/util/runtime-profile.h
index 5c79669..65e97d7 100644
--- a/be/src/util/runtime-profile.h
+++ b/be/src/util/runtime-profile.h
@@ -22,7 +22,7 @@
 #include <boost/function.hpp>
 #include <boost/thread/lock_guard.hpp>
 #include <iostream>
-
+#include <rapidjson/document.h>
 #include "common/atomic.h"
 #include "common/status.h"
 #include "util/spinlock.h"
@@ -118,6 +118,16 @@ class RuntimeProfile { // NOLINT: This struct is not packed, but there are not s
       return *reinterpret_cast<const double*>(&v);
     }
 
+    /// Builds a new Value into 'val', using (if required) the allocator from
+    /// 'document'. Should set the following fields where appropriate:
+    /// counter_name, value, kind, unit
+    virtual void ToJson(rapidjson::Document& document, rapidjson::Value* val) const;
+
+    ///  Return the name of the counter type
+    virtual string CounterType() const {
+      return "Counter";
+    }
+
     TUnit::type unit() const { return unit_; }
 
    protected:
@@ -315,6 +325,10 @@ class RuntimeProfile { // NOLINT: This struct is not packed, but there are not s
   void ToThrift(TRuntimeProfileTree* tree) const;
   void ToThrift(std::vector<TRuntimeProfileNode>* nodes) const;
 
+  /// Store profile into JSON format into a document
+  void ToJsonHelper(rapidjson::Value* parent, rapidjson::Document* d) const;
+  void ToJson(rapidjson::Document* d) const;
+
   /// Serializes the runtime profile to a string.  This first serializes the
   /// object using thrift compact binary format, then gzip compresses it and
   /// finally encodes it as base64.  This is not a lightweight operation and
@@ -599,6 +613,17 @@ class RuntimeProfile { // NOLINT: This struct is not packed, but there are not s
   static void PrintChildCounters(const std::string& prefix,
       const std::string& counter_name, const CounterMap& counter_map,
       const ChildCounterMap& child_counter_map, std::ostream* s);
+
+  /// Add all the counters of this instance into the given parent node in JSON format
+  /// Args:
+  ///   parent: the root node to add all the counters
+  ///   d: document of this json, could be used to get Allocator
+  ///   counter_name: this will be used to find its child counters in child_counter_map
+  ///   counter_map: A map of counters name to counter
+  ///   child_counter_map: A map of counter to its child counters
+  void ToJsonCounters(rapidjson::Value* parent, rapidjson::Document* d,
+      const string& counter_name, const CounterMap& counter_map,
+      const ChildCounterMap& child_counter_map) const;
 };
 
 }
diff --git a/common/thrift/ImpalaService.thrift b/common/thrift/ImpalaService.thrift
index 8d534f0..ae03051 100644
--- a/common/thrift/ImpalaService.thrift
+++ b/common/thrift/ImpalaService.thrift
@@ -549,7 +549,8 @@ struct TGetRuntimeProfileReq {
 struct TGetRuntimeProfileResp {
   1: required TCLIService.TStatus status
 
-  // Will be set on success if TGetRuntimeProfileReq.format was STRING or BASE64.
+  // Will be set on success if TGetRuntimeProfileReq.format
+  // was STRING, BASE64 or JSON.
   2: optional string profile
 
   // Will be set on success if TGetRuntimeProfileReq.format was THRIFT.
diff --git a/common/thrift/RuntimeProfile.thrift b/common/thrift/RuntimeProfile.thrift
index 0d944c8..95d370a 100644
--- a/common/thrift/RuntimeProfile.thrift
+++ b/common/thrift/RuntimeProfile.thrift
@@ -42,6 +42,9 @@ enum TRuntimeProfileFormat {
 
   // TRuntimeProfileTree.
   THRIFT = 2
+
+  // JSON profile
+  JSON = 3
 }
 
 // Counter data
diff --git a/tests/hs2/test_hs2.py b/tests/hs2/test_hs2.py
index 223e29c..14d1b19 100644
--- a/tests/hs2/test_hs2.py
+++ b/tests/hs2/test_hs2.py
@@ -652,6 +652,29 @@ class TestHS2(HS2TestSuite):
     assert statement in get_profile_resp.profile
     assert "Query State: FINISHED" in get_profile_resp.profile, get_profile_resp.profile
 
+    # Check get JSON format profile
+    get_profile_req.format = 3  # json format
+    get_profile_resp = self.hs2_client.GetRuntimeProfile(get_profile_req)
+    TestHS2.check_response(get_profile_resp)
+
+    try:
+      json_res = json.loads(get_profile_resp.profile)
+    except ValueError:
+      assert False, "Download JSON format query profile cannot be parsed." \
+          "Response text:{0}".format(get_profile_resp.profile)
+
+    # The query statement should exist in json info_strings
+    if ("child_profiles" not in json_res["contents"]) or \
+        ("info_strings" not in json_res["contents"]["child_profiles"][0]):
+      assert False, "JSON content is invalid. Content: {0}"\
+        .format(get_profile_resp.profile)
+
+    for info_string in json_res["contents"]["child_profiles"][0]["info_strings"]:
+      if info_string["key"] == "Sql Statement":
+        assert statement in info_string["value"], \
+          "JSON content is invalid. Content: {0}".format(get_profile_resp.profile)
+        break
+
   @needs_session(conf_overlay={"use:database": "functional"})
   def test_change_default_database(self):
     statement = "SELECT 1 FROM alltypes LIMIT 1"
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
index aee8238..e5bee26 100644
--- a/tests/webserver/test_web_pages.py
+++ b/tests/webserver/test_web_pages.py
@@ -553,27 +553,47 @@ class TestWebPage(ImpalaTestSuite):
     assert not backend_row['is_quiescing']
     assert len(backend_row['admit_mem_limit']) > 0
 
-  def test_download_text_profile(self):
+  def test_download_profile(self):
     """Test download text profile for a query"""
     query = "select count(*) from functional.alltypes"
     query_id = self.client.execute(query).query_id
     profile_page_url = "{0}query_profile?query_id={1}".format(
         self.ROOT_URL, query_id)
-    # Check the text download tag is there.
-    responses = self.get_and_check_status(
-        profile_page_url, "Download Text Profile",
+    # Check the download tag is there.
+    for profile_format in ["Text", "Json"]:
+      responses = self.get_and_check_status(
+        profile_page_url, profile_format,
         ports_to_test=self.IMPALAD_TEST_PORT)
-    assert len(responses) == 1
-    download_link = "query_profile_plain_text?query_id={0}".format(
-        query_id)
-    assert download_link in responses[0].text
-    # Get the response from download link and validate it by checking
-    # the query is in the file.
-    responses = self.get_and_check_status(
-        self.ROOT_URL + download_link, query, self.IMPALAD_TEST_PORT)
-    # Check the query id is in the content of the reponse.
-    assert len(responses) == 1
-    assert query_id in responses[0].text
+
+      assert len(responses) == 1
+
+      if profile_format == 'Text':
+        download_link = "query_profile_plain_text?query_id={0}".format(query_id)
+        assert download_link in responses[0].text
+        # Get the response from download link and validate it by checking
+        # the query is in the file.
+        responses = self.get_and_check_status(
+            self.ROOT_URL + download_link, query, self.IMPALAD_TEST_PORT)
+        # Check the query id is in the content of the reponse.
+        assert len(responses) == 1
+        assert query_id in responses[0].text
+      elif profile_format == 'Json':
+        download_link = "query_profile_json?query_id={0}".format(query_id)
+        assert download_link in responses[0].text
+        # Get the response from download link and validate it by checking
+        # the query is in the file.
+        responses = self.get_and_check_status(
+            self.ROOT_URL + download_link, query, self.IMPALAD_TEST_PORT)
+
+        assert len(responses) == 1
+        # Check the return content is valid json
+        try:
+          json_res = json.loads(responses[0].text)
+        except ValueError:
+          assert False, "Downloaded JSON format query profile cannot be parsed. " \
+              "Json profile:{0}".format(responses[0].text)
+        # Find the query id in json
+        assert query_id in json_res["contents"]["profile_name"], json_res
 
   def test_prometheus_metrics(self):
     """Test to check prometheus metrics"""
diff --git a/www/query_profile.tmpl b/www/query_profile.tmpl
index 4f93fdb..aabe685 100644
--- a/www/query_profile.tmpl
+++ b/www/query_profile.tmpl
@@ -24,19 +24,19 @@ under the License.
 
 {{> www/query_detail_tabs.tmpl }}
 
-<h4>
-  <a href="/query_profile_encoded?query_id={{query_id}}"
-      download="thrift_profile_{{query_id}}">
-    Download Thrift Profile
-  </a>
-</h4>
-
-<h4>
-  <a href="/query_profile_plain_text?query_id={{query_id}}"
-      download="profile_{{query_id}}">
-   Download Text Profile
-  </a>
-</h4>
+<div>
+    <h4>Download Profile (Available Formats):
+        <a style="font-size:16px;" class="btn btn-primary"
+            href="/query_profile_encoded?query_id={{query_id}}"
+            download="thrift_profile_{{query_id}}">Thrift</a>
+        <a style="font-size:16px;" class="btn btn-primary"
+            href="/query_profile_json?query_id={{query_id}}"
+            download="json_profile_{{query_id}}">Json</a>
+        <a style="font-size:16px;" class="btn btn-primary"
+            href="/query_profile_plain_text?query_id={{query_id}}"
+            download="profile_{{query_id}}">Text</a>
+    </h4>
+</div>
 
 <pre>{{profile}}</pre>