Selaa lähdekoodia

HDDS-1674. Make ScmBlockLocationProtocol message type based.

Closes #984
S O'Donnell 5 vuotta sitten
vanhempi
commit
b95a58e231

+ 39 - 4
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java

@@ -25,6 +25,9 @@ import java.util.stream.Collectors;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdds.client.ContainerBlockID;
 import org.apache.hadoop.hdds.client.ContainerBlockID;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.SCMBlockLocationRequest;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.SCMBlockLocationResponse;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.Type;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateBlockResponse;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateBlockResponse;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockResponseProto;
@@ -72,6 +75,16 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
     this.rpcProxy = rpcProxy;
     this.rpcProxy = rpcProxy;
   }
   }
 
 
+  /**
+   * Returns a SCMBlockLocationRequest builder with specified type.
+   * @param cmdType type of the request
+   */
+  private SCMBlockLocationRequest.Builder createSCMBlockRequest(Type cmdType) {
+    return SCMBlockLocationRequest.newBuilder()
+        .setCmdType(cmdType)
+        .setTraceID(TracingUtil.exportCurrentSpan());
+  }
+
   /**
   /**
    * Asks SCM where a block should be allocated. SCM responds with the
    * Asks SCM where a block should be allocated. SCM responds with the
    * set of datanodes that should be used creating this block.
    * set of datanodes that should be used creating this block.
@@ -96,12 +109,19 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
             .setType(type)
             .setType(type)
             .setFactor(factor)
             .setFactor(factor)
             .setOwner(owner)
             .setOwner(owner)
-            .setTraceID(TracingUtil.exportCurrentSpan())
             .setExcludeList(excludeList.getProtoBuf())
             .setExcludeList(excludeList.getProtoBuf())
             .build();
             .build();
+
+    SCMBlockLocationRequest wrapper = createSCMBlockRequest(
+        Type.AllocateScmBlock)
+        .setAllocateScmBlockRequest(request)
+        .build();
+
     final AllocateScmBlockResponseProto response;
     final AllocateScmBlockResponseProto response;
+    final SCMBlockLocationResponse wrappedResponse;
     try {
     try {
-      response = rpcProxy.allocateScmBlock(NULL_RPC_CONTROLLER, request);
+      wrappedResponse = rpcProxy.send(NULL_RPC_CONTROLLER, wrapper);
+      response = wrappedResponse.getAllocateScmBlockResponse();
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw transformServiceException(e);
       throw transformServiceException(e);
     }
     }
@@ -141,9 +161,16 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
         .addAllKeyBlocks(keyBlocksProto)
         .addAllKeyBlocks(keyBlocksProto)
         .build();
         .build();
 
 
+    SCMBlockLocationRequest wrapper = createSCMBlockRequest(
+        Type.DeleteScmKeyBlocks)
+        .setDeleteScmKeyBlocksRequest(request)
+        .build();
+
     final DeleteScmKeyBlocksResponseProto resp;
     final DeleteScmKeyBlocksResponseProto resp;
+    final SCMBlockLocationResponse wrappedResponse;
     try {
     try {
-      resp = rpcProxy.deleteScmKeyBlocks(NULL_RPC_CONTROLLER, request);
+      wrappedResponse = rpcProxy.send(NULL_RPC_CONTROLLER, wrapper);
+      resp = wrappedResponse.getDeleteScmKeyBlocksResponse();
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw transformServiceException(e);
       throw transformServiceException(e);
     }
     }
@@ -191,8 +218,16 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
     HddsProtos.GetScmInfoRequestProto request =
     HddsProtos.GetScmInfoRequestProto request =
         HddsProtos.GetScmInfoRequestProto.getDefaultInstance();
         HddsProtos.GetScmInfoRequestProto.getDefaultInstance();
     HddsProtos.GetScmInfoResponseProto resp;
     HddsProtos.GetScmInfoResponseProto resp;
+
+    SCMBlockLocationRequest wrapper = createSCMBlockRequest(
+        Type.GetScmInfo)
+        .setGetScmInfoRequest(request)
+        .build();
+
+    final SCMBlockLocationResponse wrappedResponse;
     try {
     try {
-      resp = rpcProxy.getScmInfo(NULL_RPC_CONTROLLER, request);
+      wrappedResponse = rpcProxy.send(NULL_RPC_CONTROLLER, wrapper);
+      resp = wrappedResponse.getGetScmInfoResponse();
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw transformServiceException(e);
       throw transformServiceException(e);
     }
     }

+ 56 - 9
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java

@@ -22,6 +22,7 @@ import com.google.protobuf.ServiceException;
 import io.opentracing.Scope;
 import io.opentracing.Scope;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
     .AllocateBlockResponse;
     .AllocateBlockResponse;
 import org.apache.hadoop.hdds.scm.ScmInfo;
 import org.apache.hadoop.hdds.scm.ScmInfo;
@@ -42,6 +43,12 @@ import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
     .DeleteScmKeyBlocksRequestProto;
     .DeleteScmKeyBlocksRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
     .DeleteScmKeyBlocksResponseProto;
     .DeleteScmKeyBlocksResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
+    .SCMBlockLocationResponse;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
+    .SCMBlockLocationRequest;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
+    .Status;
 import org.apache.hadoop.hdds.tracing.TracingUtil;
 import org.apache.hadoop.hdds.tracing.TracingUtil;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
 import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
@@ -71,13 +78,51 @@ public final class ScmBlockLocationProtocolServerSideTranslatorPB
     this.impl = impl;
     this.impl = impl;
   }
   }
 
 
+
+  private SCMBlockLocationResponse.Builder createSCMBlockResponse(
+      ScmBlockLocationProtocolProtos.Type cmdType,
+      String traceID) {
+    return SCMBlockLocationResponse.newBuilder()
+        .setCmdType(cmdType)
+        .setTraceID(traceID);
+  }
+
   @Override
   @Override
+  public SCMBlockLocationResponse send(RpcController controller,
+      SCMBlockLocationRequest request) throws ServiceException {
+    String traceId = request.getTraceID();
+
+    SCMBlockLocationResponse.Builder response = createSCMBlockResponse(
+        request.getCmdType(),
+        traceId);
+
+    switch (request.getCmdType()) {
+    case AllocateScmBlock:
+      response.setAllocateScmBlockResponse(
+          allocateScmBlock(traceId, request.getAllocateScmBlockRequest()));
+      break;
+    case DeleteScmKeyBlocks:
+      response.setDeleteScmKeyBlocksResponse(
+          deleteScmKeyBlocks(traceId, request.getDeleteScmKeyBlocksRequest()));
+      break;
+    case GetScmInfo:
+      response.setGetScmInfoResponse(
+          getScmInfo(traceId, request.getGetScmInfoRequest()));
+      break;
+    default:
+      throw new ServiceException("Unknown Operation");
+    }
+    response.setSuccess(true)
+        .setStatus(Status.OK);
+    return response.build();
+  }
+
   public AllocateScmBlockResponseProto allocateScmBlock(
   public AllocateScmBlockResponseProto allocateScmBlock(
-      RpcController controller, AllocateScmBlockRequestProto request)
+      String traceId, AllocateScmBlockRequestProto request)
       throws ServiceException {
       throws ServiceException {
-    try (Scope scope = TracingUtil
+    try(Scope scope = TracingUtil
         .importAndCreateScope("ScmBlockLocationProtocol.allocateBlock",
         .importAndCreateScope("ScmBlockLocationProtocol.allocateBlock",
-            request.getTraceID())) {
+            traceId)) {
       List<AllocatedBlock> allocatedBlocks =
       List<AllocatedBlock> allocatedBlocks =
           impl.allocateBlock(request.getSize(),
           impl.allocateBlock(request.getSize(),
               request.getNumBlocks(), request.getType(),
               request.getNumBlocks(), request.getType(),
@@ -107,13 +152,14 @@ public final class ScmBlockLocationProtocolServerSideTranslatorPB
     }
     }
   }
   }
 
 
-  @Override
   public DeleteScmKeyBlocksResponseProto deleteScmKeyBlocks(
   public DeleteScmKeyBlocksResponseProto deleteScmKeyBlocks(
-      RpcController controller, DeleteScmKeyBlocksRequestProto req)
+      String traceId, DeleteScmKeyBlocksRequestProto req)
       throws ServiceException {
       throws ServiceException {
     DeleteScmKeyBlocksResponseProto.Builder resp =
     DeleteScmKeyBlocksResponseProto.Builder resp =
         DeleteScmKeyBlocksResponseProto.newBuilder();
         DeleteScmKeyBlocksResponseProto.newBuilder();
-    try {
+    try(Scope scope = TracingUtil
+        .importAndCreateScope("ScmBlockLocationProtocol.deleteKeyBlocks",
+            traceId)) {
       List<BlockGroup> infoList = req.getKeyBlocksList().stream()
       List<BlockGroup> infoList = req.getKeyBlocksList().stream()
           .map(BlockGroup::getFromProto).collect(Collectors.toList());
           .map(BlockGroup::getFromProto).collect(Collectors.toList());
       final List<DeleteBlockGroupResult> results =
       final List<DeleteBlockGroupResult> results =
@@ -132,12 +178,13 @@ public final class ScmBlockLocationProtocolServerSideTranslatorPB
     return resp.build();
     return resp.build();
   }
   }
 
 
-  @Override
   public HddsProtos.GetScmInfoResponseProto getScmInfo(
   public HddsProtos.GetScmInfoResponseProto getScmInfo(
-      RpcController controller, HddsProtos.GetScmInfoRequestProto req)
+      String traceId, HddsProtos.GetScmInfoRequestProto req)
       throws ServiceException {
       throws ServiceException {
     ScmInfo scmInfo;
     ScmInfo scmInfo;
-    try {
+    try(Scope scope = TracingUtil
+        .importAndCreateScope("ScmBlockLocationProtocol.getInfo",
+            traceId)) {
       scmInfo = impl.getScmInfo();
       scmInfo = impl.getScmInfo();
     } catch (IOException ex) {
     } catch (IOException ex) {
       throw new ServiceException(ex);
       throw new ServiceException(ex);

+ 56 - 20
hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto

@@ -33,6 +33,60 @@ import "hdds.proto";
 
 
 // SCM Block protocol
 // SCM Block protocol
 
 
+enum Type {
+  AllocateScmBlock   = 11;
+  DeleteScmKeyBlocks = 12;
+  GetScmInfo         = 13;
+}
+
+message SCMBlockLocationRequest {
+  required Type cmdType = 1; // Type of the command
+
+  // A string that identifies this command, we generate  Trace ID in Ozone
+  // frontend and this allows us to trace that command all over ozone.
+  optional string traceID = 2;
+
+  optional UserInfo userInfo = 3;
+
+  optional AllocateScmBlockRequestProto       allocateScmBlockRequest   = 11;
+  optional DeleteScmKeyBlocksRequestProto     deleteScmKeyBlocksRequest = 12;
+  optional hadoop.hdds.GetScmInfoRequestProto getScmInfoRequest         = 13;
+}
+
+message SCMBlockLocationResponse {
+  required Type cmdType = 1; // Type of the command
+
+  // A string that identifies this command, we generate  Trace ID in Ozone
+  // frontend and this allows us to trace that command all over ozone.
+  optional string traceID = 2;
+
+  optional bool success = 3 [default=true];
+
+  optional string message = 4;
+
+  required Status status = 5;
+
+  optional string leaderOMNodeId = 6;
+
+  optional AllocateScmBlockResponseProto       allocateScmBlockResponse   = 11;
+  optional DeleteScmKeyBlocksResponseProto     deleteScmKeyBlocksResponse = 12;
+  optional hadoop.hdds.GetScmInfoResponseProto getScmInfoResponse         = 13;
+}
+
+/**
+    User information which will be extracted during RPC context and used
+    during validating Acl.
+*/
+message UserInfo {
+  optional string userName = 1;
+  optional string remoteAddress = 3;
+}
+
+enum Status {
+  OK = 1;
+  UNKNOWN = 2;
+}
+
 /**
 /**
 * Request send to SCM asking allocate block of specified size.
 * Request send to SCM asking allocate block of specified size.
 */
 */
