You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by wz...@apache.org on 2022/05/10 23:27:38 UTC

[impala] branch master updated (e6ed98c22 -> 2abcf3194)

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

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


    from e6ed98c22 IMPALA-11201: update gitignore files
     new 397d1d15a IMPALA-10745: Support Kerberos over HTTP for impala-shell
     new 2abcf3194 IMPALA-11248: Fix infinite warning dialog issue when datatable cells have undefined values in www/rpcz web page

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 infra/python/deps/requirements.txt |   1 +
 shell/ImpalaHttpClient.py          | 233 ++++++++++++++++++++++++++-----------
 shell/impala_client.py             |  22 ++--
 shell/impala_shell.py              |   3 +-
 shell/packaging/requirements.txt   |   1 +
 shell/shell_exceptions.py          |   8 ++
 www/rpcz.tmpl                      |  48 ++++----
 7 files changed, 209 insertions(+), 107 deletions(-)


[impala] 02/02: IMPALA-11248: Fix infinite warning dialog issue when datatable cells have undefined values in www/rpcz web page

Posted by wz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 2abcf31949ecdda95bf60e0ef463bdc168b53bf1
Author: carolinchen <ca...@tencent.com>
AuthorDate: Wed Apr 27 10:58:14 2022 +0800

    IMPALA-11248: Fix infinite warning dialog issue when datatable cells have undefined values in www/rpcz web page
    
    This patch fixes datatables script show warning dialog repeatedly in
    some kernel environment, we find the issue in centOS linux release 7.2
    and tencent tlinux release 2.2. If the javascript without initial value
    set, then the front-end page prompts that the variable is not defined,
    which cause the datatables script to repeatedly show warning dialog
    pop up with message such as follow:
    
    DataTables warning: table id=inbound_per_conn_metrics - Requested
    unknown parameter '6' for row 0, column 6.
    For more information about this error, please see
    http://datatables.net/tn/4
    
    Testing:
    - Ran Q4 of TPC-DS against tpcds_parquet table manually. Verified that
    the page display the KRPC inbound/outbound connections table properly
    without any warning dialog and auto-refresh normally.
    - Ran exhaustive tests with the patch.
    
    Change-Id: I7f284b4419ba7dd72baa960dcd5700da22dccdff
    Reviewed-on: http://gerrit.cloudera.org:8080/18452
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 www/rpcz.tmpl | 48 ++++++++++++++++++++++++------------------------
 1 file changed, 24 insertions(+), 24 deletions(-)

diff --git a/www/rpcz.tmpl b/www/rpcz.tmpl
index cbe704f06..f2e2e19e2 100644
--- a/www/rpcz.tmpl
+++ b/www/rpcz.tmpl
@@ -298,18 +298,18 @@ function update_krpc_conn_metrics_datatable(json) {
   var table = $('#per_conn_metrics').DataTable();
   var rows = $.map(json["per_conn_metrics"], function(row) {
     return [[row["remote_ip"], row["num_calls_in_flight"], row["outbound_queue_size"],
-             row["socket_stats"]["rtt"],
-             row["socket_stats"]["rttvar"],
-             row["socket_stats"]["snd_cwnd"],
-             row["socket_stats"]["total_retrans"],
-             row["socket_stats"]["pacing_rate"],
-             row["socket_stats"]["max_pacing_rate"],
-             row["socket_stats"]["bytes_acked"],
-             row["socket_stats"]["bytes_received"],
-             row["socket_stats"]["segs_out"],
-             row["socket_stats"]["segs_in"],
-             row["socket_stats"]["send_queue_bytes"],
-             row["socket_stats"]["receive_queue_bytes"]]];
+             row["socket_stats"]["rtt"] ?? '',
+             row["socket_stats"]["rttvar"] ?? '',
+             row["socket_stats"]["snd_cwnd"] ?? '',
+             row["socket_stats"]["total_retrans"] ?? '',
+             row["socket_stats"]["pacing_rate"] ?? '',
+             row["socket_stats"]["max_pacing_rate"] ?? '',
+             row["socket_stats"]["bytes_acked"] ?? '',
+             row["socket_stats"]["bytes_received"] ?? '',
+             row["socket_stats"]["segs_out"] ?? '',
+             row["socket_stats"]["segs_in"] ?? '',
+             row["socket_stats"]["send_queue_bytes"] ?? '',
+             row["socket_stats"]["receive_queue_bytes"] ?? '']];
   });
 
   table.clear().rows.add(rows).draw();
