You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/08/02 03:28:30 UTC

[GitHub] [ozone] kaijchen opened a new pull request, #3644: HDDS-7077. EC: Block is not deleted on datanodes

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

   ## What changes were proposed in this pull request?
   
   EC Block is never deleted due to BlockDeletingService#isDeletionAllowed() always returning false.
   
   ```
   2022-08-02 10:53:47,890 [BlockDeletingService#3] WARN  background.BlockDeletingService (BlockDeletingService.java:getTasks(171)) - Unexpected error occurs during deleting blocks.
   java.lang.IllegalArgumentException: Invalid UUID string: 
   	at java.util.UUID.fromString(UUID.java:194)
   	at org.apache.hadoop.ozone.container.keyvalue.statemachine.background.BlockDeletingService.isDeletionAllowed(BlockDeletingService.java:200)
   ```
   
   This patch is a workaround for now, and requires redesign in the future.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-7077
   
   ## How was this patch tested?
   
   Integration test `TestBlockDeletion#testBlockDeletion()`.
   


-- 
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] JacksonYao287 commented on a diff in pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java:
##########
@@ -194,13 +194,17 @@ private boolean isDeletionAllowed(ContainerData containerData,
       return false;
     } else if (!containerData.isClosed()) {
       return false;
+    } else if (containerData.getOriginPipelineId() == null

Review Comment:
   what if a ratis container does not have a originPipelineID exceptionally? is it a good behavior to just return true directlly?
   
   IMHO, the problem here is caused by 
   `if (ozoneContainer.getWriteChannel() instanceof XceiverServerRatis)`.
   we should not differentiate containers by the write channel , we should differentiate them by container type. so the code should be like
   ```
   ContainerType contianerType = containerData.getContainerType();
   switch(contianerType) {
   case Ratis:
   return isDeletionAllowedForRatis()
   default:
   //for non-ratis container, they do not have pipelineid
   return true:
   }
   ```
   what is more, we could also add a sanity check for ratis container pipeline in `isDeletionAllowedForRatis()` to make sure the pipelineid is not empty
   



-- 
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] kaijchen commented on a diff in pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java:
##########
@@ -194,13 +194,17 @@ private boolean isDeletionAllowed(ContainerData containerData,
       return false;
     } else if (!containerData.isClosed()) {
       return false;
+    } else if (containerData.getOriginPipelineId() == null

Review Comment:
   > what if a ratis container does not have a originPipelineID exceptionally? is it a good behavior to just return true directlly?
   
   If a ratis container does not have a originPipelineID, how do we get the `minReplicatedIndex` then?
   
   > we should not differentiate containers by the write channel , we should differentiate them by container type.
   
   I agree with this. But it should be done in another 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] umamaheswararao merged pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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


-- 
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] sodonnel commented on a diff in pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java:
##########
@@ -198,8 +198,15 @@ private boolean isDeletionAllowed(ContainerData containerData,
       if (ozoneContainer.getWriteChannel() instanceof XceiverServerRatis) {
         XceiverServerRatis ratisServer =
             (XceiverServerRatis) ozoneContainer.getWriteChannel();
-        PipelineID pipelineID = PipelineID
-            .valueOf(UUID.fromString(containerData.getOriginPipelineId()));
+        PipelineID pipelineID;

Review Comment:
   After some debugging, the issue is that `containerData.getOriginPipelineId()` is an empty string for EC Containers. I have not been able to track down how the pipelineID gets added to the `writeChunk` call for Ratis requests - I tried to find it in the debugger, but I suspect it happens inside the Xceiver Ratis code. Either way, I did see that the writeChunk command, which creates the container on the first write has a pipelineID set for Ratis writes, but not for EC writes. This kind of makes sense, as EC writes don't really care about the pipeline on a DN - they are all standalone writes.
   
   The code here at line 198, will always end up returning true, even if its not a Ratis write. I ran into this before trying to do something else on the DN where I was trying to detect an EC or Ratis container.
   
   All the logic inside this IF branch is related Ratis stuff - whether the pipeline still exists or not.
   
   Would we be better adding a new IF branch, something like:
   
   ```
   else if (containerData.getOriginPipelineId == emptyString or Null) {
     // This is not a ratis container so allow the delete
     return true;
   } else {
     // existing code as it is
   }
   ```
   
   This would avoid the ugly exception handling, and I've been told catching exceptions like this is Java is relatively slow (ie avoid using exception handling for common cases), so if there are a lot of block deletes for EC, it may cause a performance problem as this exception will always get thrown.



-- 
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] kaijchen commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   > Ok, I think this case may be when first stripe failed in some of block groups?
   > If this case is consistently reproduce able, then enable debug, so that we will know there are stripe failures and rewritten to new nodes. In this case that previously failed stripe blocks left out. Just wanted to make sure this is what happening. If that is the case, it's not hard data itself, we need to figure out a way to delete them.
   
   Exactly, see HDDS-7177 for details.


-- 
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] sodonnel commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   > After setting ozone.scm.ha.ratis.snapshot.threshold to 1, the problem seems to be fixed.
   
   What does this change control? Will this affect Ratis containers negatively if set to 1?


