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 2020/12/25 09:32:46 UTC

[GitHub] [ozone] GlenGeng opened a new pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

GlenGeng opened a new pull request #1738:
URL: https://github.com/apache/ozone/pull/1738


   ## What changes were proposed in this pull request?
   
   Use `SCMHAManagerImpl` and `SCMRatisServerImpl` instead of `MockSCMHAManager` and `MockRatisServer` in `MiniOzoneCluster`.
   The `MockRatisServer` and `MockSCMHAManager` can not fully test the HA code.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-4622
   
   ## How was this patch tested?
   
   CI
   


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

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



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


[GitHub] [ozone] amaliujia commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/RatisUtil.java
##########
@@ -76,7 +76,8 @@ public static void setRaftStorageDir(final RaftProperties properties,
                                        final ConfigurationSource conf) {
     String storageDir = haConf.getRatisStorageDir();
     if (Strings.isNullOrEmpty(storageDir)) {
-      storageDir = ServerUtils.getDefaultRatisDirectory(conf);
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      storageDir = (new File(metaDirPath, "scm-ha")).getPath();

Review comment:
       I think it's prefered to use SCMHA Configuration to set the storageDir




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

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



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


[GitHub] [ozone] amaliujia commented on pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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


   O seems like we are more or less doing the similar change :) https://github.com/apache/ozone/pull/1733


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

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



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


[GitHub] [ozone] amaliujia commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
##########
@@ -475,7 +475,6 @@ public static void quasiCloseContainer(ContainerManagerV2 containerManager,
   public static StorageContainerManager getScmSimple(OzoneConfiguration conf)
       throws IOException, AuthenticationException {
     SCMConfigurator configurator = new SCMConfigurator();
-    configurator.setSCMHAManager(MockSCMHAManager.getInstance(true));

Review comment:
       You will need this line: 
   `conf.setBoolean(ScmConfigKeys.OZONE_SCM_HA_ENABLE_KEY, true);`
   
   `SCMManagerImpl` will return isLeader immediately without enabling this HA:
   
   ```
       if (!SCMHAUtils.isSCMHAEnabled(conf)) {
         // When SCM HA is not enabled, the current SCM is always the leader.
         return Optional.of((long)0);
       }
   ```




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

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



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


[GitHub] [ozone] GlenGeng commented on pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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


   > O seems like we are more or less doing the similar change :) #1733
   
   We can fix this issue together, and you can take away my changes if needed.


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

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



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


[GitHub] [ozone] amaliujia commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
##########
@@ -475,7 +475,6 @@ public static void quasiCloseContainer(ContainerManagerV2 containerManager,
   public static StorageContainerManager getScmSimple(OzoneConfiguration conf)
       throws IOException, AuthenticationException {
     SCMConfigurator configurator = new SCMConfigurator();
-    configurator.setSCMHAManager(MockSCMHAManager.getInstance(true));

Review comment:
       So currently the `cluster.waitForClusterToBeReady` returns quickly even the SCM has not become leader yet. 
   
   I am guessing CI can still pass become you also have changed the rpc timeout parameters. So either the cluster wait for ready or the parameter change can solve the `not leader problem`. 
   
   I am not sure if we need both. Maybe we only need to keep `cluster.waitForClusterToBeReady`?




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

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



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


[GitHub] [ozone] GlenGeng commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/RatisUtil.java
##########
@@ -76,7 +76,8 @@ public static void setRaftStorageDir(final RaftProperties properties,
                                        final ConfigurationSource conf) {
     String storageDir = haConf.getRatisStorageDir();
     if (Strings.isNullOrEmpty(storageDir)) {
-      storageDir = ServerUtils.getDefaultRatisDirectory(conf);
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      storageDir = (new File(metaDirPath, "scm-ha")).getPath();

Review comment:
       agree




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

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



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


[GitHub] [ozone] amaliujia commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
##########
@@ -475,7 +475,6 @@ public static void quasiCloseContainer(ContainerManagerV2 containerManager,
   public static StorageContainerManager getScmSimple(OzoneConfiguration conf)
       throws IOException, AuthenticationException {
     SCMConfigurator configurator = new SCMConfigurator();
-    configurator.setSCMHAManager(MockSCMHAManager.getInstance(true));

Review comment:
       So currently the `cluster.waitForClusterToBeReady` returns quickly even the SCM has not become leader yet. 
   
   I am guessing CI can still pass become you also have changed the rpc timeout parameters. So either the cluster wait for ready or the parameter change can solve the `not leader problem`. 
   
   I am not if we need both. Maybe we only need to keep `cluster.waitForClusterToBeReady`?




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

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



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


[GitHub] [ozone] amaliujia commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##########
@@ -181,126 +181,73 @@ public Pipeline createPipeline(ReplicationType type, ReplicationFactor factor,
   @Override
   public Pipeline getPipeline(PipelineID pipelineID)
       throws PipelineNotFoundException {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipeline(pipelineID);
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipeline(pipelineID);
   }
 
   @Override
   public boolean containsPipeline(PipelineID pipelineID) {
-    lock.readLock().lock();
     try {
       getPipeline(pipelineID);
       return true;
     } catch (PipelineNotFoundException e) {
       return false;
-    } finally {
-      lock.readLock().unlock();
     }
   }
 
   @Override
   public List<Pipeline> getPipelines() {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipelines();
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipelines();
   }
 
   @Override
   public List<Pipeline> getPipelines(ReplicationType type) {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipelines(type);
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipelines(type);
   }
 
   @Override
   public List<Pipeline> getPipelines(ReplicationType type,
                               ReplicationFactor factor) {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipelines(type, factor);
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipelines(type, factor);
   }
 
   @Override
   public List<Pipeline> getPipelines(ReplicationType type,
                               Pipeline.PipelineState state) {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipelines(type, state);
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipelines(type, state);
   }
 
   @Override
   public List<Pipeline> getPipelines(ReplicationType type,
                               ReplicationFactor factor,
                               Pipeline.PipelineState state) {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipelines(type, factor, state);
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipelines(type, factor, state);
   }
 
   @Override
   public List<Pipeline> getPipelines(
       ReplicationType type, ReplicationFactor factor,
       Pipeline.PipelineState state, Collection<DatanodeDetails> excludeDns,
       Collection<PipelineID> excludePipelines) {
-    lock.readLock().lock();
-    try {
-      return stateManager
+    return stateManager

Review comment:
       so the dead lock solution seems is removing the read write lock? Is there an explanation why we can remove locks here?




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

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



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


[GitHub] [ozone] amaliujia commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/RatisUtil.java
##########
@@ -100,9 +101,9 @@ private static void setRaftRpcProperties(final RaftProperties properties,
     Rpc.setRequestTimeout(properties, TimeDuration.valueOf(
         conf.getRatisRequestTimeout(), TimeUnit.MILLISECONDS));
     Rpc.setTimeoutMin(properties, TimeDuration.valueOf(
-        conf.getRatisRequestMinTimeout(), TimeUnit.MILLISECONDS));

Review comment:
       I think you don't need to update these parameters. The `cluster.waitForClusterToBeReady` is supposed to wait for SCM become leader thus solve the not leader exception.




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

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



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


[GitHub] [ozone] GlenGeng commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
##########
@@ -475,7 +475,6 @@ public static void quasiCloseContainer(ContainerManagerV2 containerManager,
   public static StorageContainerManager getScmSimple(OzoneConfiguration conf)
       throws IOException, AuthenticationException {
     SCMConfigurator configurator = new SCMConfigurator();
-    configurator.setSCMHAManager(MockSCMHAManager.getInstance(true));

Review comment:
       Good catch!




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

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



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


[GitHub] [ozone] amaliujia commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/RatisUtil.java
##########
@@ -76,7 +76,8 @@ public static void setRaftStorageDir(final RaftProperties properties,
                                        final ConfigurationSource conf) {
     String storageDir = haConf.getRatisStorageDir();
     if (Strings.isNullOrEmpty(storageDir)) {
-      storageDir = ServerUtils.getDefaultRatisDirectory(conf);
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      storageDir = (new File(metaDirPath, "scm-ha")).getPath();

Review comment:
       After https://github.com/apache/ozone/pull/1739 this can be set through HA config.




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

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



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


[GitHub] [ozone] GlenGeng commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/RatisUtil.java
##########
@@ -100,9 +101,9 @@ private static void setRaftRpcProperties(final RaftProperties properties,
     Rpc.setRequestTimeout(properties, TimeDuration.valueOf(
         conf.getRatisRequestTimeout(), TimeUnit.MILLISECONDS));
     Rpc.setTimeoutMin(properties, TimeDuration.valueOf(
-        conf.getRatisRequestMinTimeout(), TimeUnit.MILLISECONDS));

Review comment:
       You are right, we only need the wait leader in cluster.waitForClusterToBeReady. The problem here is, we've set the min/max timeout un-properly. I will create a separate jira for his fix.




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

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



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


[GitHub] [ozone] GlenGeng commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##########
@@ -181,126 +181,73 @@ public Pipeline createPipeline(ReplicationType type, ReplicationFactor factor,
   @Override
   public Pipeline getPipeline(PipelineID pipelineID)
       throws PipelineNotFoundException {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipeline(pipelineID);
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipeline(pipelineID);
   }
 
   @Override
   public boolean containsPipeline(PipelineID pipelineID) {
-    lock.readLock().lock();
     try {
       getPipeline(pipelineID);
       return true;
     } catch (PipelineNotFoundException e) {
       return false;
-    } finally {
-      lock.readLock().unlock();
     }
   }
 
   @Override
   public List<Pipeline> getPipelines() {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipelines();
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipelines();
   }
 
   @Override
   public List<Pipeline> getPipelines(ReplicationType type) {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipelines(type);
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipelines(type);
   }
 
   @Override
   public List<Pipeline> getPipelines(ReplicationType type,
                               ReplicationFactor factor) {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipelines(type, factor);
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipelines(type, factor);
   }
 
   @Override
   public List<Pipeline> getPipelines(ReplicationType type,
                               Pipeline.PipelineState state) {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipelines(type, state);
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipelines(type, state);
   }
 
   @Override
   public List<Pipeline> getPipelines(ReplicationType type,
                               ReplicationFactor factor,
                               Pipeline.PipelineState state) {
-    lock.readLock().lock();
-    try {
-      return stateManager.getPipelines(type, factor, state);
-    } finally {
-      lock.readLock().unlock();
-    }
+    return stateManager.getPipelines(type, factor, state);
   }
 
   @Override
   public List<Pipeline> getPipelines(
       ReplicationType type, ReplicationFactor factor,
       Pipeline.PipelineState state, Collection<DatanodeDetails> excludeDns,
       Collection<PipelineID> excludePipelines) {
-    lock.readLock().lock();
-    try {
-      return stateManager
+    return stateManager

Review comment:
       I will create a jira for this issue, can give the explanation of this solution there.




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

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



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


[GitHub] [ozone] GlenGeng closed pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

Posted by GitBox <gi...@apache.org>.
GlenGeng closed pull request #1738:
URL: https://github.com/apache/ozone/pull/1738


   


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

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



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


[GitHub] [ozone] amaliujia commented on a change in pull request #1738: HDDS-4622: Use singe server raft cluster in MiniOzoneCluster.

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



##########
File path: hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
##########
@@ -475,7 +475,6 @@ public static void quasiCloseContainer(ContainerManagerV2 containerManager,
   public static StorageContainerManager getScmSimple(OzoneConfiguration conf)
       throws IOException, AuthenticationException {
     SCMConfigurator configurator = new SCMConfigurator();
-    configurator.setSCMHAManager(MockSCMHAManager.getInstance(true));

Review comment:
       You will need this line: 
   `conf.setBoolean(ScmConfigKeys.OZONE_SCM_HA_ENABLE_KEY, true);`
   
   `SCMHAManagerImpl` will return isLeader immediately without enabling this HA:
   
   ```
       if (!SCMHAUtils.isSCMHAEnabled(conf)) {
         // When SCM HA is not enabled, the current SCM is always the leader.
         return Optional.of((long)0);
       }
   ```




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

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



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