From ozone-commits-return-356-archive-asf-public=cust-asf.ponee.io@hadoop.apache.org Thu Dec 12 10:13:34 2019 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by mx-eu-01.ponee.io (Postfix) with SMTP id 74DC818061A for ; Thu, 12 Dec 2019 11:13:33 +0100 (CET) Received: (qmail 29153 invoked by uid 500); 12 Dec 2019 10:13:32 -0000 Mailing-List: contact ozone-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: ozone-commits@hadoop.apache.org Delivered-To: mailing list ozone-commits@hadoop.apache.org Received: (qmail 29140 invoked by uid 99); 12 Dec 2019 10:13:32 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 12 Dec 2019 10:13:32 +0000 Received: by gitbox.apache.org (ASF Mail Server at gitbox.apache.org, from userid 33) id A4D948D80D; Thu, 12 Dec 2019 10:13:32 +0000 (UTC) Date: Thu, 12 Dec 2019 10:13:32 +0000 To: "ozone-commits@hadoop.apache.org" Subject: [hadoop-ozone] branch master updated: HDDS-2688. Refactor container response builders to hadoop-hdds/common (#324) MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit Message-ID: <157614561257.23924.9693261355091893490@gitbox.apache.org> From: shashikant@apache.org X-Git-Host: gitbox.apache.org X-Git-Repo: hadoop-ozone X-Git-Refname: refs/heads/master X-Git-Reftype: branch X-Git-Oldrev: 10bff97f1406991c8cfa5d9745a0de38cfeea55b X-Git-Newrev: 650678928c0f8032a3935757884e20710eebc775 X-Git-Rev: 650678928c0f8032a3935757884e20710eebc775 X-Git-NotificationType: ref_changed_plus_diff X-Git-Multimail-Version: 1.5.dev Auto-Submitted: auto-generated This is an automated email from the ASF dual-hosted git repository. shashikant pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hadoop-ozone.git The following commit(s) were added to refs/heads/master by this push: new 6506789 HDDS-2688. Refactor container response builders to hadoop-hdds/common (#324) 6506789 is described below commit 650678928c0f8032a3935757884e20710eebc775 Author: Doroszlai, Attila <6454655+adoroszlai@users.noreply.github.com> AuthorDate: Thu Dec 12 11:13:25 2019 +0100 HDDS-2688. Refactor container response builders to hadoop-hdds/common (#324) --- .../ContainerCommandResponseBuilders.java | 269 +++++++++++++++++++++ .../ozone/container/common/helpers/BlockData.java | 5 - .../container/common/helpers/ContainerUtils.java | 78 +----- .../container/common/impl/HddsDispatcher.java | 7 +- .../ozone/container/keyvalue/KeyValueHandler.java | 124 +++++----- .../container/keyvalue/helpers/BlockUtils.java | 88 +------ .../container/keyvalue/helpers/ChunkUtils.java | 16 +- .../keyvalue/helpers/KeyValueContainerUtil.java | 27 --- .../container/keyvalue/helpers/SmallFileUtils.java | 91 ------- .../container/common/impl/TestHddsDispatcher.java | 6 +- 10 files changed, 346 insertions(+), 365 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java new file mode 100644 index 0000000..3d9ded9 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -0,0 +1,269 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdds.scm.protocolPB; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto.Builder; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetBlockResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetCommittedBlockLengthResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetSmallFileResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; + +/** + * A set of helper functions to create responses to container commands. + */ +public final class ContainerCommandResponseBuilders { + + /** + * Returns a Container Command Response Builder with the specified result + * and message. + * @param request requestProto message. + * @param result result of the command. + * @param message response message. + * @return ContainerCommand Response Builder. + */ + public static Builder getContainerCommandResponse( + ContainerCommandRequestProto request, Result result, String message) { + + return ContainerCommandResponseProto.newBuilder() + .setCmdType(request.getCmdType()) + .setTraceID(request.getTraceID()) + .setResult(result) + .setMessage(message); + } + + /** + * Returns a Container Command Response Builder. This call is used to build + * success responses. Calling function can add other fields to the response + * as required. + * @param request requestProto message. + * @return ContainerCommand Response Builder with result as SUCCESS. + */ + public static Builder getSuccessResponseBuilder( + ContainerCommandRequestProto request) { + + return ContainerCommandResponseProto.newBuilder() + .setCmdType(request.getCmdType()) + .setTraceID(request.getTraceID()) + .setResult(Result.SUCCESS); + } + + /** + * Returns a Container Command Response. This call is used for creating null + * success responses. + * @param request requestProto message. + * @return ContainerCommand Response with result as SUCCESS. + */ + public static ContainerCommandResponseProto getSuccessResponse( + ContainerCommandRequestProto request) { + + return getSuccessResponseBuilder(request) + .setMessage("") + .build(); + } + + /** + * We found a command type but no associated payload for the command. Hence + * return malformed Command as response. + * + * @param request - Protobuf message. + * @return ContainerCommandResponseProto - MALFORMED_REQUEST. + */ + public static ContainerCommandResponseProto malformedRequest( + ContainerCommandRequestProto request) { + + return getContainerCommandResponse(request, Result.MALFORMED_REQUEST, + "Cmd type does not match the payload.") + .build(); + } + + /** + * We found a command type that is not supported yet. + * + * @param request - Protobuf message. + * @return ContainerCommandResponseProto - UNSUPPORTED_REQUEST. + */ + public static ContainerCommandResponseProto unsupportedRequest( + ContainerCommandRequestProto request) { + + return getContainerCommandResponse(request, Result.UNSUPPORTED_REQUEST, + "Server does not support this command yet.") + .build(); + } + + /** + * Returns putBlock response success. + * @param msg - Request. + * @return Response. + */ + public static ContainerCommandResponseProto putBlockResponseSuccess( + ContainerCommandRequestProto msg, BlockData blockData) { + + PutBlockResponseProto.Builder putBlock = PutBlockResponseProto.newBuilder() + .setCommittedBlockLength(getCommittedBlockLengthResponseBuilder( + blockData.getSize(), blockData.getBlockID())); + + return getSuccessResponseBuilder(msg) + .setPutBlock(putBlock) + .build(); + } + + /** + * Returns successful blockResponse. + * @param msg - Request. + * @return Response. + */ + public static ContainerCommandResponseProto getBlockResponseSuccess( + ContainerCommandRequestProto msg) { + + return getSuccessResponse(msg); + } + + public static ContainerCommandResponseProto getBlockDataResponse( + ContainerCommandRequestProto msg, BlockData data) { + + GetBlockResponseProto.Builder getBlock = GetBlockResponseProto.newBuilder() + .setBlockData(data); + + return getSuccessResponseBuilder(msg) + .setGetBlock(getBlock) + .build(); + } + + /** + * Returns successful getCommittedBlockLength Response. + * @param msg - Request. + * @return Response. + */ + public static ContainerCommandResponseProto getBlockLengthResponse( + ContainerCommandRequestProto msg, long blockLength) { + + GetCommittedBlockLengthResponseProto.Builder committedBlockLength = + getCommittedBlockLengthResponseBuilder(blockLength, + msg.getGetCommittedBlockLength().getBlockID()); + + return getSuccessResponseBuilder(msg) + .setGetCommittedBlockLength(committedBlockLength) + .build(); + } + + public static GetCommittedBlockLengthResponseProto.Builder + getCommittedBlockLengthResponseBuilder(long blockLength, + DatanodeBlockID blockID) { + + return GetCommittedBlockLengthResponseProto.newBuilder() + .setBlockLength(blockLength) + .setBlockID(blockID); + } + + /** + * Gets a response for the putSmallFile RPC. + * @param msg - ContainerCommandRequestProto + * @return - ContainerCommandResponseProto + */ + public static ContainerCommandResponseProto getPutFileResponseSuccess( + ContainerCommandRequestProto msg, BlockData blockData) { + + PutSmallFileResponseProto.Builder putSmallFile = + PutSmallFileResponseProto.newBuilder() + .setCommittedBlockLength(getCommittedBlockLengthResponseBuilder( + blockData.getSize(), blockData.getBlockID())); + + return getSuccessResponseBuilder(msg) + .setCmdType(Type.PutSmallFile) + .setPutSmallFile(putSmallFile) + .build(); + } + + /** + * Gets a response to the read small file call. + * @param msg - Msg + * @param data - Data + * @param info - Info + * @return Response. + */ + public static ContainerCommandResponseProto getGetSmallFileResponseSuccess( + ContainerCommandRequestProto msg, ByteString data, ChunkInfo info) { + + Preconditions.checkNotNull(msg); + + ReadChunkResponseProto.Builder readChunk = + ReadChunkResponseProto.newBuilder() + .setChunkData(info) + .setData((data)) + .setBlockID(msg.getGetSmallFile().getBlock().getBlockID()); + + GetSmallFileResponseProto.Builder getSmallFile = + GetSmallFileResponseProto.newBuilder().setData(readChunk); + + return getSuccessResponseBuilder(msg) + .setCmdType(Type.GetSmallFile) + .setGetSmallFile(getSmallFile) + .build(); + } + /** + * Returns a ReadContainer Response. + * + * @param request Request + * @param containerData - data + * @return Response. + */ + public static ContainerCommandResponseProto getReadContainerResponse( + ContainerCommandRequestProto request, ContainerDataProto containerData) { + + Preconditions.checkNotNull(containerData); + + ReadContainerResponseProto.Builder response = + ReadContainerResponseProto.newBuilder() + .setContainerData(containerData); + + return getSuccessResponseBuilder(request) + .setReadContainer(response) + .build(); + } + + public static ContainerCommandResponseProto getReadChunkResponse( + ContainerCommandRequestProto request, ByteString data) { + + ReadChunkResponseProto.Builder response = + ReadChunkResponseProto.newBuilder() + .setChunkData(request.getReadChunk().getChunkData()) + .setData(data) + .setBlockID(request.getReadChunk().getBlockID()); + + return getSuccessResponseBuilder(request) + .setReadChunk(response) + .build(); + } + + private ContainerCommandResponseBuilders() { + throw new UnsupportedOperationException("no instances"); + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java index 4cac5a3..87fcf42 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java @@ -115,11 +115,6 @@ public class BlockData { return builder.build(); } - public int getSerializedSize() { - //TODO: the serialized size may be computed without creating the proto. - return getProtoBufMessage().getSerializedSize(); - } - /** * Adds metadata. * diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java index 85e9b9d..4ada1e1 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java @@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.container.common.helpers; import static org.apache.commons.io.FilenameUtils.removeExtension; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_CHECKSUM_ERROR; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_ALGORITHM; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getContainerCommandResponse; import static org.apache.hadoop.ozone.container.common.impl.ContainerData.CHARSET_ENCODING; import java.io.File; @@ -35,7 +36,6 @@ import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.ozone.OzoneConsts; @@ -49,7 +49,7 @@ import org.yaml.snakeyaml.Yaml; import com.google.common.base.Preconditions; /** - * A set of helper functions to create proper responses. + * A mix of helper functions for containers. */ public final class ContainerUtils { @@ -61,78 +61,6 @@ public final class ContainerUtils { } /** - * Returns a Container Command Response Builder with the specified result - * and message. - * @param request requestProto message. - * @param result result of the command. - * @param message response message. - * @return ContainerCommand Response Builder. - */ - public static ContainerCommandResponseProto.Builder - getContainerCommandResponse( - ContainerCommandRequestProto request, Result result, String message) { - return ContainerCommandResponseProto.newBuilder() - .setCmdType(request.getCmdType()) - .setTraceID(request.getTraceID()) - .setResult(result) - .setMessage(message); - } - - /** - * Returns a Container Command Response Builder. This call is used to build - * success responses. Calling function can add other fields to the response - * as required. - * @param request requestProto message. - * @return ContainerCommand Response Builder with result as SUCCESS. - */ - public static ContainerCommandResponseProto.Builder getSuccessResponseBuilder( - ContainerCommandRequestProto request) { - return - ContainerCommandResponseProto.newBuilder() - .setCmdType(request.getCmdType()) - .setTraceID(request.getTraceID()) - .setResult(Result.SUCCESS); - } - - /** - * Returns a Container Command Response. This call is used for creating null - * success responses. - * @param request requestProto message. - * @return ContainerCommand Response with result as SUCCESS. - */ - public static ContainerCommandResponseProto getSuccessResponse( - ContainerCommandRequestProto request) { - ContainerCommandResponseProto.Builder builder = - getContainerCommandResponse(request, Result.SUCCESS, ""); - return builder.build(); - } - - /** - * We found a command type but no associated payload for the command. Hence - * return malformed Command as response. - * - * @param request - Protobuf message. - * @return ContainerCommandResponseProto - MALFORMED_REQUEST. - */ - public static ContainerCommandResponseProto malformedRequest( - ContainerCommandRequestProto request) { - return getContainerCommandResponse(request, Result.MALFORMED_REQUEST, - "Cmd type does not match the payload.").build(); - } - - /** - * We found a command type that is not supported yet. - * - * @param request - Protobuf message. - * @return ContainerCommandResponseProto - UNSUPPORTED_REQUEST. - */ - public static ContainerCommandResponseProto unsupportedRequest( - ContainerCommandRequestProto request) { - return getContainerCommandResponse(request, Result.UNSUPPORTED_REQUEST, - "Server does not support this command yet.").build(); - } - - /** * Logs the error and returns a response to the caller. * * @param log - Logger @@ -264,7 +192,7 @@ public final class ContainerUtils { } /** - * Return the SHA-256 chesksum of the containerData. + * Return the SHA-256 checksum of the containerData. * @param containerDataYamlStr ContainerData as a Yaml String * @return Checksum of the container data * @throws StorageContainerException diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java index 92924b8..3e4a911 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java @@ -74,6 +74,9 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.malformedRequest; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.unsupportedRequest; + /** * Ozone Container dispatcher takes a call from the netty server and routes it * to the right handler function. @@ -273,7 +276,7 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor { if (!msg.hasCreateContainer()) { audit(action, eventType, params, AuditEventStatus.FAILURE, new Exception("MALFORMED_REQUEST")); - return ContainerUtils.malformedRequest(msg); + return malformedRequest(msg); } containerType = msg.getCreateContainer().getContainerType(); } @@ -359,7 +362,7 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor { // log failure audit(action, eventType, params, AuditEventStatus.FAILURE, new Exception("UNSUPPORTED_REQUEST")); - return ContainerUtils.unsupportedRequest(msg); + return unsupportedRequest(msg); } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java index 5da0b8f..5ba9d6d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java @@ -70,10 +70,6 @@ import org.apache.hadoop.ozone.container.common.volume.HddsVolume; import org.apache.hadoop.ozone.container.common.volume .RoundRobinVolumeChoosingPolicy; import org.apache.hadoop.ozone.container.common.volume.VolumeSet; -import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils; -import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; -import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil; -import org.apache.hadoop.ozone.container.keyvalue.helpers.SmallFileUtils; import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerFactory; import org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl; import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; @@ -88,6 +84,18 @@ import static org.apache.hadoop.hdds.HddsConfigKeys .HDDS_DATANODE_VOLUME_CHOOSING_POLICY; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos. Result.*; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getBlockDataResponse; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getBlockLengthResponse; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getBlockResponseSuccess; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getGetSmallFileResponseSuccess; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getPutFileResponseSuccess; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getReadChunkResponse; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getReadContainerResponse; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getSuccessResponse; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.malformedRequest; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.putBlockResponseSuccess; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.unsupportedRequest; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -212,7 +220,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Create Container request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } // Create Container request should be passed a null container as the // container would be created here. @@ -251,7 +259,7 @@ public class KeyValueHandler extends Handler { return ContainerUtils.logAndReturnError(LOG, ex, request); } } - return ContainerUtils.getSuccessResponse(request); + return getSuccessResponse(request); } public void populateContainerPathFields(KeyValueContainer container, @@ -277,7 +285,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Read Container request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } // The container can become unhealthy after the lock is released. @@ -289,8 +297,8 @@ public class KeyValueHandler extends Handler { } KeyValueContainerData containerData = kvContainer.getContainerData(); - return KeyValueContainerUtil.getReadContainerResponse( - request, containerData); + return getReadContainerResponse( + request, containerData.getProtoBufMessage()); } @@ -306,7 +314,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Update Container request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } boolean forceUpdate = request.getUpdateContainer().getForceUpdate(); @@ -324,7 +332,7 @@ public class KeyValueHandler extends Handler { } catch (StorageContainerException ex) { return ContainerUtils.logAndReturnError(LOG, ex, request); } - return ContainerUtils.getSuccessResponse(request); + return getSuccessResponse(request); } /** @@ -342,7 +350,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Delete container request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } boolean forceDelete = request.getDeleteContainer().getForceDelete(); @@ -351,7 +359,7 @@ public class KeyValueHandler extends Handler { } catch (StorageContainerException ex) { return ContainerUtils.logAndReturnError(LOG, ex, request); } - return ContainerUtils.getSuccessResponse(request); + return getSuccessResponse(request); } /** @@ -366,7 +374,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Update Container request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } try { markContainerForClose(kvContainer); @@ -379,7 +387,7 @@ public class KeyValueHandler extends Handler { IO_EXCEPTION), request); } - return ContainerUtils.getSuccessResponse(request); + return getSuccessResponse(request); } /** @@ -394,21 +402,24 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Put Key request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } - BlockData blockData; + final ContainerProtos.BlockData blockDataProto; try { checkContainerOpen(kvContainer); - blockData = BlockData.getFromProtoBuf( + BlockData blockData = BlockData.getFromProtoBuf( request.getPutBlock().getBlockData()); Preconditions.checkNotNull(blockData); long bcsId = dispatcherContext == null ? 0 : dispatcherContext.getLogIndex(); blockData.setBlockCommitSequenceId(bcsId); - final long numBytes = blockData.getSerializedSize(); blockManager.putBlock(kvContainer, blockData); + + blockDataProto = blockData.getProtoBufMessage(); + + final long numBytes = blockDataProto.getSerializedSize(); metrics.incContainerBytesStats(Type.PutBlock, numBytes); } catch (StorageContainerException ex) { return ContainerUtils.logAndReturnError(LOG, ex, request); @@ -418,7 +429,7 @@ public class KeyValueHandler extends Handler { request); } - return BlockUtils.putBlockResponseSuccess(request, blockData); + return putBlockResponseSuccess(request, blockDataProto); } /** @@ -432,7 +443,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Get Key request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } // The container can become unhealthy after the lock is released. @@ -443,11 +454,12 @@ public class KeyValueHandler extends Handler { return ContainerUtils.logAndReturnError(LOG, sce, request); } - BlockData responseData; + ContainerProtos.BlockData responseData; try { BlockID blockID = BlockID.getFromProtobuf( request.getGetBlock().getBlockID()); - responseData = blockManager.getBlock(kvContainer, blockID); + responseData = blockManager.getBlock(kvContainer, blockID) + .getProtoBufMessage(); final long numBytes = responseData.getSerializedSize(); metrics.incContainerBytesStats(Type.GetBlock, numBytes); @@ -459,7 +471,7 @@ public class KeyValueHandler extends Handler { request); } - return BlockUtils.getBlockDataResponse(request, responseData); + return getBlockDataResponse(request, responseData); } /** @@ -473,7 +485,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Get Key request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } // The container can become unhealthy after the lock is released. @@ -497,7 +509,7 @@ public class KeyValueHandler extends Handler { IO_EXCEPTION), request); } - return BlockUtils.getBlockLengthResponse(request, blockLength); + return getBlockLengthResponse(request, blockLength); } /** @@ -511,7 +523,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Delete Key request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } try { @@ -529,7 +541,7 @@ public class KeyValueHandler extends Handler { request); } - return BlockUtils.getBlockResponseSuccess(request); + return getBlockResponseSuccess(request); } /** @@ -544,7 +556,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Read Chunk request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } // The container can become unhealthy after the lock is released. @@ -555,12 +567,11 @@ public class KeyValueHandler extends Handler { return ContainerUtils.logAndReturnError(LOG, sce, request); } - ChunkInfo chunkInfo; ByteBuffer data; try { BlockID blockID = BlockID.getFromProtobuf( request.getReadChunk().getBlockID()); - chunkInfo = ChunkInfo.getFromProtoBuf(request.getReadChunk() + ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(request.getReadChunk() .getChunkData()); Preconditions.checkNotNull(chunkInfo); @@ -581,16 +592,7 @@ public class KeyValueHandler extends Handler { Preconditions.checkNotNull(data, "Chunk data is null"); - ContainerProtos.ReadChunkResponseProto.Builder response = - ContainerProtos.ReadChunkResponseProto.newBuilder(); - response.setChunkData(chunkInfo.getProtoBufMessage()); - response.setData(byteBufferToByteString.apply(data)); - response.setBlockID(request.getReadChunk().getBlockID()); - - ContainerCommandResponseProto.Builder builder = - ContainerUtils.getSuccessResponseBuilder(request); - builder.setReadChunk(response); - return builder.build(); + return getReadChunkResponse(request, byteBufferToByteString.apply(data)); } /** @@ -625,7 +627,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Delete Chunk request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } // The container can become unhealthy after the lock is released. @@ -655,7 +657,7 @@ public class KeyValueHandler extends Handler { IO_EXCEPTION), request); } - return ChunkUtils.getChunkResponseSuccess(request); + return getSuccessResponse(request); } /** @@ -670,7 +672,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Write Chunk request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } try { @@ -710,7 +712,7 @@ public class KeyValueHandler extends Handler { request); } - return ChunkUtils.getChunkResponseSuccess(request); + return getSuccessResponse(request); } /** @@ -727,42 +729,43 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Put Small File request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } - PutSmallFileRequestProto putSmallFileReq = - request.getPutSmallFile(); - BlockData blockData; + PutSmallFileRequestProto putSmallFileReq = request.getPutSmallFile(); + final ContainerProtos.BlockData blockDataProto; try { checkContainerOpen(kvContainer); - BlockID blockID = BlockID.getFromProtobuf(putSmallFileReq.getBlock() - .getBlockData().getBlockID()); - blockData = BlockData.getFromProtoBuf( + BlockData blockData = BlockData.getFromProtoBuf( putSmallFileReq.getBlock().getBlockData()); Preconditions.checkNotNull(blockData); - ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf( - putSmallFileReq.getChunkInfo()); + ContainerProtos.ChunkInfo chunkInfoProto = putSmallFileReq.getChunkInfo(); + ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(chunkInfoProto); Preconditions.checkNotNull(chunkInfo); + ByteBuffer data = putSmallFileReq.getData().asReadOnlyByteBuffer(); if (dispatcherContext == null) { dispatcherContext = new DispatcherContext.Builder().build(); } + BlockID blockID = blockData.getBlockID(); + // chunks will be committed as a part of handling putSmallFile // here. There is no need to maintain this info in openContainerBlockMap. chunkManager .writeChunk(kvContainer, blockID, chunkInfo, data, dispatcherContext); List chunks = new LinkedList<>(); - chunks.add(chunkInfo.getProtoBufMessage()); + chunks.add(chunkInfoProto); blockData.setChunks(chunks); blockData.setBlockCommitSequenceId(dispatcherContext.getLogIndex()); blockManager.putBlock(kvContainer, blockData); - metrics.incContainerBytesStats(Type.PutSmallFile, data.capacity()); + blockDataProto = blockData.getProtoBufMessage(); + metrics.incContainerBytesStats(Type.PutSmallFile, data.capacity()); } catch (StorageContainerException ex) { return ContainerUtils.logAndReturnError(LOG, ex, request); } catch (IOException ex) { @@ -771,7 +774,7 @@ public class KeyValueHandler extends Handler { PUT_SMALL_FILE_ERROR), request); } - return SmallFileUtils.getPutFileResponseSuccess(request, blockData); + return getPutFileResponseSuccess(request, blockDataProto); } /** @@ -787,7 +790,7 @@ public class KeyValueHandler extends Handler { LOG.debug("Malformed Get Small File request. trace ID: {}", request.getTraceID()); } - return ContainerUtils.malformedRequest(request); + return malformedRequest(request); } // The container can become unhealthy after the lock is released. @@ -819,8 +822,7 @@ public class KeyValueHandler extends Handler { chunkInfo = chunk; } metrics.incContainerBytesStats(Type.GetSmallFile, dataBuf.size()); - return SmallFileUtils.getGetSmallFileResponseSuccess(request, dataBuf, - ChunkInfo.getFromProtoBuf(chunkInfo)); + return getGetSmallFileResponseSuccess(request, dataBuf, chunkInfo); } catch (StorageContainerException e) { return ContainerUtils.logAndReturnError(LOG, e, request); } catch (IOException ex) { @@ -836,7 +838,7 @@ public class KeyValueHandler extends Handler { ContainerCommandResponseProto handleUnsupportedOp( ContainerCommandRequestProto request) { // TODO : remove all unsupported operations or handle them. - return ContainerUtils.unsupportedRequest(request); + return unsupportedRequest(request); } /** diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java index da7c857..35e0b0c 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java @@ -21,20 +21,8 @@ package org.apache.hadoop.ozone.container.keyvalue.helpers; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos - .ContainerCommandRequestProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos - .ContainerCommandResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos - .GetBlockResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos. - GetCommittedBlockLengthResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos. - PutBlockResponseProto; -import org.apache.hadoop.hdds.scm.container.common.helpers - .StorageContainerException; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.ozone.container.common.helpers.BlockData; -import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; import org.apache.hadoop.ozone.container.common.utils.ContainerCache; import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB; @@ -103,7 +91,6 @@ public final class BlockUtils { * * @param cache - Cache for DB Handles. */ - @SuppressWarnings("unchecked") public static void shutdownCache(ContainerCache cache) { cache.shutdownCache(); } @@ -119,83 +106,12 @@ public final class BlockUtils { try { ContainerProtos.BlockData blockData = ContainerProtos.BlockData.parseFrom( bytes); - BlockData data = BlockData.getFromProtoBuf(blockData); - return data; + return BlockData.getFromProtoBuf(blockData); } catch (IOException e) { throw new StorageContainerException("Failed to parse block data from " + "the bytes array.", NO_SUCH_BLOCK); } } - /** - * Returns putBlock response success. - * @param msg - Request. - * @return Response. - */ - public static ContainerCommandResponseProto putBlockResponseSuccess( - ContainerCommandRequestProto msg, BlockData blockData) { - ContainerProtos.BlockData blockDataProto = blockData.getProtoBufMessage(); - GetCommittedBlockLengthResponseProto.Builder - committedBlockLengthResponseBuilder = - getCommittedBlockLengthResponseBuilder(blockData.getSize(), - blockDataProto.getBlockID()); - PutBlockResponseProto.Builder putKeyResponse = - PutBlockResponseProto.newBuilder(); - putKeyResponse - .setCommittedBlockLength(committedBlockLengthResponseBuilder); - ContainerProtos.ContainerCommandResponseProto.Builder builder = - ContainerUtils.getSuccessResponseBuilder(msg); - builder.setPutBlock(putKeyResponse); - return builder.build(); - } - /** - * Returns successful blockResponse. - * @param msg - Request. - * @return Response. - */ - public static ContainerCommandResponseProto getBlockResponseSuccess( - ContainerCommandRequestProto msg) { - return ContainerUtils.getSuccessResponse(msg); - } - - public static ContainerCommandResponseProto getBlockDataResponse( - ContainerCommandRequestProto msg, BlockData data) { - GetBlockResponseProto.Builder getBlock = ContainerProtos - .GetBlockResponseProto - .newBuilder(); - getBlock.setBlockData(data.getProtoBufMessage()); - ContainerProtos.ContainerCommandResponseProto.Builder builder = - ContainerUtils.getSuccessResponseBuilder(msg); - builder.setGetBlock(getBlock); - return builder.build(); - } - - /** - * Returns successful getCommittedBlockLength Response. - * @param msg - Request. - * @return Response. - */ - public static ContainerCommandResponseProto getBlockLengthResponse( - ContainerCommandRequestProto msg, long blockLength) { - GetCommittedBlockLengthResponseProto.Builder - committedBlockLengthResponseBuilder = - getCommittedBlockLengthResponseBuilder(blockLength, - msg.getGetCommittedBlockLength().getBlockID()); - ContainerProtos.ContainerCommandResponseProto.Builder builder = - ContainerUtils.getSuccessResponseBuilder(msg); - builder.setGetCommittedBlockLength(committedBlockLengthResponseBuilder); - return builder.build(); - } - - public static GetCommittedBlockLengthResponseProto.Builder - getCommittedBlockLengthResponseBuilder(long blockLength, - ContainerProtos.DatanodeBlockID blockID) { - ContainerProtos.GetCommittedBlockLengthResponseProto.Builder - getCommittedBlockLengthResponseBuilder = ContainerProtos. - GetCommittedBlockLengthResponseProto.newBuilder(); - getCommittedBlockLengthResponseBuilder.setBlockLength(blockLength); - getCommittedBlockLengthResponseBuilder.setBlockID(blockID); - return getCommittedBlockLengthResponseBuilder; - } } \ No newline at end of file diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java index 6dc6823..a298c85 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java @@ -34,13 +34,10 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; -import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils; import org.apache.hadoop.ozone.container.common.volume.VolumeIOStats; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; import org.apache.hadoop.util.Time; @@ -53,6 +50,7 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Res import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.IO_EXCEPTION; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNABLE_TO_FIND_CHUNK; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNABLE_TO_FIND_DATA_DIR; + import org.apache.ratis.util.function.CheckedSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -286,18 +284,6 @@ public final class ChunkUtils { (Boolean.valueOf(overWrite)); } - /** - * Returns a CreateContainer Response. This call is used by create and delete - * containers which have null success responses. - * - * @param msg Request - * @return Response. - */ - public static ContainerCommandResponseProto getChunkResponseSuccess( - ContainerCommandRequestProto msg) { - return ContainerUtils.getSuccessResponse(msg); - } - @VisibleForTesting static T processFileExclusively( Path path, CheckedSupplier op diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java index 3733b06..501680a 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java @@ -26,11 +26,6 @@ import java.util.Map; import com.google.common.primitives.Longs; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos - .ContainerCommandRequestProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos - .ContainerCommandResponseProto; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils; @@ -132,28 +127,6 @@ public final class KeyValueContainerUtil { } /** - * Returns a ReadContainer Response. - * - * @param request Request - * @param containerData - data - * @return Response. - */ - public static ContainerCommandResponseProto getReadContainerResponse( - ContainerCommandRequestProto request, - KeyValueContainerData containerData) { - Preconditions.checkNotNull(containerData); - - ContainerProtos.ReadContainerResponseProto.Builder response = - ContainerProtos.ReadContainerResponseProto.newBuilder(); - response.setContainerData(containerData.getProtoBufMessage()); - - ContainerCommandResponseProto.Builder builder = - ContainerUtils.getSuccessResponseBuilder(request); - builder.setReadContainer(response); - return builder.build(); - } - - /** * Parse KeyValueContainerData and verify checksum. * @param kvContainerData * @param config diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/SmallFileUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/SmallFileUtils.java deleted file mode 100644 index 5aacee9..0000000 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/SmallFileUtils.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.ozone.container.keyvalue.helpers; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.ozone.container.common.helpers.BlockData; -import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; -import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils; -import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos - .ContainerCommandRequestProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos - .ContainerCommandResponseProto; - -/** - * File Utils are helper routines used by putSmallFile and getSmallFile - * RPCs. - */ -public final class SmallFileUtils { - /** - * Never Constructed. - */ - private SmallFileUtils() { - } - - /** - * Gets a response for the putSmallFile RPC. - * @param msg - ContainerCommandRequestProto - * @return - ContainerCommandResponseProto - */ - public static ContainerCommandResponseProto getPutFileResponseSuccess( - ContainerCommandRequestProto msg, BlockData blockData) { - ContainerProtos.PutSmallFileResponseProto.Builder getResponse = - ContainerProtos.PutSmallFileResponseProto.newBuilder(); - ContainerProtos.BlockData blockDataProto = blockData.getProtoBufMessage(); - ContainerProtos.GetCommittedBlockLengthResponseProto.Builder - committedBlockLengthResponseBuilder = BlockUtils - .getCommittedBlockLengthResponseBuilder(blockDataProto.getSize(), - blockDataProto.getBlockID()); - getResponse.setCommittedBlockLength(committedBlockLengthResponseBuilder); - ContainerCommandResponseProto.Builder builder = - ContainerUtils.getSuccessResponseBuilder(msg); - builder.setCmdType(ContainerProtos.Type.PutSmallFile); - builder.setPutSmallFile(getResponse); - return builder.build(); - } - - /** - * Gets a response to the read small file call. - * @param msg - Msg - * @param data - Data - * @param info - Info - * @return Response. - */ - public static ContainerCommandResponseProto getGetSmallFileResponseSuccess( - ContainerCommandRequestProto msg, ByteString data, ChunkInfo info) { - Preconditions.checkNotNull(msg); - - ContainerProtos.ReadChunkResponseProto.Builder readChunkresponse = - ContainerProtos.ReadChunkResponseProto.newBuilder(); - readChunkresponse.setChunkData(info.getProtoBufMessage()); - readChunkresponse.setData((data)); - readChunkresponse.setBlockID(msg.getGetSmallFile().getBlock().getBlockID()); - - ContainerProtos.GetSmallFileResponseProto.Builder getSmallFile = - ContainerProtos.GetSmallFileResponseProto.newBuilder(); - getSmallFile.setData(readChunkresponse.build()); - ContainerCommandResponseProto.Builder builder = - ContainerUtils.getSuccessResponseBuilder(msg); - builder.setCmdType(ContainerProtos.Type.GetSmallFile); - builder.setGetSmallFile(getSmallFile); - return builder.build(); - } - -} diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java index 9db11d0..5ed8b2c 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto; import org.apache.hadoop.ozone.common.Checksum; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; -import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils; import org.apache.hadoop.ozone.container.common.interfaces.Container; import org.apache.hadoop.ozone.container.common.interfaces.Handler; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine; @@ -63,6 +62,7 @@ import java.util.function.Consumer; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getContainerCommandResponse; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -179,8 +179,8 @@ public class TestHddsDispatcher { dd.getUuidString(), 1L, 1L); HddsDispatcher mockDispatcher = Mockito.spy(hddsDispatcher); - ContainerCommandResponseProto.Builder builder = ContainerUtils - .getContainerCommandResponse(writeChunkRequest, + ContainerCommandResponseProto.Builder builder = + getContainerCommandResponse(writeChunkRequest, ContainerProtos.Result.DISK_OUT_OF_SPACE, ""); // Return DISK_OUT_OF_SPACE response when writing chunk // with container creation. --------------------------------------------------------------------- To unsubscribe, e-mail: ozone-commits-unsubscribe@hadoop.apache.org For additional commands, e-mail: ozone-commits-help@hadoop.apache.org