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:53:59 UTC

[kudu] 01/03: jwt: add control points for test binaries

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 3d56a26b962d83f3f2789b2f2c7c2967322e98e9
Author: Andrew Wong <aw...@cloudera.com>
AuthorDate: Mon May 2 12:08:50 2022 -0700

    jwt: add control points for test binaries
    
    This hooks the configurations of the MiniOidc into the Java test binary,
    allowing test code to add statically-defined JWKS associated with
    specific account IDs, to be fetched via OIDC Discovery Endpoint.
    
    Change-Id: I489a67e93610467c5a2caabcf3e5603cbb49d118
    Reviewed-on: http://gerrit.cloudera.org:8080/18476
    Tested-by: Kudu Jenkins
    Reviewed-by: Attila Bukor <ab...@apache.org>
---
 .../org/apache/kudu/backup/TestKuduBackup.scala    |  6 +--
 .../apache/kudu/test/cluster/MiniKuduCluster.java  | 45 ++++++++++++++++++++--
 .../org/apache/kudu/test/TestMiniKuduCluster.java  | 17 ++++++++
 src/kudu/tools/tool.proto                          | 10 ++++-
 src/kudu/tools/tool_action_test.cc                 | 24 ++++++++++++
 5 files changed, 94 insertions(+), 8 deletions(-)

