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/06/29 05:36:27 UTC

[GitHub] [hadoop-ozone] nandakumar131 opened a new pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

nandakumar131 opened a new pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145


   ## What changes were proposed in this pull request?
   SCM HA uses the new ContainerManagerImpl. This change introduces container related operations in ContainerManagerImpl.
   
   ## What is the link to the Apache JIRA
   HDDS-3895
   
   ## How was this patch tested?
   Unit test will be added as part of HDDS-3896
   


----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] xiaoyuyao commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#discussion_r449309362



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManagerV2.java
##########
@@ -130,6 +141,21 @@ ContainerInfo getContainer(ContainerID containerID)
   void addContainer(ContainerInfoProto containerInfo)
       throws IOException;
 
+  /**
+   *
+   */
+  @Replicate
+  void updateContainerState(HddsProtos.ContainerID id,

Review comment:
       Can we add unit test if any the updateContainerState with @Replicate?




----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] nandakumar131 commented on pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
nandakumar131 commented on pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#issuecomment-675004188


   It is not just about one node pipeline, we have the same issue in three node pipeline as well. If all the three nodes goes stale (because of restart), we will mark the pipeline as closed and will not remove.
   
   The problem is not just there is no instant removal, it is never deleted.


----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#issuecomment-674816824


   @nandakumar131 Ratis One pipeline is not instantly removed in StaleNodeHandler. We will come up with a 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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] xiaoyuyao commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#discussion_r449190798



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
##########
@@ -93,94 +98,45 @@ public ContainerManagerImpl(
   }
 
   @Override
