Browse Source

LIVY-273. Add Spnego support for Java/scala and Python Job API client (#265)

* Add Spnego support for Java Job API client

Change-Id: I8cce80d75a549e573da70be9ef1aeb94457829a4

* Add Python Job API kerberos support

Change-Id: I022b3c14a439321deda3ae5e0cacbc2b18ef9830

* Fix python style

Change-Id: I9e9f00dde93c471c6fa1fd5f71842945f527c73b

* Address the comments

Change-Id: I4994122e8e28276efe4f00fc7ff8e02e0a7a6c60
Saisai Shao 8 years ago
parent
commit
b75bc11b65

+ 3 - 2
.travis.yml

@@ -26,6 +26,7 @@ cache:
 
 
 before_install:
 before_install:
   - sudo apt-get -y install python3-pip python-dev
   - sudo apt-get -y install python3-pip python-dev
+  - sudo apt-get -y install libkrb5-dev
   - sudo apt-get -y remove python-setuptools
   - sudo apt-get -y remove python-setuptools
   - pip install --user --upgrade pip setuptools
   - pip install --user --upgrade pip setuptools
   - pip3 install --user --upgrade pip setuptools
   - pip3 install --user --upgrade pip setuptools
@@ -36,8 +37,8 @@ install:
   - mvn $MVN_FLAG install -Dskip -DskipTests -DskipITs -Dmaven.javadoc.skip=true -B -V
   - mvn $MVN_FLAG install -Dskip -DskipTests -DskipITs -Dmaven.javadoc.skip=true -B -V
 
 
 before_script:
 before_script:
-  - pip install --user requests pytest flaky flake8
-  - pip3 install --user requests pytest flaky
+  - pip install --user requests pytest flaky flake8 requests-kerberos
+  - pip3 install --user requests pytest flaky requests-kerberos
 
 
 script:
 script:
   - mvn $MVN_FLAG verify -e
   - mvn $MVN_FLAG verify -e

+ 1 - 0
README.rst

@@ -45,6 +45,7 @@ MacOS:
 Required python packages for building Livy:
 Required python packages for building Livy:
   * cloudpickle
   * cloudpickle
   * requests
   * requests
+  * requests-kerberos
   * flake8
   * flake8
   * flaky
   * flaky
   * pytest
   * pytest

+ 28 - 1
client-http/src/main/java/com/cloudera/livy/client/http/HttpConf.java

@@ -32,7 +32,13 @@ class HttpConf extends ClientConf<HttpConf> {
     JOB_INITIAL_POLL_INTERVAL("job.initial_poll_interval", "100ms"),
     JOB_INITIAL_POLL_INTERVAL("job.initial_poll_interval", "100ms"),
     JOB_MAX_POLL_INTERVAL("job.max_poll_interval", "5s"),
     JOB_MAX_POLL_INTERVAL("job.max_poll_interval", "5s"),
 
 
-    CONTENT_COMPRESS_ENABLE("content.compress.enable", true);
+    CONTENT_COMPRESS_ENABLE("content.compress.enable", true),
+
+    // Kerberos related configuration
+    SPNEGO_ENABLED("spnego.enable", false),
+    AUTH_LOGIN_CONFIG("auth.login.config", null),
+    KRB5_DEBUG_ENABLED("krb5.debug", false),
+    KRB5_CONF("krb5.conf", null);
 
 
     private final String key;
     private final String key;
     private final Object dflt;
     private final Object dflt;
@@ -51,6 +57,27 @@ class HttpConf extends ClientConf<HttpConf> {
 
 
   HttpConf(Properties config) {
   HttpConf(Properties config) {
     super(config);
     super(config);
+
+    if (getBoolean(Entry.SPNEGO_ENABLED)) {
+      if (get(Entry.AUTH_LOGIN_CONFIG ) == null) {
+        throw new IllegalArgumentException(Entry.AUTH_LOGIN_CONFIG.key + " should not be null");
+      }
+
+      if (get(Entry.KRB5_CONF) == null) {
+        throw new IllegalArgumentException(Entry.KRB5_CONF.key + " should not be null");
+      }
+
+      System.setProperty("java.security.auth.login.config", get(Entry.AUTH_LOGIN_CONFIG));
+      System.setProperty("java.security.krb5.conf", get(Entry.KRB5_CONF));
+      System.setProperty(
+        "sun.security.krb5.debug", String.valueOf(getBoolean(Entry.KRB5_DEBUG_ENABLED)));
+      // This is needed to get Kerberos credentials from the environment, instead of
+      // requiring the application to manually obtain the credentials.
+      System.setProperty("javax.security.auth.useSubjectCredsOnly", "false");
+    }
   }
   }
 
 
+  boolean isSpnegoEnabled() {
+    return getBoolean(Entry.SPNEGO_ENABLED);
+  }
 }
 }

+ 13 - 4
client-http/src/main/java/com/cloudera/livy/client/http/LivyConnection.java

@@ -30,17 +30,22 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpHeaders;
 import org.apache.http.HttpHeaders;
 import org.apache.http.HttpStatus;
 import org.apache.http.HttpStatus;
+import org.apache.http.auth.AuthSchemeProvider;
 import org.apache.http.auth.AuthScope;
 import org.apache.http.auth.AuthScope;
 import org.apache.http.auth.Credentials;
 import org.apache.http.auth.Credentials;
 import org.apache.http.auth.UsernamePasswordCredentials;
 import org.apache.http.auth.UsernamePasswordCredentials;
 import org.apache.http.client.CredentialsProvider;
 import org.apache.http.client.CredentialsProvider;
+import org.apache.http.client.config.AuthSchemes;
 import org.apache.http.client.config.RequestConfig;
 import org.apache.http.client.config.RequestConfig;
 import org.apache.http.client.methods.*;
 import org.apache.http.client.methods.*;
 import org.apache.http.client.protocol.HttpClientContext;
 import org.apache.http.client.protocol.HttpClientContext;
+import org.apache.http.config.Registry;
+import org.apache.http.config.RegistryBuilder;
 import org.apache.http.entity.ByteArrayEntity;
 import org.apache.http.entity.ByteArrayEntity;
 import org.apache.http.entity.mime.MultipartEntityBuilder;
 import org.apache.http.entity.mime.MultipartEntityBuilder;
 import org.apache.http.entity.mime.content.FileBody;
 import org.apache.http.entity.mime.content.FileBody;
 import org.apache.http.impl.NoConnectionReuseStrategy;
 import org.apache.http.impl.NoConnectionReuseStrategy;
+import org.apache.http.impl.auth.SPNegoSchemeFactory;
 import org.apache.http.impl.client.BasicCredentialsProvider;
 import org.apache.http.impl.client.BasicCredentialsProvider;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClientBuilder;
 import org.apache.http.impl.client.HttpClientBuilder;
@@ -123,10 +128,6 @@ class LivyConnection {
       };
       };
     }
     }
 
 
