You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2019/11/08 01:51:15 UTC

[kudu] 02/02: webserver: add support for Knox URL rewriting

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

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

commit 58177651a4e3ba3da9ddee46ec5e79bc011f8406
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Mon Oct 28 21:56:49 2019 -0700

    webserver: add support for Knox URL rewriting
    
    This patch modifies the web UI to support proxying by Apache Knox. When Kudu
    is deployed with Knox, the Kudu server is expected to be firewalled off and
    all web UI access is mediated by the Knox gateway. Precisely how this works
    is best illustrated via example. Suppose we want to access a web UI at
    foo.bar.com:8051 via Knox running on localhost. Instead of accessing:
    
      http://foo.bar.com:8051/varz
    
    We must access:
    
      https://localhost:8443/gateway/test/kuduui/varz?scheme=http&host=foo.bar.com&port=8051
    
    Let's break this down:
    - localhost:8443 is the location of the Knox gateway.
    - The gateway/test/kuduui subpath is part of the Knox topology definition,
      which tells Knox that we're interested in accessing a Kudu web UI in the
      'test' topology.
    - Ultimately we're interested in the /varz page in the Kudu web UI.
    - Because there's a web UI in each Kudu process, the query parameters tell
      Knox which Kudu server we're interested in, and how we want to acccess it.
    
    When Knox receives this HTTP request, it rewrites it to use the simpler form
    and sends it to Kudu. That's only half of the work though; Knox must also
    rewrite the HTTP response because all of the links in the HTML were created
    by a web UI unaware of its firewalled state. By the way of example, if we
    kept a partial URL like /varz intact, the client would try to access
    https://localhost:8443/varz when following the link. So Knox needs to
    rewrite /varz into the "long" form described above. URLs pointing to other Kudu
    servers (e.g. http://baz.bar.com:8051/) must also be rewritten because the
    client can't access those servers directly.
    
    So how do we do all of this? The first part is a KUDUUI service definition
    in Knox[1]. The definition uses pattern matching to identify which web UI
    URLs need to be rewritten and how. Unfortunately, the matching isn't robust
    enough to match "/.*" (including "/"). So we need to help it out. When we
    detect a request proxied by Knox, we prepend a special identifier to all
    non-external links in the response. The KUDUUI service definition searches
    for this identifier and rewrites all URLs that include it.[2]
    
    Almost everything in this patch either directly or indirectly facilitates
    that work. Other interesting things going on:
    - cpp-mustache was upgraded. Todd added some patches to recursively resolve
      a variable through all parent contexts. This is necessary if we're to find
      {{base_url}} at the top-level JSON context regardless of where it's used.
      The library now depends on a C++11-compliant compiler.
    - If we're responding to Knox, we need to avoid URL-encoding any query
      parameter values, because for some reason Knox does this on its own when
      it rewrites HTTP responses.
    - Standing up a Knox gateway is difficult, and given that Knox integration
      is quite ancillary to core Kudu, I didn't think that implementing a
      "MiniKnox" made sense. Instead, I wrote a new test that crawls all web UIs
      in a mini cluster and tests all links in all pages. I used the Gumbo HTML
      parser (added to thirdparty in previous patches) to simplify this work.
    
    What doesn't work?
    - The /config page references a font glyph that can't be proxied, because
      the link to the font is embedded in bootstrap.min.css and we don't rewrite
      links in CSS files. The effect is a small box instead of a lock icon, and
      an ugly error in the Knox logs.
    - Similarly, the /metrics.html and /tracing.html pages (and JS) can't be
      proxied because they're not templates and can't easily be made into
      templates. Clients who wish to use them will need to set up an SSH tunnel
      in order to do so.
    
    1. See https://issues.apache.org/jira/browse/KNOX-2072 for details.
    2. Another approach is to add host/port info to all non-external links.
       That's what Impala did in IMPALA-8897, and it's nice in that it minimizes
       URL rewriting in HTTP responses. But it's also fraught in that Impala
       doesn't always know its own hostname.
    
    Change-Id: Iee92cb094b81609356acf858b7c549b6c281a7e5
    Reviewed-on: http://gerrit.cloudera.org:8080/14573
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/integration-tests/CMakeLists.txt          |   3 +-
 src/kudu/integration-tests/registration-test.cc    |   2 +-
 .../integration-tests/webserver-crawl-itest.cc     | 322 +++++++++++++++++++++
 src/kudu/master/master_path_handlers.cc            |   4 +-
 src/kudu/server/webserver.cc                       |  81 ++++--
 src/kudu/server/webserver.h                        |  20 +-
 src/kudu/tserver/tserver_path_handlers.cc          |   3 +-
 src/kudu/util/CMakeLists.txt                       |   1 +
 src/kudu/util/thread.cc                            |  11 +-
 src/kudu/util/web_callback_registry.cc             |  28 ++
 src/kudu/util/web_callback_registry.h              |  20 +-
 thirdparty/build-definitions.sh                    |   2 +-
 thirdparty/vars.sh                                 |   2 +-
 www/dashboards.mustache                            |   6 +-
 www/home.mustache                                  |   2 +-
 www/log-anchors.mustache                           |   2 +-
 www/scans.mustache                                 |   2 +-
 www/table.mustache                                 |   2 +-
 www/tables.mustache                                |   2 +-
 www/tablet-rowsetlayout-svg.mustache               |   2 +-
 www/tablet-servers.mustache                        |   2 +-
 www/tablet.mustache                                |   8 +-
 www/tablets.mustache                               |   4 +-
 www/threadz.mustache                               |   4 +-
 24 files changed, 463 insertions(+), 72 deletions(-)

diff --git a/src/kudu/integration-tests/CMakeLists.txt b/src/kudu/integration-tests/CMakeLists.txt
index 64d8fef..c76a445 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -54,7 +54,7 @@ add_dependencies(itest_util
   kudu-tserver)
 
 # Tests
-SET_KUDU_TEST_LINK_LIBS(itest_util)
+SET_KUDU_TEST_LINK_LIBS(itest_util gumbo-parser gumbo-query)
 ADD_KUDU_TEST(all_types-itest
   PROCESSORS 4
   NUM_SHARDS 8)
@@ -131,6 +131,7 @@ ADD_KUDU_TEST(ts_recovery-itest PROCESSORS 4)
 ADD_KUDU_TEST(ts_sentry-itest NUM_SHARDS 2)
 ADD_KUDU_TEST(ts_tablet_manager-itest)
 ADD_KUDU_TEST(update_scan_delta_compact-test RUN_SERIAL true)
+ADD_KUDU_TEST(webserver-crawl-itest LABELS no_dist_test)
 ADD_KUDU_TEST(webserver-stress-itest RUN_SERIAL true)
 ADD_KUDU_TEST(write_throttling-itest)
 
diff --git a/src/kudu/integration-tests/registration-test.cc b/src/kudu/integration-tests/registration-test.cc
index 0f6d93c..e5f6916 100644
--- a/src/kudu/integration-tests/registration-test.cc
+++ b/src/kudu/integration-tests/registration-test.cc
@@ -396,7 +396,7 @@ TEST_F(RegistrationTest, TestExposeHttpsURLs) {
   // dealing with figuring out what the hostname should be, just
   // use a more permissive regex which doesn't check the host.
   string expected_url_regex = strings::Substitute(
-      "https://[a-zA-Z0-9.-]+:$0/", opts->port);
+      "https://[a-zA-Z0-9.-]+:$0", opts->port);
 
   // Need "eventually" here because the tserver may take a few seconds
   // to re-register while starting up.
diff --git a/src/kudu/integration-tests/webserver-crawl-itest.cc b/src/kudu/integration-tests/webserver-crawl-itest.cc
new file mode 100644
index 0000000..ac85e80
--- /dev/null
+++ b/src/kudu/integration-tests/webserver-crawl-itest.cc
@@ -0,0 +1,322 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <algorithm>
+#include <deque>
+#include <ostream>
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gq/Document.h>
+#include <gq/Node.h>
+#include <gq/Selection.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/strcat.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/integration-tests/test_workload.h"
+#include "kudu/mini-cluster/external_mini_cluster.h"
+#include "kudu/security/test/test_certs.h"
+#include "kudu/util/curl_util.h"
+#include "kudu/util/env.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/url-coding.h"
+
+using kudu::cluster::ExternalMiniCluster;
+using kudu::cluster::ExternalMiniClusterOptions;
+using std::deque;
+using std::string;
+using std::tuple;
+using std::unordered_set;
+using std::vector;
+using strings::SkipEmpty;
+using strings::Split;
+
+namespace kudu {
+
+enum class UseSsl { NO, YES, };
+enum class ImpersonateKnox { NO, YES, };
+
+// Beautifies test output if a test scenario fails.
+std::ostream& operator<<(std::ostream& o, UseSsl opt) {
+  switch (opt) {
+    case UseSsl::NO:
+      return o << "UseSsl::NO";
+    case UseSsl::YES:
+      return o << "UseSsl::YES";
+  }
+  return o;
+}
+
+std::ostream& operator<<(std::ostream& o, ImpersonateKnox opt) {
+  switch (opt) {
+    case ImpersonateKnox::NO:
+      return o << "ImpersonateKnox::NO";
+    case ImpersonateKnox::YES:
+      return o << "ImpersonateKnox::YES";
+  }
+  return o;
+}
+
+class WebserverCrawlITest : public KuduTest,
+                            public ::testing::WithParamInterface<tuple<UseSsl, ImpersonateKnox>> {
+ protected:
+  static constexpr const char* const kKnoxPrefix = "/KNOX-BASE";
+  static constexpr const char* const kHttpScheme = "http://";
+  static constexpr const char* const kHttpsScheme = "https://";
+
+  // A web server is responsible for escaping illegal URL characters in all
+  // links embedded in a response. In Kudu's web UI, a prime example is the
+  // /threadz page, which includes links to individual thread group pages by
+  // name, and thread group names may include spaces. However, when a real Knox
+  // gateway mediates the connection, it unconditionally URL-encodes all query
+  // parameter values. Kudu is aware of this behavior and when Knox is detected
+  // in the connection, Kudu no longer does any escaping, expecting that Knox
+  // will do it instead.
+  //
+  // Impersonating Knox means that our links will not be escaped and will be
+  // uncrawlable unless we escape them in the test. This function just does
+  // that: it URL-encodes all query parameter values just as Knox would.
+  static void KnoxifyURL(string* url) {
+    int idx = url->find('?');
+    if (idx == string::npos || idx == url->length() - 1) {
+      // No query parameters found in the URL.
+      return;
+    }
+    string query = url->substr(idx + 1);
+    vector<string> encoded_query_params;
+    for (const auto& param : Split(query, "&", SkipEmpty())) {
+      vector<string> kv = Split(param, "=");
+      ASSERT_EQ(2, kv.size());
+      encoded_query_params.emplace_back(kv[0] + "=" +
+                                        UrlEncodeToString(kv[1]));
+    }
+    string encoded_query_param_str = JoinStrings(encoded_query_params, "&");
+
+    // Replace the original query parameter string with the new one containing
+    // URL-encoded parameter values.
+    *url = StringReplace(*url, query, encoded_query_param_str,
+                         /* replace_all= */ false);
+  }
+};
+
+INSTANTIATE_TEST_CASE_P(,
+    WebserverCrawlITest,
+    ::testing::Combine(
+        ::testing::Values(UseSsl::NO, UseSsl::YES),
+        ::testing::Values(ImpersonateKnox::NO, ImpersonateKnox::YES)));
+
+TEST_P(WebserverCrawlITest, TestAllWebPages) {
+  bool use_ssl = std::get<0>(GetParam()) == UseSsl::YES;
+  bool impersonate_knox = std::get<1>(GetParam()) == ImpersonateKnox::YES;
+  const char* scheme = use_ssl ? kHttpsScheme : kHttpScheme;
+
+  // We must set up a proper docroot so that we can render the mustache templates.
+  //
+  // Unfortunately, this means hardcoding the test's location, and prevents us
+  // from testing it with dist-test (see CMakeLists.txt).
+  string bin_path;
+  ASSERT_OK(Env::Default()->GetExecutablePath(&bin_path));
+  string docroot = StrCat(DirName(bin_path), "/../../../www");
+  bool is_dir;
+  ASSERT_OK(Env::Default()->IsDirectory(docroot, &is_dir));
+  ASSERT_TRUE(is_dir);
+
+  vector<string> flags = { StrCat("--webserver_doc_root=", docroot) };
+  if (use_ssl) {
+    string cert_file;
+    string pk_file;
+    string pw;
+    ASSERT_OK(security::CreateTestSSLCertWithEncryptedKey(
+        GetTestDataDirectory(), &cert_file, &pk_file, &pw));
+    flags.emplace_back(StrCat("--webserver_certificate_file=", cert_file));
+    flags.emplace_back(StrCat("--webserver_private_key_file=", pk_file));
+    flags.emplace_back(StrCat("--webserver_private_key_password_cmd=echo ", pw));
+  }
+
+  // Use multiple masters so that the /masters pages contain interesting links.
+  ExternalMiniClusterOptions opts;
+  opts.num_masters = 3;
+  opts.num_tablet_servers = 3;
+  opts.extra_master_flags = flags;
+  opts.extra_tserver_flags = flags;
+  ExternalMiniCluster cluster(std::move(opts));
+  ASSERT_OK(cluster.Start());
+
+  // Create a table and write soem data so that log anchors page gets populated.
+  TestWorkload work(&cluster);
+  work.set_num_replicas(3);
+  work.set_num_read_threads(4);
+  work.set_num_tablets(6);
+  work.Setup();
+  work.Start();
+  while (work.rows_inserted() < 1000) {
+    SleepFor(MonoDelta::FromMilliseconds(10));
+  }
+  work.StopAndJoin();
+
+  // Tracks all URLs left to process. When empty, the search is finished.
+  deque<string> urls;
+
+  // Tracks every URL seen by the test. The URL is full; it includes the scheme,
+  // host information, path, and query parameters.
+  //
+  // Note that a URL is added _before_ it is actually visited so that we can
+  // avoid duplicate visits if a page has the same link multiple times.
+  unordered_set<string> urls_seen;
+
+  // Populate 'urls' with the roots of all the web UIs.
+  for (int i = 0; i < cluster.num_masters(); i++) {
+    string url = scheme + cluster.master(i)->bound_http_hostport().ToString();
+    urls.emplace_back(url);
+    urls_seen.emplace(std::move(url));
+  }
+  for (int i = 0; i < cluster.num_tablet_servers(); i++) {
+    string url = scheme + cluster.tablet_server(i)->bound_http_hostport().ToString();
+    urls.emplace_back(url);
+    urls_seen.emplace(std::move(url));
+  }
+
+  // Process one link, adding it to the queue if has yet to be crawled.
+  auto process_link = [&](const string& host, string link) {
+    SCOPED_TRACE(link);
+    if (link.empty()) {
+      // The HTML parser produces empty links in some JS and CSS pages; skip them.
+      return;
+    }
+    if (HasPrefixString(link, "#")) {
+      // An anchor without a path doesn't need to be crawled.
+      return;
+    }
+
+    // Verify that the link's scheme matches how we've configured the web UI.
+    ASSERT_FALSE(HasPrefixString(link, use_ssl ? kHttpScheme : kHttpsScheme));
+
+    if (HasPrefixString(link, scheme)) {
+      // Full URLs should not be modified and can be visited directly.
+      if (EmplaceIfNotPresent(&urls_seen, link)) {
+        urls.emplace_back(std::move(link));
+      }
+      return;
+    }
+
+    // From here on out we're dealing with a URL with an absolute path.
+
+    if (impersonate_knox) {
+      // The web UI should have returned a link beginning with the special
+      // Knox token. Verify this and remove it so that we can actually crawl
+      // the link.
+      ASSERT_TRUE(HasPrefixString(link, kKnoxPrefix));
+      link = StringReplace(link, kKnoxPrefix, "", /* replace_all= */ false);
+      NO_FATALS(KnoxifyURL(&link));
+    } else {
+      ASSERT_FALSE(HasPrefixString(link, kKnoxPrefix));
+    }
+
+    // Root paths are canonicalized into empty strings to match the behavior
+    // of the initial URL seeding as well as the various external URLs generated
+    // by the web UIs.
+    if (link == "/") {
+      link = "";
+    }
+
+    // Sanity check that there are no spaces in the link; they should have been
+    // URL-encoded, either by the web UI or by KnoxifyURL.
+    ASSERT_EQ(string::npos, link.find(' '));
+
+    string full_url = host + link;
+    if (!ContainsKey(urls_seen, full_url)) {
+      urls.emplace_back(full_url);
+      urls_seen.emplace(std::move(full_url));
+    }
+  };
+
+  // Process all links in the page as defined by a particular pairing of element
+  // and attribute type.
+  auto process_page = [&](const string& host,
+                          gq::CDocument* page,
+                          const char* elem,
+                          const char* attr) {
+    gq::CSelection sel = page->find(elem);
+    for (int i = 0; i < sel.nodeNum(); i++) {
+      string link = sel.nodeAt(i).attribute(attr);
+      NO_FATALS(process_link(host, std::move(link)));
+    }
+  };
+
+  // Crawl the web UIs.
+  //
+  // TODO(adar): the crawl could be faster if squeasel used TCP_NODELAY in its
+  // sockets. As it stands, a repeated fetch to the same host is slowed by about
+  // ~40ms due to Nagle's algorithm and delayed TCP ACKs.
+  EasyCurl curl;
+
+  // We use a self-signed cert, so we need to disable cert verification in curl.
+  if (use_ssl) {
+    curl.set_verify_peer(false);
+  }
+
+  faststring response;
+  vector<string> headers;
+  if (impersonate_knox) {
+    // Pretend we're Knox when communicating with the web UI.
+    //
+    // Note: the header value doesn't actually matter; only the key matters.
+    headers.emplace_back("X-Forwarded-Context: test");
+  }
+  while (!urls.empty()) {
+    string url = urls.front();
+    urls.pop_front();
+    int ret = FindNth(url, '/', 3);
+    string host = ret == string::npos ? url : url.substr(0, ret);
+
+    // Every link should be reachable.
+    ASSERT_OK(curl.FetchURL(url, &response, headers));
+    string resp_str = response.ToString();
+    SCOPED_TRACE(resp_str);
+
+    gq::CDocument page;
+    page.parse(resp_str);
+
+    // e.g. <a href="/rpcz">RPCs</a>
+    NO_FATALS(process_page(host, &page, "a", "href"));
+
+    // e.g. <script src='/bootstrap/js/bootstrap.min.js' defer></script>
+    NO_FATALS(process_page(host, &page, "script", "src"));
+  }
+
+  // Dump the results for troubleshooting.
+  vector<string> sorted_urls(urls_seen.begin(), urls_seen.end());
+  std::sort(sorted_urls.begin(), sorted_urls.end());
+  LOG(INFO) << "Dumping visited URLs";
+  for (const auto& u : sorted_urls) {
+    LOG(INFO) << u;
+  }
+}
+
+} // namespace kudu
diff --git a/src/kudu/master/master_path_handlers.cc b/src/kudu/master/master_path_handlers.cc
index 3c0c373..bcb5d6b 100644
--- a/src/kudu/master/master_path_handlers.cc
+++ b/src/kudu/master/master_path_handlers.cc
@@ -168,7 +168,7 @@ void MasterPathHandlers::HandleTabletServers(const Webserver::WebRequest& /*req*
     desc->GetRegistration(&reg);
     ts_json["uuid"] = ts_uuid;
     if (!reg.http_addresses().empty()) {
-      string webserver = Substitute("$0://$1:$2/",
+      string webserver = Substitute("$0://$1:$2",
                                     reg.https_enabled() ? "https" : "http",
                                     reg.http_addresses(0).host(),
                                     reg.http_addresses(0).port());
@@ -567,7 +567,7 @@ void MasterPathHandlers::HandleMasters(const Webserver::WebRequest& /*req*/,
     ServerRegistrationPB reg = master.registration();
     master_json["uuid"] = master.instance_id().permanent_uuid();
     if (!reg.http_addresses().empty()) {
-      master_json["target"] = Substitute("$0://$1:$2/",
+      master_json["target"] = Substitute("$0://$1:$2",
                                          reg.https_enabled() ? "https" : "http",
                                          reg.http_addresses(0).host(),
                                          reg.http_addresses(0).port());
diff --git a/src/kudu/server/webserver.cc b/src/kudu/server/webserver.cc
index 4a7684e..3dad6d7 100644
--- a/src/kudu/server/webserver.cc
+++ b/src/kudu/server/webserver.cc
@@ -58,6 +58,7 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/net/sockaddr.h"
+#include "kudu/util/string_case.h"
 #include "kudu/util/url-coding.h"
 #include "kudu/util/version_info.h"
 #include "kudu/util/zlib.h"
@@ -137,7 +138,7 @@ Sockaddr GetRemoteAddress(const struct sq_request_info* req) {
 // out-parameters will be written to, and the function will return either OK or
 // Incomplete depending on whether additional SPNEGO steps are required.
 Status RunSpnegoStep(const char* authz_header,
-                     WebCallbackRegistry::HttpResponseHeaders* resp_headers,
+                     WebCallbackRegistry::ArgumentMap* resp_headers,
                      string* authn_user) {
   static const char* const kNegotiateHdrName = "WWW-Authenticate";
   static const char* const kNegotiateHdrValue = "Negotiate";
@@ -531,14 +532,20 @@ sq_callback_result_t Webserver::RunPathHandler(
     struct sq_connection* connection,
     struct sq_request_info* request_info,
     PrerenderedWebResponse* resp) {
-  // Should we render with css styles?
-  bool use_style = true;
-
   WebRequest req;
   if (request_info->query_string != nullptr) {
     req.query_string = request_info->query_string;
     BuildArgumentMap(request_info->query_string, &req.parsed_args);
   }
+  for (int i = 0; i < request_info->num_headers; i++) {
+    const auto& h = request_info->http_headers[i];
+    string key = h.name;
+
+    // Canonicalize header names to lower case so that we needn't worry about
+    // doing case-insensitive comparisons throughout.
+    ToLowerCase(&key);
+    req.request_headers[key] = h.value;
+  }
   req.request_method = request_info->request_method;
   if (req.request_method == "POST") {
     const char* content_len_str = sq_get_header(connection, "Content-Length");
@@ -576,10 +583,6 @@ sq_callback_result_t Webserver::RunPathHandler(
     }
   }
 
-  if (!handler.is_styled() || ContainsKey(req.parsed_args, "raw")) {
-    use_style = false;
-  }
-
   // Enable or disable redaction from the web UI based on the setting of --redact.
   // This affects operations like default value and scan predicate pretty printing.
   if (kudu::g_should_redact == kudu::RedactContext::ALL) {
@@ -589,17 +592,22 @@ sq_callback_result_t Webserver::RunPathHandler(
     handler.callback()(req, resp);
   }
 
-  SendResponse(connection, resp, use_style ? StyleMode::STYLED : StyleMode::UNSTYLED);
+  // Should we render with css styles?
+  StyleMode use_style = handler.is_styled() && !ContainsKey(req.parsed_args, "raw") ?
+                        StyleMode::STYLED : StyleMode::UNSTYLED;
+  SendResponse(connection, resp, &req, use_style);
   return SQ_HANDLED_OK;
 }
 
 void Webserver::SendResponse(struct sq_connection* connection,
                              PrerenderedWebResponse* resp,
+                             const WebRequest* req,
                              StyleMode mode) {
   // If styling was requested, rerender and replace the prerendered output.
   if (mode == StyleMode::STYLED) {
+    DCHECK(req);
     stringstream ss;
-    RenderMainTemplate(resp->output.str(), &ss);
+    RenderMainTemplate(*req, resp->output.str(), &ss);
     resp->output.str(ss.str());
   }
 
@@ -673,13 +681,22 @@ void Webserver::SendResponse(struct sq_connection* connection,
   sq_write(connection, complete_response.c_str(), complete_response.length());
 }
 
+void Webserver::AddKnoxVariables(const WebRequest& req, EasyJson* json) {
+  if (WebCallbackRegistry::IsProxiedViaKnox(req)) {
+    (*json)["base_url"] = "/KNOX-BASE";
+  } else {
+    (*json)["base_url"] = "";
+  }
+}
+
 void Webserver::RegisterPathHandler(const string& path, const string& alias,
     const PathHandlerCallback& callback, bool is_styled, bool is_on_nav_bar) {
   string render_path = (path == "/") ? "/home" : path;
-  auto wrapped_cb = [=](const WebRequest& args, PrerenderedWebResponse* rendered_resp) {
+  auto wrapped_cb = [=](const WebRequest& req, PrerenderedWebResponse* rendered_resp) {
     WebResponse resp;
-    callback(args, &resp);
+    callback(req, &resp);
     stringstream out;
+    AddKnoxVariables(req, &resp.output);
     Render(render_path, resp.output, is_styled, &out);
     rendered_resp->status_code = resp.status_code;
     rendered_resp->response_headers = std::move(resp.response_headers);
@@ -711,39 +728,40 @@ static const char* const kMainTemplate = R"(
   <head>
     <title>Kudu</title>
     <meta charset='utf-8'/>
-    <link href='/bootstrap/css/bootstrap.min.css' rel='stylesheet' media='screen' />
-    <link href='/bootstrap/css/bootstrap-table.min.css' rel='stylesheet' media='screen' />
-    <script src='/jquery-3.2.1.min.js' defer></script>
-    <script src='/bootstrap/js/bootstrap.min.js' defer></script>
-    <script src='/bootstrap/js/bootstrap-table.min.js' defer></script>
-    <script src='/kudu.js' defer></script>
-    <link href='/kudu.css' rel='stylesheet' />
+    <link href='{{base_url}}/bootstrap/css/bootstrap.min.css' rel='stylesheet' media='screen'/>
+    <link href='{{base_url}}/bootstrap/css/bootstrap-table.min.css' rel='stylesheet' media='screen'/>
+    <script src='{{base_url}}/jquery-3.2.1.min.js' defer></script>
+    <script src='{{base_url}}/bootstrap/js/bootstrap.min.js' defer></script>
+    <script src='{{base_url}}/bootstrap/js/bootstrap-table.min.js' defer></script>
+    <script src='{{base_url}}/kudu.js' defer></script>
+    <link href='{{base_url}}/kudu.css' rel='stylesheet'/>
+    <link rel='icon' href='{{base_url}}/favicon.ico'>
   </head>
   <body>
 
     <nav class="navbar navbar-default">
       <div class="container-fluid">
         <div class="navbar-header">
-          <a class="navbar-brand" style="padding-top: 5px;" href="/">
-            <img src="/logo.png" width='61' height='45' alt="Kudu" />
+          <a class="navbar-brand" style="padding-top: 5px;" href="{{base_url}}/">
+            <img src="{{base_url}}/logo.png" width='61' height='45' alt="Kudu"/>
           </a>
         </div>
         <div id="navbar" class="navbar-collapse collapse">
           <ul class="nav navbar-nav">
            {{#path_handlers}}
-            <li><a class="nav-link"href="{{path}}">{{alias}}</a></li>
+            <li><a class="nav-link" href="{{base_url}}{{path}}">{{alias}}</a></li>
            {{/path_handlers}}
           </ul>
         </div><!--/.nav-collapse -->
       </div><!--/.container-fluid -->
     </nav>
-      {{^static_pages_available}}
-      <div style="color: red">
-        <strong>Static pages not available. Configure KUDU_HOME or use the --webserver_doc_root
-        flag to fix page styling.</strong>
-      </div>
-      {{/static_pages_available}}
-      {{{content}}}
+    {{^static_pages_available}}
+    <div style="color: red">
+      <strong>Static pages not available. Configure KUDU_HOME or use the --webserver_doc_root
+      flag to fix page styling.</strong>
+    </div>
+    {{/static_pages_available}}
+    {{{content}}}
     </div>
     {{#footer_html}}
     <footer class="footer"><div class="container text-muted">
@@ -754,10 +772,13 @@ static const char* const kMainTemplate = R"(
 </html>
 )";
 
-void Webserver::RenderMainTemplate(const string& content, stringstream* output) {
+void Webserver::RenderMainTemplate(
+    const WebRequest& req, const string& content, stringstream* output) {
   EasyJson ej;
   ej["static_pages_available"] = static_pages_available();
   ej["content"] = content;
+  AddKnoxVariables(req, &ej);
+
   {
     shared_lock<RWMutex> l(lock_);
     ej["footer_html"] = footer_html_;
diff --git a/src/kudu/server/webserver.h b/src/kudu/server/webserver.h
index 68568a6..42ca903 100644
--- a/src/kudu/server/webserver.h
+++ b/src/kudu/server/webserver.h
@@ -14,8 +14,8 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_UTIL_WEBSERVER_H
-#define KUDU_UTIL_WEBSERVER_H
+
+#pragma once
 
 #include <iosfwd>
 #include <map>
@@ -112,6 +112,9 @@ class Webserver : public WebCallbackRegistry {
     PrerenderedPathHandlerCallback callback_;
   };
 
+  // Add any necessary Knox-related variables to 'json' based on the headers in 'args'.
+  static void AddKnoxVariables(const WebRequest& req, EasyJson* json);
+
   bool static_pages_available() const;
 
   // Build the string to pass to mongoose specifying where to bind.
@@ -126,8 +129,11 @@ class Webserver : public WebCallbackRegistry {
   bool MustacheTemplateAvailable(const std::string& path) const;
 
   // Renders the main HTML template with the pre-rendered string 'content'
-  // in the main body of the page, into 'output'.
-  void RenderMainTemplate(const std::string& content, std::stringstream* output);
+  // in the main body of the page into 'output'. Additional state specific to
+  // the HTTP request that may affect rendering is available in 'req' if needed.
+  void RenderMainTemplate(const WebRequest& req,
+                          const std::string& content,
+                          std::stringstream* output);
 
   // Renders the template corresponding to 'path' (if available), using
   // fields in 'ej'.
@@ -161,6 +167,9 @@ class Webserver : public WebCallbackRegistry {
 
   // Sends a response back thru 'connection'.
   //
+  // 'req' may be null if we're early enough in processing that we haven't
+  // parsed the request yet (e.g. an early error out).
+  //
   // If 'mode' is STYLED, includes page styling elements like CSS, navigation bar, etc.
   enum class StyleMode {
     STYLED,
@@ -168,6 +177,7 @@ class Webserver : public WebCallbackRegistry {
   };
   void SendResponse(struct sq_connection* connection,
                     PrerenderedWebResponse* resp,
+                    const WebRequest* req = nullptr,
                     StyleMode mode = StyleMode::UNSTYLED);
 
   const WebserverOptions opts_;
@@ -197,5 +207,3 @@ class Webserver : public WebCallbackRegistry {
 };
 
 } // namespace kudu
-
-#endif // KUDU_UTIL_WEBSERVER_H
diff --git a/src/kudu/tserver/tserver_path_handlers.cc b/src/kudu/tserver/tserver_path_handlers.cc
index d865703..d1c35bf 100644
--- a/src/kudu/tserver/tserver_path_handlers.cc
+++ b/src/kudu/tserver/tserver_path_handlers.cc
@@ -343,8 +343,7 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& /*
       replica_json["id"] = status.tablet_id();
       if (replica->tablet() != nullptr) {
         EasyJson link_json = replica_json.Set("link", EasyJson::kObject);
-        link_json["id"] = status.tablet_id();
-        link_json["url"] = Substitute("/tablet?id=$0", UrlEncodeToString(status.tablet_id()));
+        link_json["url"] = Substitute("/tablet?id=$0", status.tablet_id());
       }
       replica_json["partition"] =
           tmeta->partition_schema().PartitionDebugString(tmeta->partition(),
diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index e45d8ce..f31fdac 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -231,6 +231,7 @@ set(UTIL_SRCS
   url-coding.cc
   version_info.cc
   version_util.cc
+  web_callback_registry.cc
   website_util.cc
   yamlreader.cc
   zlib.cc
diff --git a/src/kudu/util/thread.cc b/src/kudu/util/thread.cc
index fb68977..223fbab 100644
--- a/src/kudu/util/thread.cc
+++ b/src/kudu/util/thread.cc
@@ -443,7 +443,16 @@ void ThreadMgr::ThreadPathHandler(const WebCallbackRegistry::WebRequest& req,
     EasyJson groups = output.Set("groups", EasyJson::kArray);
     for (const auto& elem : thread_categories_info) {
       string category_arg;
-      UrlEncode(elem.first, &category_arg);
+      if (WebCallbackRegistry::IsProxiedViaKnox(req)) {
+        // Knox encodes query parameter values when it rewrites HTTP responses.
+        // If we also encoded, we'd end up with broken URLs. For example, we'd
+        // encode the query parameter 'group=service pool' to
+        // 'group=service%20pool', then Knox would encode it again to
+        // 'group=service%2520pool'.
+        category_arg = elem.first;
+      } else {
+        UrlEncode(elem.first, &category_arg);
+      }
       EasyJson g = groups.PushBack(EasyJson::kObject);
       g["encoded_group_name"] = category_arg;
       g["group_name"] = elem.first;
diff --git a/src/kudu/util/web_callback_registry.cc b/src/kudu/util/web_callback_registry.cc
new file mode 100644
index 0000000..b6e4e7b
--- /dev/null
+++ b/src/kudu/util/web_callback_registry.cc
@@ -0,0 +1,28 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/util/web_callback_registry.h"
+
+#include "kudu/gutil/map-util.h"
+
+namespace kudu {
+
+bool WebCallbackRegistry::IsProxiedViaKnox(const WebRequest& req) {
+  return ContainsKey(req.request_headers, "x-forwarded-context");
+}
+
+} // namespace kudu
diff --git a/src/kudu/util/web_callback_registry.h b/src/kudu/util/web_callback_registry.h
index f48a628..3b7ff13 100644
--- a/src/kudu/util/web_callback_registry.h
+++ b/src/kudu/util/web_callback_registry.h
@@ -14,10 +14,10 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_UTIL_WEB_CALLBACK_REGISTRY_H
-#define KUDU_UTIL_WEB_CALLBACK_REGISTRY_H
 
-#include <iosfwd>
+#pragma once
+
+#include <sstream>
 #include <string>
 #include <unordered_map>
 
@@ -54,6 +54,9 @@ class WebCallbackRegistry {
     // The query string, parsed into key/value argument pairs.
     ArgumentMap parsed_args;
 
+    // The HTTP request headers.
+    ArgumentMap request_headers;
+
     // The raw query string passed in the URL. May be empty.
     std::string query_string;
 
@@ -64,15 +67,13 @@ class WebCallbackRegistry {
     std::string post_data;
   };
 
-  typedef std::unordered_map<std::string, std::string> HttpResponseHeaders;
-
   // A response to an HTTP request whose body is rendered by template.
   struct WebResponse {
     // Determines the status code of the HTTP response.
     HttpStatusCode status_code = HttpStatusCode::Ok;
 
     // Additional headers added to the HTTP response.
-    HttpResponseHeaders response_headers;
+    ArgumentMap response_headers;
 
     // A JSON object to be rendered to HTML by a mustache template.
     EasyJson output;
@@ -84,7 +85,7 @@ class WebCallbackRegistry {
     HttpStatusCode status_code = HttpStatusCode::Ok;
 
     // Additional headers added to the HTTP response.
-    HttpResponseHeaders response_headers;
+    ArgumentMap response_headers;
 
     // The fully-rendered HTML response body.
     std::ostringstream output;
@@ -123,8 +124,9 @@ class WebCallbackRegistry {
                                               const PrerenderedPathHandlerCallback& callback,
                                               bool is_styled,
                                               bool is_on_nav_bar) = 0;
+
+  // Returns true if 'req' was proxied via Knox, false otherwise.
+  static bool IsProxiedViaKnox(const WebRequest& req);
 };
 
 } // namespace kudu
-
-#endif /* KUDU_UTIL_WEB_CALLBACK_REGISTRY_H */
diff --git a/thirdparty/build-definitions.sh b/thirdparty/build-definitions.sh
index d81b01c..bdd7a4e 100644
--- a/thirdparty/build-definitions.sh
+++ b/thirdparty/build-definitions.sh
@@ -645,7 +645,7 @@ build_mustache() {
   mkdir -p $MUSTACHE_BDIR
   pushd $MUSTACHE_BDIR
   # We add $PREFIX/include for boost and $PREFIX_COMMON/include for rapidjson.
-  ${CXX:-g++} $EXTRA_CXXFLAGS -I$PREFIX/include -I$PREFIX_COMMON/include -O3 -DNDEBUG -fPIC -c "$MUSTACHE_SOURCE/mustache.cc"
+  ${CXX:-g++} -std=c++11 $EXTRA_CXXFLAGS -I$PREFIX/include -I$PREFIX_COMMON/include -O3 -DNDEBUG -fPIC -c "$MUSTACHE_SOURCE/mustache.cc"
   ar rs libmustache.a mustache.o
   cp libmustache.a $PREFIX/lib/
   cp $MUSTACHE_SOURCE/mustache.h $PREFIX/include/
diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh
index d0d2e45..af13b32 100644
--- a/thirdparty/vars.sh
+++ b/thirdparty/vars.sh
@@ -108,7 +108,7 @@ SQUEASEL_SOURCE=$TP_SOURCE_DIR/$SQUEASEL_NAME
 #  export NAME=mustache-$(git rev-parse HEAD)
 #  git archive HEAD --prefix=$NAME/ -o /tmp/$NAME.tar.gz
 #  s3cmd put -P /tmp/$NAME.tar.gz s3://cloudera-thirdparty-libs/$NAME.tar.gz
-MUSTACHE_VERSION=87a592e8aa04497764c533acd6e887618ca7b8a8
+MUSTACHE_VERSION=b290952d8eb93d085214d8c8c9eab8559df9f606
 MUSTACHE_NAME=mustache-$MUSTACHE_VERSION
 MUSTACHE_SOURCE=$TP_SOURCE_DIR/$MUSTACHE_NAME
 
diff --git a/www/dashboards.mustache b/www/dashboards.mustache
index d86ae27..bb6e8a8 100644
--- a/www/dashboards.mustache
+++ b/www/dashboards.mustache
@@ -21,7 +21,7 @@ under the License.
 <table class='table table-striped'>
   <thead><tr><th>Dashboard</th><th>Description</th></tr></thead>
   <tbody
-  <tr><td><a href="/scans">Scans</a></td><td>List of currently running and recently completed scans.</td></tr>
-  <tr><td><a href="/transactions">Transactions</a></td><td>List of transactions that are currently running.</td></tr>
-  <tr><td><a href="/maintenance-manager">Maintenance Manager</a></td><td>List of operations that are currently running and those that are registered.</td></tr>
+  <tr><td><a href="{{base_url}}/scans">Scans</a></td><td>List of currently running and recently completed scans.</td></tr>
+  <tr><td><a href="{{base_url}}/transactions">Transactions</a></td><td>List of transactions that are currently running.</td></tr>
+  <tr><td><a href="{{base_url}}/maintenance-manager">Maintenance Manager</a></td><td>List of operations that are currently running and those that are registered.</td></tr>
 </tbody></table>
diff --git a/www/home.mustache b/www/home.mustache
index 831d100..bedb258 100644
--- a/www/home.mustache
+++ b/www/home.mustache
@@ -19,7 +19,7 @@ under the License.
 
 <h2>Status Pages</h2>
 {{#path_handlers}}
-<a href="{{path}}">{{alias}}</a><br/>
+<a href="{{base_url}}{{path}}">{{alias}}</a><br/>
 {{/path_handlers}}
 <hr/>
 <h2>Version Info</h2>
diff --git a/www/log-anchors.mustache b/www/log-anchors.mustache
index a70ce0e..e4ab7f4 100644
--- a/www/log-anchors.mustache
+++ b/www/log-anchors.mustache
@@ -20,6 +20,6 @@ under the License.
   <div class="text-error">{{.}}</div>
 {{/error}}
 {{^error}}
-  <h1>Log Anchors for Tablet <a href="/tablet?id={{tablet_id}}">{{tablet_id}}</a></h1>
+  <h1>Log Anchors for Tablet <a href="{{base_url}}/tablet?id={{tablet_id}}">{{tablet_id}}</a></h1>
   <pre>{{log_anchors}}</pre>
 {{/error}}
diff --git a/www/scans.mustache b/www/scans.mustache
index f12705f..4e97978 100644
--- a/www/scans.mustache
+++ b/www/scans.mustache
@@ -35,7 +35,7 @@ under the License.
   <tbody>
     {{#scans}}
     <tr>
-      <td><a href="/tablet?id={{tablet_id}}"><samp>{{tablet_id}}</samp></a></td>
+      <td><a href="{{base_url}}/tablet?id={{tablet_id}}"><samp>{{tablet_id}}</samp></a></td>
       <td><samp>{{scanner_id}}</samp></td>
       <td>{{state}}</td>
       {{! The query string is pre-formatted HTML, so don't escape it (triple-brace). }}
diff --git a/www/table.mustache b/www/table.mustache
index c940513..b96f6e3 100644
--- a/www/table.mustache
+++ b/www/table.mustache
@@ -53,7 +53,7 @@ under the License.
     {{#columns}}
       <tr>
         <th>
-          {{#is_key}}<img src="key.png" width=12 height=6 />&nbsp;&nbsp;{{/is_key}}{{name}}
+          {{#is_key}}<img src="{{base_url}}/key.png" width=12 height=6 />&nbsp;&nbsp;{{/is_key}}{{name}}
         </th>
         <td>{{id}}</a></td>
         <td>{{type}}</td>
diff --git a/www/tables.mustache b/www/tables.mustache
index d20d935..44c8d21 100644
--- a/www/tables.mustache
+++ b/www/tables.mustache
@@ -41,7 +41,7 @@ There are {{num_tables}} tables.
    {{#tables}}
     <tr>
       <td>{{name}}</td>
-      <td><a href="/table?id={{id}}">{{id}}</a></td>
+      <td><a href="{{base_url}}/table?id={{id}}">{{id}}</a></td>
       <td>{{state}}</td>
       <td>{{message}}</td>
       <td>{{create time}}</td>
diff --git a/www/tablet-rowsetlayout-svg.mustache b/www/tablet-rowsetlayout-svg.mustache
index dc81f11..f52721b 100644
--- a/www/tablet-rowsetlayout-svg.mustache
+++ b/www/tablet-rowsetlayout-svg.mustache
@@ -20,6 +20,6 @@ under the License.
   <div class="text-error">{{.}}</div>
 {{/error}}
 {{^error}}
-  <h1>Rowset Layout Diagram for Tablet <a href="/tablet?id={{tablet_id}}">{{tablet_id}}</a></h1>
+  <h1>Rowset Layout Diagram for Tablet <a href="{{base_url}}/tablet?id={{tablet_id}}">{{tablet_id}}</a></h1>
   {{{rowset_layout}}}
 {{/error}}
diff --git a/www/tablet-servers.mustache b/www/tablet-servers.mustache
index 6a3d16e..65a1ace 100644
--- a/www/tablet-servers.mustache
+++ b/www/tablet-servers.mustache
@@ -95,7 +95,7 @@ under the License.
     <tbody>
     {{#dead_tservers}}
       <tr>
-        <td>{{#target}}<a href="{{.}}">{{/target}}{{uuid}}{{#target}}</a>{{/target}}</td>
+        <td>{{#target}}<a href="{{.}}{{base_url}}">{{/target}}{{uuid}}{{#target}}</a>{{/target}}</td>
         <td>{{time_since_hb}}</td>
         <td><pre><code>{{registration}}</code></pre></td>
       </tr>
diff --git a/www/tablet.mustache b/www/tablet.mustache
index 2483d0f..d4a0124 100644
--- a/www/tablet.mustache
+++ b/www/tablet.mustache
@@ -45,7 +45,7 @@ under the License.
     {{#columns}}
       <tr>
         <th>
-          {{#is_key}}<img src="key.png" width=12 height=6 />&nbsp;&nbsp;{{/is_key}}{{name}}
+          {{#is_key}}<img src="{{base_url}}/key.png" width=12 height=6 />&nbsp;&nbsp;{{/is_key}}{{name}}
         </th>
         <td>{{id}}</a></td>
         <td>{{type}}</td>
@@ -62,13 +62,13 @@ under the License.
   <h2>Other Tablet Info Pages</h2>
   <ul>
     <li>
-      <a href="tablet-rowsetlayout-svg?id={{tablet_id}}">Rowset Layout Diagram</a>
+      <a href="{{base_url}}/tablet-rowsetlayout-svg?id={{tablet_id}}">Rowset Layout Diagram</a>
     </li>
     <li>
-      <a href="tablet-consensus-status?id={{tablet_id}}">Consensus Status</a>
+      <a href="{{base_url}}/tablet-consensus-status?id={{tablet_id}}">Consensus Status</a>
     </li>
     <li>
-      <a href="log-anchors?id={{tablet_id}}">Tablet Log Anchors</a>
+      <a href="{{base_url}}/log-anchors?id={{tablet_id}}">Tablet Log Anchors</a>
     </li>
   </ul>
 {{/error}}
diff --git a/www/tablets.mustache b/www/tablets.mustache
index 2a54ada..63c52f9 100644
--- a/www/tablets.mustache
+++ b/www/tablets.mustache
@@ -48,7 +48,7 @@ There are no tablet replicas.
       <tr>
         <td>{{table_name}}</td>
         <td>
-          {{#link}}<a href="{{url}}">{{id}}</a>{{/link}}
+          {{#link}}<a href="{{base_url}}{{url}}">{{id}}</a>{{/link}}
           {{^link}}{{id}}{{/link}}
         </td>
         <td>{{partition}}</td>
@@ -94,7 +94,7 @@ There are no tablet replicas.
       <tr>
         <td>{{table_name}}</td>
         <td>
-          {{#link}}<a href="{{url}}">{{id}}</a>{{/link}}
+          {{#link}}<a href="{{base_url}}{{url}}">{{id}}</a>{{/link}}
           {{^link}}{{id}}{{/link}}
         </td>
         <td>{{partition}}</td>
diff --git a/www/threadz.mustache b/www/threadz.mustache
index efabdd7..77f575f 100644
--- a/www/threadz.mustache
+++ b/www/threadz.mustache
@@ -48,7 +48,7 @@ under the License.
 {{^requested_thread_group}}
 <h2>Thread Groups</h2>
 <h4>{{total_threads_running}} thread(s) running</h4>
-<a href='/threadz?group=all'><h3>All Threads</h3></a>
+<a href='{{base_url}}/threadz?group=all'><h3>All Threads</h3></a>
 <table class='table table-hover' data-sort-name='group' data-toggle='table'>
   <thead>
     <tr>
@@ -59,7 +59,7 @@ under the License.
   <tbody>
     {{#groups}}
     <tr>
-      <td><a href='/threadz?group={{encoded_group_name}}'>{{group_name}}</a></td>
+      <td><a href='{{base_url}}/threadz?group={{encoded_group_name}}'>{{group_name}}</a></td>
       <td>{{threads_running}}</td>
     </tr>
     {{/groups}}