You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by la...@apache.org on 2021/07/29 16:06:26 UTC

[impala] 03/04: IMPALA-10779: Print the username closing a session or cancelling a query from the WebUI

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

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

commit 8a2758f1e88500867d8d2fbaffb62493fca2215a
Author: Fucun Chu <ch...@hotmail.com>
AuthorDate: Fri Jul 23 23:11:54 2021 +0800

    IMPALA-10779: Print the username closing a session or cancelling a query from the WebUI
    
    This patch appends the username of the client who made the request to
    close a session or cancel a query from the coordinator's debug WebUI.
    
    Tests:
    - Added a new fe test for LDAP auth to verify that the new status gets
      printed in runtime profile and coordinator log when a query is
      cancelled in this way.
    
    Change-Id: I02c92b5caee61d1f9f381cd2906a850e02c54d55
    Reviewed-on: http://gerrit.cloudera.org:8080/17726
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/kudu/util/web_callback_registry.h           |  3 +
 be/src/service/impala-http-handler.cc              |  8 +--
 be/src/util/webserver.cc                           |  3 +
 .../apache/impala/customcluster/LdapHS2Test.java   | 13 ++++
 .../impala/customcluster/LdapWebserverTest.java    | 69 ++++++++++++++++++++++
 tests/webserver/test_web_pages.py                  |  7 ++-
 6 files changed, 96 insertions(+), 7 deletions(-)

diff --git a/be/src/kudu/util/web_callback_registry.h b/be/src/kudu/util/web_callback_registry.h
index d2e4ecc..75fa5a6 100644
--- a/be/src/kudu/util/web_callback_registry.h
+++ b/be/src/kudu/util/web_callback_registry.h
@@ -68,6 +68,9 @@ class WebCallbackRegistry {
 
     // The socket address of the requester, <host>:<port>.
     std::string source_socket;
+
+    // Authenticated user, or 'anonymous' if no auth used
+    std::string source_user = "anonymous";
   };
 
   // A response to an HTTP request whose body is rendered by template.
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index 677f50c..caffd7b 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -247,8 +247,8 @@ void ImpalaHttpHandler::CancelQueryHandler(const Webserver::WebRequest& req,
     document->AddMember("error", error, document->GetAllocator());
     return;
   }
-  Status cause(Substitute("Cancelled from Impala's debug web interface by client at $0"
-                           , req.source_socket));
+  Status cause(Substitute("Cancelled from Impala's debug web interface by user:"
+                          " '$0' at $1", req.source_user, req.source_socket));
   // Web UI doesn't have access to secret so we can't validate it. We assume that
   // web UI is allowed to close queries.
   status = server_->UnregisterQuery(unique_id, true, &cause);
@@ -270,8 +270,8 @@ void ImpalaHttpHandler::CloseSessionHandler(const Webserver::WebRequest& req,
     document->AddMember("error", error, document->GetAllocator());
     return;
   }
