You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by tm...@apache.org on 2019/09/10 18:43:45 UTC

[impala] branch master updated: IMPALA-8917: Remove hostname from webui links if Knox isn't being used

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

tmarshall 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 de77c61  IMPALA-8917: Remove hostname from webui links if Knox isn't being used
de77c61 is described below

commit de77c61f383794601c79d9783d36235482740417
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
AuthorDate: Wed Sep 4 15:57:18 2019 -0700

    IMPALA-8917: Remove hostname from webui links if Knox isn't being used
    
    IMPALA-8897 added the hostname to all links on the debug webui in
    order to facilitate proxying connections through Apache Knox. This
    makes the webui difficult to use in situations where the hostname is
    not DNS-resolvable.
    
    This patch fixes this by only including the hostname with links if
    Knox proxying is actually being used, which we determine by looking
    for the 'x-forwarded-context' header in the request, which Knox adds
    to all requests.
    
    It also removes the hidden form fields that were added to support Knox
    integration when not being accessed through Knox.
    
    It also adds a class comment on Webserver explaining the requirements
    for keeping the webui compatible with Knox.
    
    Testing:
    - Added a test that checks that links on the webui are made absolute
      when the 'x-forwarded-context' header is in the request.
    
    Change-Id: Ifcf77058dc6ce1d72422a9e3ca7868cdffacff76
    Reviewed-on: http://gerrit.cloudera.org:8080/14199
    Reviewed-by: Thomas Tauber-Marshall <tm...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/util/webserver.cc          | 45 +++++++++++++++++++++++----------------
 be/src/util/webserver.h           | 20 +++++++++++++++--
 tests/webserver/test_web_pages.py | 26 ++++++++++++++++------
 www/form-hidden-inputs.tmpl       |  4 +++-
 4 files changed, 68 insertions(+), 27 deletions(-)

diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index 4873447..d510ee9 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -431,26 +431,34 @@ void Webserver::Init() {
       "$0://$1:$2", IsSecure() ? "https" : "http", hostname_, http_address_.port);
 }
 
