You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2017/02/23 00:54:04 UTC

kudu git commit: java: fetch and adopt authn token and CA from master

Repository: kudu
Updated Branches:
  refs/heads/master ed44a20df -> 3cd434e51


java: fetch and adopt authn token and CA from master

This makes the Java client fetch a token and CA information from the
master upon first connect, and also exposes APIs to export and import
the security information in a serialized protobuf form.

A new test connects to a Kerberized cluster and serializes this
authentication info. It then drops Kerberos credentials, and passes the
credentials to a new client, which is still able to connect to the
cluster using the serialized info.

The new APIs introduced here are marked Unstable since we may need to
get more experience using them in execution frameworks before locking
them down.

Change-Id: I24e71f0f5feefa36106a50074d40731911eff64b
Reviewed-on: http://gerrit.cloudera.org:8080/6077
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/3cd434e5
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/3cd434e5
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/3cd434e5

Branch: refs/heads/master
Commit: 3cd434e51af9d092baea34dd8cdd21317edcb168
Parents: ed44a20
Author: Todd Lipcon <to...@apache.org>
Authored: Fri Feb 17 00:15:33 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Feb 22 23:51:58 2017 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/client/AsyncKuduClient.java |  62 ++++++++++-
 .../kudu/client/ConnectToClusterResponse.java   |   9 +-
 .../java/org/apache/kudu/client/KuduClient.java |  27 +++++
 .../org/apache/kudu/client/SecurityContext.java | 102 +++++++++++++++++--
 .../org/apache/kudu/client/TestKuduClient.java  |  15 ++-
 .../org/apache/kudu/client/TestNegotiator.java  |   3 +-
 .../org/apache/kudu/client/TestSecurity.java    |  76 ++++++++++++++
 src/kudu/client/CMakeLists.txt                  |   3 +-
 src/kudu/client/client.proto                    |  10 ++
 9 files changed, 284 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/3cd434e5/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
index cbb4ad5..acda901 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
@@ -30,6 +30,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.kudu.client.ExternalConsistencyMode.CLIENT_PROPAGATED;
 
 import java.net.UnknownHostException;
+import java.security.cert.CertificateException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -51,6 +52,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.net.HostAndPort;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.protobuf.ByteString;
 import com.google.protobuf.Message;
 import com.stumbleupon.async.Callback;
 import com.stumbleupon.async.Deferred;
@@ -510,6 +512,48 @@ public class AsyncKuduClient implements AutoCloseable {
   }
 
   /**
+   * Export serialized authentication data that may be passed to a different
+   * client instance and imported to provide that client the ability to connect
+   * to the cluster.
+   */
+  @InterfaceStability.Unstable
+  public Deferred<byte[]> exportAuthenticationCredentials() {
+    byte[] authnData = securityContext.exportAuthenticationCredentials();
+    if (authnData != null) {
+      return Deferred.fromResult(authnData);
+    }
+    // We have no authn data -- connect to the master, which will fetch
+    // new info.
+    return getMasterTableLocationsPB(null)
+        .addCallback(new MasterLookupCB(masterTable, null, 1))
+        .addCallback(new Callback<byte[], Object>() {
+          @Override
+          public byte[] call(Object ignored) {
+            // Connecting to the cluster should have also fetched the
+            // authn data.
+            return securityContext.exportAuthenticationCredentials();
+          }
+        });
+  }
+
+  /**
+   * Import data allowing this client to authenticate to the cluster.
+   * This will typically be used before making any connections to servers
+   * in the cluster.
+   *
+   * Note that, if this client has already been used by one user, this
+   * method cannot be used to switch authenticated users. Attempts to
+   * do so have undefined results, and may throw an exception.
+   *
+   * @param authnData then authentication data provided by a prior call to
+   * {@link #exportAuthenticationCredentials()}
+   */
+  @InterfaceStability.Unstable
+  public void importAuthenticationCredentials(byte[] authnData) {
+    securityContext.importAuthenticationCredentials(authnData);
+  }
+
+  /**
    * Get the timeout used for operations on sessions and scanners.
    * @return a timeout in milliseconds
    */