-    // This is needed to get Kerberos credentials from the environment, instead of
-    // requiring the application to manually obtain the credentials.
-    System.setProperty("javax.security.auth.useSubjectCredsOnly", "false");
-
     CredentialsProvider credsProvider = new BasicCredentialsProvider();
     CredentialsProvider credsProvider = new BasicCredentialsProvider();
     credsProvider.setCredentials(AuthScope.ANY, credentials);
     credsProvider.setCredentials(AuthScope.ANY, credentials);
 
 
@@ -141,6 +142,14 @@ class LivyConnection {
       .setDefaultCredentialsProvider(credsProvider)
       .setDefaultCredentialsProvider(credsProvider)
       .setUserAgent("livy-client-http");
       .setUserAgent("livy-client-http");
 
 
+    if (config.isSpnegoEnabled()) {
+      Registry<AuthSchemeProvider> authSchemeProviderRegistry =
+        RegistryBuilder.<AuthSchemeProvider>create()
+          .register(AuthSchemes.SPNEGO, new SPNegoSchemeFactory())
+          .build();
+      builder.setDefaultAuthSchemeRegistry(authSchemeProviderRegistry);
+    }
+
     this.server = uri;
     this.server = uri;
     this.client = builder.build();
     this.client = builder.build();
     this.mapper = new ObjectMapper();
     this.mapper = new ObjectMapper();

+ 1 - 0
python-api/setup.py

@@ -35,6 +35,7 @@ requirements = [
     'futures>=3.0.5',
     'futures>=3.0.5',
     'requests>=2.10.0',
     'requests>=2.10.0',
     'responses>=0.5.1',
     'responses>=0.5.1',
+    'requests-kerberos>=0.11.0',
 ]
 ]
 
 
 setup(
 setup(

+ 17 - 4
python-api/src/main/python/livy/client.py

@@ -28,6 +28,7 @@ from configparser import ConfigParser
 from concurrent.futures import ThreadPoolExecutor
 from concurrent.futures import ThreadPoolExecutor
 from future.moves.urllib.parse import ParseResult, urlparse
 from future.moves.urllib.parse import ParseResult, urlparse
 from io import open, StringIO
 from io import open, StringIO
+from requests_kerberos import HTTPKerberosAuth, REQUIRED
 from livy.job_handle import JobHandle
 from livy.job_handle import JobHandle
 
 
 
 
@@ -76,13 +77,13 @@ class HttpClient(object):
                 path=match.group(1), params=uri.params, query=uri.query,
                 path=match.group(1), params=uri.params, query=uri.query,
                 fragment=uri.fragment)
                 fragment=uri.fragment)
             self._set_uri(base)
             self._set_uri(base)
