You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "ChenSammi (via GitHub)" <gi...@apache.org> on 2023/02/07 08:33:45 UTC

[GitHub] [ozone] ChenSammi commented on a diff in pull request #4196: HDDS-7815. Extract S3 secret layer

ChenSammi commented on code in PR #4196:
URL: https://github.com/apache/ozone/pull/4196#discussion_r1098305161


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/S3SecretStoreMap.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Map based implementation of {@link S3SecretStore}.
+ */
+public class S3SecretStoreMap implements S3SecretStore {
+  private final Map<String, S3SecretValue> map = new ConcurrentHashMap<>();
+
+  public S3SecretStoreMap(Map<String, S3SecretValue> map) {
+    this.map.putAll(map);
+  }
+
+  @Override
+  public void store(String kerberosId, S3SecretValue secret)
+      throws IOException {
+    map.put(kerberosId, secret);
+  }
+
+  @Override
+  public S3SecretValue get(String kerberosID) throws IOException {
+    return map.get(kerberosID);
+  }
+
+  @Override
+  public void revokeSecret(String kerberosId) throws IOException {
+    map.remove(kerberosId);
+  }
+
+  @Override
+  public S3Batcher batcher() {
+    return null;
+  }
+}

Review Comment:
   Please move this implementation to test package if it only for test. 



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3SecretManagerImpl.java:
##########
@@ -88,22 +64,43 @@ public String getS3UserSecretString(String awsAccessKey)
         "awsAccessKeyId cannot be null or empty.");
     LOG.trace("Get secret for awsAccessKey:{}", awsAccessKey);
 
-    S3SecretValue s3Secret;
-    omMetadataManager.getLock().acquireReadLock(S3_SECRET_LOCK, awsAccessKey);
-    try {
-      s3Secret = omMetadataManager.getS3SecretTable().get(awsAccessKey);
-      if (s3Secret == null) {
-        throw new OzoneSecurityException("S3 secret not found for " +
-            "awsAccessKeyId " + awsAccessKey, S3_SECRET_NOT_FOUND);
-      }
-    } finally {
-      omMetadataManager.getLock().releaseReadLock(S3_SECRET_LOCK, awsAccessKey);
+    S3SecretValue s3Secret = s3SecretStore.get(awsAccessKey);
+    if (s3Secret == null) {
+      throw new OzoneSecurityException("S3 secret not found for " +
+          "awsAccessKeyId " + awsAccessKey, S3_SECRET_NOT_FOUND);
     }
 
     return s3Secret.getAwsSecret();
   }
 
-  public OMMetadataManager getOmMetadataManager() {
-    return omMetadataManager;
+  @Override
+  public void store(String kerberosId, S3SecretValue secretValue)
+      throws IOException {
+    s3SecretStore.store(kerberosId, secretValue);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Secret for accessKey:{}, proto:{}", kerberosId, secretValue);

Review Comment:
   secretValue should not be logged.



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/S3SecretStore.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+
+import java.io.IOException;
+
+/**
+ * S3 secret store interface.
+ */
+public interface S3SecretStore {
+
+  /**
+   * Store provided s3 secret with associated kerberos ID.
+   * @param kerberosId kerberos ID.
+   * @param secret s3 secret.
+   * @throws IOException in case when secret storing failed.
+   */
+  void store(String kerberosId, S3SecretValue secret) throws IOException;

Review Comment:
   Consistent naming is preferred,  either (store, get, revoke) or (storeSecret, getSecret, revokeSecret).  Consider OmMetadataManagerImpl implements the S3SecretStore. And there are many functions in OmMetadataManagerImpl. I more prefer (storeSecret, getSecret, revokeSecret) . 



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java:
##########
@@ -154,24 +147,20 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       assert (accessId.equals(updateGetS3SecretRequest.getKerberosID()));
     }
 
+    S3SecretManager s3SecretManager = ozoneManager.getS3SecretManager();
     try {
-      acquiredLock = omMetadataManager.getLock()
-          .acquireWriteLock(S3_SECRET_LOCK, accessId);
-

Review Comment:
   This lock protects both getS3Secret and cache operation. It cannot be removed. 



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/OMSetSecretResponse.java:
##########
@@ -1,25 +1,26 @@
-/**
- * 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.
+/*
+  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.

Review Comment:
   Please revert this piece of change. 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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