@@ -319,18 +319,18 @@ function update_krpc_inbound_conn_metrics_datatable(json) {
   var table = $('#inbound_per_conn_metrics').DataTable();
   var rows = $.map(json["inbound_per_conn_metrics"], function(row) {
     return [[row["remote_ip"], row["num_calls_in_flight"],
-             row["socket_stats"]["rtt"],
-             row["socket_stats"]["rttvar"],
-             row["socket_stats"]["snd_cwnd"],
-             row["socket_stats"]["total_retrans"],
-             row["socket_stats"]["pacing_rate"],
-             row["socket_stats"]["max_pacing_rate"],
-             row["socket_stats"]["bytes_acked"],
-             row["socket_stats"]["bytes_received"],
-             row["socket_stats"]["segs_out"],
-             row["socket_stats"]["segs_in"],
-             row["socket_stats"]["send_queue_bytes"],
-             row["socket_stats"]["receive_queue_bytes"]]];
+             row["socket_stats"]["rtt"] ?? '',
+             row["socket_stats"]["rttvar"] ?? '',
+             row["socket_stats"]["snd_cwnd"] ?? '',
+             row["socket_stats"]["total_retrans"] ?? '',
+             row["socket_stats"]["pacing_rate"] ?? '',
+             row["socket_stats"]["max_pacing_rate"] ?? '',
+             row["socket_stats"]["bytes_acked"] ?? '',
+             row["socket_stats"]["bytes_received"] ?? '',
+             row["socket_stats"]["segs_out"] ?? '',
+             row["socket_stats"]["segs_in"] ?? '',
+             row["socket_stats"]["send_queue_bytes"] ?? '',
+             row["socket_stats"]["receive_queue_bytes"] ?? '']];
   });
 
   table.clear().rows.add(rows).draw();


[impala] 01/02: IMPALA-10745: Support Kerberos over HTTP for impala-shell