diff --git a/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala b/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala
index f1421b5ff..0a61f989c 100644
--- a/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala
+++ b/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala
@@ -609,7 +609,7 @@ class TestKuduBackup extends KuduTestSuite {
     // Check the range bounds and the hash schema of each range of the restored table.
     val restoredTable = kuduClient.openTable(s"$tableName-restore")
     assertEquals(
-        "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
+      "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
         "100 <= VALUES < 200 HASH(key) PARTITIONS 3]",
       restoredTable.getFormattedRangePartitionsWithHashSchema(10000).toString
     )
@@ -635,7 +635,7 @@ class TestKuduBackup extends KuduTestSuite {
     // Check the range bounds and the hash schema of each range of the restored table.
     val restoredTable = kuduClient.openTable(s"$tableName-restore")
     assertEquals(
-        "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
+      "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
         "100 <= VALUES < 200 HASH(key) PARTITIONS 3, " +
         "200 <= VALUES < 300 HASH(key) PARTITIONS 4]",
       restoredTable.getFormattedRangePartitionsWithHashSchema(10000).toString
@@ -694,7 +694,7 @@ class TestKuduBackup extends KuduTestSuite {
     // Check the range bounds and the hash schema of each range of the restored table.
     val restoredTable = kuduClient.openTable(s"$tableName-restore")
     assertEquals(
-        "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
+      "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
         "100 <= VALUES < 200 HASH(key) PARTITIONS 3, " +
         "200 <= VALUES < 300 HASH(key) PARTITIONS 5, " +
         "300 <= VALUES < 400 HASH(key) PARTITIONS 6]",
diff --git a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/cluster/MiniKuduCluster.java b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/cluster/MiniKuduCluster.java
index cd5eb2acb..31bcdc6f9 100644
--- a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/cluster/MiniKuduCluster.java
+++ b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/cluster/MiniKuduCluster.java
@@ -50,7 +50,10 @@ import org.apache.kudu.test.TempDirUtils;
 import org.apache.kudu.tools.Tool.ControlShellRequestPB;
 import org.apache.kudu.tools.Tool.ControlShellResponsePB;
 import org.apache.kudu.tools.Tool.CreateClusterRequestPB;
+import org.apache.kudu.tools.Tool.CreateClusterRequestPB.JwksOptionsPB;
 import org.apache.kudu.tools.Tool.CreateClusterRequestPB.MiniKdcOptionsPB;
+import org.apache.kudu.tools.Tool.CreateClusterRequestPB.MiniOidcOptionsPB;
+import org.apache.kudu.tools.Tool.CreateJwtRequestPB;
 import org.apache.kudu.tools.Tool.DaemonIdentifierPB;
 import org.apache.kudu.tools.Tool.DaemonInfoPB;
 import org.apache.kudu.tools.Tool.GetKDCEnvVarsRequestPB;
@@ -110,9 +113,11 @@ public final class MiniKuduCluster implements AutoCloseable {
   private final ImmutableList<String> locationInfo;
   private final String clusterRoot;
   private final String principal;
+  private final boolean enableClientJwt;
 
   private MiniKdcOptionsPB kdcOptionsPb;
   private final Common.HmsMode hmsMode;
+  private MiniOidcOptionsPB oidcOptionsPb;
 
   private MiniKuduCluster(boolean enableKerberos,
       int numMasters,
@@ -123,7 +128,9 @@ public final class MiniKuduCluster implements AutoCloseable {
       MiniKdcOptionsPB kdcOptionsPb,
       String clusterRoot,
       Common.HmsMode hmsMode,
-      String principal) {
+      String principal,
+      boolean enableClientJwt,
+      MiniOidcOptionsPB oidcOptionsPb) {
     this.enableKerberos = enableKerberos;
     this.numMasters = numMasters;
     this.numTservers = numTservers;
@@ -133,6 +140,8 @@ public final class MiniKuduCluster implements AutoCloseable {
     this.kdcOptionsPb = kdcOptionsPb;
     this.principal = principal;
     this.hmsMode = hmsMode;
+    this.enableClientJwt = enableClientJwt;
+    this.oidcOptionsPb = oidcOptionsPb;
 
     if (clusterRoot == null) {
       // If a cluster root was not set, create a unique temp directory to use.
@@ -227,7 +236,8 @@ public final class MiniKuduCluster implements AutoCloseable {
         .addAllExtraTserverFlags(extraTserverFlags)
         .setMiniKdcOptions(kdcOptionsPb)
         .setClusterRoot(clusterRoot)
-        .setPrincipal(principal);
+        .setPrincipal(principal)
+        .setMiniOidcOptions(oidcOptionsPb);
 
     // Set up the location mapping command flag if there is location info.
     if (!locationInfo.isEmpty()) {
@@ -323,6 +333,18 @@ public final class MiniKuduCluster implements AutoCloseable {
     return principal;
   }
 
+  public String createJwtFor(String accountId, String subject, boolean isValid) throws IOException {
+    ControlShellResponsePB resp = sendRequestToCluster(ControlShellRequestPB.newBuilder()
+        .setCreateJwt(CreateJwtRequestPB
+            .newBuilder()
+            .setAccountId(accountId)
+            .setSubject(subject)
+            .setIsValidKey(isValid)
+            .build())
+        .build());
+    return resp.getCreateJwt().getJwt();
+  }
+
   /**
    * Starts a master identified by a host and port.
    * Does nothing if the server was already running.
@@ -703,8 +725,10 @@ public final class MiniKuduCluster implements AutoCloseable {
     private final List<String> locationInfo = new ArrayList<>();
     private String clusterRoot = null;
     private String principal = "kudu";
+    private boolean enableClientJwt = false;
 
     private MiniKdcOptionsPB.Builder kdcOptionsPb = MiniKdcOptionsPB.newBuilder();
+    private MiniOidcOptionsPB.Builder oidcOptionsPb = MiniOidcOptionsPB.newBuilder();
     private Common.HmsMode hmsMode = Common.HmsMode.NONE;
 
     public MiniKuduClusterBuilder numMasterServers(int numMasterServers) {
@@ -726,6 +750,11 @@ public final class MiniKuduCluster implements AutoCloseable {
       return this;
     }
 
+    public MiniKuduClusterBuilder enableClientJwt() {
+      enableClientJwt = true;
+      return this;
+    }
+
     public MiniKuduClusterBuilder enableHiveMetastoreIntegration() {
       hmsMode = Common.HmsMode.ENABLE_METASTORE_INTEGRATION;
       return this;
@@ -788,6 +817,15 @@ public final class MiniKuduCluster implements AutoCloseable {
       return this;
     }
 
+    public MiniKuduClusterBuilder addJwks(String accountId, boolean isValid) {
+      this.oidcOptionsPb.addJwksOptions(
+          JwksOptionsPB.newBuilder()
+              .setAccountId(accountId)
+              .setIsValidKey(isValid)
+              .build());
+      return this;
+    }
+
     /**
      * Builds and starts a new {@link MiniKuduCluster} using builder state.
      * @return the newly started {@link MiniKuduCluster}
@@ -798,7 +836,8 @@ public final class MiniKuduCluster implements AutoCloseable {
           new MiniKuduCluster(enableKerberos,
               numMasterServers, numTabletServers,
               extraTabletServerFlags, extraMasterServerFlags, locationInfo,
-              kdcOptionsPb.build(), clusterRoot, hmsMode, principal);
+              kdcOptionsPb.build(), clusterRoot, hmsMode, principal, enableClientJwt,
+              oidcOptionsPb.build());
       try {
         cluster.start();
       } catch (IOException e) {
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 6844fcdd4..a36f38d0a 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
@@ -37,6 +37,8 @@ import org.apache.kudu.client.TimeoutTracker;
 import org.apache.kudu.test.cluster.FakeDNS;
 import org.apache.kudu.test.cluster.MiniKuduCluster;
 import org.apache.kudu.test.junit.RetryRule;
+import org.apache.kudu.tools.Tool.CreateClusterRequestPB.JwksOptionsPB;
+import org.apache.kudu.tools.Tool.CreateClusterRequestPB.MiniOidcOptionsPB;
 
 public class TestMiniKuduCluster {
 
@@ -107,6 +109,21 @@ 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);
+
+      assertNotNull(jwt);
+    }
+  }
+
   @Test(timeout = 50000)
   public void testKerberos() throws Exception {
     FakeDNS.getInstance().install();
diff --git a/src/kudu/tools/tool.proto b/src/kudu/tools/tool.proto
index 1fa7878a1..94b20aba3 100644
--- a/src/kudu/tools/tool.proto
+++ b/src/kudu/tools/tool.proto
@@ -72,7 +72,7 @@ message CreateClusterRequestPB {
   // Options pertaining to a single JWKS.
   message JwksOptionsPB {
     optional string account_id = 1;
-    optional bool is_valid_key = 2;
+    optional bool is_valid_key = 2 [ default = true ];
   }
 
   message MiniOidcOptionsPB {
@@ -223,6 +223,7 @@ message ControlShellResponsePB {
     GetMastersResponsePB get_masters = 2;
     GetTServersResponsePB get_tservers = 3;
     GetKDCEnvVarsResponsePB get_kdc_env_vars = 4;
+    CreateJwtResponsePB create_jwt = 5;
   }
 }
 
@@ -249,6 +250,7 @@ message ControlShellRequestPB {
     SetDaemonFlagRequestPB set_daemon_flag = 12;
     PauseDaemonRequestPB pause_daemon = 13;
     ResumeDaemonRequestPB resume_daemon = 14;
+    CreateJwtRequestPB create_jwt = 15;
   }
 }
 
@@ -548,7 +550,7 @@ message TablesInfoPB {
   repeated TableInfoPB tables = 1;
 }
 
-message CreateJwtPB {
+message CreateJwtRequestPB {
   // The account ID with which a JWT will be created.
   optional string account_id = 1;
 
@@ -558,3 +560,7 @@ message CreateJwtPB {
   // Whether or not the returned token should supply a valid key ID.
   optional bool is_valid_key = 3;
 }
+
+message CreateJwtResponsePB {
+  optional string jwt = 1;
+}
diff --git a/src/kudu/tools/tool_action_test.cc b/src/kudu/tools/tool_action_test.cc
index 11153aa98..df49b83b3 100644
--- a/src/kudu/tools/tool_action_test.cc
+++ b/src/kudu/tools/tool_action_test.cc
@@ -43,6 +43,7 @@
 #include "kudu/tools/tool.pb.h"
 #include "kudu/util/env.h"
 #include "kudu/util/env_util.h"
+#include "kudu/util/mini_oidc.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
@@ -179,6 +180,14 @@ Status ProcessRequest(const ControlShellRequestPB& req,
           opts.principal = cc.principal();
         }
       }
+      if (cc.has_mini_oidc_options()) {
+        opts.enable_client_jwt = true;
+        const auto& mini_oidc_opts = cc.mini_oidc_options();
+        for (const auto& jwks_opt : mini_oidc_opts.jwks_options()) {
+          opts.mini_oidc_options.account_ids.emplace(
+              jwks_opt.account_id(), jwks_opt.is_valid_key());
+        }
+      }
 
       cluster->reset(new ExternalMiniCluster(std::move(opts)));
       break;
@@ -341,6 +350,21 @@ Status ProcessRequest(const ControlShellRequestPB& req,
       RETURN_NOT_OK(daemon->Resume());
       break;
     }
+    case ControlShellRequestPB::kCreateJwt:
+    {
+      const auto& r = req.create_jwt();
+      if (!r.has_account_id() || !r.has_subject()) {
+        return Status::InvalidArgument("JWT creation requires account ID and subject");
+      }
+      RETURN_NOT_OK(CheckClusterExists(*cluster));
+      auto* oidc = (*cluster)->oidc();
+      if (!oidc) {
+        return Status::ConfigurationError("mini-cluster must be configured with MiniOidc");
+      }
+      auto jwt = oidc->CreateJwt(r.account_id(), r.subject(), r.is_valid_key());
+      *resp->mutable_create_jwt()->mutable_jwt() = jwt;
+      break;
+    }
     default:
       RETURN_NOT_OK(Status::InvalidArgument("unknown cluster control request"));
   }