You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2019/06/21 00:04:55 UTC

[kudu] 02/02: [java] KUDU-2873: attach authz tokens to SplitKeyRangeRequests

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

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

commit 8c68c946d467a466fa2d7f21c283fc083d380d29
Author: Andrew Wong <aw...@apache.org>
AuthorDate: Thu Jun 20 12:03:47 2019 -0700

    [java] KUDU-2873: attach authz tokens to SplitKeyRangeRequests
    
    This adds the fetching of the token, and tests to make sure that it
    exercises the same token reacquisition logic we have for writes and
    scans.
    
    Note: tserver-side testing for the endpoint can be found in
    tablet_server_authorization-test.cc.
    
    Change-Id: I2b4140076206f25b34d8025b45118e3ff7b4b826
    Reviewed-on: http://gerrit.cloudera.org:8080/13692
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 .../org/apache/kudu/client/SplitKeyRangeRequest.java  | 19 ++++++++++++++++++-
 .../apache/kudu/client/TestAuthTokenReacquire.java    | 13 +++++++++++++
 2 files changed, 31 insertions(+), 1 deletion(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/SplitKeyRangeRequest.java b/java/kudu-client/src/main/java/org/apache/kudu/client/SplitKeyRangeRequest.java
index 02644d9..f106245 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/SplitKeyRangeRequest.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/SplitKeyRangeRequest.java
@@ -26,6 +26,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.jboss.netty.util.Timer;
 
 import org.apache.kudu.Common.KeyRangePB;
+import org.apache.kudu.security.Token;
 import org.apache.kudu.tserver.Tserver;
 import org.apache.kudu.util.Pair;
 
@@ -40,6 +41,9 @@ class SplitKeyRangeRequest extends KuduRpc<SplitKeyRangeResponse> {
   private final byte[] partitionKey;
   private final long splitSizeBytes;
 
+  /** The token with which to authorize this RPC. */
+  private Token.SignedTokenPB authzToken;
+
   /**
    * Create a new RPC request
    * @param table table to lookup
@@ -81,11 +85,24 @@ class SplitKeyRangeRequest extends KuduRpc<SplitKeyRangeResponse> {
       builder.setStopPrimaryKey(UnsafeByteOperations.unsafeWrap(endPrimaryKey));
     }
     builder.setTargetChunkSizeBytes(splitSizeBytes);
+    if (authzToken != null) {
+      builder.setAuthzToken(authzToken);
+    }
 
     return builder.build();
   }
 
   @Override
+  boolean needsAuthzToken() {
+    return true;
+  }
+
+  @Override
+  void bindAuthzToken(Token.SignedTokenPB token) {
+    authzToken = token;
+  }
+
+  @Override
   String serviceName() {
     return TABLET_SERVER_SERVICE_NAME;
   }
@@ -116,4 +133,4 @@ class SplitKeyRangeRequest extends KuduRpc<SplitKeyRangeResponse> {
   byte[] partitionKey() {
     return this.partitionKey;
   }
-}
\ No newline at end of file
+}
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthTokenReacquire.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthTokenReacquire.java
index 2ab753c..242d948 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthTokenReacquire.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthTokenReacquire.java
@@ -20,6 +20,7 @@ package org.apache.kudu.client;
 import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
 import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
 import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -190,6 +191,11 @@ public class TestAuthTokenReacquire {
     assertEquals(0, session.countPendingErrors());
   }
 
+  private List<KeyRange> splitKeyRange(KuduTable table) throws Exception {
+    // Note: the nulls are for key bounds; we don't really care about them.
+    return table.getAsyncClient().getTableKeyRanges(table, null, null, null, null,
+        AsyncKuduClient.FETCH_TABLETS_PER_RANGE_LOOKUP, 1, DEFAULT_SLEEP).join();
+  }
 
   @Test
   public void testBasicWorkflow() throws Exception {
@@ -229,6 +235,13 @@ public class TestAuthTokenReacquire {
     assertEquals(key, countRowsInTable(scanTable));
     assertFalse(asyncClient.getAuthzToken(tableId).equals(originalToken));
 
+    // Now wait for the authz token to expire and send a request to split the
+    // key range. It should succeed and get a new authz token.
+    originalToken = asyncClient.getAuthzToken(tableId);
+    expireTokens();
+    assertFalse(splitKeyRange(scanTable).isEmpty());
+    assertFalse(asyncClient.getAuthzToken(tableId).equals(originalToken));
+
     // Force the client to get a new authn token and delete the table.
     originalToken = asyncClient.securityContext.getAuthenticationToken();
     dropConnectionsAndExpireTokens();