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/03 23:10:56 UTC

[GitHub] [ozone] GeorgeJahad opened a new pull request, #3653: HDDS-6964. [Snapshot] Split out shared "Path based access" code from OM.

GeorgeJahad opened a new pull request, #3653:
URL: https://github.com/apache/ozone/pull/3653

   ## What changes were proposed in this pull request?
   
   This PR refactors some of the OzoneManager code to make it available for use by the upcoming Om Snapshotting feature.
   
   ### OmMReader class
   Reading the metadata for OM snapshots is quite similiar to reading the metadata for the active fs.  The main difference is a different RocksDb instance is used by the OmMetadataManager, KeyManager, etc.  The code that reads that metadata is currently in the OzoneManager class.
   
   This PR refactors that code and moves it into a new class called the OmMReader, (Ozone Metadata Reader.)  To replace the code removed from the ozoneManager, an OmMReader will be constructed using the RocksDb for the active fs.  For snapshots, the OmMReader will be constructed with the rocksDb instance corresponding to that snapshot.
   
   ### Locks and Metrics
   The code in OmMReader makes extensive use of the OzoneManagerLock and OMMetrics classes.  Snapshotting will need its own version of those  so I've created two new interfaces that correspond to them: OmLock and OmMReaderMetrics.  These interfaces are used in the OmMReader code so that the OzoneManager and Snapshotting can each have their own.  (They each pass their own version into the OmMReader.)
   
   Note that snapshot version of the lock won't be a lock at all.  It just seemed cleaner to use an interface instead of putting "if statements" everywhere.  Instead of a real lock, the snapshot version will always return true on "acquireReadLock()" and false on "acquireWriteLock()".
   
   ### Removal of obsolete IOzoneAcl code
   
   In the process of doing the refactoring, I discovered that much of the IOzoneAcl interface is no longer in use.  So I removed those methods, (and fixed the few tests that still invoked them.)
   
   These are the methods [removed](https://github.com/apache/ozone/blob/18ea21661/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/IOzoneAcl.java#L40-L60)
   
   ### KeyManagerImpl changes
   In addition to the IOzoneAcl deletion mentioned above, I also had to change the KeyManagerImpl to not read the metadataManager through the ozoneManager.
   
   In a few places, it was unnecessarily reading metadata like so: "ozoneManager.getMetadataManager()" instead of using the metadataManager field initialized by its constructor.  
   
   That would prevent the snapshot rocksDb instance from being read so I switched those to use the metadataManager field.
   
   ### Initialization of the OmMReader field in OzoneManager
   
   After reviewing the code, I feel the correct place to init the OmMReader is in the instantiateServices() method [here](https://github.com/apache/ozone/blob/dd01353137322009edba4e20f07f697ff22d2a82/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java#L694).
   
   The KeyManager and others are set there, and the OmMReader should be reset anytime that method is called and those are reset.
   
   #### metrics field
   However, the omMReader needs the omMetrics which is initialized after instantiateServices() is called in the ozone manager [constructor](https://github.com/apache/ozone/blob/18ea21661/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java#L568).
   
   To work around that, I moved the OMMetrics.create() to before instantiateServices() is called in the constructor:
   ```
   +    metrics = OMMetrics.create();
        instantiateServices(false);
   ```
   
   #### omRpcAddress field
   The omRpcAddress has the same problem but I was hesitant to move that code, so omMReader doesn't store it as a field.  It just accesses it through the om, like so:  "ozoneManager.getOmRpcServerAddr()"
   
   
   ### volatile fields
   
   The new omMReader field in the OzoneManager is not final, and will be changed each time instantiateServices() is called, (as it should.)
   
   But the OzoneManager is multithreaded code, and, IMHO, such fields should be defined "volatile".  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.  In any case, I have not made the omMReader field volatile for now.
   
   Let me know if you think that is correct.
   
   ### Reviewing the changes
   
   There are lot of changes in this PR but most are very straightforward.  The only part that is hard to review is the code that is moved from the OzoneManager to the OmMReader and the only reason that is hard is because the diff tool isn't smart enough to find the changes in the new file.
   
   So I created a python script that uses string matching to generate a version of OmMReader that has the original unchanged OzoneManager methods pasted in.  You can see the diffs [here](https://github.com/GeorgeJahad/ozone/compare/b2ffbbed2e8d9ea815cff4d2306e2798c3dad84a..compareMReader1?w=1)
   
   
   As you'll see, these are the main things changed, in the code moved to the OmMReader:
   
   1. A few private methods were made package-private.
   2. The "LOG" and "AUDIT" field names were changed to "log" and "audit".  (Since they are no longer static final, checkstyle won't let them be all caps.)
   3. Changed the omRpcAddress access to "ozoneManager.getOmRpcServerAddr()" as mentioned above.
   
   
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6964
   
   ## How was this patch tested?
   
   All tests continue to work.
   
   In my own repo, I've added some code that excercises the OmMReader to read snapshots.  That will be part of the next PR.
   


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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955283855


##########
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:
   > If so, then it might be better just to remove the unused constructors here and deal with any merge conflicts when we do the merge to master. (It shouldn't be too bad.)
   
   Alright, let's do this on the snapshot branch then. We might have to deal with some conflicts if other PRs are changing `OzoneManager` around at the same time but yeah it shouldn't been too bad (most likely just `import` conflicts).



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


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

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955227972


##########
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:
   @smengcl I'm fine with creating a cleanup PR against master and then backporting, but @prashantpogde seemed like he was in a bit of a hurry to get this done.  
   
   If so, then it might be better just to remove the unused constructors here and deal with any merge conflicts when we do the merge to master.  (It shouldn't be too bad.)  Either way is fine with me.  Just let me know.



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


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

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955438247


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -151,14 +151,12 @@
 import org.apache.hadoop.hdds.utils.TransactionInfo;
 import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
-import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.apache.hadoop.ozone.om.snapshot.OzoneManagerSnapshotProvider;
 import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager;
 import org.apache.hadoop.ozone.om.upgrade.OMUpgradeFinalizer;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerAdminProtocolProtos.OzoneManagerAdminService;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DBUpdatesRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;

Review Comment:
   pushed



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


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

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on PR #3653:
URL: https://github.com/apache/ozone/pull/3653#issuecomment-1219073408

   @GeorgeJahad let us discuss the refactoring changes in the weekly call tomorrow.


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


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

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955435380


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -151,14 +151,12 @@
 import org.apache.hadoop.hdds.utils.TransactionInfo;
 import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
-import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.apache.hadoop.ozone.om.snapshot.OzoneManagerSnapshotProvider;
 import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager;
 import org.apache.hadoop.ozone.om.upgrade.OMUpgradeFinalizer;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerAdminProtocolProtos.OzoneManagerAdminService;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DBUpdatesRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;

Review Comment:
   The constructors were removed in a previous checkin here: https://github.com/apache/ozone/pull/3653/commits/8da67ae5660eeb091594333d9ee5da5494cb25ef#diff-bde0dade7dd5ddda419499f4f999d25d40fcec1412e0ce809c36ffd1be473f22L176
   
   I missed the KeyArgs.  Fixing now.



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


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

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955319608


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java:
##########
@@ -214,7 +214,8 @@ public void checkAcls(OzoneManager ozoneManager,
       String vol, String bucket, String key) throws IOException {
     checkAcls(ozoneManager, resType, storeType, aclType, vol, bucket, key,
         ozoneManager.getVolumeOwner(vol, aclType, resType),
-        ozoneManager.getBucketOwner(vol, bucket, aclType, resType));
+        ozoneManager
+        .getBucketOwner(vol, bucket, aclType, resType));

Review Comment:
   fixed, 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.

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


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

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on PR #3653:
URL: https://github.com/apache/ozone/pull/3653#issuecomment-1210078938

   A flaky test, unrelated to snapshotting, was preventing the tests from passing since I did the rebase.  I debugged it and added the fix to this pr.  I describe the problem [here](https://issues.apache.org/jira/browse/HDDS-7110)
   
   This is the [fix](https://github.com/apache/ozone/pull/3653/commits/4dfef32c99ae4ab352711cc18ec24233b1cda157)
   


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


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

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


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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955303867


##########
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:
   Thanks @GeorgeJahad . Good read.
   
   I see how stale read could potentially become a problem now. e.g. after another `instantiateServices()` call while other threads are still actively using `metadataManager`, `multiTenantManager`, `volumeManager`, etc. This might cause legitimate issue in other worker threads ~~like `OpenKeyCleanupService`, `KeyDeletingService`~~ (sry, bad example, these are re-inited in `KeyManagerImpl` so should be alright). Thanks for raising awareness of this potential issue.
   
   IIRC, making those references `volatile` shouldn't impose much performance impact on variable reads (at least on x86 because of [TSO](https://docs.oracle.com/cd/E23824_01/html/819-3196/hwovr-15.html#:~:text=store%20ordering%20(PSO).-,Total%20Store%20Ordering,-(TSO))). Let's open a separate jira on master branch to discuss this with others 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.

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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955312394


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java:
##########
@@ -214,7 +214,8 @@ public void checkAcls(OzoneManager ozoneManager,
       String vol, String bucket, String key) throws IOException {
     checkAcls(ozoneManager, resType, storeType, aclType, vol, bucket, key,
         ozoneManager.getVolumeOwner(vol, aclType, resType),
-        ozoneManager.getBucketOwner(vol, bucket, aclType, resType));
+        ozoneManager
+        .getBucketOwner(vol, bucket, aclType, resType));

Review Comment:
   nit: accidental new line?



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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955303867


##########
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:
   Thanks @GeorgeJahad . Good read.
   
   I see how stale read could potentially become a problem now. e.g. after another `instantiateServices()` call while other threads are still actively using `metadataManager`, `multiTenantManager`, `volumeManager`, etc. This might cause legitimate issue in other worker threads ~~like `OpenKeyCleanupService`, `KeyDeletingService`~~ (sry, bad example, these are re-inited in `KeyManagerImpl` so should be alright). Thanks for raising awareness of this potential issue.
   
   IIRC, making those references `volatile` shouldn't impose much performance impact on variable reads (at least on x86 because of [TSO](https://docs.oracle.com/cd/E23824_01/html/819-3196/hwovr-15.html#:~:text=store%20ordering%20(PSO).-,Total%20Store%20Ordering,-(TSO))). Let's open a separate jira on master branch to discuss this with others as well (and to check if other existing OM threads are impacted by 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.

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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955303867


##########
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:
   Thanks @GeorgeJahad . Good read.
   
   I see how stale read could potentially become a problem now. e.g. after another `instantiateServices()` call while other threads are still actively using `metadataManager`, `multiTenantManager`, `volumeManager`, etc. This might cause legitimate issue in other worker threads ~~like `OpenKeyCleanupService`, `KeyDeletingService`~~ (sry, bad example, these are re-inited in `KeyManagerImpl` so should be alright). Thanks for raising awareness of this potential issue.
   
   IIRC, making those references `volatile` shouldn't impose much performance impact on variable reads (at least on x86 because of [TSO](https://docs.oracle.com/cd/E23824_01/html/819-3196/hwovr-15.html#:~:text=store%20ordering%20(PSO).-,Total%20Store%20Ordering,-(TSO))). Let's open a separate jira on master branch to discuss this with others as well (and to check if other existing OM threads could be affected by stale read).



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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955303867


##########
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:
   Thanks @GeorgeJahad . Good read.
   
   I see how stale read could potentially become a problem now. e.g. after another `instantiateServices()` call while other threads are still actively using `metadataManager`, `multiTenantManager`, `volumeManager`, etc. This might cause legitimate issue in other worker threads ~~like `OpenKeyCleanupService`, `KeyDeletingService`~~ (sry, bad example, these are re-inited in `KeyManagerImpl` so should be alright). Thanks for raising awareness of this potential issue.
   
   IIRC, making those references `volatile` shouldn't impose much performance impact on variable reads (at least on x86 because of [TSO](https://docs.oracle.com/cd/E23824_01/html/819-3196/hwovr-15.html#:~:text=store%20ordering%20(PSO).-,Total%20Store%20Ordering,-(TSO))). Let's open a separate jira on master branch to discuss this with others as well (and to check if other existing OM threads are affected by 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.

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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955303867


##########
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:
   Thanks @GeorgeJahad . Good read.
   
   I see how stale read could potentially become a problem now. e.g. after another `instantiateServices()` call while other threads are still actively using `metadataManager`, `multiTenantManager`, `volumeManager`, etc. This might cause legitimate issue in other worker threads ~~like `OpenKeyCleanupService`, `KeyDeletingService`~~ (sry, bad example, these are re-inited in `KeyManagerImpl` so should be alright). Thanks for raising awareness of this potential issue.
   
   IIUC, making those references `volatile` shouldn't impose much performance impact on variable reads (at least on x86 because of [TSO](https://docs.oracle.com/cd/E23824_01/html/819-3196/hwovr-15.html#:~:text=store%20ordering%20(PSO).-,Total%20Store%20Ordering,-(TSO))). Let's open a separate jira on master branch to discuss this with others as well (and to check if other existing OM threads could be affected by stale read).



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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955303867


##########
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:
   Thanks @GeorgeJahad . Good read.
   
   I see how stale read could potentially become a problem now. e.g. after another `instantiateServices()` call while other threads are still actively using `metadataManager`, `multiTenantManager`, `volumeManager`, etc. This might cause legitimate issue in other worker threads ~~like `OpenKeyCleanupService`, `KeyDeletingService`~~ (sry, bad example, those are re-inited in `KeyManagerImpl` so should be alright). Thanks for raising awareness of this potential issue.
   
   IIRC, making those references `volatile` shouldn't impose much performance impact on variable reads (at least on x86 because of [TSO](https://docs.oracle.com/cd/E23824_01/html/819-3196/hwovr-15.html#:~:text=store%20ordering%20(PSO).-,Total%20Store%20Ordering,-(TSO))). Let's open a separate jira on master branch to discuss this with others 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.

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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955303867


##########
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:
   Thanks @GeorgeJahad . Good read.
   
   I see how stale read could potentially become a problem now. e.g. after another `instantiateServices()` call while other threads are still actively using `metadataManager`, `multiTenantManager`, `volumeManager`, etc. This might cause legitimate issue in other worker threads ~~like `OpenKeyCleanupService`, `KeyDeletingService`~~ (those are re-inited in `KeyManagerImpl` so should be alright). Thanks for raising awareness of this potential issue.
   
   IIRC, making those references `volatile` shouldn't impose much performance impact on variable reads (at least on x86 because of [TSO](https://docs.oracle.com/cd/E23824_01/html/819-3196/hwovr-15.html#:~:text=store%20ordering%20(PSO).-,Total%20Store%20Ordering,-(TSO))). Let's open a separate jira on master branch to discuss this with others 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.

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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955303867


##########
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:
   Thanks @GeorgeJahad . Good read.
   
   I see how stale read could potentially become a problem now. e.g. after another `instantiateServices()` call while other threads are still actively using `metadataManager`, `multiTenantManager`, `volumeManager`, etc. This might cause legitimate issue in worker threads like `OpenKeyCleanupService`, `KeyDeletingService`. Thanks for raising awareness of this potential issue.
   
   IIRC, making those references `volatile` shouldn't impose much performance impact on variable reads (at least on x86 because of [TSO](https://docs.oracle.com/cd/E23824_01/html/819-3196/hwovr-15.html#:~:text=store%20ordering%20(PSO).-,Total%20Store%20Ordering,-(TSO))). Let's open a separate jira on master branch to discuss this with others 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.

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


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

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955228173


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

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


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

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r955404521


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -151,14 +151,12 @@
 import org.apache.hadoop.hdds.utils.TransactionInfo;
 import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
-import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.apache.hadoop.ozone.om.snapshot.OzoneManagerSnapshotProvider;
 import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager;
 import org.apache.hadoop.ozone.om.upgrade.OMUpgradeFinalizer;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerAdminProtocolProtos.OzoneManagerAdminService;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DBUpdatesRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;

Review Comment:
   Corresponding to this comment: https://github.com/apache/ozone/pull/3653#discussion_r954754529



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


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

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3653:
URL: https://github.com/apache/ozone/pull/3653#discussion_r954992484


##########
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:
   It is true that the writes are atomic.  The problem is that Java makes no guarantee that the writes will be visible to other threads, if they aren't properly "synchronized", either with the volatile keyword or by using synchronized blocks.
   
   Here is a relatively short blog post explaining the issue:
   https://www.logicbig.com/tutorials/core-java-tutorial/java-multi-threading/visibility-volatile.html
   
   The best explanation I've found is in my favorite java book, "Java Concurrency In Practice", chapter three, but it is a bit long:
   https://leon-wtf.github.io/doc/java-concurrency-in-practice.pdf#%5B%7B%22num%22%3A63%2C%22gen%22%3A0%7D%2C%7B%22name%22%3A%22XYZ%22%7D%2C-6%2C466%2C1%5D



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