-- 
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] kaijchen commented on a diff in pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java:
##########
@@ -198,8 +198,15 @@ private boolean isDeletionAllowed(ContainerData containerData,
       if (ozoneContainer.getWriteChannel() instanceof XceiverServerRatis) {
         XceiverServerRatis ratisServer =
             (XceiverServerRatis) ozoneContainer.getWriteChannel();
-        PipelineID pipelineID = PipelineID
-            .valueOf(UUID.fromString(containerData.getOriginPipelineId()));
+        PipelineID pipelineID;

Review Comment:
   This change only checks empty or null pipeline id, which is sufficient for the EC case.
   
   However, if there is a malformed container whose pipeline id is some random non-empty string,
   should we be able to delete blocks in it?



-- 
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] umamaheswararao commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   Yeah, the config ozone.scm.ha.ratis.snapshot.threshold should applicable to both RATIS and EC right? Is it the same behavior for RATIS keys?


-- 
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] umamaheswararao commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   Ok, I think this case is when first stripe failed in some of block groups?
   If this case is consistently reproduce able, then enable debug, so that we will know there are stripe failures and rewritten to new nodes. In this case that previously failed stripe blocks left out. Just wanted to make sure this is what happening. If that is the case, it's not hard data itself, we need to figure out a way to delete them. 


-- 
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] kaijchen commented on pull request #3644: HDDS-7077. EC: Block is not deleted on datanodes

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

   Cc @umamaheswararao @JacksonYao287 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


[GitHub] [ozone] kaijchen commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   Verified block deletion is working on a real cluster:
   
   Before deletion:
   
   ![before_deletion](https://user-images.githubusercontent.com/5821159/182536528-9a5ba379-7630-4727-903f-29791bb3fd17.png)
   
   After deletion (1 hour 30 mins later):
   
   ![after_deletion](https://user-images.githubusercontent.com/5821159/182536598-70f09438-e194-41f1-b69f-e720873f7adf.png)


-- 
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] kaijchen commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   > Verified block deletion is working on a real cluster:
   
   This result was on 8bec15ec755ae005104ba8807179bff955e17e35, but I cannot reproduce it later.
   Seems there is still other problems.


-- 
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] kaijchen commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   Reverted 8bec15ec755ae005104ba8807179bff955e17e35 to keep it clear for future work.


-- 
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] kaijchen commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   Followup: just did some large scale test (create-close-delete), found there are chunks of exact 1024 bytes remaining on disk.
   I don't think it is related to the deleting service though. There might be issues in block metadata tracking. @sodonnel @umamaheswararao 
   
   ```console
   $ find /data1/ozone/hdds/hdds/ -name '*.block' | xargs du | awk '{print $1}' | sort -n | uniq -c
        74 1024
   ```
   
   Metrics:
   
   <img width="757" alt="metrics" src="https://user-images.githubusercontent.com/5821159/182784886-7cb0d408-fbfd-4348-a4ec-72063fb74166.png">
   


-- 
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] kaijchen commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   > What does this change control? Will this affect Ratis containers negatively if set to 1?
   
   It is some batch size on scm. Command will be flushed only if the batch is full.
   Maybe @JacksonYao287 can explain it in detail. And there will be a PR on this in the near future.


