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/30 03:54:39 UTC

[impala] branch master updated: IMPALA-8897: Update debug webui to help with Apache Knox rewrite rules

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


The following commit(s) were added to refs/heads/master by this push:
     new b7dfc18  IMPALA-8897: Update debug webui to help with Apache Knox rewrite rules
b7dfc18 is described below

commit b7dfc18c59e831fa265d14ef4f7d26e33120b67f
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
AuthorDate: Thu Aug 22 11:16:22 2019 -0700

    IMPALA-8897: Update debug webui to help with Apache Knox rewrite rules
    
    This patch adds the 'host:port' to all links on the webserver. This
    will facilitate proxying connections to the debug webui through Knox
    by allowing us to create rewrite rules that do the transform:
    
    <a href="scheme://host:port/path">...</a>
    =>
    <a href="<knox-host>/topology/impalaui/path?scheme-scheme&host=host&port=port">...</a>
    
    which allows us to have a single IMPALAUI Knox service that can proxy
    connections to any impalad/statestored/catalogd webui in a cluster.
    
    Note that this works because currently all of the links on Impala's
    webui are within the same webserver (it would also be possible to add
    links to other Impala daemon webuis within a cluster, eg. if we wanted
    to add webui links on the /backends page). If we ever need to add
    links to external pages, the Knox service definition will likely need
    to be modified.
    
    This patch also adds hidden fields to all forms for the scheme, host,
    and port value, so that GET requests from forms will result in the
    same form as the transformed url shown above.
    
    Testing:
    - Ran the webserver and manually clicked around on a bunch of links to
      ensure everything works as expected.
    - Ran in a cluster and verified the new Knox service defintion works
      as intended with this change.
    - Added a test that uses a regex to check for template files that
      don't conform to the requirements.
    
    Change-Id: If1195709a0f21f39d9a1e484880a0c46c9967ed2
    Reviewed-on: http://gerrit.cloudera.org:8080/14151
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/service/impala-beeswax-server.cc |  2 +-
 be/src/service/impala-hs2-server.cc     |  2 +-
 be/src/util/webserver.cc                | 38 +++++++++++++++++++++++----------
 be/src/util/webserver.h                 | 11 +++++++++-
 bin/rat_exclude_files.txt               |  1 +
 tests/common/file_utils.py              | 15 +++++++------
 tests/custom_cluster/test_redaction.py  |  4 ++--
 tests/webserver/test_web_pages.py       | 16 ++++++++++++++
 www/admission_controller.tmpl           | 10 ++++-----
 www/catalog.tmpl                        | 18 ++++++++--------
 www/common-header.tmpl                  | 16 +++++++-------
 www/form-hidden-inputs.tmpl             |  3 +++
 www/log_level.tmpl                      | 10 ++++-----
 www/queries.tmpl                        | 10 ++++-----
 www/query_detail_tabs.tmpl              | 16 +++++++-------
 www/query_profile.tmpl                  |  6 +++---
 www/sessions.tmpl                       |  2 +-
 www/threadz.tmpl                        |  6 +++---
 www/threadz_tabs.tmpl                   |  4 ++--
 19 files changed, 119 insertions(+), 71 deletions(-)

diff --git a/be/src/service/impala-beeswax-server.cc b/be/src/service/impala-beeswax-server.cc
index ab0d895..899900b 100644
--- a/be/src/service/impala-beeswax-server.cc
+++ b/be/src/service/impala-beeswax-server.cc
@@ -454,7 +454,7 @@ void ImpalaServer::PingImpalaService(TPingImpalaServiceResp& return_val) {
 
   VLOG_RPC << "PingImpalaService()";
   return_val.version = GetVersionString(true);
-  return_val.webserver_address = ExecEnv::GetInstance()->webserver()->Url();
+  return_val.webserver_address = ExecEnv::GetInstance()->webserver()->url();
   VLOG_RPC << "PingImpalaService(): return_val=" << ThriftDebugString(return_val);
 }
 
diff --git a/be/src/service/impala-hs2-server.cc b/be/src/service/impala-hs2-server.cc
index 697ef67..68848f9 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -1053,7 +1053,7 @@ void ImpalaServer::PingImpalaHS2Service(TPingImpalaHS2ServiceResp& return_val,
   }
 
   return_val.__set_version(GetVersionString(true));
