You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/05/12 05:08:20 UTC

[GitHub] [ozone] smengcl opened a new pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

smengcl opened a new pull request #2239:
URL: https://github.com/apache/ozone/pull/2239


   https://issues.apache.org/jira/browse/HDDS-5206


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r634774354



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java
##########
@@ -73,10 +73,14 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     String kerberosID = s3GetSecretRequest.getKerberosID();
 
     UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+    // Permission check. Users need to be themselves or have admin privilege
     if (!user.getUserName().equals(kerberosID)) {
-      throw new OMException("User mismatch. Requested user name is " +
-          "mismatched " + kerberosID +", with current user " +
-          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+      if (!ozoneManager.isAdmin(kerberosID)) {
+        throw new OMException("Requested user name '" + kerberosID +
+            "' doesn't match current user '" + user.getUserName() +
+            "', nor does current user has administrator privilege.",
+            OMException.ResultCodes.USER_MISMATCH);
+      }

Review comment:
       Hey @vivekratnavel thanks for the comment.  But the added `else` wouldn't work because it is triggered when an admin is revoking another user's secret.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r630728117



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3RevokeSecretRequest.java
##########
@@ -0,0 +1,149 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.request.s3.security;
+
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.security.S3RevokeSecretResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RevokeS3SecretRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_SECRET_LOCK;
+
+/**
+ * Handles RevokeS3Secret request.
+ */
+public class S3RevokeSecretRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3RevokeSecretRequest.class);
+
+  public S3RevokeSecretRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    final RevokeS3SecretRequest s3RevokeSecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    final String kerberosID = s3RevokeSecretRequest.getKerberosID();
+    final UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+
+    if (!user.getUserName().equals(kerberosID)) {
+      throw new OMException("User mismatch. Requested user name is " +
+          "mismatched " + kerberosID +", with current user " +
+          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+    }
+
+    final RevokeS3SecretRequest revokeS3SecretRequest =
+            RevokeS3SecretRequest.newBuilder()
+                    .setKerberosID(kerberosID).build();
+
+    OMRequest.Builder omRequest = OMRequest.newBuilder()
+        .setUserInfo(getUserInfo())
+        .setRevokeS3SecretRequest(revokeS3SecretRequest)
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    if (getOmRequest().hasTraceID()) {
+      omRequest.setTraceID(getOmRequest().getTraceID());
+    }
+
+    return omRequest.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    OMClientResponse omClientResponse = null;
+    OMResponse.Builder omResponse =
+            OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredLock = false;
+    IOException exception = null;
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    final RevokeS3SecretRequest revokeS3SecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    String kerberosID = revokeS3SecretRequest.getKerberosID();
+    try {
+      acquiredLock =
+         omMetadataManager.getLock().acquireWriteLock(S3_SECRET_LOCK,
+             kerberosID);
+
+      // Remove if entry exists in table
+      if (omMetadataManager.getS3SecretTable().isExist(kerberosID)) {
+//        omMetadataManager.getS3SecretTable().delete(kerberosID);

Review comment:
       Remove this line and only rely on S3RevokeSecretResponse to remove the key from secret table?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/S3RevokeSecretResponse.java
##########
@@ -0,0 +1,58 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.response.s3.security;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.S3_SECRET_TABLE;
+
+/**
+ * Response for RevokeS3Secret request.
+ */
+@CleanupTableInfo(cleanupTables = {S3_SECRET_TABLE})
+public class S3RevokeSecretResponse extends OMClientResponse {
+
+  private final S3SecretValue s3SecretValue;
+
+  public S3RevokeSecretResponse(@Nullable S3SecretValue s3SecretValue,
+                                @Nonnull OMResponse omResponse) {
+    super(omResponse);
+    this.s3SecretValue = s3SecretValue;
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    if (s3SecretValue != null && getOMResponse().getStatus() == Status.OK) {
+      omMetadataManager.getS3SecretTable().deleteWithBatch(batchOperation,
+          s3SecretValue.getKerberosID());

Review comment:
       referred here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r631267215



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3RevokeSecretRequest.java
##########
@@ -0,0 +1,149 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.request.s3.security;
+
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.security.S3RevokeSecretResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RevokeS3SecretRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_SECRET_LOCK;
+
+/**
+ * Handles RevokeS3Secret request.
+ */
+public class S3RevokeSecretRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3RevokeSecretRequest.class);
+
+  public S3RevokeSecretRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    final RevokeS3SecretRequest s3RevokeSecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    final String kerberosID = s3RevokeSecretRequest.getKerberosID();
+    final UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+
+    if (!user.getUserName().equals(kerberosID)) {

Review comment:
       Agreed. This is actually the next step for this. I'm also planning to add S3 secret get/revoke/rotate for in "ozone admin".
   
   Since you mentioned, I would just add admin check in this PR as well.

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/s3/RevokeS3SecretHandler.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.security.UserGroupInformation;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+import java.util.Scanner;
+
+/**
+ * Executes revokesecret calls.
+ */
+@Command(name = "revokesecret",
+    description = "Revoke s3 secret for current user")
+public class RevokeS3SecretHandler extends S3Handler {
+
+  @Option(names = "-y",
+      description = "Continue without interactive user confirmation")
+  private boolean yes;
+
+  @Override
+  protected boolean isApplicable() {
+    return securityEnabled();
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException {
+    String userName = UserGroupInformation.getCurrentUser().getUserName();
+
+    if (!yes) {

Review comment:
       Will do




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r633795854



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/s3/RevokeS3SecretHandler.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.security.UserGroupInformation;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+import java.util.Scanner;
+
+/**
+ * Executes revokesecret calls.
+ */
+@Command(name = "revokesecret",
+    description = "Revoke s3 secret for current user")
+public class RevokeS3SecretHandler extends S3Handler {
+
+  @Option(names = "-y",
+      description = "Continue without interactive user confirmation")
+  private boolean yes;
+
+  @Override
+  protected boolean isApplicable() {
+    return securityEnabled();
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException {
+    String userName = UserGroupInformation.getCurrentUser().getUserName();
+
+    if (!yes) {

Review comment:
       I have modified existing GetS3Secret test in TestSecureOzoneCluster to involve RevokeS3Secret, as it makes sense to me to test revoke right after get.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl merged pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
smengcl merged pull request #2239:
URL: https://github.com/apache/ozone/pull/2239


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r631267215



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3RevokeSecretRequest.java
##########
@@ -0,0 +1,149 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.request.s3.security;
+
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.security.S3RevokeSecretResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RevokeS3SecretRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_SECRET_LOCK;
+
+/**
+ * Handles RevokeS3Secret request.
+ */
+public class S3RevokeSecretRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3RevokeSecretRequest.class);
+
+  public S3RevokeSecretRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    final RevokeS3SecretRequest s3RevokeSecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    final String kerberosID = s3RevokeSecretRequest.getKerberosID();
+    final UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+
+    if (!user.getUserName().equals(kerberosID)) {

Review comment:
       Agreed. This is actually the next step for this. I'm also planning to add S3 secret get/revoke/rotate command in "ozone admin s3 ...".
   
   Since you mentioned, I would just add admin check in this PR as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] vivekratnavel commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
vivekratnavel commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r634737204



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
##########
@@ -380,6 +381,14 @@ OmMultipartUploadList listMultipartUploads(String volumeName,
    */
   S3SecretValue getS3Secret(String kerberosID) throws IOException;
 
+  /**
+   * Revokes s3Secret of given kerberos user.
+   * @param kerberosID
+   * @return S3SecretValue

Review comment:
       ```suggestion
      
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] vivekratnavel commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
vivekratnavel commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r634743673



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java
##########
@@ -73,10 +73,14 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     String kerberosID = s3GetSecretRequest.getKerberosID();
 
     UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+    // Permission check. Users need to be themselves or have admin privilege
     if (!user.getUserName().equals(kerberosID)) {
-      throw new OMException("User mismatch. Requested user name is " +
-          "mismatched " + kerberosID +", with current user " +
-          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+      if (!ozoneManager.isAdmin(kerberosID)) {
+        throw new OMException("Requested user name '" + kerberosID +
+            "' doesn't match current user '" + user.getUserName() +
+            "', nor does current user has administrator privilege.",
+            OMException.ResultCodes.USER_MISMATCH);
+      }

Review comment:
       ```suggestion
         } else {
             throw new OMException("Requested user name '" + kerberosID +
               "' doesn't match current user '" + user.getUserName() + "'");
         }
   ```

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java
##########
@@ -73,10 +73,14 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     String kerberosID = s3GetSecretRequest.getKerberosID();
 
     UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+    // Permission check. Users need to be themselves or have admin privilege
     if (!user.getUserName().equals(kerberosID)) {
-      throw new OMException("User mismatch. Requested user name is " +
-          "mismatched " + kerberosID +", with current user " +
-          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+      if (!ozoneManager.isAdmin(kerberosID)) {

Review comment:
       We might want to throw an exception when username does not match the kerberos ID.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r633750539



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3RevokeSecretRequest.java
##########
@@ -0,0 +1,149 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.request.s3.security;
+
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.security.S3RevokeSecretResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RevokeS3SecretRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_SECRET_LOCK;
+
+/**
+ * Handles RevokeS3Secret request.
+ */
+public class S3RevokeSecretRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3RevokeSecretRequest.class);
+
+  public S3RevokeSecretRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    final RevokeS3SecretRequest s3RevokeSecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    final String kerberosID = s3RevokeSecretRequest.getKerberosID();
+    final UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+
+    if (!user.getUserName().equals(kerberosID)) {
+      throw new OMException("User mismatch. Requested user name is " +
+          "mismatched " + kerberosID +", with current user " +
+          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+    }
+
+    final RevokeS3SecretRequest revokeS3SecretRequest =
+            RevokeS3SecretRequest.newBuilder()
+                    .setKerberosID(kerberosID).build();
+
+    OMRequest.Builder omRequest = OMRequest.newBuilder()

Review comment:
       We don't check for permission other than in `preExecute` in S3RevokeSecretRequest, I think I will remove `.setUserInfo(getUserInfo())`. Thanks!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#issuecomment-842618697


   @xiaoyuyao @bharatviswa504 I have addressed all comments. Please take another look. Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r634773197



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java
##########
@@ -73,10 +73,14 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     String kerberosID = s3GetSecretRequest.getKerberosID();
 
     UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+    // Permission check. Users need to be themselves or have admin privilege
     if (!user.getUserName().equals(kerberosID)) {
-      throw new OMException("User mismatch. Requested user name is " +
-          "mismatched " + kerberosID +", with current user " +
-          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+      if (!ozoneManager.isAdmin(kerberosID)) {

Review comment:
       The current logic already has the exception thrown, when both same user check and admin check fails. Prompt is also changed accordingly.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r631214391



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/s3/RevokeS3SecretHandler.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.security.UserGroupInformation;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+import java.util.Scanner;
+
+/**
+ * Executes revokesecret calls.
+ */
+@Command(name = "revokesecret",
+    description = "Revoke s3 secret for current user")
+public class RevokeS3SecretHandler extends S3Handler {
+
+  @Option(names = "-y",
+      description = "Continue without interactive user confirmation")
+  private boolean yes;
+
+  @Override
+  protected boolean isApplicable() {
+    return securityEnabled();
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException {
+    String userName = UserGroupInformation.getCurrentUser().getUserName();
+
+    if (!yes) {

Review comment:
       Do we want to add a check this command to run only when security enabled?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3RevokeSecretRequest.java
##########
@@ -0,0 +1,149 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.request.s3.security;
+
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.security.S3RevokeSecretResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RevokeS3SecretRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_SECRET_LOCK;
+
+/**
+ * Handles RevokeS3Secret request.
+ */
+public class S3RevokeSecretRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3RevokeSecretRequest.class);
+
+  public S3RevokeSecretRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    final RevokeS3SecretRequest s3RevokeSecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    final String kerberosID = s3RevokeSecretRequest.getKerberosID();
+    final UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+
+    if (!user.getUserName().equals(kerberosID)) {
+      throw new OMException("User mismatch. Requested user name is " +
+          "mismatched " + kerberosID +", with current user " +
+          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+    }
+
+    final RevokeS3SecretRequest revokeS3SecretRequest =
+            RevokeS3SecretRequest.newBuilder()
+                    .setKerberosID(kerberosID).build();
+
+    OMRequest.Builder omRequest = OMRequest.newBuilder()

Review comment:
       As we don't perform any acl check there is no real use of modifying OMRequest to add UserInfo.
   Or is there plan to check acl for this request?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3RevokeSecretRequest.java
##########
@@ -0,0 +1,149 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.request.s3.security;
+
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.security.S3RevokeSecretResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RevokeS3SecretRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_SECRET_LOCK;
+
+/**
+ * Handles RevokeS3Secret request.
+ */
+public class S3RevokeSecretRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3RevokeSecretRequest.class);
+
+  public S3RevokeSecretRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    final RevokeS3SecretRequest s3RevokeSecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    final String kerberosID = s3RevokeSecretRequest.getKerberosID();
+    final UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+
+    if (!user.getUserName().equals(kerberosID)) {
+      throw new OMException("User mismatch. Requested user name is " +
+          "mismatched " + kerberosID +", with current user " +
+          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+    }
+
+    final RevokeS3SecretRequest revokeS3SecretRequest =
+            RevokeS3SecretRequest.newBuilder()
+                    .setKerberosID(kerberosID).build();
+
+    OMRequest.Builder omRequest = OMRequest.newBuilder()
+        .setUserInfo(getUserInfo())
+        .setRevokeS3SecretRequest(revokeS3SecretRequest)
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    if (getOmRequest().hasTraceID()) {
+      omRequest.setTraceID(getOmRequest().getTraceID());
+    }
+
+    return omRequest.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    OMClientResponse omClientResponse = null;
+    OMResponse.Builder omResponse =
+            OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredLock = false;
+    IOException exception = null;
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    final RevokeS3SecretRequest revokeS3SecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    String kerberosID = revokeS3SecretRequest.getKerberosID();
+    try {
+      acquiredLock =
+         omMetadataManager.getLock().acquireWriteLock(S3_SECRET_LOCK,
+             kerberosID);
+
+      // Remove if entry exists in table
+      if (omMetadataManager.getS3SecretTable().isExist(kerberosID)) {
+//        omMetadataManager.getS3SecretTable().delete(kerberosID);
+        omClientResponse = new S3RevokeSecretResponse(
+                new S3SecretValue(kerberosID, null),

Review comment:
       Minor: Can we pass direct KerberosID instead of creating S3SecretValue object?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] vivekratnavel commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
vivekratnavel commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r634987415



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java
##########
@@ -73,10 +73,14 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     String kerberosID = s3GetSecretRequest.getKerberosID();
 
     UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+    // Permission check. Users need to be themselves or have admin privilege
     if (!user.getUserName().equals(kerberosID)) {
-      throw new OMException("User mismatch. Requested user name is " +
-          "mismatched " + kerberosID +", with current user " +
-          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+      if (!ozoneManager.isAdmin(kerberosID)) {
+        throw new OMException("Requested user name '" + kerberosID +
+            "' doesn't match current user '" + user.getUserName() +
+            "', nor does current user has administrator privilege.",
+            OMException.ResultCodes.USER_MISMATCH);
+      }

Review comment:
       @smengcl Thanks for the clarification!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r631268296



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3RevokeSecretRequest.java
##########
@@ -0,0 +1,149 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.request.s3.security;
+
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.security.S3RevokeSecretResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RevokeS3SecretRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_SECRET_LOCK;
+
+/**
+ * Handles RevokeS3Secret request.
+ */
+public class S3RevokeSecretRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3RevokeSecretRequest.class);
+
+  public S3RevokeSecretRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    final RevokeS3SecretRequest s3RevokeSecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    final String kerberosID = s3RevokeSecretRequest.getKerberosID();
+    final UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+
+    if (!user.getUserName().equals(kerberosID)) {
+      throw new OMException("User mismatch. Requested user name is " +
+          "mismatched " + kerberosID +", with current user " +
+          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+    }
+
+    final RevokeS3SecretRequest revokeS3SecretRequest =
+            RevokeS3SecretRequest.newBuilder()
+                    .setKerberosID(kerberosID).build();
+
+    OMRequest.Builder omRequest = OMRequest.newBuilder()
+        .setUserInfo(getUserInfo())
+        .setRevokeS3SecretRequest(revokeS3SecretRequest)
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    if (getOmRequest().hasTraceID()) {
+      omRequest.setTraceID(getOmRequest().getTraceID());
+    }
+
+    return omRequest.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    OMClientResponse omClientResponse = null;
+    OMResponse.Builder omResponse =
+            OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredLock = false;
+    IOException exception = null;
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    final RevokeS3SecretRequest revokeS3SecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    String kerberosID = revokeS3SecretRequest.getKerberosID();
+    try {
+      acquiredLock =
+         omMetadataManager.getLock().acquireWriteLock(S3_SECRET_LOCK,
+             kerberosID);
+
+      // Remove if entry exists in table
+      if (omMetadataManager.getS3SecretTable().isExist(kerberosID)) {
+//        omMetadataManager.getS3SecretTable().delete(kerberosID);
+        omClientResponse = new S3RevokeSecretResponse(
+                new S3SecretValue(kerberosID, null),

Review comment:
       Yes.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#issuecomment-844579549


   Just realized I have a typo in the permission check, should be `isAdmin(user.getUserName())` instead of `kerberosID`. Will post an addendum in a min.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r633795252



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3RevokeSecretRequest.java
##########
@@ -0,0 +1,149 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.request.s3.security;
+
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.security.S3RevokeSecretResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RevokeS3SecretRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_SECRET_LOCK;
+
+/**
+ * Handles RevokeS3Secret request.
+ */
+public class S3RevokeSecretRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3RevokeSecretRequest.class);
+
+  public S3RevokeSecretRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    final RevokeS3SecretRequest s3RevokeSecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    final String kerberosID = s3RevokeSecretRequest.getKerberosID();
+    final UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+
+    if (!user.getUserName().equals(kerberosID)) {

Review comment:
       @xiaoyuyao Currently, only the user himself is allowed to invoke S3 getsecret. revokesecret is based on that. In the second commit I have changed it to allow admins to get/revoke secret for other users as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r631183030



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3RevokeSecretRequest.java
##########
@@ -0,0 +1,149 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.request.s3.security;
+
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.security.S3RevokeSecretResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RevokeS3SecretRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_SECRET_LOCK;
+
+/**
+ * Handles RevokeS3Secret request.
+ */
+public class S3RevokeSecretRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3RevokeSecretRequest.class);
+
+  public S3RevokeSecretRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    final RevokeS3SecretRequest s3RevokeSecretRequest =
+        getOmRequest().getRevokeS3SecretRequest();
+    final String kerberosID = s3RevokeSecretRequest.getKerberosID();
+    final UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+
+    if (!user.getUserName().equals(kerberosID)) {

Review comment:
       I think we may want to allow not only the original kerberos user but also the admin can revoke s3 secret. Can you confirm the use case here? The code below seems allow only the original kerberos use to revoke. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r631273225



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/s3/RevokeS3SecretHandler.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.security.UserGroupInformation;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+import java.util.Scanner;
+
+/**
+ * Executes revokesecret calls.
+ */
+@Command(name = "revokesecret",
+    description = "Revoke s3 secret for current user")
+public class RevokeS3SecretHandler extends S3Handler {
+
+  @Option(names = "-y",
+      description = "Continue without interactive user confirmation")
+  private boolean yes;
+
+  @Override
+  protected boolean isApplicable() {
+    return securityEnabled();
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException {
+    String userName = UserGroupInformation.getCurrentUser().getUserName();
+
+    if (!yes) {

Review comment:
       The security enabled check is actually already achieved by `protected boolean isApplicable()` above in this class.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r634775927



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java
##########
@@ -73,10 +73,14 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     String kerberosID = s3GetSecretRequest.getKerberosID();
 
     UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+    // Permission check. Users need to be themselves or have admin privilege
     if (!user.getUserName().equals(kerberosID)) {
-      throw new OMException("User mismatch. Requested user name is " +
-          "mismatched " + kerberosID +", with current user " +
-          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+      if (!ozoneManager.isAdmin(kerberosID)) {
+        throw new OMException("Requested user name '" + kerberosID +
+            "' doesn't match current user '" + user.getUserName() +
+            "', nor does current user has administrator privilege.",
+            OMException.ResultCodes.USER_MISMATCH);
+      }

Review comment:
       I have replaced the nested if with a single one to make the logic cleaner, hopefully.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret in Ozone CLI

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r631192591



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/s3/RevokeS3SecretHandler.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.security.UserGroupInformation;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+import java.util.Scanner;
+
+/**
+ * Executes revokesecret calls.
+ */
+@Command(name = "revokesecret",
+    description = "Revoke s3 secret for current user")
+public class RevokeS3SecretHandler extends S3Handler {
+
+  @Option(names = "-y",
+      description = "Continue without interactive user confirmation")
+  private boolean yes;
+
+  @Override
+  protected boolean isApplicable() {
+    return securityEnabled();
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException {
+    String userName = UserGroupInformation.getCurrentUser().getUserName();
+
+    if (!yes) {

Review comment:
       Can we add some acceptance tests for this?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#issuecomment-844344071


   Thanks @xiaoyuyao @bharatviswa504 @vivekratnavel for reviewing this. Will merge shortly.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] smengcl commented on a change in pull request #2239: HDDS-5206. Support revoking S3 secret

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2239:
URL: https://github.com/apache/ozone/pull/2239#discussion_r634774354



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java
##########
@@ -73,10 +73,14 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     String kerberosID = s3GetSecretRequest.getKerberosID();
 
     UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+    // Permission check. Users need to be themselves or have admin privilege
     if (!user.getUserName().equals(kerberosID)) {
-      throw new OMException("User mismatch. Requested user name is " +
-          "mismatched " + kerberosID +", with current user " +
-          user.getUserName(), OMException.ResultCodes.USER_MISMATCH);
+      if (!ozoneManager.isAdmin(kerberosID)) {
+        throw new OMException("Requested user name '" + kerberosID +
+            "' doesn't match current user '" + user.getUserName() +
+            "', nor does current user has administrator privilege.",
+            OMException.ResultCodes.USER_MISMATCH);
+      }

Review comment:
       Hey @vivekratnavel thanks for the comment.  But the added `else` wouldn't work because it is triggered when an admin is revoking another user's secret. We want to allow admins to revoke other user's secret, throwing exception would prevent this.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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