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

[kudu] 03/03: jwt: Java client usage of JWTs

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

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

commit a3a7c97be031f8fc32402e430eff1a89c19dbdfb
Author: Zoltan Chovan <zc...@cloudera.com>
AuthorDate: Thu Apr 13 19:26:04 2023 +0200

    jwt: Java client usage of JWTs
    
    This change introduces modifications to the Negotiator and the
    SecurityContext  classes to be able to handle JWT authentication from
    the Java client.
    
    Also adding a test where the client is set up with a JWT that was
    generated by the MiniOIDC that starts along with the
    ExternalMiniCluster.
    
    Change-Id: Id02dcd7ee57a838f4763500e78053e21aac21c09
    Reviewed-on: http://gerrit.cloudera.org:8080/18477
    Reviewed-by: Attila Bukor <ab...@apache.org>
    Tested-by: Attila Bukor <ab...@apache.org>
---
 .../java/org/apache/kudu/client/Negotiator.java    | 24 +++++++++++++
 .../org/apache/kudu/client/SecurityContext.java    | 24 +++++++++++++
 .../java/org/apache/kudu/test/KuduTestHarness.java |  4 +++
 .../org/apache/kudu/test/TestMiniKuduCluster.java  | 41 +++++++++++++++++-----
 4 files changed, 84 insertions(+), 9 deletions(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java b/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
index 5fe628598..f17dfce4e 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
@@ -82,6 +82,7 @@ import org.apache.kudu.rpc.RpcHeader.AuthenticationTypePB;
 import org.apache.kudu.rpc.RpcHeader.NegotiatePB;
 import org.apache.kudu.rpc.RpcHeader.NegotiatePB.NegotiateStep;
 import org.apache.kudu.rpc.RpcHeader.RpcFeatureFlag;
+import org.apache.kudu.security.Token.JwtRawPB;
 import org.apache.kudu.security.Token.SignedTokenPB;
 import org.apache.kudu.util.SecurityUtil;
 
@@ -176,6 +177,7 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
    * ensure that it doesn't change over the course of a negotiation attempt.
    */
   private final SignedTokenPB authnToken;
+  private final JwtRawPB jsonWebToken;
 
   private enum AuthnTokenNotUsedReason {
     NONE_AVAILABLE("no token is available"),
@@ -267,6 +269,9 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
       this.authnToken = null;
       this.authnTokenNotUsedReason = AuthnTokenNotUsedReason.NONE_AVAILABLE;
     }
+    JwtRawPB jwt = securityContext.getJsonWebToken();
+    // TODO(zchovan): also guard this on the presence of certs.
+    this.jsonWebToken = jwt;
   }
 
   public void sendHello(ChannelHandlerContext ctx) {
@@ -558,6 +563,12 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
               "but client had no valid token");
         }
         break;
+      case JWT:
+        if (jsonWebToken == null) {
+          throw new IllegalArgumentException("server chose JWT authentication " +
+              "but client had no valid JWT");
+        }
+        break;
       default:
         throw new IllegalArgumentException("server chose bad authn type " + chosenAuthnType);
     }
@@ -745,6 +756,9 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
       case TOKEN:
         sendTokenExchange(ctx);
         break;
+      case JWT:
+        sendJwtExchange(ctx);
+        break;
       default:
         throw new AssertionError("unreachable");
     }
@@ -764,6 +778,16 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
     sendSaslMessage(ctx, builder.build());
   }
 
