You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by si...@apache.org on 2021/11/11 03:12:23 UTC

[ozone] branch HDDS-4944 updated: HDDS-5939. [Multi-Tenant] Implement `ozone tenant user getsecret` that does not generate secret when accessId does not exist (#2804)

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

siyao pushed a commit to branch HDDS-4944
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/HDDS-4944 by this push:
     new 72d6701  HDDS-5939. [Multi-Tenant] Implement `ozone tenant user getsecret` that does not generate secret when accessId does not exist (#2804)
72d6701 is described below

commit 72d67018e6e3b672aa8798eee158588d92f1444b
Author: Siyao Meng <50...@users.noreply.github.com>
AuthorDate: Wed Nov 10 19:12:07 2021 -0800

    HDDS-5939. [Multi-Tenant] Implement `ozone tenant user getsecret` that does not generate secret when accessId does not exist (#2804)
---
 .../apache/hadoop/ozone/client/ObjectStore.java    |   5 +
 .../ozone/client/protocol/ClientProtocol.java      |  18 ++-
 .../apache/hadoop/ozone/client/rpc/RpcClient.java  |  15 +++
 .../hadoop/ozone/om/exceptions/OMException.java    |   2 +-
 .../ozone/om/protocol/OzoneManagerProtocol.java    |  10 ++
 ...OzoneManagerProtocolClientSideTranslatorPB.java |  16 +++
 .../hadoop/ozone/shell/TestOzoneTenantShell.java   |  69 ++++++++---
 .../src/main/proto/OmClientProtocol.proto          |  12 +-
 .../org/apache/hadoop/ozone/om/OzoneManager.java   |  10 ++
 .../om/request/s3/security/S3GetSecretRequest.java | 127 ++++++++++++++-------
 .../s3/tenant/OMTenantAssignAdminRequest.java      |   2 +-
 .../s3/tenant/OMTenantRevokeAdminRequest.java      |   2 +-
 .../tenant/OMTenantRevokeUserAccessIdRequest.java  |   2 +-
 ...minHandler.java => TenantGetSecretHandler.java} |  63 +++++-----
 .../shell/tenant/TenantRevokeAdminHandler.java     |   3 -
 .../ozone/shell/tenant/TenantSetSecretHandler.java |  40 +++++++
 .../ozone/shell/tenant/TenantUserCommands.java     |   4 +-
 17 files changed, 306 insertions(+), 94 deletions(-)

diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
index 41a471f..5119cb3 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
@@ -173,6 +173,11 @@ public class ObjectStore {
     return proxy.getS3Secret(kerberosID);
   }
 
+  public S3SecretValue getS3Secret(String kerberosID, boolean createIfNotExist)
+          throws IOException {
+    return proxy.getS3Secret(kerberosID, createIfNotExist);
+  }
+
   public void revokeS3Secret(String kerberosID) throws IOException {
     proxy.revokeS3Secret(kerberosID);
   }
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
index 9eec9d0..5f94abd 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
@@ -547,7 +547,9 @@ public interface ClientProtocol {
       throws IOException;
 
   /**
-   * returns S3 Secret given kerberos user.
+   * Returns S3 Secret given kerberos user.
+   * Will generate a secret access key for the accessId (=kerberosID)
+   * if it doesn't exist.
    * @param kerberosID
    * @return S3SecretValue
    * @throws IOException
@@ -555,6 +557,20 @@ public interface ClientProtocol {
   S3SecretValue getS3Secret(String kerberosID) throws IOException;
 
   /**
+   * Returns S3 Secret given kerberos user.
+   * Optionally generate a secret access key for the accessId (=kerberosID)
+   * if it doesn't exist if createIfNotExist is true.
+   * When createIfNotExist is false and accessId (=kerberosID) doesn't
+   * exist, OM throws OMException with ACCESSID_NOT_FOUND to the client.
+   * @param kerberosID
+   * @param createIfNotExist
+   * @return S3SecretValue
+   * @throws IOException
+   */
+  S3SecretValue getS3Secret(String kerberosID, boolean createIfNotExist)
+          throws IOException;
+
+  /**
    * Revoke S3 Secret of given kerberos user.
    * @param kerberosID
    * @throws IOException
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index aebfff0..bba1e5e 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -627,6 +627,21 @@ public class RpcClient implements ClientProtocol {
   }
 
   /**
+   * Returns s3 secret given a kerberos user.
+   * @param kerberosID
+   * @return S3SecretValue
+   * @throws IOException
+   */
+  @Override
+  public S3SecretValue getS3Secret(String kerberosID, boolean createIfNotExist)
+          throws IOException {
+    Preconditions.checkArgument(Strings.isNotBlank(kerberosID),
+            "kerberosID cannot be null or empty.");
+    // No need to check createIfNotExist here which is a primitive
+    return ozoneManagerClient.getS3Secret(kerberosID, createIfNotExist);
+  }
+
+  /**
    * {@inheritDoc}
    */
   @Override
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
index a049a20..8c2f38f 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
@@ -248,7 +248,7 @@ public class OMException extends IOException {
     TENANT_ALREADY_EXISTS,
     INVALID_TENANT_NAME,
 
-    TENANT_USER_ACCESSID_NOT_FOUND,
+    ACCESSID_NOT_FOUND,
     TENANT_USER_ACCESSID_ALREADY_EXISTS,
     INVALID_TENANT_USER_NAME,
     INVALID_ACCESSID,
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
index 06b328d..fddeec2 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
@@ -452,6 +452,16 @@ public interface OzoneManagerProtocol
   S3SecretValue getS3Secret(String kerberosID) throws IOException;
 
   /**
+   * Gets s3Secret for given kerberos user.
+   * @param kerberosID
+   * @param createIfNotExist
+   * @return S3SecretValue
+   * @throws IOException
+   */
+  S3SecretValue getS3Secret(String kerberosID, boolean createIfNotExist)
+          throws IOException;
+
+  /**
    * Revokes s3Secret of given kerberos user.
    * @param kerberosID
    * @throws IOException
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
index 168635e..85b528f 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
@@ -879,6 +879,22 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
   }
 
   @Override
+  public S3SecretValue getS3Secret(String kerberosID, boolean createIfNotExist)
+          throws IOException {
+    GetS3SecretRequest request = GetS3SecretRequest.newBuilder()
+            .setKerberosID(kerberosID)
+            .setCreateIfNotExist(createIfNotExist)
+            .build();
+    OMRequest omRequest = createOMRequest(Type.GetS3Secret)
+            .setGetS3SecretRequest(request)
+            .build();
+    final GetS3SecretResponse resp = handleError(submitRequest(omRequest))
+            .getGetS3SecretResponse();
+
+    return S3SecretValue.fromProtobuf(resp.getS3Secret());
+  }
+
+  @Override
   public void revokeS3Secret(String kerberosID) throws IOException {
     RevokeS3SecretRequest request = RevokeS3SecretRequest.newBuilder()
             .setKerberosID(kerberosID)
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
index 4556d4f..c37e048 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
@@ -376,6 +376,7 @@ public class TestOzoneTenantShell {
    * and revoke user flow.
    */
   @Test
+  @SuppressWarnings("methodlength")
   public void testOzoneTenantBasicOperations() throws IOException {
 
     List<String> lines = FileUtils.readLines(AUDIT_LOG_FILE, (String)null);
@@ -424,6 +425,13 @@ public class TestOzoneTenantShell {
     checkOutput(out, "Policy", false);
     checkOutput(err, "", true);
 
+    // Attempt user getsecret before assignment, should fail
+    executeHA(tenantShell, new String[] {
+        "user", "getsecret", "finance$bob"});
+    checkOutput(out, "", false);
+    checkOutput(err, "AccessId 'finance$bob' doesn't exist\n",
+        true);
+
     // Assign user accessId
     // Equivalent to `ozone tenant user assign bob --tenant=finance`
     executeHA(tenantShell, new String[] {
@@ -433,6 +441,19 @@ public class TestOzoneTenantShell {
     checkOutput(err, "Assigned 'bob' to 'finance' with accessId"
         + " 'finance$bob'.\n", true);
 
+    // Try user getsecret again after assignment, should succeed
+    executeHA(tenantShell, new String[] {
+        "user", "getsecret", "finance$bob"});
+    checkOutput(out, "awsAccessKey=finance$bob\n", false);
+    checkOutput(err, "", true);
+
+    // Try user getsecret again with -e option
+    executeHA(tenantShell, new String[] {
+        "user", "getsecret", "-e", "finance$bob"});
+    checkOutput(out, "export AWS_ACCESS_KEY_ID='finance$bob'\n",
+            false);
+    checkOutput(err, "", true);
+
     executeHA(tenantShell, new String[] {
         "user", "assign", "bob", "--tenant=research"});
     checkOutput(out, "export AWS_ACCESS_KEY_ID='research$bob'\n"
@@ -521,33 +542,53 @@ public class TestOzoneTenantShell {
         "user", "revoke", "research$bob"});
     checkOutput(out, "", true);
     checkOutput(err, "Revoked accessId", false);
+
+    // TODO: Clean up: remove tenant when tenant remove CLI is implemented
   }
 
-  private void testListTenantUsers() throws IOException {
+  @Test
+  public void testListTenantUsers() throws IOException {
+    executeHA(tenantShell, new String[] {"create", "tenant1"});
+    checkOutput(out, "Created tenant 'tenant1'.\n", true);
+    checkOutput(err, "", true);
+
+    executeHA(tenantShell, new String[] {
+        "user", "assign", "alice", "--tenant=tenant1"});
+    checkOutput(out, "export AWS_ACCESS_KEY_ID='tenant1$alice'\n"
+        + "export AWS_SECRET_ACCESS_KEY='", false);
+    checkOutput(err, "Assigned 'alice' to 'tenant1'" +
+        " with accessId 'tenant1$alice'.\n", true);
+
     executeHA(tenantShell, new String[] {
-        "user", "assign", "alice@EXAMPLE.COM", "--tenant=research"});
-    checkOutput(out, "export AWS_ACCESS_KEY_ID='research$alice@EXAMPLE.COM'\n"
+        "user", "assign", "bob", "--tenant=tenant1"});
+    checkOutput(out, "export AWS_ACCESS_KEY_ID='tenant1$bob'\n"
         + "export AWS_SECRET_ACCESS_KEY='", false);
-    checkOutput(err, "Assigned 'alice@EXAMPLE.COM' to 'research'" +
-        " with accessId 'research$alice@EXAMPLE.COM'.\n", true);
+    checkOutput(err, "Assigned 'bob' to 'tenant1'" +
+        " with accessId 'tenant1$bob'.\n", true);
 
     executeHA(tenantShell, new String[] {
-        "user", "list", "--tenant=research"});
-    checkOutput(out,
-        "- User 'bob@EXAMPLE.COM' with accessId 'research$bob@EXAMPLE.COM'\n"
-            + "- User 'alice@EXAMPLE.COM' with accessId 'research$alice@EXAMPLE"
-            + ".COM'\n", true);
+        "user", "list", "--tenant=tenant1"});
+    checkOutput(out, "- User 'bob' with accessId 'tenant1$bob'\n" +
+        "- User 'alice' with accessId 'tenant1$alice'\n", true);
     checkOutput(err, "", true);
 
     executeHA(tenantShell, new String[] {
-        "user", "list", "--tenant=research", "--prefix=b"});
-    checkOutput(out, "- User 'bob@EXAMPLE.COM' with accessId " +
-        "'research$bob@EXAMPLE.COM'\n", true);
+        "user", "list", "--tenant=tenant1", "--prefix=b"});
+    checkOutput(out, "- User 'bob' with accessId " +
+        "'tenant1$bob'\n", true);
     checkOutput(err, "", true);
 
     executeHA(tenantShell, new String[] {
         "user", "list", "--tenant=unknown"});
     checkOutput(err, "Failed to Get Users in tenant 'unknown': " +
         "Tenant 'unknown' not found!\n", true);
+
+    // Clean up
+    executeHA(tenantShell, new String[] {
+        "user", "revoke", "tenant1$alice"});
+    checkOutput(out, "", true);
+    checkOutput(err, "Revoked accessId", false);
+
+    // TODO: Clean up: remove tenant when tenant remove CLI is implemented
   }
-}
\ No newline at end of file
+}
diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
index 2cb1c66..87ae19f 100644
--- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
+++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
@@ -405,7 +405,7 @@ enum Status {
     TENANT_ALREADY_EXISTS = 76;
     INVALID_TENANT_NAME = 77;
 
-    TENANT_USER_ACCESSID_NOT_FOUND = 78;
+    ACCESSID_NOT_FOUND = 78;
     TENANT_USER_ACCESSID_ALREADY_EXISTS = 79;  // TODO: Remove if not used
     INVALID_TENANT_USER_NAME = 80;
     INVALID_ACCESSID = 81;
@@ -1376,12 +1376,22 @@ message S3Secret {
 
 message GetS3SecretRequest {
     required string kerberosID = 1;
+    optional bool createIfNotExist = 2;
 }
 
 message GetS3SecretResponse {
     required S3Secret s3Secret = 2;
 }
 
+message SetSecretRequest {
+    optional string accessId = 1;
+    optional string secret = 2;
+}
+
+message SetSecretResponse {
+    optional bool success = 1;
+}
+
 message TenantInfo {
     optional string tenantName = 1;
     optional string bucketNamespaceName = 2;
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 4b59387..9be641a 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -3075,6 +3075,16 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     return s3SecretManager.getS3Secret(kerberosID);
   }
 
+  @Override
+  /**
+   * {@inheritDoc}
+   */
+  public S3SecretValue getS3Secret(String kerberosID, boolean createIfNotExist)
+          throws IOException {
+    throw new NotImplementedException(
+            "non-Ratis getS3Secret(String, boolean) is not implemented");
+  }
+
   /**
    * Create tenant.
    */
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java
index 781c95c..9cb03c1 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java
@@ -71,28 +71,21 @@ public class S3GetSecretRequest extends OMClientRequest {
         getOmRequest().getGetS3SecretRequest();
 
     // Generate S3 Secret to be used by OM quorum.
-    String kerberosID = s3GetSecretRequest.getKerberosID();
+    // Note 1: The proto field kerberosID is effectively accessId already.
+    // It is still named kerberosID because kerberosID == accessId before
+    // multi-tenancy. TODO: Rename the kerberosID field later in master branch.
+    String accessId = s3GetSecretRequest.getKerberosID();
 
     final UserGroupInformation ugi = ProtobufRpcEngine.Server.getRemoteUser();
     final String username = ugi.getUserName();
     // Permission check. Users need to be themselves or have admin privilege
-    if (!username.equals(kerberosID) &&
-        !ozoneManager.isAdmin(ugi)) {
-      throw new OMException("Requested user name '" + kerberosID +
+    if (!username.equals(accessId) && !ozoneManager.isAdmin(ugi)) {
+      throw new OMException("Requested accessId '" + accessId +
           "' doesn't match current user '" + username +
           "', nor does current user has administrator privilege.",
           OMException.ResultCodes.USER_MISMATCH);
     }
 
-    // Generate secret. Used only when doesn't the kerberosID entry doesn't
-    //  exist in DB, discarded otherwise.
-    String s3Secret = DigestUtils.sha256Hex(OmUtils.getSHADigest());
-
-    UpdateGetS3SecretRequest updateGetS3SecretRequest =
-        UpdateGetS3SecretRequest.newBuilder()
-            .setAwsSecret(s3Secret)
-            .setKerberosID(kerberosID).build();
-
     // Client issues GetS3Secret request, when received by OM leader
     // it will generate s3Secret. Original GetS3Secret request is
     // converted to UpdateGetS3Secret request with the generated token
@@ -102,16 +95,42 @@ public class S3GetSecretRequest extends OMClientRequest {
     // client does not need any proto changes.
     OMRequest.Builder omRequest = OMRequest.newBuilder()
         .setUserInfo(getUserInfo())
-        .setUpdateGetS3SecretRequest(updateGetS3SecretRequest)
         .setCmdType(getOmRequest().getCmdType())
         .setClientId(getOmRequest().getClientId());
 
+    // createIfNotExist defaults to true if not specified.
+    boolean createIfNotExist = !s3GetSecretRequest.hasCreateIfNotExist()
+            || s3GetSecretRequest.getCreateIfNotExist();
+
+    // Recompose GetS3SecretRequest just in case createIfNotExist is missing
+    final GetS3SecretRequest newGetS3SecretRequest =
+            GetS3SecretRequest.newBuilder()
+                    .setKerberosID(accessId)  // See Note 1 above
+                    .setCreateIfNotExist(createIfNotExist)
+                    .build();
+    omRequest.setGetS3SecretRequest(newGetS3SecretRequest);
+
+    // When createIfNotExist is true, pass UpdateGetS3SecretRequest message;
+    // otherwise, just use GetS3SecretRequest message.
+    if (createIfNotExist) {
+      // Generate secret here because this will be written to DB only when
+      // createIfNotExist is true and accessId entry doesn't exist in DB.
+      String s3Secret = DigestUtils.sha256Hex(OmUtils.getSHADigest());
+
+      final UpdateGetS3SecretRequest updateGetS3SecretRequest =
+              UpdateGetS3SecretRequest.newBuilder()
+                      .setKerberosID(accessId)  // See Note 1 above
+                      .setAwsSecret(s3Secret)
+                      .build();
+
+      omRequest.setUpdateGetS3SecretRequest(updateGetS3SecretRequest);
+    }
+
     if (getOmRequest().hasTraceID()) {
       omRequest.setTraceID(getOmRequest().getTraceID());
     }
 
     return omRequest.build();
-
   }
 
   @Override
@@ -124,32 +143,47 @@ public class S3GetSecretRequest extends OMClientRequest {
     boolean acquiredLock = false;
     IOException exception = null;
     OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
-    UpdateGetS3SecretRequest updateGetS3SecretRequest =
-        getOmRequest().getUpdateGetS3SecretRequest();
-    String kerberosID = updateGetS3SecretRequest.getKerberosID();
+
+    final GetS3SecretRequest getS3SecretRequest =
+            getOmRequest().getGetS3SecretRequest();
+    assert(getS3SecretRequest.hasCreateIfNotExist());
+    final boolean createIfNotExist = getS3SecretRequest.getCreateIfNotExist();
+    // See Note 1 above
+    final String accessId = getS3SecretRequest.getKerberosID();
+    String awsSecret = null;
+    if (createIfNotExist) {
+      final UpdateGetS3SecretRequest updateGetS3SecretRequest =
+              getOmRequest().getUpdateGetS3SecretRequest();
+      awsSecret = updateGetS3SecretRequest.getAwsSecret();
+      assert(accessId.equals(updateGetS3SecretRequest.getKerberosID()));
+    }
+
     try {
-      String awsSecret = updateGetS3SecretRequest.getAwsSecret();
       // Note: We use the same S3_SECRET_LOCK for TenantAccessIdTable.
       acquiredLock = omMetadataManager.getLock()
-          .acquireWriteLock(S3_SECRET_LOCK, kerberosID);
+          .acquireWriteLock(S3_SECRET_LOCK, accessId);
 
       // Check multi-tenant table first: tenantAccessIdTable
       final S3SecretValue assignS3SecretValue;
       final OmDBAccessIdInfo omDBAccessIdInfo =
-          omMetadataManager.getTenantAccessIdTable().get(kerberosID);
+          omMetadataManager.getTenantAccessIdTable().get(accessId);
       if (omDBAccessIdInfo == null) {
         // Not found in TenantAccessIdTable. Fallback to S3SecretTable.
         final S3SecretValue s3SecretValue =
-            omMetadataManager.getS3SecretTable().get(kerberosID);
+            omMetadataManager.getS3SecretTable().get(accessId);
 
         if (s3SecretValue == null) {
-          // Still not found in S3SecretTable. Will add new entry in this case.
-          assignS3SecretValue = new S3SecretValue(kerberosID, awsSecret);
-          // Add cache entry first.
-          omMetadataManager.getS3SecretTable().addCacheEntry(
-              new CacheKey<>(kerberosID),
-              new CacheValue<>(
-                  Optional.of(assignS3SecretValue), transactionLogIndex));
+          if (createIfNotExist) {
+            // Still not found in S3SecretTable. Add new entry in this case
+            assignS3SecretValue = new S3SecretValue(accessId, awsSecret);
+            // Add cache entry first.
+            omMetadataManager.getS3SecretTable().addCacheEntry(
+                    new CacheKey<>(accessId),
+                    new CacheValue<>(Optional.of(assignS3SecretValue),
+                            transactionLogIndex));
+          } else {
+            assignS3SecretValue = null;
+          }
         } else {
           // Found in S3SecretTable.
           awsSecret = s3SecretValue.getAwsSecret();
@@ -161,16 +195,25 @@ public class S3GetSecretRequest extends OMClientRequest {
         assignS3SecretValue = null;
       }
 
+      // Throw ACCESSID_NOT_FOUND to the client if accessId doesn't exist
+      //  when createIfNotExist is false.
+      if (awsSecret == null) {
+        assert(!createIfNotExist);
+        throw new OMException("accessId '" + accessId + "' doesn't exist",
+                OMException.ResultCodes.ACCESSID_NOT_FOUND);
+      }
+
       // Compose response
       final GetS3SecretResponse.Builder getS3SecretResponse =
-          GetS3SecretResponse.newBuilder().setS3Secret(
-              S3Secret.newBuilder()
-                  .setAwsSecret(awsSecret)
-                  .setKerberosID(kerberosID));
-      // If entry exists, assignS3SecretValue will be null,
-      // so we won't overwrite the entry.
+              GetS3SecretResponse.newBuilder().setS3Secret(
+                      S3Secret.newBuilder()
+                              .setAwsSecret(awsSecret)
+                              .setKerberosID(accessId)  // See Note 1 above
+              );
+      // If entry exists or createIfNotExist is false, assignS3SecretValue
+      // will be null, so we won't write or overwrite the entry.
       omClientResponse = new S3GetSecretResponse(assignS3SecretValue,
-          omResponse.setGetS3SecretResponse(getS3SecretResponse).build());
+              omResponse.setGetS3SecretResponse(getS3SecretResponse).build());
 
     } catch (IOException ex) {
       exception = ex;
@@ -181,13 +224,12 @@ public class S3GetSecretRequest extends OMClientRequest {
           ozoneManagerDoubleBufferHelper);
       if (acquiredLock) {
         omMetadataManager.getLock().releaseWriteLock(S3_SECRET_LOCK,
-            kerberosID);
+            accessId);
       }
     }
 
-
     Map<String, String> auditMap = new HashMap<>();
-    auditMap.put(OzoneConsts.S3_GETSECRET_USER, kerberosID);
+    auditMap.put(OzoneConsts.S3_GETSECRET_USER, accessId);
 
     // audit log
     auditLog(ozoneManager.getAuditLogger(), buildAuditMessage(
@@ -195,10 +237,11 @@ public class S3GetSecretRequest extends OMClientRequest {
         exception, getOmRequest().getUserInfo()));
 
     if (exception == null) {
-      LOG.debug("Successfully generated secret for accessKey '{}'", kerberosID);
+      LOG.debug("Success: GetSecret for accessKey '{}', createIfNotExist '{}'",
+              accessId, createIfNotExist);
     } else {
-      LOG.error("Failed to generate secret for accessKey '{}': {}", kerberosID,
-          exception);
+      LOG.error("Failed to GetSecret for accessKey '{}', createIfNotExist " +
+                      "'{}': {}", accessId, createIfNotExist, exception);
     }
     return omClientResponse;
   }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantAssignAdminRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantAssignAdminRequest.java
index dbd2fa5..b99426c 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantAssignAdminRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantAssignAdminRequest.java
@@ -91,7 +91,7 @@ public class OMTenantAssignAdminRequest extends OMClientRequest {
 
     if (accessIdInfo == null) {
       throw new OMException("accessId '" + accessId + "' not found.",
-          OMException.ResultCodes.TENANT_USER_ACCESSID_NOT_FOUND);
+          OMException.ResultCodes.ACCESSID_NOT_FOUND);
     }
 
     // Check if accessId is assigned to the tenant
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRevokeAdminRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRevokeAdminRequest.java
index 705dd89..b725e34 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRevokeAdminRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRevokeAdminRequest.java
@@ -90,7 +90,7 @@ public class OMTenantRevokeAdminRequest extends OMClientRequest {
 
     if (accessIdInfo == null) {
       throw new OMException("accessId '" + accessId + "' not found.",
-          OMException.ResultCodes.TENANT_USER_ACCESSID_NOT_FOUND);
+          OMException.ResultCodes.ACCESSID_NOT_FOUND);
     }
 
     // Check if accessId is assigned to the tenant
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRevokeUserAccessIdRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRevokeUserAccessIdRequest.java
index e364f12..9c16297 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRevokeUserAccessIdRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRevokeUserAccessIdRequest.java
@@ -93,7 +93,7 @@ public class OMTenantRevokeUserAccessIdRequest extends OMClientRequest {
     if (accessIdInfo == null) {
       // Note: This potentially leaks which accessIds exists in OM.
       throw new OMException("accessId '" + accessId + "' doesn't exist",
-          OMException.ResultCodes.TENANT_USER_ACCESSID_NOT_FOUND);
+          OMException.ResultCodes.ACCESSID_NOT_FOUND);
     }
 
     final String tenantName = accessIdInfo.getTenantId();
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantRevokeAdminHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantGetSecretHandler.java
similarity index 51%
copy from hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantRevokeAdminHandler.java
copy to hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantGetSecretHandler.java
index ad94a9a..dcbc0e3 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantRevokeAdminHandler.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantGetSecretHandler.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.shell.tenant;
 import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.shell.OzoneAddress;
 import picocli.CommandLine;
 
@@ -27,48 +28,54 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.PERMISSION_DENIED;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.ACCESSID_NOT_FOUND;
 
 /**
- * ozone tenant user revoke-admin.
+ * ozone tenant user get-secret.
  */
-@CommandLine.Command(name = "revoke-admin",
-    aliases = {"revokeadmin"},
-    description = "Revoke admin role from accessIds in a tenant")
-public class TenantRevokeAdminHandler extends TenantHandler {
-
-  @CommandLine.Spec
-  private CommandLine.Model.CommandSpec spec;
+@CommandLine.Command(name = "get-secret",
+    aliases = {"getsecret"},
+    description = "Get secret for tenant user accessIds. " +
+        "This differs from `ozone s3 getsecret` that this would not " +
+        "generate a key/secret pair when the accessId doesn't exist.")
+public class TenantGetSecretHandler extends TenantHandler {
 
   @CommandLine.Parameters(description = "List of accessIds", arity = "1..")
   private List<String> accessIds = new ArrayList<>();
 
-  @CommandLine.Option(names = {"-t", "--tenant"},
-      description = "Tenant name")
-  private String tenantName;
+  @CommandLine.Option(names = "-e",
+      description = "Print out variables together with 'export' prefix")
+  private boolean export;
 
   @Override
-  protected void execute(OzoneClient client, OzoneAddress address) {
-    final ObjectStore objStore = client.getObjectStore();
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException {
+    final ObjectStore objectStore = client.getObjectStore();
 
     for (final String accessId : accessIds) {
+
       try {
-        // TODO: Make tenantRevokeAdmin return accessId, tenantName, user later.
-        objStore.tenantRevokeAdmin(accessId, tenantName);
-        err().println("Revoked admin role of '" + accessId +
-            (tenantName != null ? "' from tenant '" + tenantName + "'" : ""));
-      } catch (IOException e) {
-        err().println("Failed to revoke admin role of '" + accessId +
-            (tenantName != null ? "' from tenant '" + tenantName + "'" : "") +
-            ": " + e.getMessage());
-        if (e instanceof OMException) {
-          final OMException omEx = (OMException) e;
-          // Don't bother continuing the loop if current user isn't Ozone admin
-          if (omEx.getResult().equals(PERMISSION_DENIED)) {
-            break;
-          }
+        final S3SecretValue secret =
+            objectStore.getS3Secret(accessId, false);
+        if (export) {
+          out().println("export AWS_ACCESS_KEY_ID='" +
+              secret.getAwsAccessKey() + "'");
+          out().println("export AWS_SECRET_ACCESS_KEY='" +
+              secret.getAwsSecret() + "'");
+        } else {
+          out().println(secret);
+        }
+      } catch (OMException omEx) {
+        if (omEx.getResult().equals(ACCESSID_NOT_FOUND)) {
+          // Print to stderr here in order not to contaminate stdout just in
+          // case -e is specified.
+          err().println("AccessId '" + accessId + "' doesn't exist");
+          // Continue the loop if it's just ACCESSID_NOT_FOUND
+        } else {
+          throw omEx;
         }
       }
+
     }
   }
 }
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantRevokeAdminHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantRevokeAdminHandler.java
index ad94a9a..dec27c2 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantRevokeAdminHandler.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantRevokeAdminHandler.java
@@ -37,9 +37,6 @@ import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.PERM
     description = "Revoke admin role from accessIds in a tenant")
 public class TenantRevokeAdminHandler extends TenantHandler {
 
-  @CommandLine.Spec
-  private CommandLine.Model.CommandSpec spec;
-
   @CommandLine.Parameters(description = "List of accessIds", arity = "1..")
   private List<String> accessIds = new ArrayList<>();
 
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantSetSecretHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantSetSecretHandler.java
new file mode 100644
index 0000000..9ca6133
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantSetSecretHandler.java
@@ -0,0 +1,40 @@
+/*
+ * 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.hadoop.ozone.shell.tenant;
+
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import picocli.CommandLine;
+
+/**
+ * ozone tenant user set-secret.
+ */
+@CommandLine.Command(name = "set-secret",
+    aliases = {"setsecret"},
+    description = "Set secret for a tenant user accessId.")
+public class TenantSetSecretHandler extends TenantHandler {
+
+  @CommandLine.Parameters(description = "AccessId", arity = "1")
+  private String accessId;
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address) {
+//    final ObjectStore objectStore = client.getObjectStore();
+  }
+}
+
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantUserCommands.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantUserCommands.java
index 356fafa..86a051f 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantUserCommands.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantUserCommands.java
@@ -40,7 +40,9 @@ import java.util.concurrent.Callable;
         TenantRevokeUserAccessIdHandler.class,
         TenantAssignAdminHandler.class,
         TenantRevokeAdminHandler.class,
-        TenantListUsersHandler.class
+        TenantListUsersHandler.class,
+        TenantGetSecretHandler.class,
+        TenantSetSecretHandler.class
     },
     mixinStandardHelpOptions = true,
     versionProvider = HddsVersionProvider.class)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@ozone.apache.org
For additional commands, e-mail: commits-help@ozone.apache.org