-  Status cause(Substitute("Session closed from Impala's debug web interface by client at"
-                          " $0", req.source_socket));
+  Status cause(Substitute("Session closed from Impala's debug web interface by user:"
+                          " '$0' at $1", req.source_user, req.source_socket));
   // Web UI doesn't have access to secret so we can't validate it. We assume that
   // web UI is allowed to close sessions.
   status = server_->CloseSessionInternal(unique_id,
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index 5a6c1e2..27c7af9 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -730,6 +730,9 @@ sq_callback_result_t Webserver::BeginRequestCallback(struct sq_connection* conne
     req.query_string = request_info->query_string;
     BuildArgumentMap(request_info->query_string, &req.parsed_args);
   }
+  if (request_info->remote_user != nullptr) {
+    req.source_user = request_info->remote_user;
+  }
 
   HttpStatusCode response = HttpStatusCode::Ok;
   ContentType content_type = HTML;
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapHS2Test.java b/fe/src/test/java/org/apache/impala/customcluster/LdapHS2Test.java
index 8757b1e..f7848d0 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapHS2Test.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapHS2Test.java
@@ -97,6 +97,19 @@ public class LdapHS2Test {
     return execResp.getOperationHandle();
   }
 
+  /**
+   * Executes async 'query'.
+   */
+  static TOperationHandle execQueryAsync(TCLIService.Iface client,
+      TSessionHandle sessionHandle, String query)
+      throws Exception {
+    TExecuteStatementReq execReq = new TExecuteStatementReq(sessionHandle, query);
+    TExecuteStatementResp execResp = client.ExecuteStatement(execReq);
+    verifySuccess(execResp.getStatus());
+
+    return execResp.getOperationHandle();
+  }
+
   private void verifyMetrics(long expectedBasicAuthSuccess, long expectedBasicAuthFailure)
       throws Exception {
     long actualBasicAuthSuccess = (long) metrics.getMetric(
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java b/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java
index 6e68db6..4e50d8f 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java
@@ -28,16 +28,22 @@ import java.net.URL;
 import java.net.URLConnection;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Range;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.directory.server.core.annotations.CreateDS;
 import org.apache.directory.server.core.annotations.CreatePartition;
 import org.apache.directory.server.annotations.CreateLdapServer;
 import org.apache.directory.server.annotations.CreateTransport;
 import org.apache.directory.server.core.annotations.ApplyLdifFiles;
 import org.apache.directory.server.core.integ.CreateLdapServerRule;
+import org.apache.hive.service.rpc.thrift.*;
 import org.apache.impala.util.Metrics;
 import org.apache.log4j.Logger;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.transport.THttpClient;
 import org.junit.After;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -297,6 +303,56 @@ public class LdapWebserverTest {
     verifyJwtAuthMetrics(Range.closed(1L, 1L), Range.closed(1L, 1L));
   }
 
+  /**
+   * Print the username closing a session or cancelling a query from the WebUI.
+   */
+  @Test
+  public void testDisplaySrcUsernameInQueryCause() throws Exception {
+    setUp("", "");
+    // Create client
+    THttpClient transport = new THttpClient("http://localhost:28000");
+    Map<String, String> headers = new HashMap<String, String>();
+    // Authenticate as 'Test1Ldap' with password '12345'
+    headers.put("Authorization", "Basic VGVzdDFMZGFwOjEyMzQ1");
+    transport.setCustomHeaders(headers);
+    transport.open();
+    TCLIService.Iface client = new TCLIService.Client(new TBinaryProtocol(transport));
+
+    // Open a session which will get username 'Test1Ldap'.
+    TOpenSessionReq openReq = new TOpenSessionReq();
+    TOpenSessionResp openResp = client.OpenSession(openReq);
+
+    // Execute a long running query then cancel it from the WebUI.
+    // Check the runtime profile and the INFO logs for the cause message.
+    TOperationHandle operationHandle = LdapHS2Test.execQueryAsync(
+        client, openResp.getSessionHandle(), "select sleep(10000)");
+    String queryId = PrintId(operationHandle.getOperationId());
+    String cancelQueryUrl = String.format("/cancel_query?query_id=%s", queryId);
+    String textProfileUrl = String.format("/query_profile_plain_text?query_id=%s",
+            queryId);
+    metrics_.readContent(cancelQueryUrl);
+    String response =  metrics_.readContent(textProfileUrl);
+    String cancelStatus = String.format("Cancelled from Impala&apos;s debug web interface"
+        + " by user: &apos;%s&apos; at", TEST_USER_1);
+    assertTrue(response.contains(cancelStatus));
+    // Wait for logs to flush
+    TimeUnit.SECONDS.sleep(6);
+    response = metrics_.readContent("/logs");
+    assertTrue(response.contains(cancelStatus));
+
+    // Session closing from the WebUI does not produce the cause message in the profile,
+    // so we will skip checking the runtime profile.
+    String sessionId = PrintId(openResp.getSessionHandle().getSessionId());
+    String closeSessionUrl =  String.format("/close_session?session_id=%s", sessionId);
+    metrics_.readContent(closeSessionUrl);
+    // Wait for logs to flush
+    TimeUnit.SECONDS.sleep(6);
+    String closeStatus = String.format("Session closed from Impala&apos;s debug web"
+        + " interface by user: &apos;%s&apos; at", TEST_USER_1);
+    response = metrics_.readContent("/logs");
+    assertTrue(response.contains(closeStatus));
+  }
+
   // Helper method to make a get call to the webserver using the input basic
   // auth token and x-forward-for token.
   private void attemptConnection(String basic_auth_token, String xff_address)
@@ -311,4 +367,17 @@ public class LdapWebserverTest {
     }
     connection.getInputStream();
   }
+
+  // Helper method to get query id or session id
+  private static String PrintId(THandleIdentifier handle) {
+    // The binary representation is present in the query handle but we need to
+    // massage it into the expected string representation.
+    byte[] guid_bytes = handle.getGuid();
+    assertEquals(guid_bytes.length,16);
+    byte[] low_bytes = ArrayUtils.subarray(guid_bytes, 0, 8);
+    byte[] high_bytes = ArrayUtils.subarray(guid_bytes, 8, 16);
+    ArrayUtils.reverse(low_bytes);
+    ArrayUtils.reverse(high_bytes);
+    return Hex.encodeHexString(low_bytes) + ":" + Hex.encodeHexString(high_bytes);
+  }
 }
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
index 01661a4..bfed002 100644
--- a/tests/webserver/test_web_pages.py
+++ b/tests/webserver/test_web_pages.py
@@ -823,10 +823,11 @@ class TestWebPage(ImpalaTestSuite):
       .format("25000"), query_id)
     requests.get(cancel_query_url)
     response = requests.get(text_profile_url)
-    cancel_status = "Cancelled from Impala&apos;s debug web interface by client at"
+    cancel_status = "Cancelled from Impala&apos;s debug web interface by user: " \
+                    "&apos;anonymous&apos; at"
     assert cancel_status in response.text
     self.assert_impalad_log_contains("INFO", "Cancelled from Impala\'s debug web "
-      "interface by client at", expected_count=-1)
+      "interface by user: 'anonymous' at", expected_count=-1)
     # Session closing from the WebUI does not produce the cause message in the profile,
     # so we will skip checking the runtime profile.
     results = self.execute_query("select current_session()")
@@ -835,7 +836,7 @@ class TestWebPage(ImpalaTestSuite):
       ("25000"), session_id)
     requests.get(close_session_url)
     self.assert_impalad_log_contains("INFO", "Session closed from Impala\'s debug "
-      "web interface by client at", expected_count=-1)
+      "web interface by user: 'anonymous' at", expected_count=-1)
 
   def test_catalog_operations_endpoint(self):
     """Test to check that the /operations endpoint returns 200 OK."""