@@ -1066,7 +1110,23 @@ public class AsyncKuduClient implements AutoCloseable {
             new Callback<Master.GetTableLocationsResponsePB, ConnectToClusterResponse>() {
               @Override
               public Master.GetTableLocationsResponsePB call(ConnectToClusterResponse resp) {
-                // Once we've connected, we translate the located master into a TableLocations
+                // If the response has security info, adopt it.
+                if (resp.getConnectResponse().hasAuthnToken()) {
+                  securityContext.setAuthenticationToken(
+                      resp.getConnectResponse().getAuthnToken());
+                }
+                List<ByteString> caCerts = resp.getConnectResponse().getCaCertDerList();
+                if (!caCerts.isEmpty()) {
+                  try {
+                    securityContext.trustCertificates(caCerts);
+                  } catch (CertificateException e) {
+                    LOG.warn("Ignoring invalid CA cert from leader {}: {}",
+                        resp.getLeaderHostAndPort(),
+                        e.getMessage());
+                  }
+                }
+
+                // Translate the located master into a TableLocations
                 // since the rest of our locations caching code expects this type.
                 return resp.getAsTableLocations();
               }

http://git-wip-us.apache.org/repos/asf/kudu/blob/3cd434e5/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java b/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
index 50b483e..6cb687f 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
@@ -40,7 +40,6 @@ class ConnectToClusterResponse {
   /** The response from that master */
   private final ConnectToMasterResponsePB connectResponse;
 
-
   public ConnectToClusterResponse(HostAndPort hostAndPort,
       ConnectToMasterResponsePB connectResponse) {
     super();
@@ -48,6 +47,14 @@ class ConnectToClusterResponse {
     this.connectResponse = connectResponse;
   }
 
+  public ConnectToMasterResponsePB getConnectResponse() {
+    return connectResponse;
+  }
+
+  public HostAndPort getLeaderHostAndPort() {
+    return leaderHostAndPort;
+  }
+
   /**
    * Return the location of the located leader master as if this had been a normal
    * tablet lookup. This is necessary so that we can cache the master location as

http://git-wip-us.apache.org/repos/asf/kudu/blob/3cd434e5/java/kudu-client/src/main/java/org/apache/kudu/client/KuduClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduClient.java b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduClient.java
index 141acad..371d196 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduClient.java
@@ -256,6 +256,33 @@ public class KuduClient implements AutoCloseable {
   }
 
   /**
+   * Export serialized authentication data that may be passed to a different
+   * client instance and imported to provide that client the ability to connect
+   * to the cluster.
+   */
+  @InterfaceStability.Unstable
+  public byte[] exportAuthenticationCredentials() throws KuduException {
+    return joinAndHandleException(asyncClient.exportAuthenticationCredentials());
+  }
+
+  /**
+   * Import data allowing this client to authenticate to the cluster.
+   * This will typically be used before making any connections to servers
+   * in the cluster.
+   *
+   * Note that, if this client has already been used by one user, this
+   * method cannot be used to switch authenticated users. Attempts to
+   * do so have undefined results, and may throw an exception.
+   *
+   * @param authnData then authentication data provided by a prior call to
+   * {@link #exportAuthenticationCredentials()}
+   */
+  @InterfaceStability.Unstable
+  public void importAuthenticationCredentials(byte[] authnData) {
+    asyncClient.importAuthenticationCredentials(authnData);
+  }
+
+  /**
    * Get the timeout used for operations on sessions and scanners.
    * @return a timeout in milliseconds
    */

http://git-wip-us.apache.org/repos/asf/kudu/blob/3cd434e5/java/kudu-client/src/main/java/org/apache/kudu/client/SecurityContext.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/SecurityContext.java b/java/kudu-client/src/main/java/org/apache/kudu/client/SecurityContext.java
index 51195da..a0af81f 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/SecurityContext.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/SecurityContext.java
@@ -21,6 +21,8 @@ import java.security.KeyStore;
 import java.security.cert.CertificateException;
 import java.security.cert.CertificateFactory;
 import java.security.cert.X509Certificate;
+import java.util.Collections;
+import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
 import javax.annotation.Nullable;
@@ -33,9 +35,14 @@ import javax.net.ssl.X509TrustManager;
 import javax.security.auth.Subject;
 
 import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
 
+import org.apache.kudu.client.Client.AuthenticationCredentialsPB;
 import org.apache.kudu.security.Token.SignedTokenPB;
+import org.apache.kudu.security.Token.TokenPB;
 
 /**
  * Class associated with a single AsyncKuduClient which stores security-related
@@ -65,6 +72,11 @@ class SecurityContext {
   @Nullable
   private final Subject subject;
 
+  /**
+   * The currently trusted CA certs, in DER format.
+   */
+  private List<ByteString> trustedCertDers = Collections.emptyList();
+
   public SecurityContext(Subject subject) {
     try {
       this.subject = subject;
@@ -84,6 +96,56 @@ class SecurityContext {
     return subject;
   }
 
+  public synchronized byte[] exportAuthenticationCredentials() {
+    if (authnToken == null || !hasTrustedCerts()) {
+      return null;
+    }
+
+    return AuthenticationCredentialsPB.newBuilder()
+        .setAuthnToken(authnToken)
+        .addAllCaCertDers(trustedCertDers)
+        .build().toByteArray();
+  }
+
+  private static String getUserFromToken(SignedTokenPB token)
+      throws InvalidProtocolBufferException {
+    TokenPB pb = TokenPB.parseFrom(token.getTokenData());
+    return pb.getAuthn().getUsername();
+  }
+
+  private static void checkUserMatches(SignedTokenPB oldToken, SignedTokenPB newToken)
+      throws InvalidProtocolBufferException {
+    String oldUser = getUserFromToken(oldToken);
+    String newUser = getUserFromToken(newToken);
+
+    if (!oldUser.equals(newUser)) {
+      throw new IllegalArgumentException(String.format(
+          "cannot import authentication data from a different user: old='%s', new='%s'",
+          oldUser, newUser));
+    }
+  }
+
+  public synchronized void importAuthenticationCredentials(byte[] authnData) {
+    try {
+      AuthenticationCredentialsPB pb = AuthenticationCredentialsPB.parseFrom(authnData);
+      if (authnToken != null) {
+        checkUserMatches(authnToken, pb.getAuthnToken());
+      }
+      // TODO(todd): also check that, if there is a JAAS subject, that
+      // the subject in the imported authn token matces the Kerberos
+      // principal in the JAAS subject. Alternatively, this could
+      // completely disable the JAAS authentication path (assumedly if
+      // we import a token, we want to _only_ act as the user in that
+      // token, and would rather have a connection failure than flip
+      // back to GSSAPI transparently.
+      trustCertificates(pb.getCaCertDersList());
+      authnToken = pb.getAuthnToken();
+
+    } catch (InvalidProtocolBufferException | CertificateException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
   /**
    * @return the current authentication token, or null if we have no valid token
    */
@@ -117,19 +179,22 @@ class SecurityContext {
   /**
    * @return true if any cert has been marked as trusted
    */
-  public boolean hasTrustedCerts() {
-    return trustManager.delegate.get() != null;
+  public synchronized boolean hasTrustedCerts() {
+    return !trustedCertDers.isEmpty();
   }
 
   /**
-   * Mark the given CA cert (provided in DER form) as the trusted CA cert for the
-   * client. Replaces any previously trusted cert.
-   * @throws CertificateException if the cert was invalid
+   * Create a trust manager which will trust all of the given CA certs.
    */
-  public void trustCertificate(ByteString certDer) throws CertificateException {
+  private static X509TrustManager createTrustManagerForCerts(Iterable<ByteString> certDers)
+      throws CertificateException {
     CertificateFactory certFactory = CertificateFactory.getInstance("X.509");
-    X509Certificate cert = (X509Certificate)certFactory.generateCertificate(
-        certDer.newInput());
+
+    List<X509Certificate> certs = Lists.newArrayList();
+    for (ByteString certDer : certDers) {
+      certs.add((X509Certificate)certFactory.generateCertificate(
+          certDer.newInput()));
+    }
 
     // This is implemented by making a new TrustManager and swapping it out under
     // our delegating trust manager. It might seem more straight-forward to instead
@@ -139,7 +204,10 @@ class SecurityContext {
     try {
       KeyStore certKeyStore = KeyStore.getInstance(KeyStore.getDefaultType());
       certKeyStore.load(null);
-      certKeyStore.setCertificateEntry("my-ca", cert);
+      int i = 0;
+      for (X509Certificate cert : certs) {
+        certKeyStore.setCertificateEntry(String.format("cert-%d",  i++), cert);
+      }
 
       TrustManagerFactory tmf = TrustManagerFactory.getInstance(
           TrustManagerFactory.getDefaultAlgorithm());
@@ -148,7 +216,7 @@ class SecurityContext {
       if (managers.length != 1) {
         throw new RuntimeException("TrustManagerFactory generated multiple TrustManagers");
       }
-      trustManager.delegate.set((X509TrustManager) managers[0]);
+      return (X509TrustManager) managers[0];
     } catch (Exception e) {
       Throwables.propagateIfInstanceOf(e, CertificateException.class);
       throw Throwables.propagate(e);
@@ -156,6 +224,19 @@ class SecurityContext {
   }
 
   /**
+   * Mark the given CA cert (provided in DER form) as the trusted CA cert for the
+   * client. Replaces any previously trusted cert.
+   * @throws CertificateException if the cert was invalid
+   */
+  public void trustCertificates(List<ByteString> certDers) throws CertificateException {
+    X509TrustManager tm = createTrustManagerForCerts(certDers);
+    synchronized (this) {
+      trustManager.delegate.set(tm);
+      trustedCertDers = ImmutableList.copyOf(certDers);
+    }
+  }
+
+  /**
    * TrustManager implementation which will trust any certificate.
    */
   private static class TrustAnyCert implements X509TrustManager {
@@ -199,5 +280,4 @@ class SecurityContext {
       return delegate.get().getAcceptedIssuers();
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/3cd434e5/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
index f5435f1..488d7ea 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
@@ -21,12 +21,7 @@ import static org.apache.kudu.client.KuduPredicate.ComparisonOp.GREATER_EQUAL;
 import static org.apache.kudu.client.KuduPredicate.ComparisonOp.LESS;
 import static org.apache.kudu.client.KuduPredicate.ComparisonOp.LESS_EQUAL;
 import static org.apache.kudu.client.RowResult.timestampToString;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 import static org.junit.matchers.JUnitMatchers.containsString;
 
 import java.util.ArrayList;
@@ -37,14 +32,12 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterators;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.kudu.ColumnSchema;
-import org.apache.kudu.Common;
 import org.apache.kudu.Schema;
 import org.apache.kudu.Type;
 
@@ -522,6 +515,12 @@ public class TestKuduClient extends BaseKuduTest {
     return count.get();
   }
 
+  @Test
+  public void testGetAuthnToken() throws Exception {
+    byte[] token = client.exportAuthenticationCredentials().join();
+    assertNotNull(token);
+  }
+
   /**
    * Tests scan tokens by creating a set of scan tokens, serializing them, and
    * then executing them in parallel with separate client instances. This

http://git-wip-us.apache.org/repos/asf/kudu/blob/3cd434e5/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java
index b4885f3..945160c 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java
@@ -50,6 +50,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.Message;
@@ -354,7 +355,7 @@ public class TestNegotiator {
    */
   @Test
   public void testTokenAuthWithTrustedCerts() throws Exception {
-    secContext.trustCertificate(ByteString.copyFromUtf8(CA_CERT_DER));
+    secContext.trustCertificates(ImmutableList.of(ByteString.copyFromUtf8(CA_CERT_DER)));
     secContext.setAuthenticationToken(SignedTokenPB.getDefaultInstance());
     startNegotiation(false);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/3cd434e5/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
new file mode 100644
index 0000000..35fc389
--- /dev/null
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.kudu.client;
+
+import static org.junit.Assert.assertNotNull;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.kudu.util.SecurityUtil;
+
+public class TestSecurity extends BaseKuduTest {
+
+  private static final String TABLE_NAME = "TestSecurity-table";
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    miniClusterBuilder.enableKerberos()
+    .addMasterFlag("--rpc_trace_negotiation");
+
+    BaseKuduTest.setUpBeforeClass();
+  }
+
+  /**
+   * Test that a client can export its authentication data and pass it to
+   * a new client which has no Kerberos credentials, which will then
+   * be able to authenticate to the masters and tablet servers using tokens.
+   */
+  @Test
+  public void testImportExportAuthenticationCredentials() throws InterruptedException, Exception {
+    byte[] authnData = client.exportAuthenticationCredentials().join();
+    assertNotNull(authnData);
+    String oldTicketCache = System.getProperty(SecurityUtil.KUDU_TICKETCACHE_PROPERTY);
+    System.clearProperty(SecurityUtil.KUDU_TICKETCACHE_PROPERTY);
+    try {
+      KuduClient newClient = new KuduClient.KuduClientBuilder(masterAddresses)
+          .defaultAdminOperationTimeoutMs(2000)
+          .build();
+
+      // Test that a client with no credentials cannot list servers.
+      try {
+        newClient.listTabletServers();
+        Assert.fail("should not have been able to connect to a secure cluster " +
+            "with no credentials");
+      } catch (Exception e) {
+        // Expected.
+        // TODO(todd): assert on the particular exception type and error string
+      }
+
+      // If we import the authentication data from the old authenticated client,
+      // we should now be able to perform all of the normal client operations.
+      newClient.importAuthenticationCredentials(authnData);
+      KuduTable table = newClient.createTable(TABLE_NAME, basicSchema,
+          getBasicCreateTableOptions());
+      KuduSession session = newClient.newSession();
+      session.apply(createBasicSchemaInsert(table, 1));
+      session.flush();
+    } finally {
+      // Restore ticket cache for other test cases.
+      System.setProperty(SecurityUtil.KUDU_TICKETCACHE_PROPERTY, oldTicketCache);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/kudu/blob/3cd434e5/src/kudu/client/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/client/CMakeLists.txt b/src/kudu/client/CMakeLists.txt
index e7dd8ae..5d38a81 100644
--- a/src/kudu/client/CMakeLists.txt
+++ b/src/kudu/client/CMakeLists.txt
@@ -21,7 +21,8 @@ PROTOBUF_GENERATE_CPP(
   BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
   PROTO_FILES client.proto)
 set(CLIENT_PROTO_LIBS
-  kudu_common_proto)
+  kudu_common_proto
+  token_proto)
 ADD_EXPORTABLE_LIBRARY(client_proto
   SRCS ${CLIENT_PROTO_SRCS}
   DEPS ${CLIENT_PROTO_LIBS}

http://git-wip-us.apache.org/repos/asf/kudu/blob/3cd434e5/src/kudu/client/client.proto
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.proto b/src/kudu/client/client.proto
index a26e447..17e2057 100644
--- a/src/kudu/client/client.proto
+++ b/src/kudu/client/client.proto
@@ -22,6 +22,7 @@ package kudu.client;
 option java_package = "org.apache.kudu.client";
 
 import "kudu/common/common.proto";
+import "kudu/security/token.proto";
 import "kudu/util/pb_util.proto";
 
 // Serialization format for client scan tokens. Scan tokens are serializable
@@ -92,3 +93,12 @@ message ScanTokenPB {
   // Whether the scan should be fault tolerant.
   optional bool fault_tolerant = 14 [default = false];
 }
+
+
+// All of the data necessary to authenticate to a cluster from a client with
+// no Kerberos credentials.
+message AuthenticationCredentialsPB {
+  optional security.SignedTokenPB authn_token = 1;
+
+  repeated bytes ca_cert_ders = 2;
+}