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 23:20:50 UTC

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

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