@@ -42,7 +96,6 @@ message AllocateScmBlockRequestProto {
   required ReplicationType type = 3;
   required ReplicationType type = 3;
   required hadoop.hdds.ReplicationFactor factor = 4;
   required hadoop.hdds.ReplicationFactor factor = 4;
   required string owner = 5;
   required string owner = 5;
-  optional string traceID = 6;
   optional ExcludeListProto excludeList = 7;
   optional ExcludeListProto excludeList = 7;
 }
 }
 
 
@@ -73,8 +126,6 @@ message KeyBlocks {
  */
  */
 message DeleteScmKeyBlocksResponseProto {
 message DeleteScmKeyBlocksResponseProto {
   repeated DeleteKeyBlocksResultProto results = 1;
   repeated DeleteKeyBlocksResultProto results = 1;
-  optional string traceID = 2;
-
 }
 }
 
 
 /**
 /**
@@ -122,21 +173,6 @@ message AllocateScmBlockResponseProto {
  */
  */
 service ScmBlockLocationProtocolService {
 service ScmBlockLocationProtocolService {
 
 
-  /**
-   * Creates a block entry in SCM.
-   */
-  rpc allocateScmBlock(AllocateScmBlockRequestProto)
-      returns (AllocateScmBlockResponseProto);
-
-  /**
-   * Deletes blocks for a set of object keys from SCM.
-   */
-  rpc deleteScmKeyBlocks(DeleteScmKeyBlocksRequestProto)
-      returns (DeleteScmKeyBlocksResponseProto);
-
-  /**
-   * Gets the scmInfo from SCM.
-   */
-  rpc getScmInfo(hadoop.hdds.GetScmInfoRequestProto)
-      returns (hadoop.hdds.GetScmInfoResponseProto);
+  rpc send(SCMBlockLocationRequest)
+      returns (SCMBlockLocationResponse);
 }
 }