-- 
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] umamaheswararao commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   The root cause for this issue is:
   For ratis clients, we are writing the chunk through XaciverClientRatis. In that class uses ContainerCommandRequestMessage.
   At server, it is using, groupID as pipelineID.
   ContainerCommandRequestMessage#toProto
        // TODO: setting pipeline id can be avoided if the client is sending it.
       //       In such case, just have to validate the pipeline id.
       final ContainerCommandRequestProto.Builder b = header.toBuilder();
       if (groupId != null) {
         b.setPipelineID(groupId.getUuid().toString());
       }
   
   Here is a todo someone written. If client starts sending it, they don;t need this.
   
   I tested to make sure if client passes, whether it is passing or now.
   
   ```
   diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
   index 5415ecb88..675b7edb4 100644
   --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
   +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
   @@ -34,6 +34,7 @@
    import java.util.concurrent.TimeUnit;
    
    import org.apache.hadoop.hdds.HddsUtils;
   +import org.apache.hadoop.hdds.StringUtils;
    import org.apache.hadoop.hdds.conf.ConfigurationSource;
    import org.apache.hadoop.hdds.function.SupplierWithIOException;
    import org.apache.hadoop.hdds.protocol.DatanodeDetails;
   @@ -60,6 +61,7 @@
    import io.opentracing.Scope;
    import io.opentracing.Span;
    import io.opentracing.util.GlobalTracer;
   +import org.apache.logging.log4j.util.Strings;
    import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
    import org.apache.ratis.thirdparty.io.grpc.Status;
    import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts;
   @@ -462,10 +464,14 @@ public XceiverClientReply sendCommandAsync(
    
        try (Scope ignored = GlobalTracer.get().activateSpan(span)) {
    
   -      ContainerCommandRequestProto finalPayload =
   +      ContainerCommandRequestProto.Builder builder =
              ContainerCommandRequestProto.newBuilder(request)
   -              .setTraceID(TracingUtil.exportCurrentSpan())
   -              .build();
   +              .setTraceID(TracingUtil.exportCurrentSpan());
   +
   +      if (Strings.isEmpty(builder.getPipelineID())) {
   +        builder.setPipelineID(pipeline.getId().getId().toString());
   +      }
   +      ContainerCommandRequestProto finalPayload = builder.build();
          XceiverClientReply asyncReply =
              sendCommandAsync(finalPayload, pipeline.getFirstNode());
          if (shouldBlockAndWaitAsyncReply(request)) {
   
   ```
   
   But I don't recommend to change the code in XceiverClientGrpc.
   Probably we should send this info from much before layers. 
   
   We can also bypass, how the current patch is doing. But I am worried what else we uncovered by not setting pipelineID. Many flows we are depending on existing flows as well. 
   So, I recommend to set the pipelineID from client side for EC keys.
   
   
   
   
   


-- 
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] kaijchen commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   Thanks @sodonnel and @umamaheswararao for the review.
   Yes the root cause is EC containers does not have pipelineID assigned, and the solution is not simple as @umamaheswararao has tried.
   
   The BlockDeletingService was designed for Ratis containers before, but it has to deal with other types of containers now. I think this PR is a temporary solution for this problem, and there should be some redesign or refactor of the block deletion service in the future.


-- 
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] umamaheswararao commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   Here is another place I tried:
   
   ```
   diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
   index e024d79b9..507ba4737 100644
   --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
   +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
   @@ -345,10 +345,10 @@ public static XceiverClientReply writeChunkAsync(
                .setData(data);
        String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
        ContainerCommandRequestProto.Builder builder =
   -        ContainerCommandRequestProto.newBuilder()
   -            .setCmdType(Type.WriteChunk)
   -            .setContainerID(blockID.getContainerID())
   -            .setDatanodeUuid(id)
   +        ContainerCommandRequestProto.newBuilder().setCmdType(Type.WriteChunk)
   +            .setPipelineID(
   +                xceiverClient.getPipeline().getId().getId().toString())
   +            .setContainerID(blockID.getContainerID()).setDatanodeUuid(id)
   ```
   
   But will add for Ratis flows as well.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] kaijchen commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   Thanks @sodonnel @umamaheswararao and @JacksonYao287 for the review.


-- 
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] kaijchen commented on pull request #3644: HDDS-7077. EC: Fix block deletion not allowed due to missing pipelineID

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

   > This result was on [8bec15e](https://github.com/apache/ozone/commit/8bec15ec755ae005104ba8807179bff955e17e35), but I cannot reproduce it later.
   > Seems there is still other problems.
   
   After setting `ozone.scm.ha.ratis.snapshot.threshold` to `1`, the problem seems to be fixed.


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