-            self._conn = _LivyConnection(base)
+            self._conn = _LivyConnection(base, self._config)
             self._session_id = int(match.group(2))
             self._session_id = int(match.group(2))
             self._reconnect_to_existing_session()
             self._reconnect_to_existing_session()
         else:
         else:
             self._set_uri(uri)
             self._set_uri(uri)
             session_conf_dict = dict(self._config.items(self._CONFIG_SECTION))
             session_conf_dict = dict(self._config.items(self._CONFIG_SECTION))
-            self._conn = _LivyConnection(uri)
+            self._conn = _LivyConnection(uri, self._config)
             self._session_id = self._create_new_session(
             self._session_id = self._create_new_session(
                 session_conf_dict).json()['id']
                 session_conf_dict).json()['id']
         self._executor = ThreadPoolExecutor(max_workers=1)
         self._executor = ThreadPoolExecutor(max_workers=1)
@@ -434,10 +435,22 @@ class _LivyConnection(object):
         'Accept': 'application/json',
         'Accept': 'application/json',
     }
     }
 
 
-    def __init__(self, uri):
+    _SPNEGO_ENABLED_CONF = 'livy.client.http.spnego.enable'
+
+    def __init__(self, uri, config):
         self._server_url_prefix = uri.geturl() + self._SESSIONS_URI
         self._server_url_prefix = uri.geturl() + self._SESSIONS_URI
         self._requests = requests
         self._requests = requests
         self.lock = threading.Lock()
         self.lock = threading.Lock()
+        self._spnego_enabled = \
+            config.getboolean('env', self._SPNEGO_ENABLED_CONF) \
+            if config.has_option('env', self._SPNEGO_ENABLED_CONF) else False
+
+    def _spnego_auth(self):
+        if self._spnego_enabled:
+            return HTTPKerberosAuth(mutual_authentication=REQUIRED,
+                                    sanitize_mutual_error_response=False)
+        else:
+            return None
 
 
     def send_request(
     def send_request(
         self,
         self,
@@ -483,7 +496,7 @@ class _LivyConnection(object):
                 request_url = self._server_url_prefix + suffix_url
                 request_url = self._server_url_prefix + suffix_url
                 return self._requests.request(method, request_url,
                 return self._requests.request(method, request_url,
                     timeout=self._TIMEOUT, headers=local_headers, files=files,
                     timeout=self._TIMEOUT, headers=local_headers, files=files,
-                    json=data)
+                    json=data, auth=self._spnego_auth())
         finally:
         finally:
             if files is not None:
             if files is not None:
                 files.clear()
                 files.clear()