-  return_val.__set_webserver_address(ExecEnv::GetInstance()->webserver()->Url());
+  return_val.__set_webserver_address(ExecEnv::GetInstance()->webserver()->url());
   VLOG_RPC << "PingImpalaHS2Service(): return_val=" << ThriftDebugString(return_val);
 }
 }
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index d5f299a..4873447 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -241,6 +241,7 @@ Webserver::Webserver()
   http_address_ = MakeNetworkAddress(
       FLAGS_webserver_interface.empty() ? "0.0.0.0" : FLAGS_webserver_interface,
       FLAGS_webserver_port);
+  Init();
 }
 
 Webserver::Webserver(const int port)
@@ -248,6 +249,7 @@ Webserver::Webserver(const int port)
       error_handler_(UrlHandler(bind<void>(&Webserver::ErrorHandler, this, _1, _2),
           "error.tmpl", false)) {
   http_address_ = MakeNetworkAddress("0.0.0.0", port);
+  Init();
 }
 
 Webserver::~Webserver() {
@@ -284,17 +286,6 @@ bool Webserver::IsSecure() const {
   return !FLAGS_webserver_certificate_file.empty();
 }
 
-string Webserver::Url() {
-  string hostname = http_address_.hostname;
-  if (IsWildcardAddress(http_address_.hostname)) {
-    if (!GetHostname(&hostname).ok()) {
-      hostname = http_address_.hostname;
-    }
-  }
-  return Substitute("$0://$1:$2", IsSecure() ? "https" : "http",
-      hostname, http_address_.port);
-}
-
 Status Webserver::Start() {
   LOG(INFO) << "Starting webserver on " << TNetworkAddressToString(http_address_);
 
@@ -429,6 +420,17 @@ void Webserver::Stop() {
   }
 }
 
+void Webserver::Init() {
+  hostname_ = http_address_.hostname;
+  if (IsWildcardAddress(http_address_.hostname)) {
+    if (!GetHostname(&hostname_).ok()) {
+      hostname_ = http_address_.hostname;
+    }
+  }
+  url_ = Substitute(
+      "$0://$1:$2", IsSecure() ? "https" : "http", hostname_, http_address_.port);
+}
+
 void Webserver::GetCommonJson(Document* document) {
   DCHECK(document != nullptr);
   Value obj(kObjectType);
@@ -436,6 +438,20 @@ void Webserver::GetCommonJson(Document* document) {
       rapidjson::StringRef(google::ProgramInvocationShortName()),
       document->GetAllocator());
 
+  // We require that all links on the webui are qualified with the url.
+  Value url_value(url().c_str(), document->GetAllocator());
+  obj.AddMember("host-url", url_value, document->GetAllocator());
+
+  Value scheme_value(IsSecure() ? "https" : "http", document->GetAllocator());
+  obj.AddMember("scheme", scheme_value, document->GetAllocator());
+
+  Value hostname_value(hostname_.c_str(), document->GetAllocator());
+  obj.AddMember("hostname", hostname_value, document->GetAllocator());
+
+  Value port_value;
+  port_value.SetInt(http_address_.port);
+  obj.AddMember("port", port_value, document->GetAllocator());
+
   Value lst(kArrayType);
   for (const UrlHandlerMap::value_type& handler: url_handlers_) {
     if (handler.second.is_on_nav_bar()) {
diff --git a/be/src/util/webserver.h b/be/src/util/webserver.h
index e062e26..7422674 100644
--- a/be/src/util/webserver.h
+++ b/be/src/util/webserver.h
@@ -105,7 +105,7 @@ class Webserver {
   bool IsSecure() const;
 
   /// Returns the URL to the webserver as a string.
-  string Url();
+  string url() { return url_; }
 
   /// Returns the appropriate MIME type for a given ContentType.
   static const std::string GetMimeType(const ContentType& content_type);
@@ -151,6 +151,9 @@ class Webserver {
     std::string template_filename_;
   };
 
+  /// Sets the values of 'url_' and 'hostname_'.
+  void Init();
+
   /// Squeasel callback for log events. Returns squeasel success code.
   static int LogMessageCallbackStatic(const struct sq_connection* connection,
       const char* message);
@@ -203,6 +206,12 @@ class Webserver {
   /// The address of the interface on which to run this webserver.
   TNetworkAddress http_address_;
 
+  /// Formatted string representing 'http_address_'.
+  std::string url_;
+
+  /// The resolved hostname from 'http_address_'.
+  std::string hostname_;
+
   /// Handle to Squeasel context; owned and freed by Squeasel internally
   struct sq_context* context_;
 
diff --git a/bin/rat_exclude_files.txt b/bin/rat_exclude_files.txt
index e4592e7..9b7fa18 100644
--- a/bin/rat_exclude_files.txt
+++ b/bin/rat_exclude_files.txt
@@ -174,6 +174,7 @@ be/src/thirdparty/pcg-cpp-0.98/LICENSE.txt
 # a larger file where the larger file would have duplicate licensing headers.
 www/all_child_groups.tmpl
 www/common-footer.tmpl
+www/form-hidden-inputs.tmpl
 
 # GNU tar artifact
 pax_global_header
diff --git a/tests/common/file_utils.py b/tests/common/file_utils.py
index e634c92..e91befa 100644
--- a/tests/common/file_utils.py
+++ b/tests/common/file_utils.py
@@ -20,6 +20,7 @@
 # directories.
 
 import os
+import re
 from subprocess import check_call
 
 from tests.util.filesystem_utils import get_fs_path
@@ -71,15 +72,16 @@ def create_table_and_copy_files(impala_client, create_stmt, unique_database, tab
   impala_client.execute(create_stmt)
 
 
-def grep_dir(dir, search):
-  '''Recursively search for files that contain 'search' and return a list of matched
-     lines grouped by file.
+def grep_dir(dir, search, filename_search=""):
+  '''Recursively search for files that contain 'search' and have a filename that matches
+     'filename_search' and return a list of matched lines grouped by file.
   '''
+  filename_matcher = re.compile(filename_search)
   matching_files = dict()
   for dir_name, _, file_names in os.walk(dir):
     for file_name in file_names:
       file_path = os.path.join(dir_name, file_name)
-      if os.path.islink(file_path):
+      if os.path.islink(file_path) or not filename_matcher.search(file_path):
         continue
       with open(file_path) as file:
         matching_lines = grep_file(file, search)
@@ -89,12 +91,13 @@ def grep_dir(dir, search):
 
 
 def grep_file(file, search):
-  '''Return lines in 'file' that contain the 'search' term. 'file' must already be
+  '''Return lines in 'file' that match the 'search' regex. 'file' must already be
      opened.
   '''
+  matcher = re.compile(search)
   matching_lines = list()
   for line in file:
-    if search in line:
+    if matcher.search(line):
       matching_lines.append(line)
   return matching_lines
 
diff --git a/tests/custom_cluster/test_redaction.py b/tests/custom_cluster/test_redaction.py
index 1e436eb..fd95a16 100644
--- a/tests/custom_cluster/test_redaction.py
+++ b/tests/custom_cluster/test_redaction.py
@@ -282,7 +282,7 @@ class TestRedaction(CustomClusterTestSuite, unittest.TestCase):
     self.assert_query_profile_contains(self.find_last_query_id(), user_profile_pattern)
     # Wait for the logs to be written.
     sleep(5)
-    self.assert_log_redaction(email, "*email*")
+    self.assert_log_redaction(email, "\*email\*")
 
     # Even if the query is invalid, redaction should still be applied.
     credit_card = '1234-5678-1234-5678'
@@ -294,7 +294,7 @@ class TestRedaction(CustomClusterTestSuite, unittest.TestCase):
     self.assert_query_profile_contains(self.find_last_query_id(), user_profile_pattern)
     sleep(5)
     # Apparently an invalid query doesn't generate an audit log entry.
-    self.assert_log_redaction(credit_card, "*credit card*", expect_audit=False)
+    self.assert_log_redaction(credit_card, "\*credit card\*", expect_audit=False)
 
     # Assert that the username in the query stmt is redacted but not from the user fields.
     self.execute_query_expect_success(self.client, query_template % current_user)
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
index df12ad4..beca9f5 100644
--- a/tests/webserver/test_web_pages.py
+++ b/tests/webserver/test_web_pages.py
@@ -16,6 +16,7 @@
 # under the License.
 
 from tests.common.environ import ImpalaTestClusterFlagsDetector
+from tests.common.file_utils import grep_dir
 from tests.common.skip import SkipIfBuildType
 from tests.common.impala_cluster import ImpalaCluster
 from tests.common.impala_test_suite import ImpalaTestSuite
@@ -655,3 +656,18 @@ class TestWebPage(ImpalaTestSuite):
     """Test to check that the /healthz endpoint returns 200 OK."""
     page = requests.get("http://localhost:25000/healthz")
     assert page.status_code == requests.codes.ok
+
+  def test_knox_compatability(self):
+    """Checks that the template files conform to the requirements for compatability with
+    the Apache Knox service definition."""
+    # Matches all 'a' links with an 'href' that doesn't start with either '#' (which stays
+    # on the same page an so doesn't need to the hostname) or '{{ __common__.host-url }}'
+    # Note that if we ever need to add a link that doesn't conform to this, we will
+    # probably also have to change the Knox service definition.
+    href_regex = "<a .*? href=['\"](?!({{ __common__.host-url }})|#)"
+    # Matches all 'form' tags that are not followed by including the hidden inputs.
+    form_regex = "<form [^{]*?>(?!{{>www/form-hidden-inputs.tmpl}})"
+    regex = "(%s)|(%s)" % (href_regex, form_regex)
+    results = grep_dir("/home/thomas/Impala/www", regex, ".*\.tmpl")
+    assert len(results) == 0, \
+        "All links on the webui must include the webserver host: %s" % results
diff --git a/www/admission_controller.tmpl b/www/admission_controller.tmpl
index 069f57c..cf72dc6 100644
--- a/www/admission_controller.tmpl
+++ b/www/admission_controller.tmpl
@@ -203,13 +203,13 @@ function reset_method(pool_name) {
 </h2>
 {{^get_all_pools}}
 <p id="show_all_pools" class="lead">
-  <a href='/admission'> < Show all Resource Pools</a>
+  <a href='{{ __common__.host-url }}/admission'> < Show all Resource Pools</a>
 </p>
 {{/get_all_pools}}
 <p class="lead">
   This page lists all resource pools to which queries have been submitted
   at least once and their corresponding state and statistics.<br>See the
-  <a href='/backends'>backends</a> debug page for memory admitted and reserved per
+  <a href='{{ __common__.host-url }}/backends'>backends</a> debug page for memory admitted and reserved per
   backend.
 </p>
 <p class="lead">
@@ -220,7 +220,7 @@ Time since last statestore update containing admission control topic state (ms):
 </p>
 {{#resource_pools}}
 <div class="container-fluid">
-  <h3><a href='/admission?pool_name={{pool_name}}'>{{pool_name}}</a></h3>
+  <h3><a href='{{ __common__.host-url }}/admission?pool_name={{pool_name}}'>{{pool_name}}</a></h3>
 
   <h4>Pool Config</h4>
   <table class='table table-hover table-border'>
@@ -285,7 +285,7 @@ Time since last statestore update containing admission control topic state (ms):
       <td class='memory'>{{coord_mem_limit}}</td>
       <td class='memory'>{{coord_mem_to_admit}}</td>
       <td>{{num_backends}}</td>
-      <td><a href='/query_plan?query_id={{query_id}}'>Details</a></td>
+      <td><a href='{{ __common__.host-url }}/query_plan?query_id={{query_id}}'>Details</a></td>
     </tr>
     {{/queued_queries}}
   </table>
@@ -309,7 +309,7 @@ Time since last statestore update containing admission control topic state (ms):
       <td class='memory'>{{coord_mem_limit}}</td>
       <td class='memory'>{{coord_mem_to_admit}}</td>
       <td>{{num_backends}}</td>
-      <td><a href='/query_plan?query_id={{query_id}}'>Details</a></td>
+      <td><a href='{{ __common__.host-url }}/query_plan?query_id={{query_id}}'>Details</a></td>
     </tr>
     {{/running_queries}}
   </table>
diff --git a/www/catalog.tmpl b/www/catalog.tmpl
index 8faf5c1..e6adacd 100644
--- a/www/catalog.tmpl
+++ b/www/catalog.tmpl
@@ -43,11 +43,11 @@ under the License.
             since metadata is partially fetched on demand. -->
           {{?use_local_catalog}} {{name}} {{/use_local_catalog}}
           {{^use_local_catalog}}
-           <a href="catalog_object?object_type=TABLE&object_name={{fqtn}}">{{name}}</a>
+           <a href="{{ __common__.host-url }}/catalog_object?object_type=TABLE&object_name={{fqtn}}">{{name}}</a>
           {{/use_local_catalog}}
           </td>
           <td>{{mem_estimate}}</td>
-          <td><a href="table_metrics?name={{name}}">{{name}}-metrics</a></td>
+          <td><a href="{{ __common__.host-url }}/table_metrics?name={{name}}">{{name}}-metrics</a></td>
         </tr>
         {{/large_tables}}
       </tbody>
@@ -88,11 +88,11 @@ under the License.
           <td>
           {{?use_local_catalog}} {{name}} {{/use_local_catalog}}
           {{^use_local_catalog}}
-           <a href="catalog_object?object_type=TABLE&object_name={{fqtn}}">{{name}}</a>
+           <a href="{{ __common__.host-url }}/catalog_object?object_type=TABLE&object_name={{fqtn}}">{{name}}</a>
           {{/use_local_catalog}}
           </td>
           <td>{{num_metadata_ops}}</td>
-          <td><a href="table_metrics?name={{name}}">{{name}}-metrics</a></td>
+          <td><a href="{{ __common__.host-url }}/table_metrics?name={{name}}">{{name}}-metrics</a></td>
         </tr>
         {{/frequent_tables}}
       </tbody>
@@ -132,11 +132,11 @@ under the License.
           <td>
           {{?use_local_catalog}} {{name}} {{/use_local_catalog}}
           {{^use_local_catalog}}
-           <a href="catalog_object?object_type=TABLE&object_name={{fqtn}}">{{name}}</a>
+           <a href="{{ __common__.host-url }}/catalog_object?object_type=TABLE&object_name={{fqtn}}">{{name}}</a>
           {{/use_local_catalog}}
           </td>
           <td>{{num_files}}</td>
-          <td><a href="table_metrics?name={{name}}">{{name}}-metrics</a></td>
+          <td><a href="{{ __common__.host-url }}/table_metrics?name={{name}}">{{name}}-metrics</a></td>
         </tr>
         {{/high_file_count_tables}}
       </tbody>
@@ -166,7 +166,7 @@ under the License.
 <div class="panel panel-info">
   <div class="panel-heading">
   {{^use_local_catalog}}
-    <a href='catalog_object?object_type=DATABASE&object_name={{name}}'>
+    <a href='{{ __common__.host-url }}/catalog_object?object_type=DATABASE&object_name={{name}}'>
   {{/use_local_catalog}}
       <h2 class="panel-title" id='{{name}}'>{{name}}
       <span class="pull-right">{{num_tables}} table(s)</span></h2>
@@ -190,11 +190,11 @@ under the License.
           <td>
           {{?use_local_catalog}} {{name}} {{/use_local_catalog}}
           {{^use_local_catalog}}
-           <a href="catalog_object?object_type=TABLE&object_name={{fqtn}}">{{name}}</a>
+           <a href="{{ __common__.host-url }}/catalog_object?object_type=TABLE&object_name={{fqtn}}">{{name}}</a>
           {{/use_local_catalog}}
           </td>
           {{?has_metrics}}
-          <td><a href="table_metrics?name={{fqtn}}">{{name}}-metrics</a></td>
+          <td><a href="{{ __common__.host-url }}/table_metrics?name={{fqtn}}">{{name}}-metrics</a></td>
           {{/has_metrics}}
         </tr>
         {{/tables}}
diff --git a/www/common-header.tmpl b/www/common-header.tmpl
index 033dd6b..e0c3f50 100644
--- a/www/common-header.tmpl
+++ b/www/common-header.tmpl
@@ -21,12 +21,12 @@ common-footer.tmpl) }}
 <!DOCTYPE html>
 <html>
   <head><title>Apache Impala</title>
-    <script src='/www/jquery/jquery-3.4.1.min.js'></script>
-    <script src='/www/bootstrap/js/bootstrap.min.js'></script>
-    <script src='/www/scripts/util.js'></script>
-    <link rel="stylesheet" type="text/css" href="/www/datatables.min.css"/>
-    <script type="text/javascript" src="/www/datatables.min.js"></script>
-    <link href='/www/bootstrap/css/bootstrap.min.css' rel='stylesheet' media='screen'>
+    <script src='{{ __common__.host-url }}/www/jquery/jquery-3.4.1.min.js'></script>
+    <script src='{{ __common__.host-url }}/www/bootstrap/js/bootstrap.min.js'></script>
+    <script src='{{ __common__.host-url }}/www/scripts/util.js'></script>
+    <link rel="stylesheet" type="text/css" href="{{ __common__.host-url }}/www/datatables.min.css"/>
+    <script type="text/javascript" src="{{ __common__.host-url }}/www/datatables.min.js"></script>
+    <link href='{{ __common__.host-url }}/www/bootstrap/css/bootstrap.min.css' rel='stylesheet' media='screen'>
     <style>
       @media (min-width: 1300px) {
         #nav-options {
@@ -39,12 +39,12 @@ common-footer.tmpl) }}
     <header class="navbar navbar-default navbar-static-top" id="top" role="banner">
       <div id="nav-options" class="container">
         <div class="navbar-header">
-          <a class='navbar-brand' href='/'>{{ __common__.process-name }}</a>
+          <a class='navbar-brand' href='{{ __common__.host-url }}/'>{{ __common__.process-name }}</a>
         </div>
         <nav class="collapse navbar-collapse bs-navbar-collapse" role="navigation">
           <ul class="nav navbar-nav">
             {{#__common__.navbar}}
-            <li><a href="{{link}}">{{title}}</a></li>
+            <li><a href="{{ __common__.host-url }}{{link}}">{{title}}</a></li>
             {{/__common__.navbar}}
           </ul>
         </nav>
diff --git a/www/form-hidden-inputs.tmpl b/www/form-hidden-inputs.tmpl
new file mode 100644
index 0000000..83b1fea
--- /dev/null
+++ b/www/form-hidden-inputs.tmpl
@@ -0,0 +1,3 @@
+<input type='hidden' name='scheme' value='{{ __common__.scheme }}' />
+<input type='hidden' name='host' value='{{ __common__.hostname }}' />
+<input type='hidden' name='port' value='{{ __common__.port }}' />
\ No newline at end of file
diff --git a/www/log_level.tmpl b/www/log_level.tmpl
index b95da82..f3b3059 100644
--- a/www/log_level.tmpl
+++ b/www/log_level.tmpl
@@ -28,7 +28,7 @@ under the License.
 {{?include_log4j_handlers}}
 <h2>Change Java log level (log4j)</h2>
 <div class="log-level">
-<form action="get_java_loglevel">
+<form action="get_java_loglevel">{{>www/form-hidden-inputs.tmpl}}
   <div class="form-group">
     <input type="text" class="form-control" name="class" placeholder="e.g. org.apache.impala.analysis.Analyzer">
   </div>
@@ -36,7 +36,7 @@ under the License.
   <label>{{get_java_loglevel_result}}</label>
 </form>
 <br>
-<form action="set_java_loglevel">
+<form action="set_java_loglevel">{{>www/form-hidden-inputs.tmpl}}
   <div class="form-group" name="level">
     <input type="text" class="form-control" name="class" placeholder="e.g. org.apache.impala.analysis.Analyzer">
     <br>
@@ -59,7 +59,7 @@ under the License.
     </div>
   </div>
 </form>
-<form action="reset_java_loglevel">
+<form action="reset_java_loglevel">{{>www/form-hidden-inputs.tmpl}}
   <div class="col-xs-20">
     <button type="submit" class="btn btn-primary btn-sm">Reset Java Log Levels</button>
     <strong>{{reset_java_loglevel_result}}</strong>
@@ -68,7 +68,7 @@ under the License.
 {{/include_log4j_handlers}}
 
 <h2>Change backend logging level (glog)</h2>
-<form action="set_glog_level">
+<form action="set_glog_level">{{>www/form-hidden-inputs.tmpl}}
   <div class="form-group" name="level">
     <div class="col-xs-20">
       <label>Log level:</label>
@@ -86,7 +86,7 @@ under the License.
     </div>
   </div>
 </form>
-<form action="reset_glog_level">
+<form action="reset_glog_level">{{>www/form-hidden-inputs.tmpl}}
   <div class="col-xs-20">
     <button type="submit" class="btn btn-primary btn-sm">Reset Glog Levels</button>
     {{?reset_glog_level_result}}
diff --git a/www/queries.tmpl b/www/queries.tmpl
index 038f392..374161e 100644
--- a/www/queries.tmpl
+++ b/www/queries.tmpl
@@ -55,8 +55,8 @@ archived in memory. The size of that archive is controlled with the
     <td><samp>{{last_event}}</samp></td>
     <td>{{rows_fetched}}</td>
     <td>{{resource_pool}}</td>
-    <td><a href='/query_plan?query_id={{query_id}}'>Details</a></td>
-    <td><a href='/cancel_query?query_id={{query_id}}'>Cancel</a></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>
   </tr>
 {{/executing}}
 {{/in_flight_queries}}
@@ -100,8 +100,8 @@ archived in memory. The size of that archive is controlled with the
     <td><samp>{{last_event}}</samp></td>
     <td>{{rows_fetched}}</td>
     <td>{{resource_pool}}</td>
-    <td><a href='/query_plan?query_id={{query_id}}'>Details</a></td>
-    <td><a href='/cancel_query?query_id={{query_id}}'>Close</a></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>
   </tr>
 {{/waiting}}
 {{/in_flight_queries}}
@@ -137,7 +137,7 @@ archived in memory. The size of that archive is controlled with the
     <td><samp>{{state}}</samp></td>
     <td>{{rows_fetched}}</td>
     <td>{{resource_pool}}</td>
-    <td><a href='/query_plan?query_id={{query_id}}'>Details</a></td>
+    <td><a href='{{ __common__.host-url }}/query_plan?query_id={{query_id}}'>Details</a></td>
   </tr>
 {{/completed_queries}}
 </table>
diff --git a/www/query_detail_tabs.tmpl b/www/query_detail_tabs.tmpl
index 8e413a1..ccbe049 100644
--- a/www/query_detail_tabs.tmpl
+++ b/www/query_detail_tabs.tmpl
@@ -21,12 +21,12 @@ under the License.
 {{?status}}<h4><em>Status:</em> <span id="status">{{status}}</span></h4>{{/status}}
 
 <ul class="nav nav-tabs">
-  <li id="plan-tab" role="presentation"><a href="/query_plan?query_id={{query_id}}">Plan</a></li>
-  <li id="stmt-tab" role="presentation"><a href="/query_stmt?query_id={{query_id}}">Query</a></li>
-  <li id="plan-text-tab" role="presentation"><a href="/query_plan_text?query_id={{query_id}}">Text plan</a></li>
-  <li id="summary-tab" role="presentation"><a href="/query_summary?query_id={{query_id}}">Summary</a></li>
-  <li id="profile-tab" role="presentation"><a href="/query_profile?query_id={{query_id}}">Profile</a></li>
-  <li id="memory-tab" role="presentation"><a href="/query_memory?query_id={{query_id}}">Memory</a></li>
-  <li id="backends-tab" role="presentation"><a href="/query_backends?query_id={{query_id}}">Backends</a></li>
-  <li id="finstances-tab" role="presentation"><a href="/query_finstances?query_id={{query_id}}">Fragment Instances</a></li>
+  <li id="plan-tab" role="presentation"><a href="{{ __common__.host-url }}/query_plan?query_id={{query_id}}">Plan</a></li>
+  <li id="stmt-tab" role="presentation"><a href="{{ __common__.host-url }}/query_stmt?query_id={{query_id}}">Query</a></li>
+  <li id="plan-text-tab" role="presentation"><a href="{{ __common__.host-url }}/query_plan_text?query_id={{query_id}}">Text plan</a></li>
+  <li id="summary-tab" role="presentation"><a href="{{ __common__.host-url }}/query_summary?query_id={{query_id}}">Summary</a></li>
+  <li id="profile-tab" role="presentation"><a href="{{ __common__.host-url }}/query_profile?query_id={{query_id}}">Profile</a></li>
+  <li id="memory-tab" role="presentation"><a href="{{ __common__.host-url }}/query_memory?query_id={{query_id}}">Memory</a></li>
+  <li id="backends-tab" role="presentation"><a href="{{ __common__.host-url }}/query_backends?query_id={{query_id}}">Backends</a></li>
+  <li id="finstances-tab" role="presentation"><a href="{{ __common__.host-url }}/query_finstances?query_id={{query_id}}">Fragment Instances</a></li>
 </ul>
diff --git a/www/query_profile.tmpl b/www/query_profile.tmpl
index aabe685..6a5a659 100644
--- a/www/query_profile.tmpl
+++ b/www/query_profile.tmpl
@@ -27,13 +27,13 @@ under the License.
 <div>
     <h4>Download Profile (Available Formats):
         <a style="font-size:16px;" class="btn btn-primary"
-            href="/query_profile_encoded?query_id={{query_id}}"
+            href="{{ __common__.host-url }}/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}}"
+            href="{{ __common__.host-url }}/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}}"
+            href="{{ __common__.host-url }}/query_profile_plain_text?query_id={{query_id}}"
             download="profile_{{query_id}}">Text</a>
     </h4>
 </div>
diff --git a/www/sessions.tmpl b/www/sessions.tmpl
index 8c4f206..ee17905 100644
--- a/www/sessions.tmpl
+++ b/www/sessions.tmpl
@@ -63,7 +63,7 @@ called <strong>active</strong>.</div>
       <td><small>{{expired}}</small></td>
       <td><small>{{closed}}</small></td>
       <td><small>{{ref_count}}</small></td>
-      <td><small><a href='/close_session?session_id={{session_id}}'>Close</a></small></td>
+      <td><small><a href='{{ __common__.host-url }}/close_session?session_id={{session_id}}'>Close</a></small></td>
     </tr>
     {{/sessions}}
   </tbody>
diff --git a/www/threadz.tmpl b/www/threadz.tmpl
index cc5ec38..c1cf513 100644
--- a/www/threadz.tmpl
+++ b/www/threadz.tmpl
@@ -26,16 +26,16 @@ under the License.
 <h4>{{.}} thread(s) running</h4>
 {{/total-threads}}
 
-<a href="/thread-group?all"><h3>All threads</h3></a>
+<a href="{{ __common__.host-url }}/thread-group?all"><h3>All threads</h3></a>
 
 {{#thread-groups}}
-<a href='/thread-group?group={{name}}'>
+<a href='{{ __common__.host-url }}/thread-group?group={{name}}'>
   <h3>{{name}} : (running: {{size}}, total created: {{num_created}})</h3>
 </a>
 {{/thread-groups}}
 
 {{#jvm-threads}}
-<a href='/jvm-threadz'>
+<a href='{{ __common__.host-url }}/jvm-threadz'>
   <h3>{{name}} : (Total: {{total}}, Daemon: {{daemon}})</h3>
 </a>
 {{/jvm-threads}}
diff --git a/www/threadz_tabs.tmpl b/www/threadz_tabs.tmpl
index 0dbb6e8..da4c8a6 100644
--- a/www/threadz_tabs.tmpl
+++ b/www/threadz_tabs.tmpl
@@ -20,8 +20,8 @@ under the License.
 <h2>Threads</h2>
 
 <ul class="nav nav-tabs">
-  <li id="overview-tab" role="presentation"><a href="/threadz">Overview</a></li>
+  <li id="overview-tab" role="presentation"><a href="{{ __common__.host-url }}/threadz">Overview</a></li>
   {{?jvm-threads}}
-  <li id="jvm-tab" role="presentation"><a href="/jvm-threadz">JVM</a></li>
+  <li id="jvm-tab" role="presentation"><a href="{{ __common__.host-url }}/jvm-threadz">JVM</a></li>
   {{/jvm-threads}}
 </ul>