-void Webserver::GetCommonJson(Document* document) {
+void Webserver::GetCommonJson(
+    Document* document, const struct sq_connection* connection, const WebRequest& req) {
   DCHECK(document != nullptr);
   Value obj(kObjectType);
   obj.AddMember("process-name",
       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());
+  // If Apacke Knox is being used to proxy connections to the webserver, the
+  // 'x-forwarded-context' header will be present.
+  if (sq_get_header(connection, "x-forwarded-context")) {
+    // When proxying connections through Apache Knox, we make all links on the webui
+    // absolute, which allows Knox to rewrite the links to point to the Knox host while
+    // including 'scheme', 'host', and 'port' parameters which tell Knox where do forward
+    // the request to.
+    Value url_value(url().c_str(), document->GetAllocator());
+    obj.AddMember("host-url", url_value, document->GetAllocator());
+
+    // These are used to add hidden form fields when Knox is being used to add the 'host'
+    // and related parameters to the form's request.
+    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_) {
@@ -575,7 +583,7 @@ sq_callback_result_t Webserver::BeginRequestCallback(struct sq_connection* conne
     content_type = PLAIN;
     url_handler->raw_callback()(req, &output, &response);
   } else {
-    RenderUrlWithTemplate(req, *url_handler, &output, &content_type);
+    RenderUrlWithTemplate(connection, req, *url_handler, &output, &content_type);
   }
 
   VLOG(3) << "Rendering page " << request_info->uri << " took "
@@ -679,11 +687,12 @@ sq_callback_result_t Webserver::HandleSpnego(
   return SQ_CONTINUE_HANDLING;
 }
 
-void Webserver::RenderUrlWithTemplate(const WebRequest& req,
-    const UrlHandler& url_handler, stringstream* output, ContentType* content_type) {
+void Webserver::RenderUrlWithTemplate(const struct sq_connection* connection,
+    const WebRequest& req, const UrlHandler& url_handler, stringstream* output,
+    ContentType* content_type) {
   Document document;
   document.SetObject();
-  GetCommonJson(&document);
+  GetCommonJson(&document, connection, req);
 
   const auto& arguments = req.parsed_args;
   url_handler.callback()(req, &document);
diff --git a/be/src/util/webserver.h b/be/src/util/webserver.h
index 7422674..1782cd9 100644
--- a/be/src/util/webserver.h
+++ b/be/src/util/webserver.h
@@ -45,6 +45,20 @@ enum ContentType {
 /// Wrapper class for the Squeasel web server library. Clients may register callback
 /// methods which produce Json documents which are rendered via a template file to either
 /// HTML or text.
+///
+/// Apache Knox Integration
+/// -----------------------
+/// In order to be compatible with the Knox 'impalaui' service definition, there are a few
+/// requirements that template files served by this webserver have to conform to:
+/// - Any relative links to pages on this server must be proceeded by
+///   {{ __common__.host-url }} so that they can be made into absolute urls when Knox
+///   proxying is being used.
+/// - Any forms must contain {{>www/form-hidden-inputs.tmpl}}, which adds some hidden
+///   fields when Knox proxying is being used.
+/// - Any relative urls that are accessed via javascript should be constructed with the
+///   make_url() function in common-header.tmpl.
+/// See: https://github.com/apache/knox/tree/master/gateway-service-definitions/
+///     src/main/resources/services/impalaui/1.0.0
 class Webserver {
  public:
   using ArgumentMap = kudu::WebCallbackRegistry::ArgumentMap;
@@ -179,7 +193,8 @@ class Webserver {
   /// - Argument 'raw' renders the page with PLAIN ContentType.
   /// - Argument 'json' renders the page with JSON ContentType. The underlying JSON is
   ///   pretty-printed.
-  void RenderUrlWithTemplate(const WebRequest& arguments, const UrlHandler& url_handler,
+  void RenderUrlWithTemplate(const struct sq_connection* connection,
+      const WebRequest& arguments, const UrlHandler& url_handler,
       std::stringstream* output, ContentType* content_type);
 
   /// Called when an error is encountered, e.g. when a handler for a URI cannot be found.
@@ -192,7 +207,8 @@ class Webserver {
 
   /// Adds a __common__ object to document with common data that every webpage might want
   /// to read (e.g. the names of links to write to the navbar).
-  void GetCommonJson(rapidjson::Document* document);
+  void GetCommonJson(rapidjson::Document* document,
+      const struct sq_connection* connection, const WebRequest& req);
 
   /// Lock guarding the path_handlers_ map
   boost::shared_mutex url_handlers_lock_;
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
index f3c8545..98d4801 100644
--- a/tests/webserver/test_web_pages.py
+++ b/tests/webserver/test_web_pages.py
@@ -149,7 +149,8 @@ class TestWebPage(ImpalaTestSuite):
       except ValueError:
         assert False, "Invalid JSON returned from /jmx endpoint: %s" % jmx_json
 
-  def get_and_check_status(self, url, string_to_search="", ports_to_test=None):
+  def get_and_check_status(
+      self, url, string_to_search="", ports_to_test=None, regex=False, headers=None):
     """Helper method that polls a given url and asserts the return code is ok and
     the response contains the input string."""
     if ports_to_test is None:
@@ -158,11 +159,15 @@ class TestWebPage(ImpalaTestSuite):
     responses = []
     for port in ports_to_test:
       input_url = url.format(port)
-      response = requests.get(input_url)
+      response = requests.get(input_url, headers=headers)
       assert response.status_code == requests.codes.ok, "URL: {0} Str:'{1}'\nResp:{2}"\
         .format(input_url, string_to_search, response.text)
-      assert string_to_search in response.text, "URL: {0} Str:'{1}'\nResp:{2}".format(
-        input_url, string_to_search, response.text)
+      if regex:
+        assert re.search(string_to_search, response.text), "URL: {0} Str:'{1}'\nResp:{2}"\
+          .format(input_url, string_to_search, response.text)
+      else:
+        assert string_to_search in response.text, "URL: {0} Str:'{1}'\nResp:{2}".format(
+          input_url, string_to_search, response.text)
       responses.append(response)
     return responses
 
@@ -658,8 +663,8 @@ class TestWebPage(ImpalaTestSuite):
     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
+  def test_knox_compatibility(self):
+    """Checks that the template files conform to the requirements for compatibility 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 }}'
@@ -674,3 +679,12 @@ class TestWebPage(ImpalaTestSuite):
     results = grep_dir(os.path.join(os.environ['IMPALA_HOME'], "www"), regex, ".*\.tmpl")
     assert len(results) == 0, \
         "All links on the webui must include the webserver host: %s" % results
+
+    # Check that when Knox integration is not being used, the links are relative, by
+    # checking for the root link from the header.
+    self.get_and_check_status(self.ROOT_URL, "href='/'", self.IMPALAD_TEST_PORT)
+    # Check that if the 'x-forwarded-context' header is present in the request, the links
+    # are written as absolute.
+    self.get_and_check_status(self.ROOT_URL,
+        "href='http://.*:%s/'" % self.IMPALAD_TEST_PORT[0], self.IMPALAD_TEST_PORT,
+        regex=True, headers={'X-Forwarded-Context': '/gateway'})
diff --git a/www/form-hidden-inputs.tmpl b/www/form-hidden-inputs.tmpl
index 83b1fea..961bed9 100644
--- a/www/form-hidden-inputs.tmpl
+++ b/www/form-hidden-inputs.tmpl
@@ -1,3 +1,5 @@
+{{# __common__.hostname }}
 <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
+<input type='hidden' name='port' value='{{ __common__.port }}' />
+{{/ __common__.hostname }}
\ No newline at end of file