+  private void sendJwtExchange(ChannelHandlerContext ctx) {
+    Preconditions.checkNotNull(jsonWebToken);
+    Preconditions.checkNotNull(sslHandshakeFuture);
+    Preconditions.checkState(sslHandshakeFuture.isSuccess());
+    RpcHeader.NegotiatePB.Builder builder = RpcHeader.NegotiatePB.newBuilder()
+        .setStep(NegotiateStep.JWT_EXCHANGE)
+        .setJwtRaw(jsonWebToken);
+    sendSaslMessage(ctx, builder.build());
+  }
+
   private void handleTokenExchangeResponse(ChannelHandlerContext ctx, NegotiatePB response)
       throws SaslException {
     Preconditions.checkArgument(response.getStep() == NegotiateStep.TOKEN_EXCHANGE,
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 8073c4dd6..8597e0cd9 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
@@ -53,6 +53,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.kudu.client.Client.AuthenticationCredentialsPB;
+import org.apache.kudu.security.Token.JwtRawPB;
 import org.apache.kudu.security.Token.SignedTokenPB;
 import org.apache.kudu.security.Token.TokenPB;
 import org.apache.kudu.util.Pair;
@@ -73,6 +74,10 @@ class SecurityContext {
   @Nullable
   private SignedTokenPB authnToken;
 
+  @GuardedBy("this")
+  @Nullable
+  private JwtRawPB jsonWebToken;
+
   @GuardedBy("this")
   private String realUser;
 
@@ -310,8 +315,10 @@ class SecurityContext {
       }
 
       LOG.debug("Importing authentication credentials with {} authn token, " +
+                "JWT={} , " +
                 "{} cert(s), and realUser={}",
                 pb.hasAuthnToken() ? "one" : "no",
+                pb.hasJwt() ? pb.getJwt() : "<none>",
                 pb.getCaCertDersCount(),
                 pb.hasRealUser() ? pb.getRealUser() : "<none>");
       if (pb.hasAuthnToken()) {
@@ -319,6 +326,10 @@ class SecurityContext {
       }
       trustCertificates(pb.getCaCertDersList());
 
+      if (pb.hasJwt()) {
+        jsonWebToken = pb.getJwt();
+      }
+
       if (pb.hasRealUser()) {
         realUser = pb.getRealUser();
       }
@@ -335,6 +346,11 @@ class SecurityContext {
     return authnToken;
   }
 
+  @Nullable
+  public synchronized JwtRawPB getJsonWebToken() {
+    return jsonWebToken;
+  }
+
   /**
    * Set the token that we will use to authenticate to servers. Replaces any
    * prior token.
@@ -343,6 +359,14 @@ class SecurityContext {
     authnToken = token;
   }
 
+  /**
+   * Set the JWT that we will use to authenticate to the server. Replaces any
+   * prior JWT.
+   */
+  public synchronized void setJsonWebToken(JwtRawPB jwt) {
+    jsonWebToken = jwt;
+  }
+
   /**
    * Create an SSLEngine which will trust all certificates without verification.
    */
diff --git a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/KuduTestHarness.java b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/KuduTestHarness.java
index afa470b2f..30ea8a365 100644
--- a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/KuduTestHarness.java
+++ b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/KuduTestHarness.java
@@ -487,6 +487,10 @@ public class KuduTestHarness extends ExternalResource {
     client = asyncClient.syncClient();
   }
 
+  public String createJwtFor(String accountId, String subject, boolean isValid) throws IOException {
+    return miniCluster.createJwtFor(accountId, subject, isValid);
+  }
+
   /**
    * An annotation that can be added to each test method to
    * define additional master server flags to be used when
diff --git a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java
index a36f38d0a..ddfbe6d9b 100644
--- a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java
+++ b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java
@@ -26,16 +26,21 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.net.Socket;
 
+import com.google.protobuf.ByteString;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.client.AsyncKuduClient;
+import org.apache.kudu.client.Client.AuthenticationCredentialsPB;
 import org.apache.kudu.client.HostAndPort;
 import org.apache.kudu.client.KuduClient;
 import org.apache.kudu.client.KuduClient.KuduClientBuilder;
 import org.apache.kudu.client.ListTablesResponse;
 import org.apache.kudu.client.TimeoutTracker;
+import org.apache.kudu.security.Token.JwtRawPB;
 import org.apache.kudu.test.cluster.FakeDNS;
 import org.apache.kudu.test.cluster.MiniKuduCluster;
+import org.apache.kudu.test.cluster.MiniKuduCluster.MiniKuduClusterBuilder;
 import org.apache.kudu.test.junit.RetryRule;
 import org.apache.kudu.tools.Tool.CreateClusterRequestPB.JwksOptionsPB;
 import org.apache.kudu.tools.Tool.CreateClusterRequestPB.MiniOidcOptionsPB;
@@ -49,6 +54,9 @@ public class TestMiniKuduCluster {
   @Rule
   public RetryRule retryRule = new RetryRule();
 
+  @Rule
+  public KuduTestHarness harness;
+
   @Test(timeout = 50000)
   public void test() throws Exception {
     try (MiniKuduCluster cluster = new MiniKuduCluster.MiniKuduClusterBuilder()
@@ -111,16 +119,31 @@ public class TestMiniKuduCluster {
 
   @Test(timeout = 50000)
   public void testJwt() throws Exception {
-    try (MiniKuduCluster cluster = new MiniKuduCluster.MiniKuduClusterBuilder()
-                                                      .numMasterServers(NUM_MASTERS)
-                                                      .numTabletServers(0)
-                                                      .enableClientJwt()
-                                                      .addJwks("account-id", true)
-                                                      .build();
-         KuduClient client = new KuduClientBuilder(cluster.getMasterAddressesAsString()).build()) {
-      String jwt = cluster.createJwtFor("account-id", "subject", true);
-
+    try {
+      MiniKuduClusterBuilder clusterBuilder = new MiniKuduCluster.MiniKuduClusterBuilder()
+              .numMasterServers(NUM_MASTERS)
+              .numTabletServers(0)
+              .enableClientJwt()
+              .addJwks("account-id", true);
+
+      harness = new KuduTestHarness(clusterBuilder);
+      harness.before();
+      harness.startAllMasterServers();
+
+      String jwt = harness.createJwtFor("account-id", "subject", true);
       assertNotNull(jwt);
+      AuthenticationCredentialsPB credentials = AuthenticationCredentialsPB.newBuilder()
+              .setJwt(JwtRawPB.newBuilder()
+                      .setJwtData(ByteString.copyFromUtf8(jwt))
+                      .build())
+              .build();
+
+      AsyncKuduClient c = harness.getAsyncClient();
+      c.importAuthenticationCredentials(credentials.toByteArray());
+      c.getTablesList();
+
+    } catch (Exception e) {
+      throw new RuntimeException(e);
     }
   }