Posted by wz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 397d1d15a274b843aad9b4da8eaa1ab7dc468099
Author: wzhou-code <wz...@cloudera.com>
AuthorDate: Wed Apr 27 15:30:51 2022 -0700

    IMPALA-10745: Support Kerberos over HTTP for impala-shell
    
    This patch ports the implementation of GSSAPI authentication over http
    transport from Impyla (https://github.com/cloudera/impyla/pull/415) to
    impala-shell.
    
    The implementation adds a new dependency on 'kerberos' python module,
    which is a pip-installed module distributed under Apache License Version
    2.
    When using impala-shell with Kerberos over http, it is assumed that the
    host has a preexisting kinit-cached Kerberos ticket that impala-shell
    can pass to the server automatically without the user to reenter the
    password.
    
    Testing:
     - Passed exhaustive tests.
     - Tested manually on a real cluster with a full Kerberos setup.
    
    Change-Id: Ia59ba4004490735162adbd468a00a962165c5abd
    Reviewed-on: http://gerrit.cloudera.org:8080/18493
    Reviewed-by: Csaba Ringhofer <cs...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 infra/python/deps/requirements.txt |   1 +
 shell/ImpalaHttpClient.py          | 233 ++++++++++++++++++++++++++-----------
 shell/impala_client.py             |  22 ++--
 shell/impala_shell.py              |   3 +-
 shell/packaging/requirements.txt   |   1 +
 shell/shell_exceptions.py          |   8 ++
 6 files changed, 185 insertions(+), 83 deletions(-)

diff --git a/infra/python/deps/requirements.txt b/infra/python/deps/requirements.txt
index cefff6d5b..9bfd0862b 100644
--- a/infra/python/deps/requirements.txt
+++ b/infra/python/deps/requirements.txt
@@ -40,6 +40,7 @@ impyla == 0.18a1
   # six == 1.14.0 (specified separately)
   thrift_sasl == 0.4.3
 kazoo == 2.2.1
+kerberos == 1.3.1
 pexpect == 3.3
 pg8000 == 1.10.2
 prettytable == 0.7.2
diff --git a/shell/ImpalaHttpClient.py b/shell/ImpalaHttpClient.py
index 947cecfc1..9903501bd 100644
--- a/shell/ImpalaHttpClient.py
+++ b/shell/ImpalaHttpClient.py
@@ -30,7 +30,7 @@ from collections import namedtuple
 from six.moves import urllib, http_client
 
 from thrift.transport.TTransport import TTransportBase
-from shell_exceptions import HttpError
+from shell_exceptions import HttpError, AuthenticationException
 from cookie_util import get_all_matching_cookies, get_cookie_expiry
 
 import six
@@ -112,18 +112,27 @@ class ImpalaHttpClient(TTransportBase):
       self.realhost = self.realport = self.proxy_auth = None
     if (not http_cookie_names) or (str(http_cookie_names).strip() == ""):
       self.__http_cookie_dict = None
+      self.__auth_cookie_names = None
     else:
       # Build a dictionary that maps cookie name to namedtuple.
       cookie_names = http_cookie_names.split(',')
       self.__http_cookie_dict = \
-        {cn: Cookie(cookie=None, expiry_time=None) for cn in cookie_names}
+          {cn: Cookie(cookie=None, expiry_time=None) for cn in cookie_names}
+      # Store the auth cookie names in __auth_cookie_names.
+      # Assume auth cookie names end with ".auth".
+      self.__auth_cookie_names = [cn for cn in cookie_names if cn.endswith(".auth")]
     # Set __are_matching_cookies_found as True if matching cookies are found in response.
     self.__are_matching_cookies_found = False
     self.__wbuf = BytesIO()
     self.__http = None
     self.__http_response = None
     self.__timeout = socket_timeout_s
+    # __custom_headers is used to store HTTP headers which are generated in runtime for
+    # new request.
     self.__custom_headers = None
+    self.__get_custom_headers_func = None
+    self.__basic_auth = None
+    self.__kerb_service = None
 
   @staticmethod
   def basic_proxy_auth_header(proxy):
@@ -165,27 +174,75 @@ class ImpalaHttpClient(TTransportBase):
     else:
       self.__timeout = ms / 1000.0
 
-  def setCustomHeaders(self, headers):
-    self.__custom_headers = headers
-
-  # Extract cookies from response and save those cookies for which the cookie names
-  # are in the cookie name list specified in the __init__().
-  def extractHttpCookiesFromResponse(self, headers):
-    if self.__http_cookie_dict:
-      matching_cookies = \
-          get_all_matching_cookies(self.__http_cookie_dict.keys(), self.path, headers)
-      if matching_cookies:
-        self.__are_matching_cookies_found = True
-        for c in matching_cookies:
-          self.__http_cookie_dict[c.key] = Cookie(c, get_cookie_expiry(c))
-
-  # Return the value as a cookie list for Cookie header. It's a list of name-value
+  def getCustomHeadersWithBasicAuth(self, cookie_header, has_auth_cookie):
+    custom_headers = {}
+    if cookie_header:
+      # Add cookies to HTTP header.
+      custom_headers['Cookie'] = cookie_header
+    # Add the 'Authorization' header to request even if the auth cookie is
+    # present to avoid a round trip in case the cookie is expired when server
+    # receive the request. Since the 'auth' value is calculated once, so it
+    # won't cause a performance issue.
+    custom_headers['Authorization'] = "Basic " + self.__basic_auth
+    return custom_headers
+
+  def getCustomHeadersWithNegotiateAuth(self, cookie_header, has_auth_cookie):
+    import kerberos
+    custom_headers = {}
+    if cookie_header:
+      # Add cookies to HTTP header.
+      custom_headers['Cookie'] = cookie_header
+    # For GSSAPI over http we need to dynamically generate custom request headers.
+    if not has_auth_cookie:
+      try:
+        _, krb_context = kerberos.authGSSClientInit(self.__kerb_service)
+        kerberos.authGSSClientStep(krb_context, "")
+        negotiate_details = kerberos.authGSSClientResponse(krb_context)
+        custom_headers['Authorization'] = "Negotiate " + negotiate_details
+      except kerberos.GSSError:
+        raise AuthenticationException("Kerberos authentication failure.")
+    return custom_headers
+
+  def getCustomHeadersWithoutAuth(self, cookie_header, has_auth_cookie):
+    custom_headers = {}
+    if cookie_header:
+      # Add cookies to HTTP header.
+      custom_headers['Cookie'] = cookie_header
+    return custom_headers
+
+  # Set function to generate customized HTTP headers for Ldap authorization.
+  def setLdapAuth(self, basic_auth):
+    # auth mechanism: LDAP
+    self.__basic_auth = basic_auth
+    self.__get_custom_headers_func = self.getCustomHeadersWithBasicAuth
+
+  # Set function to generate customized HTTP headers for Kerberos authorization.
+  def setKerberosAuth(self, kerb_service):
+    # auth mechanism: GSSAPI
+    self.__kerb_service = kerb_service
+    self.__get_custom_headers_func = self.getCustomHeadersWithNegotiateAuth
+
+  # Set function to generate customized HTTP headers without authorization.
+  def setNoneAuth(self):
+    # auth mechanism: None
+    self.__get_custom_headers_func = self.getCustomHeadersWithoutAuth
+
+  # Update HTTP headers based on the saved cookies and auth mechanism.
+  def refreshCustomHeaders(self):
+    if self.__get_custom_headers_func:
+      cookie_header, has_auth_cookie = self.getHttpCookieHeaderForRequest()
+      self.__custom_headers = \
+          self.__get_custom_headers_func(cookie_header, has_auth_cookie)
+
+  # Return first value as a cookie list for Cookie header. It's a list of name-value
   # pairs in the form of <cookie-name>=<cookie-value>. Pairs in the list are separated by
   # a semicolon and a space ('; ').
+  # Return second value as True if the cookie list contains auth cookie.
   def getHttpCookieHeaderForRequest(self):
     if (self.__http_cookie_dict is None) or not self.__are_matching_cookies_found:
-      return None
+      return None, False
     cookie_headers = []
+    has_auth_cookie = False
     for cn, c_tuple in self.__http_cookie_dict.items():
       if c_tuple.cookie:
         if c_tuple.expiry_time and c_tuple.expiry_time <= datetime.datetime.now():
@@ -193,18 +250,36 @@ class ImpalaHttpClient(TTransportBase):
         else:
           cookie_header = c_tuple.cookie.output(attrs=['value'], header='').strip()
           cookie_headers.append(cookie_header)
+          if not has_auth_cookie and self.__auth_cookie_names \
+              and cn in self.__auth_cookie_names:
+            has_auth_cookie = True
     if not cookie_headers:
       self.__are_matching_cookies_found = False
-      return None
+      return None, False
     else:
-      return '; '.join(cookie_headers)
+      return '; '.join(cookie_headers), has_auth_cookie
 
-  # Add HTTP cookie headers based on the saved cookies.
-  def addHttpCookiesToRequestHeaders(self):
+  # Extract cookies from response and save those cookies for which the cookie names
+  # are in the cookie name list specified in the __init__().
+  def extractHttpCookiesFromResponse(self):
     if self.__http_cookie_dict:
-      cookie_headers = self.getHttpCookieHeaderForRequest()
-      if cookie_headers:
-        self.__http.putheader('Cookie', cookie_headers)
+      matching_cookies = get_all_matching_cookies(
+          self.__http_cookie_dict.keys(), self.path, self.headers)
+      if matching_cookies:
+        self.__are_matching_cookies_found = True
+        for c in matching_cookies:
+          self.__http_cookie_dict[c.key] = Cookie(c, get_cookie_expiry(c))
+
+  # Return True if there are any saved cookies which are sent in previous request.
+  def areHttpCookiesSaved(self):
+    return self.__are_matching_cookies_found
+
+  # Clean all saved cookies.
+  def cleanHttpCookies(self):
+    if (self.__http_cookie_dict is not None) and self.__are_matching_cookies_found:
+      self.__are_matching_cookies_found = False
+      self.__http_cookie_dict = \
+          {cn: Cookie(cookie=None, expiry_time=None) for cn in self.__http_cookie_dict}
 
   def read(self, sz):
     return self.__http_response.read(sz)
@@ -216,57 +291,73 @@ class ImpalaHttpClient(TTransportBase):
     self.__wbuf.write(buf)
 
   def flush(self):
-    if self.isOpen():
-      self.close()
-    self.open()
+    # Send HTTP request and receive response.
+    # Return True if the client should retry this method.
+    def sendRequestRecvResp(data):
+      if self.isOpen():
+        self.close()
+      self.open()
+
+      # HTTP request
+      if self.using_proxy() and self.scheme == "http":
+        # need full URL of real host for HTTP proxy here (HTTPS uses CONNECT tunnel)
+        self.__http.putrequest('POST', "http://%s:%s%s" %
+                               (self.realhost, self.realport, self.path))
+      else:
+        self.__http.putrequest('POST', self.path)
+
+      # Write headers
+      self.__http.putheader('Content-Type', 'application/x-thrift')
+      data_len = len(data)
+      self.__http.putheader('Content-Length', str(data_len))
+      if data_len > ImpalaHttpClient.MIN_REQUEST_SIZE_FOR_EXPECT:
+        # Add the 'Expect' header to large requests. Note that we do not explicitly wait
+        # for the '100 continue' response before sending the data - HTTPConnection simply
+        # ignores these types of responses, but we'll get the right behavior anyways.
+        self.__http.putheader("Expect", "100-continue")
+      if self.using_proxy() and self.scheme == "http" and self.proxy_auth is not None:
+        self.__http.putheader("Proxy-Authorization", self.proxy_auth)
+
+      self.refreshCustomHeaders()
+      if not self.__custom_headers or 'User-Agent' not in self.__custom_headers:
+        user_agent = 'Python/ImpalaHttpClient'
+        script = os.path.basename(sys.argv[0])
+        if script:
+          user_agent = '%s (%s)' % (user_agent, urllib.parse.quote(script))
+        self.__http.putheader('User-Agent', user_agent)
+
+      if self.__custom_headers:
+        for key, val in six.iteritems(self.__custom_headers):
+          self.__http.putheader(key, val)
+
+      self.__http.endheaders()
+
+      # Write payload
+      self.__http.send(data)
+
+      # Get reply to flush the request
+      self.__http_response = self.__http.getresponse()
+      self.code = self.__http_response.status
+      self.message = self.__http_response.reason
+      self.headers = self.__http_response.msg
+      # A '401 Unauthorized' response might mean that we tried cookie-based
+      # authentication with one or more expired cookies.
+      # Delete the cookies and try again.
+      if self.code == 401 and self.areHttpCookiesSaved():
+        self.cleanHttpCookies()
+        return True
+      else:
+        self.extractHttpCookiesFromResponse()
+        return False
 
     # Pull data out of buffer
     data = self.__wbuf.getvalue()
     self.__wbuf = BytesIO()
 
-    # HTTP request
-    if self.using_proxy() and self.scheme == "http":
-      # need full URL of real host for HTTP proxy here (HTTPS uses CONNECT tunnel)
-      self.__http.putrequest('POST', "http://%s:%s%s" %
-                             (self.realhost, self.realport, self.path))
-    else:
-      self.__http.putrequest('POST', self.path)
-
-    # Write headers
-    self.__http.putheader('Content-Type', 'application/x-thrift')
-    data_len = len(data)
-    self.__http.putheader('Content-Length', str(data_len))
-    if data_len > ImpalaHttpClient.MIN_REQUEST_SIZE_FOR_EXPECT:
-      # Add the 'Expect' header to large requests. Note that we do not explicitly wait
-      # for the '100 continue' response before sending the data - HTTPConnection simply
-      # ignores these types of responses, but we'll get the right behavior anyways.
-      self.__http.putheader("Expect", "100-continue")
-    if self.using_proxy() and self.scheme == "http" and self.proxy_auth is not None:
-      self.__http.putheader("Proxy-Authorization", self.proxy_auth)
-
-    if not self.__custom_headers or 'User-Agent' not in self.__custom_headers:
-      user_agent = 'Python/ImpalaHttpClient'
-      script = os.path.basename(sys.argv[0])
-      if script:
-        user_agent = '%s (%s)' % (user_agent, urllib.parse.quote(script))
-      self.__http.putheader('User-Agent', user_agent)
-
-    if self.__custom_headers:
-      for key, val in six.iteritems(self.__custom_headers):
-        self.__http.putheader(key, val)
-
-    self.addHttpCookiesToRequestHeaders()
-    self.__http.endheaders()
-
-    # Write payload
-    self.__http.send(data)
-
-    # Get reply to flush the request
-    self.__http_response = self.__http.getresponse()
-    self.code = self.__http_response.status
-    self.message = self.__http_response.reason
-    self.headers = self.__http_response.msg
-    self.extractHttpCookiesFromResponse(self.headers)
+    retry = sendRequestRecvResp(data)
+    if retry:
+      # Received "401 Unauthorized" response. Delete HTTP cookies and then retry.
+      sendRequestRecvResp(data)
 
     if self.code >= 300:
       # Report any http response code that is not 1XX (informational response) or
diff --git a/shell/impala_client.py b/shell/impala_client.py
index 0790b1bb9..9571d0b32 100755
--- a/shell/impala_client.py
+++ b/shell/impala_client.py
@@ -388,14 +388,6 @@ class ImpalaClient(object):
     # timeout so that in case of any connection errors, the client retries have a better
     # chance of succeeding.
 
-    # HTTP server implemententations do not support SPNEGO yet.
-    # TODO: when we add support for Kerberos+HTTP, we need to re-enable the automatic
-    # kerberos retry logic in impala_shell.py that was disabled for HTTP because of
-    # IMPALA-8932.
-    if self.use_kerberos or self.kerberos_host_fqdn:
-      print("Kerberos not supported with HTTP endpoints.", file=sys.stderr)
-      raise NotImplementedError()
-
     host_and_port = "{0}:{1}".format(self.impalad_host, self.impalad_port)
     assert self.http_path
     # ImpalaHttpClient relies on the URI scheme (http vs https) to open an appropriate
@@ -417,10 +409,20 @@ class ImpalaClient(object):
                                    socket_timeout_s=self.http_socket_timeout_s)
 
     if self.use_ldap:
-      # Set the BASIC auth header
+      # Set the BASIC authorization
       user_passwd = "{0}:{1}".format(self.user, self.ldap_password)
       auth = base64.encodestring(user_passwd.encode()).decode().strip('\n')
-      transport.setCustomHeaders({"Authorization": "Basic {0}".format(auth)})
+      transport.setLdapAuth(auth)
+    elif self.use_kerberos or self.kerberos_host_fqdn:
+      # Set the Kerberos service
+      if self.kerberos_host_fqdn is not None:
+        kerb_host = self.kerberos_host_fqdn.split(':')[0].encode('ascii', 'ignore')
+      else:
+        kerb_host = self.impalad_host
+      kerb_service = "{0}@{1}".format(self.kerberos_service_name, kerb_host)
+      transport.setKerberosAuth(kerb_service)
+    else:
+      transport.setNoneAuth()
 
     # Without buffering Thrift would call socket.recv() each time it deserializes
     # something (e.g. a member in a struct).
diff --git a/shell/impala_shell.py b/shell/impala_shell.py
index 55387b602..a2bd4fb2d 100755
--- a/shell/impala_shell.py
+++ b/shell/impala_shell.py
@@ -953,8 +953,7 @@ class ImpalaShell(cmd.Cmd, object):
     # If the connection fails and the Kerberos has not been enabled,
     # check for a valid kerberos ticket and retry the connection
     # with kerberos enabled.
