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 2022/08/25 10:02:06 UTC

[GitHub] [ozone] smengcl commented on a diff in pull request #3653: HDDS-6964. [Snapshot] Split out shared "Path based access" code from OM.

smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r954727098


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java:
##########
@@ -693,13 +693,12 @@ public OmMultipartUploadListParts listParts(String volumeName,
     boolean isTruncated = false;
     int nextPartNumberMarker = 0;
     BucketLayout bucketLayout = BucketLayout.DEFAULT;
-    if (ozoneManager != null) {

Review Comment:
   Looks like we only intended to have this null check for some tests according to these two constructors:
   
   https://github.com/apache/ozone/blob/749b5265030ff44910843a6566a15d6849fd9f59/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L176-L191
   
   But neither constructors seem to be in-use anymore. Hmm, we could raise a clean up jira in master branch if this is the case.
   
   If so, we could revert this hunk in this PR, so the proper clean up can be done in master branch (so we can backport later without conflicts)? -- But this is assuming this PR will be merged to the snapshot branch. Last week we talked about merging this to the master branch directly. Do you want to switch the PR base branch?



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMReader.java:
##########
@@ -0,0 +1,495 @@
+/**
+ * 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 java.io.IOException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditAction;
+import org.apache.hadoop.ozone.audit.AuditEventStatus;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.AuditMessage;
+import org.apache.hadoop.ozone.audit.Auditor;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObjInfo;
+import org.apache.hadoop.ozone.security.acl.RequestContext;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.slf4j.Logger;
+import java.net.InetAddress;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.hdds.server.ServerUtils.getRemoteUserName;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS;
+import static org.apache.hadoop.ozone.om.KeyManagerImpl.getRemoteUser;
+import static org.apache.hadoop.ozone.om.OzoneManager.getS3Auth;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_REQUEST;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
+import org.apache.hadoop.ozone.security.acl.OzoneAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneNativeAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.security.acl.OzoneObj.ResourceType;
+import org.apache.hadoop.ozone.security.acl.OzoneObj.StoreType;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
+
+/**
+ * OM Metadata Reading class for the OM and Snapshot managers.
+ *
+ * This abstraction manages all the metadata key/acl reading
+ * from a rocksDb instance, for both the OM and OM snapshots.
+ */
+public class OmMReader implements Auditor {

Review Comment:
   If you don't mind, shall we rename `OmMReader` to `OmMetadataReader`? I personally think the latter is more consistent with the existing Ozone class naming schemes.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java:
##########
@@ -1053,17 +906,15 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
         .build();
 
     BucketLayout bucketLayout = BucketLayout.DEFAULT;
-    if (ozoneManager != null) {

Review Comment:
   Same here.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMReaderMetrics.java:
##########
@@ -0,0 +1,45 @@
+/**
+ * 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;
+
+/**
+ * Interface OM Metadata Reading metrics classes.
+ */
+public interface OmMReaderMetrics {

Review Comment:
   `OmMetadataReaderMetrics`
   
   `%s/mMReader/mMetadataReader/g` :)



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java:
##########
@@ -2534,15 +2385,10 @@ public List<OmKeyInfo> getPendingDeletionSubFiles(long volumeId,
 
   public boolean isBucketFSOptimized(String volName, String buckName)
       throws IOException {
-    // This will never be null in reality but can be null in unit test cases.
-    // Added safer check for unit testcases.
-    if (ozoneManager == null) {
-      return false;
-    }

Review Comment:
   Same here.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/IOzoneAcl.java:
##########
@@ -28,37 +28,6 @@
  * Interface for Ozone Acl management.
  */
 public interface IOzoneAcl {
-
-  /**
-   * Add acl for Ozone object. Return true if acl is added successfully else
-   * false.
-   * @param obj Ozone object for which acl should be added.
-   * @param acl ozone acl to be added.
-   *
-   * @throws IOException if there is error.
-   * */
-  boolean addAcl(OzoneObj obj, OzoneAcl acl) throws IOException;
-
-  /**
-   * Remove acl for Ozone object. Return true if acl is removed successfully
-   * else false.
-   * @param obj Ozone object.
-   * @param acl Ozone acl to be removed.
-   *
-   * @throws IOException if there is error.
-   * */
-  boolean removeAcl(OzoneObj obj, OzoneAcl acl) throws IOException;
-
-  /**
-   * Acls to be set for given Ozone object. This operations reset ACL for
-   * given object to list of ACLs provided in argument.
-   * @param obj Ozone object.
-   * @param acls List of acls.
-   *
-   * @throws IOException if there is error.
-   * */
-  boolean setAcl(OzoneObj obj, List<OzoneAcl> acls) throws IOException;
-

Review Comment:
   Thanks for cleaning this up. These IMHO can be safely removed as we have switched **all** write requests to Ratis. These were for the old non-Ratis write path.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -710,6 +703,8 @@ private void instantiateServices(boolean withNewSnapshot) throws IOException {
     prefixManager = new PrefixManagerImpl(metadataManager, isRatisEnabled);
     keyManager = new KeyManagerImpl(this, scmClient, configuration,
         omStorage.getOmId());

Review Comment:
   > But the keyManager, volumeManager fields etc are also changed by instantiateServices(), and they are not defined to be "volatile". That seems like a bug to me, but perhaps it was intentially done for performance reasons.
   
   According to [Oracle's Java doc](https://docs.oracle.com/javase/tutorial/essential/concurrency/atomic.html), reference variables reads and writes are atomic even without the `volatile` keyword:
   
   > - Reads and writes are atomic for reference variables and for most primitive variables (all types except long and double).
   > - Reads and writes are atomic for all variables declared volatile (including long and double variables).
   
   `volumeManager` and `keyManager` are reference variables. So it should be fine, in theory. CMIIW.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -4118,7 +3835,8 @@ private void startJVMPauseMonitor() {
     jvmPauseMonitor.start();
   }
 
-  public ResolvedBucket resolveBucketLink(KeyArgs args,
+  public ResolvedBucket resolveBucketLink(
+      OzoneManagerProtocolProtos.KeyArgs args,

Review Comment:
   `KeyArgs -> OzoneManagerProtocolProtos.KeyArgs` ?
   
   I suspect this is automatically changed by IntelliJ? If so we could restore the import line:
   
   ```java
   import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
   ```
   
   and back to using `KeyArgs`.
   
   To prevent IntelliJ from doing this in the future, I believe you can enable:
   ```
   Preferences -> Editor -> Code Style -> Java -> Imports -> Insert imports for inner classes
   ```
   [Ref](https://www.jetbrains.com/help/idea/code-style-java.html#imports_table:~:text=Insert%20imports%20for%20inner%20classes)



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -434,6 +424,8 @@ private enum State {
 
   private final boolean isSecurityEnabled;
 
+  private OmMReader omMReader;

Review Comment:
   Let's add a brief comment above this line mentioning this `omMetadataReader` points to the active ObjectStore RDB.



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