You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by el...@apache.org on 2017/10/20 22:50:58 UTC

calcite-avatica git commit: [CALCITE-2017] Support JAAS-based Kerberos login on IBM Java [Forced Update!]

Repository: calcite-avatica
Updated Branches:
  refs/heads/master c38312576 -> 2de4b1922 (forced update)


[CALCITE-2017] Support JAAS-based Kerberos login on IBM Java

There are numerous implementation differences between Oracle and IBM
Java which require special programming to work around.

Closes apache/avatica-calcite#18


Project: http://git-wip-us.apache.org/repos/asf/calcite-avatica/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite-avatica/commit/2de4b192
Tree: http://git-wip-us.apache.org/repos/asf/calcite-avatica/tree/2de4b192
Diff: http://git-wip-us.apache.org/repos/asf/calcite-avatica/diff/2de4b192

Branch: refs/heads/master
Commit: 2de4b19223aded0212b5dd20c051b881808bdb24
Parents: fcb1342
Author: Josh Elser <el...@apache.org>
Authored: Thu Oct 19 11:53:24 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri Oct 20 18:50:36 2017 -0400

----------------------------------------------------------------------
 .travis.yml                                     |  2 +
 .../avatica/remote/ClientKeytabJaasConf.java    | 65 +++++++++++++++++++
 .../avatica/remote/KerberosConnection.java      | 41 ++++--------
 .../calcite/avatica/server/HttpServer.java      | 42 +-----------
 .../avatica/server/ServerKeytabJaasConf.java    | 67 ++++++++++++++++++++
 .../calcite/avatica/AvaticaSpnegoTest.java      |  6 +-
 .../apache/calcite/avatica/SpnegoTestUtil.java  | 24 +++++--
 .../avatica/server/AvaticaJaasKrbUtil.java      | 59 +++++++++++++++++
 .../server/HttpServerSpnegoWithJaasTest.java    | 17 ++++-
 .../server/HttpServerSpnegoWithoutJaasTest.java | 17 ++++-
 10 files changed, 256 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/2de4b192/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 034f285..583fb23 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -22,6 +22,7 @@ matrix:
   include:
     - jdk: oraclejdk8
     - jdk: openjdk7
+    - jdk: ibmjava8
 branches:
   only:
     - master
@@ -43,4 +44,5 @@ cache:
   directories:
     - $HOME/.m2
 sudo: required
+group: edge
 # End .travis.yml

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/2de4b192/core/src/main/java/org/apache/calcite/avatica/remote/ClientKeytabJaasConf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/ClientKeytabJaasConf.java b/core/src/main/java/org/apache/calcite/avatica/remote/ClientKeytabJaasConf.java
new file mode 100644
index 0000000..ef060eb
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/ClientKeytabJaasConf.java
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+package org.apache.calcite.avatica.remote;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+
+/**
+ * Javax Configuration class which always returns a configuration for our keytab-based
+ * login suitable for callers which are acting as initiators (e.g. a client).
+ */
+public class ClientKeytabJaasConf extends Configuration {
+  private static final Logger LOG = LoggerFactory.getLogger(ClientKeytabJaasConf.class);
+  private final String principal;
+  private final String keytab;
+
+  public ClientKeytabJaasConf(String principal, String keytab) {
+    this.principal = principal;
+    this.keytab = keytab;
+  }
+
+  @Override public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+    Map<String, String> options = new HashMap<String, String>();
+    options.put("principal", principal);
+    options.put("refreshKrb5Config", "true");
+    if (KerberosConnection.isIbmJava()) {
+      options.put("useKeytab", keytab);
+      options.put("credsType", "both");
+    } else {
+      options.put("keyTab", keytab);
+      options.put("useKeyTab", "true");
+      options.put("isInitiator", "true");
+      options.put("doNotPrompt", "true");
+      options.put("storeKey", "true");
+    }
+
+    LOG.debug("JAAS Configuration for client keytab-based Kerberos login: {}", options);
+
+    return new AppConfigurationEntry[] {new AppConfigurationEntry(
+        KerberosConnection.getKrb5LoginModuleName(),
+        AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, options)};
+  }
+}
+
+// End ClientKeytabJaasConf.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/2de4b192/core/src/main/java/org/apache/calcite/avatica/remote/KerberosConnection.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/KerberosConnection.java b/core/src/main/java/org/apache/calcite/avatica/remote/KerberosConnection.java
index da701dc..657297e 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/KerberosConnection.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/KerberosConnection.java
@@ -23,7 +23,6 @@ import java.io.File;
 import java.lang.Thread.UncaughtExceptionHandler;
 import java.security.Principal;
 import java.util.AbstractMap;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map.Entry;
 import java.util.Objects;