-    # IMPALA-8932: Kerberos is not yet supported for hs2-http, so don't retry.
-    if not self.imp_client.connected and not self.use_kerberos and protocol != 'hs2-http':
+    if not self.imp_client.connected and not self.use_kerberos:
       try:
         if call(["klist", "-s"]) == 0:
           print("Kerberos ticket found in the credentials cache, retrying "
diff --git a/shell/packaging/requirements.txt b/shell/packaging/requirements.txt
index 5fbd23312..73e09f157 100644
--- a/shell/packaging/requirements.txt
+++ b/shell/packaging/requirements.txt
@@ -1,5 +1,6 @@
 bitarray==2.3.0
 configparser==4.0.2
+kerberos==1.3.1
 prettytable==0.7.2
 sasl==0.2.1
 setuptools>=36.8.0
diff --git a/shell/shell_exceptions.py b/shell/shell_exceptions.py
index 538ca1e97..00231aab4 100644
--- a/shell/shell_exceptions.py
+++ b/shell/shell_exceptions.py
@@ -41,6 +41,14 @@ class DisconnectedException(Exception):
       return self.value
 
 
+class AuthenticationException(Exception):
+  def __init__(self, value=""):
+    self.value = value
+
+  def __str__(self):
+    return self.value
+
+
 class QueryCancelledByShellException(Exception): pass