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

[GitHub] [ozone] symious opened a new pull request, #5020: HDDS-8974. Introduce detailed lock information

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

   ## What changes were proposed in this pull request?
   
   In Hadoop fair call queue feature, we can use RPC call's processing details to provide cost-based scheduler, this information needs the locks' wait, read and write time.
   
   In Hadoop, these information can be easily get and set via threadLocal variables, but in Ratis enabled cluster, we can't do that as Hadoop. So this ticket is to change the lock interface to return a new Class as OMLockDetails to store this information.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-8974
   
   ## How was this patch tested?
   
   unit test.
   


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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1396614116


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java:
##########
@@ -486,11 +502,12 @@ private void updateReadUnlockMetrics(Resource resource, String resourceName) {
       // Adds a snapshot to the metric readLockHeldTimeMsStat.
       omLockMetrics.setReadLockHeldTimeMsStat(
           TimeUnit.NANOSECONDS.toMillis(readLockHeldTimeNanos));
+      updateProcessingDetails(Timing.LOCKSHARED, readLockHeldTimeNanos);

Review Comment:
   Given currently the OMStateMachine has single thread to execute all write requests, this inaccurate will not happen. Future, if the single thread is optimized to multi threads, then this part need a revisit.



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1396641824


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java:
##########
@@ -296,7 +320,10 @@ private OMResponse submitRequestDirectlyToOM(OMRequest request) {
       ExitUtils.terminate(1, errorMessage, ex, LOG);
       Thread.currentThread().interrupt();
     }
-    return omClientResponse.getOMResponse();
+    OMLockDetails omLockDetails = omClientResponse.getOmLockDetails();

Review Comment:
   I see. Can we extract the "call.getProcessingDetails().add" logic to an individual function, can call it directly here? to avoid generate a new OMResponse to set  omLockDetails.toProtobufBuilder(). 



-- 
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] symious commented on pull request #5020: HDDS-8974. Introduce detailed lock information

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on PR #5020:
URL: https://github.com/apache/ozone/pull/5020#issuecomment-1624831063

   @guohao-rosicky Yes, these PRs should be helpful to improve the performance.
   
   I saw there will be some conflicts between #4885 and this one, I will resolve them when #4885 if merged.


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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1384589364


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java:
##########
@@ -253,11 +256,16 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       addResponseToDoubleBuffer(transactionLogIndex, omClientResponse,
           ozoneManagerDoubleBufferHelper);
       if (acquiredBucketLock) {
-        metadataManager.getLock().releaseWriteLock(BUCKET_LOCK, volumeName,
-            bucketName);
+        getOmLockDetails().merge(

Review Comment:
   Updated, PTAL



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OMLockDetails.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.lock;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMLockDetailsProto;
+
+import static org.apache.hadoop.ozone.om.lock.OMLockDetails.LockType.WAIT;
+
+/**
+ * This class is for recording detailed consumed time on Locks.
+ */
+public class OMLockDetails {
+
+  public static final OMLockDetails EMPTY_DETAILS_LOCK_ACQUIRED =
+      new OMLockDetails(true);
+  public static final OMLockDetails EMPTY_DETAILS_LOCK_NOT_ACQUIRED =
+      new OMLockDetails(false);
+  private boolean lockAcquired;
+  private long waitLockNanos;
+  private long readLockNanos;
+  private long writeLockNanos;
+
+  public OMLockDetails() {
+  }
+
+  public OMLockDetails(boolean lockAcquired) {
+    this.lockAcquired = lockAcquired;
+  }
+
+
+  enum LockType {

Review Comment:
   Updated, PTAL.



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1384352023


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OMLockDetails.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.lock;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMLockDetailsProto;
+
+import static org.apache.hadoop.ozone.om.lock.OMLockDetails.LockType.WAIT;
+
+/**
+ * This class is for recording detailed consumed time on Locks.
+ */
+public class OMLockDetails {
+
+  public static final OMLockDetails EMPTY_DETAILS_LOCK_ACQUIRED =
+      new OMLockDetails(true);
+  public static final OMLockDetails EMPTY_DETAILS_LOCK_NOT_ACQUIRED =
+      new OMLockDetails(false);
+  private boolean lockAcquired;
+  private long waitLockNanos;
+  private long readLockNanos;
+  private long writeLockNanos;
+
+  public OMLockDetails() {
+  }
+
+  public OMLockDetails(boolean lockAcquired) {
+    this.lockAcquired = lockAcquired;
+  }
+
+
+  enum LockType {

Review Comment:
   LockType -> LockActionType



-- 
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] guohao-rosicky commented on pull request #5020: HDDS-8974. Introduce detailed lock information

Posted by "guohao-rosicky (via GitHub)" <gi...@apache.org>.
guohao-rosicky commented on PR #5020:
URL: https://github.com/apache/ozone/pull/5020#issuecomment-1621521643

   Thanks @symious work on this. 
   
   There have been several commits recently that are related to om locks
   
   For example: this pr and https://github.com/apache/ozone/pull/4885
   
   These optimizations help om's performance


-- 
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] guohao-rosicky commented on pull request #5020: HDDS-8974. Introduce detailed lock information

Posted by "guohao-rosicky (via GitHub)" <gi...@apache.org>.
guohao-rosicky commented on PR #5020:
URL: https://github.com/apache/ozone/pull/5020#issuecomment-1628095950

   > @guohao-rosicky Yes, these PRs should be helpful to improve the performance.
   > 
   > I saw there will be some conflicts between #4885 and this one, I will resolve them when #4885 if merged.
   
   sure.


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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1397548770


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OmReadOnlyLock.java:
##########
@@ -47,13 +51,16 @@ public void releaseMultiUserLock(String firstUser, String secondUser) {
   }
 
   @Override
-  public void releaseWriteLock(Resource resource, String... resources) {
+  public OMLockDetails releaseWriteLock(Resource resource,
+      String... resources) {
   // Intentionally empty
+    return EMPTY_DETAILS_LOCK_NOT_ACQUIRED;
   }
 
   @Override
-  public void releaseReadLock(Resource resource, String... resources) {
+  public OMLockDetails releaseReadLock(Resource resource, String... resources) {
   // Intentionally empty
+    return null;

Review Comment:
   Updated.



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1384567956


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java:
##########
@@ -296,7 +320,10 @@ private OMResponse submitRequestDirectlyToOM(OMRequest request) {
       ExitUtils.terminate(1, errorMessage, ex, LOG);
       Thread.currentThread().interrupt();
     }
-    return omClientResponse.getOMResponse();
+    OMLockDetails omLockDetails = omClientResponse.getOmLockDetails();

Review Comment:
   Should be that Hadoop RPC can retrieve the lock time for read operations, so only write requests are handled 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.

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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1396721958


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -402,6 +402,7 @@ message OMResponse {
   optional SetSnapshotPropertyResponse       SetSnapshotPropertyResponse   = 128;
   optional ListStatusLightResponse           listStatusLightResponse       = 129;
   optional SnapshotInfoResponse              SnapshotInfoResponse          = 130;
+  optional OMLockDetailsProto                omLockDetailsProto            = 131;

Review Comment:
   For this one, since we also have a class named "OMLockDetails", the removal of the "Proto" will cause some confusion.



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on PR #5020:
URL: https://github.com/apache/ozone/pull/5020#issuecomment-1815690928

   @symious , thanks for the new patch.  Overall it looks good, just a few minor comments. 


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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1397547886


##########
hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/lock/TestOzoneManagerLock.java:
##########
@@ -67,6 +67,50 @@ public void reacquireResourceLock() {
     }
   }
 
+//  protected OMLockDetails acquireWriteLock(OzoneManagerLock lock,

Review Comment:
   Updated.



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OmReadOnlyLock.java:
##########
@@ -47,13 +51,16 @@ public void releaseMultiUserLock(String firstUser, String secondUser) {
   }
 
   @Override
-  public void releaseWriteLock(Resource resource, String... resources) {
+  public OMLockDetails releaseWriteLock(Resource resource,
+      String... resources) {
   // Intentionally empty
+    return null;

Review Comment:
   Updated.



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1396614938


##########
hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/lock/TestOzoneManagerLock.java:
##########
@@ -67,6 +67,50 @@ public void reacquireResourceLock() {
     }
   }
 
+//  protected OMLockDetails acquireWriteLock(OzoneManagerLock lock,

Review Comment:
   These commented code can be removed? 



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1384347969


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java:
##########
@@ -486,11 +502,12 @@ private void updateReadUnlockMetrics(Resource resource, String resourceName) {
       // Adds a snapshot to the metric readLockHeldTimeMsStat.
       omLockMetrics.setReadLockHeldTimeMsStat(
           TimeUnit.NANOSECONDS.toMillis(readLockHeldTimeNanos));
+      updateProcessingDetails(Timing.LOCKSHARED, readLockHeldTimeNanos);

Review Comment:
   I feel like this updateReadUnlockMetrics and updateWriteUnlockMetrics lock held time is inaccurate, for the hold time is saved in Resource object which is shared between different resoucesNames.  Say the Resource is VOLUME_LOCK, if two clients, each access a different volume concurrently, then this lock hold time will be calculated by (latest lock release time - earliest lock lock time).  
   @tanvipenumudy , could you help to check it if the above is true or not? 



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1384347969


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java:
##########
@@ -486,11 +502,12 @@ private void updateReadUnlockMetrics(Resource resource, String resourceName) {
       // Adds a snapshot to the metric readLockHeldTimeMsStat.
       omLockMetrics.setReadLockHeldTimeMsStat(
           TimeUnit.NANOSECONDS.toMillis(readLockHeldTimeNanos));
+      updateProcessingDetails(Timing.LOCKSHARED, readLockHeldTimeNanos);

Review Comment:
   I feel like this updateReadUnlockMetrics and updateWriteUnlockMetrics lock held time is inaccurate, for the hold time is saved in Resource object which is shared between different resoucesNames.  Say the Resource is VOLUME_LOCK, if two clients, each access a different volume concurrently, then this lock hold time will be calculated by (latest lock release time - earliest lock lock time).  
   
   @tanvipenumudy , please correct me if I'm wrong. 



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1396594252


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OmReadOnlyLock.java:
##########
@@ -47,13 +51,16 @@ public void releaseMultiUserLock(String firstUser, String secondUser) {
   }
 
   @Override
-  public void releaseWriteLock(Resource resource, String... resources) {
+  public OMLockDetails releaseWriteLock(Resource resource,
+      String... resources) {
   // Intentionally empty
+    return null;

Review Comment:
   Can we return EMPTY_DETAILS_LOCK_NOT_ACQUIRED for this and releaseReadLock?



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1396651339


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java:
##########
@@ -1806,4 +1806,28 @@ private void createLinkBucket(OzoneVolume sourceVolume, String sourceBucket,
         .setSourceBucket(sourceBucket);
     sourceVolume.createBucket(linkBucket, builder.build());
   }
+
+  @Test
+  public void testProcessingDetails() throws IOException, InterruptedException {
+    final Logger log = LoggerFactory.getLogger(
+        "org.apache.hadoop.ipc.ProcessingDetails");
+    GenericTestUtils.setLogLevel(log, Level.DEBUG);
+    GenericTestUtils.LogCapturer logCapturer =
+        GenericTestUtils.LogCapturer.captureLogs(log);
+    int keySize = 1024;
+    TestDataUtil.createKey(ozoneBucket, "key1", new String(new byte[keySize],
+        UTF_8));
+    logCapturer.stopCapturing();
+    String logContent = logCapturer.getOutput();
+
+    int nonZeroLines = 0;
+    for (String s: logContent.split("\n")) {
+      if (!s.contains("locksharedTime=0 lockexclusiveTime=0")) {

Review Comment:
   I think we should count and check the line which both lockexclusiveTime and locksharedTime are not 0. 
   
   ```
   2023-11-17 11:18:10,642 [IPC Server handler 0 on default port 15001] DEBUG ipc.ProcessingDetails (Server.java:run(3071)) - Served: [Call#72 Retry#0 org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol.send from localhost:56447 / 127.0.0.1:56447] name=send user=sammi (auth:SIMPLE) details=enqueueTime=10083 queueTime=8042 handlerTime=34624 processingTime=23494375 lockfreeTime=23494375 lockwaitTime=0 locksharedTime=0 lockexclusiveTime=0 responseTime=190084
   2023-11-17 11:18:10,664 [IPC Server handler 2 on default port 15004] DEBUG ipc.ProcessingDetails (Server.java:run(3071)) - Served: [Call#71 Retry#0 org.apache.hadoop.ozone.om.protocol.OzoneManagerPro
   ```
   
   



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1396651339


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java:
##########
@@ -1806,4 +1806,28 @@ private void createLinkBucket(OzoneVolume sourceVolume, String sourceBucket,
         .setSourceBucket(sourceBucket);
     sourceVolume.createBucket(linkBucket, builder.build());
   }
+
+  @Test
+  public void testProcessingDetails() throws IOException, InterruptedException {
+    final Logger log = LoggerFactory.getLogger(
+        "org.apache.hadoop.ipc.ProcessingDetails");
+    GenericTestUtils.setLogLevel(log, Level.DEBUG);
+    GenericTestUtils.LogCapturer logCapturer =
+        GenericTestUtils.LogCapturer.captureLogs(log);
+    int keySize = 1024;
+    TestDataUtil.createKey(ozoneBucket, "key1", new String(new byte[keySize],
+        UTF_8));
+    logCapturer.stopCapturing();
+    String logContent = logCapturer.getOutput();
+
+    int nonZeroLines = 0;
+    for (String s: logContent.split("\n")) {
+      if (!s.contains("locksharedTime=0 lockexclusiveTime=0")) {

Review Comment:
   I think we should count and check the line which both lockexclusiveTime and locksharedTime are not 0. 
   
   ```
   2023-11-17 11:18:10,642 [IPC Server handler 0 on default port 15001] DEBUG ipc.ProcessingDetails (Server.java:run(3071)) - Served: [Call#72 Retry#0 org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol.send from localhost:56447 / 127.0.0.1:56447] name=send user=sammi (auth:SIMPLE) details=enqueueTime=10083 queueTime=8042 handlerTime=34624 processingTime=23494375 lockfreeTime=23494375 lockwaitTime=0 locksharedTime=0 lockexclusiveTime=0 responseTime=190084
   2023-11-17 11:18:10,664 [IPC Server handler 2 on default port 15004] DEBUG ipc.ProcessingDetails (Server.java:run(3071)) - Served: [Call#71 Retry#0 org.apache.hadoop.ozone.om.protocol.OzoneManagerPro
   ```
   
   



-- 
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] adoroszlai commented on pull request #5020: HDDS-8974. Introduce detailed lock information

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #5020:
URL: https://github.com/apache/ozone/pull/5020#issuecomment-1619793901

   @symious please enable [`build-branch` workflow](https://github.com/symious/ozone/actions/workflows/post-commit.yml) in your fork to be able to get CI feedback before opening 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] symious commented on pull request #5020: HDDS-8974. Introduce detailed lock information

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on PR #5020:
URL: https://github.com/apache/ozone/pull/5020#issuecomment-1619833999

   @adoroszlai Noted with 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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1397548299


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -402,6 +402,7 @@ message OMResponse {
   optional SetSnapshotPropertyResponse       SetSnapshotPropertyResponse   = 128;
   optional ListStatusLightResponse           listStatusLightResponse       = 129;
   optional SnapshotInfoResponse              SnapshotInfoResponse          = 130;
+  optional OMLockDetailsProto                omLockDetailsProto            = 131;

Review Comment:
   Updated.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3ExpiredMultipartUploadsAbortRequest.java:
##########
@@ -197,9 +198,11 @@ private void updateTableCache(OzoneManager ozoneManager,
     OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
     OmBucketInfo omBucketInfo = null;
     BucketLayout bucketLayout = null;
+    OMLockDetails omLockDetails = null;
     try {
-      acquiredLock = omMetadataManager.getLock()
+      omLockDetails = omMetadataManager.getLock()
           .acquireWriteLock(BUCKET_LOCK, volumeName, bucketName);
+      acquiredLock = omLockDetails.isLockAcquired();

Review Comment:
   Updated.



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1384348788


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java:
##########
@@ -296,7 +320,10 @@ private OMResponse submitRequestDirectlyToOM(OMRequest request) {
       ExitUtils.terminate(1, errorMessage, ex, LOG);
       Thread.currentThread().interrupt();
     }
-    return omClientResponse.getOMResponse();
+    OMLockDetails omLockDetails = omClientResponse.getOmLockDetails();

Review Comment:
   The handleReadRequest responses are not checked in both submitRequestDirectlyToOM and submitReadRequestToOM. 



-- 
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] symious commented on pull request #5020: HDDS-8974. Introduce detailed lock information

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on PR #5020:
URL: https://github.com/apache/ozone/pull/5020#issuecomment-1624829684

   @adoroszlai @ChenSammi Could you help to review this 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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1396617123


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -402,6 +402,7 @@ message OMResponse {
   optional SetSnapshotPropertyResponse       SetSnapshotPropertyResponse   = 128;
   optional ListStatusLightResponse           listStatusLightResponse       = 129;
   optional SnapshotInfoResponse              SnapshotInfoResponse          = 130;
+  optional OMLockDetailsProto                omLockDetailsProto            = 131;

Review Comment:
   omLockDetailsProto > omLockDetails



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1396835024


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OmReadOnlyLock.java:
##########
@@ -47,13 +51,16 @@ public void releaseMultiUserLock(String firstUser, String secondUser) {
   }
 
   @Override
-  public void releaseWriteLock(Resource resource, String... resources) {
+  public OMLockDetails releaseWriteLock(Resource resource,
+      String... resources) {
   // Intentionally empty
+    return EMPTY_DETAILS_LOCK_NOT_ACQUIRED;
   }
 
   @Override
-  public void releaseReadLock(Resource resource, String... resources) {
+  public OMLockDetails releaseReadLock(Resource resource, String... resources) {
   // Intentionally empty
+    return null;

Review Comment:
   Can we return EMPTY_DETAILS_LOCK_NOT_ACQUIRED here too?



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1384349795


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java:
##########
@@ -253,11 +256,16 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       addResponseToDoubleBuffer(transactionLogIndex, omClientResponse,
           ozoneManagerDoubleBufferHelper);
       if (acquiredBucketLock) {
-        metadataManager.getLock().releaseWriteLock(BUCKET_LOCK, volumeName,
-            bucketName);
+        getOmLockDetails().merge(

Review Comment:
   Suggest to have a updateProcessingDetails() function in OMClientRequest to wrapper the getOmLockDetails().merge() logic 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.

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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1384352023


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OMLockDetails.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.lock;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMLockDetailsProto;
+
+import static org.apache.hadoop.ozone.om.lock.OMLockDetails.LockType.WAIT;
+
+/**
+ * This class is for recording detailed consumed time on Locks.
+ */
+public class OMLockDetails {
+
+  public static final OMLockDetails EMPTY_DETAILS_LOCK_ACQUIRED =
+      new OMLockDetails(true);
+  public static final OMLockDetails EMPTY_DETAILS_LOCK_NOT_ACQUIRED =
+      new OMLockDetails(false);
+  private boolean lockAcquired;
+  private long waitLockNanos;
+  private long readLockNanos;
+  private long writeLockNanos;
+
+  public OMLockDetails() {
+  }
+
+  public OMLockDetails(boolean lockAcquired) {
+    this.lockAcquired = lockAcquired;
+  }
+
+
+  enum LockType {

Review Comment:
   LockType -> LockOpType



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on PR #5020:
URL: https://github.com/apache/ozone/pull/5020#issuecomment-1791938304

   @symious , could you rebase the patch? I will help to review it after that. 


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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1396629572


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3ExpiredMultipartUploadsAbortRequest.java:
##########
@@ -197,9 +198,11 @@ private void updateTableCache(OzoneManager ozoneManager,
     OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
     OmBucketInfo omBucketInfo = null;
     BucketLayout bucketLayout = null;
+    OMLockDetails omLockDetails = null;
     try {
-      acquiredLock = omMetadataManager.getLock()
+      omLockDetails = omMetadataManager.getLock()
           .acquireWriteLock(BUCKET_LOCK, volumeName, bucketName);
+      acquiredLock = omLockDetails.isLockAcquired();

Review Comment:
   This omLockDetails is not set to the final OMClientResponse.



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on code in PR #5020:
URL: https://github.com/apache/ozone/pull/5020#discussion_r1397550441


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java:
##########
@@ -296,7 +320,10 @@ private OMResponse submitRequestDirectlyToOM(OMRequest request) {
       ExitUtils.terminate(1, errorMessage, ex, LOG);
       Thread.currentThread().interrupt();
     }
-    return omClientResponse.getOMResponse();
+    OMLockDetails omLockDetails = omClientResponse.getOmLockDetails();

Review Comment:
   Just realized non-ratis's processing time has already been set in "https://github.com/symious/ozone/blob/HDDS-8974/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java#L616", so removed the logics in `submitRequestDirectlyToOM`. PTAL.



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


Re: [PR] HDDS-8974. Introduce detailed lock information [ozone]

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi merged PR #5020:
URL: https://github.com/apache/ozone/pull/5020


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