@@ -33,7 +32,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosPrincipal;
 import javax.security.auth.kerberos.KerberosTicket;
-import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginContext;
 import javax.security.auth.login.LoginException;
@@ -51,6 +49,10 @@ public class KerberosConnection {
   private static final String JAAS_CONF_NAME = "AvaticaKeytabConf";
   private static final String RENEWAL_THREAD_NAME = "Avatica Kerberos Renewal Thread";
 
+  // Thanks Hadoop! Lifted from UserGroupInformation and PlatformName
+  private static final String JAVA_VENDOR_NAME = System.getProperty("java.vendor");
+  private static final boolean IS_IBM_JAVA = JAVA_VENDOR_NAME.contains("IBM");
+
   /** The percentage of the Kerberos ticket's lifetime which we should start trying to renew it */
   public static final float PERCENT_OF_LIFETIME_TO_RENEW = 0.80f;
   /** How long should we sleep between checks to renew the Kerberos ticket */
@@ -70,7 +72,8 @@ public class KerberosConnection {
    */
   public KerberosConnection(String principal, File keytab) {
     this.principal = Objects.requireNonNull(principal);
-    this.jaasConf = new KeytabJaasConf(principal, Objects.requireNonNull(keytab));
+    this.jaasConf = new ClientKeytabJaasConf(principal,
+        Objects.requireNonNull(keytab).getAbsolutePath());
   }
 
   public synchronized Subject getSubject() {
@@ -359,31 +362,13 @@ public class KerberosConnection {
   }
 
   /**
-   * Javax Configuration for performing a keytab-based Kerberos login.
+   * Returns whether or not the current environment is IBM Java. Otherwise, assumed to be Oracle
+   * Java/OpenJDK.
+   *
+   * @return True if the environment is IBM Java, false otherwise.
    */
-  static class KeytabJaasConf extends Configuration {
-    private String principal;
-    private File keytabFile;
-
-    KeytabJaasConf(String principal, File keytab) {
-      this.principal = principal;
-      this.keytabFile = keytab;
-    }
-
-    @Override public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
-      HashMap<String, String> options = new HashMap<String, String>();
-      options.put("keyTab", keytabFile.getAbsolutePath());
-      options.put("principal", principal);
-      options.put("useKeyTab", "true");
-      options.put("storeKey", "true");
-      options.put("doNotPrompt", "true");
-      options.put("renewTGT", "false");
-      options.put("refreshKrb5Config", "true");
-      options.put("isInitiator", "true");
-
-      return new AppConfigurationEntry[] {new AppConfigurationEntry(getKrb5LoginModuleName(),
-          AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, options)};
-    }
+  public static boolean isIbmJava() {
+    return IS_IBM_JAVA;
   }
 
   /**
@@ -391,7 +376,7 @@ public class KerberosConnection {
    *
    * @return The class name of the KRB5 LoginModule
    */
-  static String getKrb5LoginModuleName() {
+  public static String getKrb5LoginModuleName() {
     return System.getProperty("java.vendor").contains("IBM") ? IBM_KRB5_LOGIN_MODULE
         : SUN_KRB5_LOGIN_MODULE;
   }

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/2de4b192/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java b/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
index 627abe5..217cbeb 100644
--- a/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
+++ b/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
@@ -46,17 +46,14 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.security.Principal;
 import java.security.PrivilegedAction;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Locale;
-import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.Callable;
 
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosPrincipal;
-import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.LoginContext;
 import javax.security.auth.login.LoginException;
 
@@ -798,7 +795,8 @@ public class HttpServer {
       Subject subject = new Subject(false, principals, new HashSet<Object>(),
           new HashSet<Object>());
 
-      KeytabJaasConf conf = new KeytabJaasConf(b.kerberosPrincipal, b.keytab.toString());
+      ServerKeytabJaasConf conf = new ServerKeytabJaasConf(b.kerberosPrincipal,
+          b.keytab.toString());
       String confName = "NotUsed";
       try {
         LoginContext loginContext = new LoginContext(confName, subject, null, conf);
@@ -808,42 +806,6 @@ public class HttpServer {
         throw new RuntimeException(e);
       }
     }
-
-    /**
-     * Javax Configuration class which always returns a configuration for our keytab-based
-     * login.
-     */
-    private static class KeytabJaasConf extends javax.security.auth.login.Configuration {
-      private final String principal;
-      private final String keytab;
-
-      private KeytabJaasConf(String principal, String keytab) {
-        this.principal = principal;
-        this.keytab = keytab;
-      }
-
-      @Override public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
-        Map<String, String> options = new HashMap<String, String>();
-        options.put("storeKey", "true");
-        options.put("principal", principal);
-        options.put("keyTab", keytab);
-        options.put("doNotPrompt", "true");
-        options.put("useKeyTab", "true");
-        options.put("isInitiator", "false");
-        options.put("debug",
-            System.getProperty("sun.security.krb5.debug", "false")
-                .toLowerCase(Locale.ROOT));
-
-        return new AppConfigurationEntry[] {new AppConfigurationEntry(getKrb5LoginModuleName(),
-            AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, options)};
-      }
-    }
-
-    private static String getKrb5LoginModuleName() {
-      return System.getProperty("java.vendor").contains("IBM")
-          ? "com.ibm.security.auth.module.Krb5LoginModule"
-          : "com.sun.security.auth.module.Krb5LoginModule";
-    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/2de4b192/server/src/main/java/org/apache/calcite/avatica/server/ServerKeytabJaasConf.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/avatica/server/ServerKeytabJaasConf.java b/server/src/main/java/org/apache/calcite/avatica/server/ServerKeytabJaasConf.java
new file mode 100644
index 0000000..c0927f1
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/avatica/server/ServerKeytabJaasConf.java
@@ -0,0 +1,67 @@
+/*
+ * 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.
+ */
+package org.apache.calcite.avatica.server;
+
+import org.apache.calcite.avatica.remote.KerberosConnection;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+
+/**
+ * Javax Configuration class which always returns a configuration for our keytab-based
+ * login suitable for callers which are acting as a initiator or acceptor (e.g. a server).
+ */
+public class ServerKeytabJaasConf extends Configuration {
+  private static final Logger LOG = LoggerFactory.getLogger(ServerKeytabJaasConf.class);
+  private final String principal;
+  private final String keytab;
+
+  public ServerKeytabJaasConf(String principal, String keytab) {
+    this.principal = principal;
+    this.keytab = keytab;
+  }
+
+  @Override public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+    Map<String, String> options = new HashMap<String, String>();
+    options.put("principal", principal);
+    options.put("refreshKrb5Config", "true");
+    if (KerberosConnection.isIbmJava()) {
+      options.put("useKeytab", keytab);
+      options.put("credsType", "both");
+    } else {
+      options.put("keyTab", keytab);
+      options.put("useKeyTab", "true");
+      options.put("isInitiator", "false");
+      options.put("doNotPrompt", "true");
+      options.put("storeKey", "true");
+    }
+
+    LOG.debug("JAAS Configuration for server keytab-based Kerberos login: {}", options);
+
+    return new AppConfigurationEntry[] {new AppConfigurationEntry(
+        KerberosConnection.getKrb5LoginModuleName(),
+        AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, options)};
+  }
+}
+
+// End ServerKeytabJaasConf.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/2de4b192/server/src/test/java/org/apache/calcite/avatica/AvaticaSpnegoTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/AvaticaSpnegoTest.java b/server/src/test/java/org/apache/calcite/avatica/AvaticaSpnegoTest.java
index 34329e2..eb02f88 100644
--- a/server/src/test/java/org/apache/calcite/avatica/AvaticaSpnegoTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/AvaticaSpnegoTest.java
@@ -19,10 +19,10 @@ package org.apache.calcite.avatica;
 import org.apache.calcite.avatica.jdbc.JdbcMeta;
 import org.apache.calcite.avatica.remote.Driver;
 import org.apache.calcite.avatica.remote.LocalService;
+import org.apache.calcite.avatica.server.AvaticaJaasKrbUtil;
 import org.apache.calcite.avatica.server.HttpServer;
 
 import org.apache.kerby.kerberos.kerb.KrbException;
-import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
 import org.apache.kerby.kerberos.kerb.client.KrbConfig;
 import org.apache.kerby.kerberos.kerb.client.KrbConfigKey;
 import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
@@ -193,8 +193,8 @@ public class AvaticaSpnegoTest {
     try {
       final String tableName = "allowed_clients";
       // Create the subject for the client
-      final Subject clientSubject = JaasKrbUtil.loginUsingKeytab(SpnegoTestUtil.CLIENT_PRINCIPAL,
-          clientKeytab);
+      final Subject clientSubject = AvaticaJaasKrbUtil.loginUsingKeytab(
+          SpnegoTestUtil.CLIENT_PRINCIPAL, clientKeytab);
 
       // The name of the principal
 

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/2de4b192/server/src/test/java/org/apache/calcite/avatica/SpnegoTestUtil.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/SpnegoTestUtil.java b/server/src/test/java/org/apache/calcite/avatica/SpnegoTestUtil.java
index 4edbfd1..7d4f1d6 100644
--- a/server/src/test/java/org/apache/calcite/avatica/SpnegoTestUtil.java
+++ b/server/src/test/java/org/apache/calcite/avatica/SpnegoTestUtil.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.avatica;
 
+import org.apache.calcite.avatica.remote.KerberosConnection;
 import org.apache.calcite.avatica.remote.Service.RpcMetadataResponse;
 import org.apache.calcite.avatica.server.AvaticaHandler;
 
@@ -107,15 +108,24 @@ public class SpnegoTestUtil {
                      new FileOutputStream(configFile),
                      StandardCharsets.UTF_8))) {
       // Server login
-      writer.write("com.sun.security.jgss.accept {\n");
-      writer.write(" com.sun.security.auth.module.Krb5LoginModule required\n");
+      if (KerberosConnection.isIbmJava()) {
+        writer.write("com.ibm.security.jgss.accept {\n");
+      } else {
+        writer.write("com.sun.security.jgss.accept {\n");
+      }
+      writer.write(" " + KerberosConnection.getKrb5LoginModuleName() + " required\n");
       writer.write(" principal=\"" + SERVER_PRINCIPAL + "\"\n");
-      writer.write(" useKeyTab=true\n");
-      writer.write(" keyTab=\"" + serverKeytab.getPath() + "\"\n");
-      writer.write(" storeKey=true \n");
       // Some extra debug information from JAAS
-      //writer.write(" debug=true\n");
-      writer.write(" isInitiator=false;\n");
+      // writer.write(" debug=true\n");
+      if (KerberosConnection.isIbmJava()) {
+        writer.write(" useKeytab=\"" + serverKeytab.getPath() + "\"\n");
+        writer.write(" credsType=both;\n");
+      } else {
+        writer.write(" useKeyTab=true\n");
+        writer.write(" keyTab=\"" + serverKeytab.getPath() + "\"\n");
+        writer.write(" isInitiator=false\n");
+        writer.write(" storeKey=true;\n");
+      }
       writer.write("};\n");
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/2de4b192/server/src/test/java/org/apache/calcite/avatica/server/AvaticaJaasKrbUtil.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/server/AvaticaJaasKrbUtil.java b/server/src/test/java/org/apache/calcite/avatica/server/AvaticaJaasKrbUtil.java
new file mode 100644
index 0000000..f8a7c9b
--- /dev/null
+++ b/server/src/test/java/org/apache/calcite/avatica/server/AvaticaJaasKrbUtil.java
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ */
+package org.apache.calcite.avatica.server;
+
+import org.apache.calcite.avatica.remote.ClientKeytabJaasConf;
+
+import java.io.File;
+import java.security.Principal;
+import java.util.HashSet;
+import java.util.Set;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+
+/**
+ * Copy of JaasKrbUtil from Kerby that works with IBM Java as well as Oracle Java.
+ */
+public class AvaticaJaasKrbUtil {
+
+  private AvaticaJaasKrbUtil() {}
+
+  public static Subject loginUsingKeytab(
+      String principal, File keytabFile) throws LoginException {
+    Set<Principal> principals = new HashSet<Principal>();
+    principals.add(new KerberosPrincipal(principal));
+
+    Subject subject = new Subject(false, principals,
+        new HashSet<Object>(), new HashSet<Object>());
+
+    Configuration conf = useKeytab(principal, keytabFile);
+    String confName = "KeytabConf";
+    LoginContext loginContext = new LoginContext(confName, subject, null, conf);
+    loginContext.login();
+    return loginContext.getSubject();
+  }
+
+  public static Configuration useKeytab(String principal, File keytabFile) {
+    return new ClientKeytabJaasConf(principal, keytabFile.toString());
+  }
+}
+
+// End AvaticaJaasKrbUtil.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/2de4b192/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithJaasTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithJaasTest.java b/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithJaasTest.java
index 06daa86..95a622a 100644
--- a/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithJaasTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithJaasTest.java
@@ -20,7 +20,6 @@ import org.apache.calcite.avatica.SpnegoTestUtil;
 import org.apache.calcite.avatica.remote.AvaticaCommonsHttpClientSpnegoImpl;
 
 import org.apache.kerby.kerberos.kerb.KrbException;
-import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
 import org.apache.kerby.kerberos.kerb.client.KrbConfig;
 import org.apache.kerby.kerberos.kerb.client.KrbConfigKey;
 import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
@@ -36,7 +35,10 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
@@ -102,6 +104,15 @@ public class HttpServerSpnegoWithJaasTest {
     kdc.start();
     isKdcStarted = true;
 
+    try (FileInputStream fis = new FileInputStream(new File(kdcDir, "krb5.conf"));
+        InputStreamReader isr = new InputStreamReader(fis, StandardCharsets.UTF_8);
+        BufferedReader r = new BufferedReader(isr)) {
+      String line;
+      while ((line = r.readLine()) != null) {
+        LOG.debug("KRB5 Config line: {}", line);
+      }
+    }
+
     File keytabDir = new File(target, HttpServerSpnegoWithJaasTest.class.getSimpleName()
         + "_keytabs");
     if (keytabDir.exists()) {
@@ -187,8 +198,8 @@ public class HttpServerSpnegoWithJaasTest {
     Assume.assumeThat("Test disabled on Windows", File.separatorChar, is('/'));
 
     // Create the subject for the client
-    final Subject clientSubject = JaasKrbUtil.loginUsingKeytab(SpnegoTestUtil.CLIENT_PRINCIPAL,
-        clientKeytab);
+    final Subject clientSubject = AvaticaJaasKrbUtil.loginUsingKeytab(
+        SpnegoTestUtil.CLIENT_PRINCIPAL, clientKeytab);
     final Set<Principal> clientPrincipals = clientSubject.getPrincipals();
     // Make sure the subject has a principal
     assertFalse(clientPrincipals.isEmpty());

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/2de4b192/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithoutJaasTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithoutJaasTest.java b/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithoutJaasTest.java
index d30b760..2752cae 100644
--- a/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithoutJaasTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithoutJaasTest.java
@@ -20,7 +20,6 @@ import org.apache.calcite.avatica.SpnegoTestUtil;
 import org.apache.calcite.avatica.remote.AvaticaCommonsHttpClientSpnegoImpl;
 
 import org.apache.kerby.kerberos.kerb.KrbException;
-import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
 import org.apache.kerby.kerberos.kerb.client.KrbConfig;
 import org.apache.kerby.kerberos.kerb.client.KrbConfigKey;
 import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
@@ -35,7 +34,10 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
@@ -98,6 +100,15 @@ public class HttpServerSpnegoWithoutJaasTest {
     kdc.start();
     isKdcStarted = true;
 
+    try (FileInputStream fis = new FileInputStream(new File(kdcDir, "krb5.conf"));
+        InputStreamReader isr = new InputStreamReader(fis, StandardCharsets.UTF_8);
+        BufferedReader r = new BufferedReader(isr)) {
+      String line;
+      while ((line = r.readLine()) != null) {
+        LOG.debug("KRB5 Config line: {}", line);
+      }
+    }
+
     File keytabDir = new File(target, HttpServerSpnegoWithoutJaasTest.class.getSimpleName()
         + "_keytabs");
     if (keytabDir.exists()) {
@@ -174,8 +185,8 @@ public class HttpServerSpnegoWithoutJaasTest {
 
   @Test public void testAuthenticatedClientsAllowed() throws Exception {
     // Create the subject for the client
-    final Subject clientSubject = JaasKrbUtil.loginUsingKeytab(SpnegoTestUtil.CLIENT_PRINCIPAL,
-        clientKeytab);
+    final Subject clientSubject = AvaticaJaasKrbUtil.loginUsingKeytab(
+        SpnegoTestUtil.CLIENT_PRINCIPAL, clientKeytab);
     final Set<Principal> clientPrincipals = clientSubject.getPrincipals();
     // Make sure the subject has a principal
     assertFalse(clientPrincipals.isEmpty());