-  public Set<ContainerID> getContainerIDs() {
-    lock.readLock().lock();
-    try {
-      return containerStateManager.getContainerIDs();
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  @Override
-  public Set<ContainerInfo> getContainers() {
-    lock.readLock().lock();
-    try {
-      return containerStateManager.getContainerIDs().stream().map(id -> {
-        try {
-          return containerStateManager.getContainer(id);
-        } catch (ContainerNotFoundException e) {
-          // How can this happen? o_O
-          return null;
-        }
-      }).filter(Objects::nonNull).collect(Collectors.toSet());
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  @Override
-  public ContainerInfo getContainer(final ContainerID containerID)
+  public ContainerInfo getContainer(final ContainerID id)
       throws ContainerNotFoundException {
     lock.readLock().lock();
     try {
-      return containerStateManager.getContainer(containerID);
+      return Optional.ofNullable(containerStateManager
+          .getContainer(id.getProtobuf()))
+          .orElseThrow(() -> new ContainerNotFoundException("ID " + id));
     } finally {
       lock.readLock().unlock();
     }
   }
 
   @Override
-  public Set<ContainerInfo> getContainers(final LifeCycleState state) {
-    lock.readLock().lock();
-    try {
-      return containerStateManager.getContainerIDs(state).stream().map(id -> {
-        try {
-          return containerStateManager.getContainer(id);
-        } catch (ContainerNotFoundException e) {
-          // How can this happen? o_O
-          return null;
-        }
-      }).filter(Objects::nonNull).collect(Collectors.toSet());
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  @Override
-  public boolean exists(final ContainerID containerID) {
+  public List<ContainerInfo> listContainers(final ContainerID startID,
+                                            final int count) {
     lock.readLock().lock();
     try {
-      return (containerStateManager.getContainer(containerID) != null);
-    } catch (ContainerNotFoundException ex) {
-      return false;
+      final long startId = startID == null ? 0 : startID.getId();

Review comment:
       NIT: can we use a different name startId and startID is confusing.




----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] xiaoyuyao commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#discussion_r447268958



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
##########
@@ -279,6 +257,24 @@ public void notifyContainerReportProcessing(final boolean isFullReport,
     throw new UnsupportedOperationException("Not yet implemented!");
   }
 
+  @Override
+  public void deleteContainer(final ContainerID containerID)
+      throws IOException {
+    final HddsProtos.ContainerID id = containerID.getProtobuf();

Review comment:
       Need to hold the writeLock?




----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng merged pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
timmylicheng merged pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145


   


----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] nandakumar131 commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
nandakumar131 commented on a change in pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#discussion_r458878853



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
##########
@@ -60,6 +64,7 @@
   /**
    *
    */
+  //Can we move this lock to ContainerStateManager?

Review comment:
       I would also like to move the lock to ContainerStateManager and not have any lock in ContainerMangerImpl. I'm still thinking of corner cases where this might cause issues.
   State change calls to ContainerStateManager are made via Ratis, so we should not check some state in ContainerMangerImpl and make a Ratis call to change the state without holding any lock.
   
   I will remove the lock in followup jira after making sure that we are not missing any corner cases.




----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] GlenGeng commented on pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

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


   It seems `ContainerManager` is not replace by `ContainerManagerV2` in `StorageContainerManager`.


----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] nandakumar131 commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
nandakumar131 commented on a change in pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#discussion_r458884430



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManagerV2.java
##########
@@ -130,6 +141,21 @@ ContainerInfo getContainer(ContainerID containerID)
   void addContainer(ContainerInfoProto containerInfo)
       throws IOException;
 
+  /**
+   *
+   */
+  @Replicate
+  void updateContainerState(HddsProtos.ContainerID id,

Review comment:
       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.

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



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


[GitHub] [hadoop-ozone] xiaoyuyao commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#discussion_r449292409



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerV2.java
##########
@@ -113,7 +90,7 @@ void deleteContainer(ContainerID containerID)
    */
   void updateContainerState(ContainerID containerID,
                             LifeCycleEvent event)
-      throws ContainerNotFoundException;
+      throws IOException, InvalidStateTransitionException;

Review comment:
       NIT: javadoc update @throws




----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] GlenGeng commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManagerImpl.java
##########
@@ -291,23 +328,14 @@ ContainerInfo getMatchingContainer(final long size, String owner,
     throw new UnsupportedOperationException("Not yet implemented!");
   }
 
-
   NavigableSet<ContainerID> getMatchingContainerIDs(final String owner,
       final ReplicationType type, final ReplicationFactor factor,
       final LifeCycleState state) {
     throw new UnsupportedOperationException("Not yet implemented!");
   }
 
-  void removeContainerReplica(final ContainerID containerID,
-                              final ContainerReplica replica)
-      throws ContainerNotFoundException, ContainerReplicaNotFoundException {
-    throw new UnsupportedOperationException("Not yet implemented!");
-  }
-
-
-  void removeContainer(final ContainerID containerID)
-      throws ContainerNotFoundException {
-    throw new UnsupportedOperationException("Not yet implemented!");
+  public void removeContainer(final HddsProtos.ContainerID id) {
+    containers.removeContainer(ContainerID.getFromProtobuf(id));

Review comment:
       Show we update `containerStore` 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.

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



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


[GitHub] [hadoop-ozone] GlenGeng commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManagerImpl.java
##########
@@ -254,32 +260,63 @@ public void addContainer(final ContainerInfoProto containerInfo)
     final ContainerID containerID = container.containerID();
     final PipelineID pipelineID = container.getPipelineID();
 
-    /*
-     * TODO:
-     *  Check if the container already exist in in ContainerStateManager.
-     *  This optimization can be done after moving ContainerNotFoundException
-     *  from ContainerStateMap to ContainerManagerImpl.
-     */
+    if (!containers.contains(containerID)) {
+      containerStore.put(containerID, container);
+      try {
+        containers.addContainer(container);
+        pipelineManager.addContainerToPipeline(pipelineID, containerID);
+      } catch (Exception ex) {
+        containers.removeContainer(containerID);
+        containerStore.delete(containerID);
+        throw ex;
+      }
+    }
+  }
 
-    containerStore.put(containerID, container);
-    containers.addContainer(container);
-    pipelineManager.addContainerToPipeline(pipelineID, containerID);
-    nextContainerID.incrementAndGet();
+  @Override
+  public boolean contains(final HddsProtos.ContainerID id) {
+    // TODO: Remove the protobuf conversion after fixing ContainerStateMap.
+    return containers.contains(ContainerID.getFromProtobuf(id));
   }
 
-  void updateContainerState(final ContainerID containerID,
-                            final LifeCycleEvent event)
-      throws IOException {
-    throw new UnsupportedOperationException("Not yet implemented!");
+  public void updateContainerState(final HddsProtos.ContainerID containerID,
+                                   final LifeCycleEvent event)
+      throws IOException, InvalidStateTransitionException {
+    // TODO: Remove the protobuf conversion after fixing ContainerStateMap.
+    final ContainerID id = ContainerID.getFromProtobuf(containerID);

Review comment:
       Show we update containerStore as well ?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManagerImpl.java
##########
@@ -254,32 +260,63 @@ public void addContainer(final ContainerInfoProto containerInfo)
     final ContainerID containerID = container.containerID();
     final PipelineID pipelineID = container.getPipelineID();
 
-    /*
-     * TODO:
-     *  Check if the container already exist in in ContainerStateManager.
-     *  This optimization can be done after moving ContainerNotFoundException
-     *  from ContainerStateMap to ContainerManagerImpl.
-     */
+    if (!containers.contains(containerID)) {
+      containerStore.put(containerID, container);
+      try {
+        containers.addContainer(container);
+        pipelineManager.addContainerToPipeline(pipelineID, containerID);
+      } catch (Exception ex) {
+        containers.removeContainer(containerID);
+        containerStore.delete(containerID);
+        throw ex;
+      }
+    }
+  }
 
-    containerStore.put(containerID, container);
-    containers.addContainer(container);
-    pipelineManager.addContainerToPipeline(pipelineID, containerID);
-    nextContainerID.incrementAndGet();
+  @Override
+  public boolean contains(final HddsProtos.ContainerID id) {
+    // TODO: Remove the protobuf conversion after fixing ContainerStateMap.
+    return containers.contains(ContainerID.getFromProtobuf(id));
   }
 
-  void updateContainerState(final ContainerID containerID,
-                            final LifeCycleEvent event)
-      throws IOException {
-    throw new UnsupportedOperationException("Not yet implemented!");
+  public void updateContainerState(final HddsProtos.ContainerID containerID,
+                                   final LifeCycleEvent event)
+      throws IOException, InvalidStateTransitionException {
+    // TODO: Remove the protobuf conversion after fixing ContainerStateMap.
+    final ContainerID id = ContainerID.getFromProtobuf(containerID);

Review comment:
       Show we update `containerStore` 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.

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



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


[GitHub] [hadoop-ozone] nandakumar131 commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
nandakumar131 commented on a change in pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#discussion_r458884274



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerV2.java
##########
@@ -113,7 +90,7 @@ void deleteContainer(ContainerID containerID)
    */
   void updateContainerState(ContainerID containerID,
                             LifeCycleEvent event)
-      throws ContainerNotFoundException;
+      throws IOException, InvalidStateTransitionException;

Review comment:
       I'm planning to add detailed javadoc in follow up jiras. Will fix the @throws 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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] nandakumar131 closed pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
nandakumar131 closed pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145


   


----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] xiaoyuyao commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#discussion_r449304483



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
##########
@@ -60,6 +64,7 @@
   /**
    *
    */
+  //Can we move this lock to ContainerStateManager?

Review comment:
       Same question here. I see we have a RWLock in ContainerMangerImpl and one RWLock in ContainerStateManager. Can we rely only on the lock in ContainerStateManager?




----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] xiaoyuyao commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#discussion_r447269086



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
##########
@@ -279,6 +257,24 @@ public void notifyContainerReportProcessing(final boolean isFullReport,
     throw new UnsupportedOperationException("Not yet implemented!");
   }
 
+  @Override
+  public void deleteContainer(final ContainerID containerID)
+      throws IOException {
+    final HddsProtos.ContainerID id = containerID.getProtobuf();
+    if (!containerStateManager.contains(id)) {

Review comment:
       Can we use checkIfContainerExist()? 




----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] nandakumar131 commented on pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
nandakumar131 commented on pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#issuecomment-674561145


   Integration test failure is not related to this change. Created HDDS-4125 to fix the bug which is causing test failure.


----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] nandakumar131 commented on pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
nandakumar131 commented on pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#issuecomment-677646787


   Thanks @GlenGeng for the review. I will address them as part of HDDS-4133.


----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] GlenGeng edited a comment on pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
GlenGeng edited a comment on pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#issuecomment-677442329


   It seems `ContainerManager` is not replaced by `ContainerManagerV2` in `StorageContainerManager` yet.


----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on a change in pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#discussion_r473780927



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
##########
@@ -60,6 +64,7 @@
   /**
    *
    */
+  //Can we move this lock to ContainerStateManager?

Review comment:
       Agree with @xiaoyuyao 's suggestion.
   @nandakumar131 Can we also track it in a separate Jira? 




----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#issuecomment-678924285


   +1 overall. 
   
   Comments on this PR will be addressed in following Jiras. HDDS-4132 and HDDS-4133.
   
   Thanks @nandakumar131 for contribution. Merging...


----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on pull request #1145: HDDS-3895. Implement container related operations in ContainerManagerImpl

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on pull request #1145:
URL: https://github.com/apache/hadoop-ozone/pull/1145#issuecomment-677483163


   > It seems `ContainerManager` is not replaced by `ContainerManagerV2` in `StorageContainerManager` yet.
   
   @nandakumar131 @GlenGeng Create https://issues.apache.org/jira/browse/HDDS-4132 to track switching to ContainerManagerV2.
   
   @nandakumar131 Could you create other follow-up tasks?


----------------------------------------------------------------
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: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org