Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,11 @@ public ContainerProtos.BlockData getProtoBufMessage() {
return builder.build();
}

public int getSerializedSize() {
//TODO: the serialized size may be computed without creating the proto.
return getProtoBufMessage().getSerializedSize();
}

/**
* Adds metadata.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -267,8 +267,7 @@ private void scanData(DataTransferThrottler throttler, Canceler canceler)
ByteString expected = cData.getChecksums().get(i);
ByteString actual = cal.computeChecksum(buffer, 0, v)
.getChecksums().get(0);
if (!Arrays.equals(expected.toByteArray(),
actual.toByteArray())) {
if (!expected.equals(actual)) {
throw new OzoneChecksumException(String
.format("Inconsistent read for chunk=%s len=%d expected" +
" checksum %s actual checksum %s for block %s",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -387,7 +387,6 @@ ContainerCommandResponseProto handlePutBlock(
ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {

long blockLength;
if (!request.hasPutBlock()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Put Key request. trace ID: {}",
Expand All @@ -406,7 +405,7 @@ ContainerCommandResponseProto handlePutBlock(
long bcsId =
dispatcherContext == null ? 0 : dispatcherContext.getLogIndex();
blockData.setBlockCommitSequenceId(bcsId);
long numBytes = blockData.getProtoBufMessage().toByteArray().length;
final long numBytes = blockData.getSerializedSize();
blockManager.putBlock(kvContainer, blockData);
metrics.incContainerBytesStats(Type.PutBlock, numBytes);
} catch (StorageContainerException ex) {
Expand Down Expand Up @@ -447,7 +446,7 @@ ContainerCommandResponseProto handleGetBlock(
BlockID blockID = BlockID.getFromProtobuf(
request.getGetBlock().getBlockID());
responseData = blockManager.getBlock(kvContainer, blockID);
long numBytes = responseData.getProtoBufMessage().toByteArray().length;
final long numBytes = responseData.getSerializedSize();
metrics.incContainerBytesStats(Type.GetBlock, numBytes);

} catch (StorageContainerException ex) {
Expand Down Expand Up @@ -818,8 +817,8 @@ ContainerCommandResponseProto handleGetSmallFile(
chunkInfo = chunk;
}
metrics.incContainerBytesStats(Type.GetSmallFile, dataBuf.size());
return SmallFileUtils.getGetSmallFileResponseSuccess(request, dataBuf
.toByteArray(), ChunkInfo.getFromProtoBuf(chunkInfo));
return SmallFileUtils.getGetSmallFileResponseSuccess(request, dataBuf,
ChunkInfo.getFromProtoBuf(chunkInfo));
} catch (StorageContainerException e) {
return ContainerUtils.logAndReturnError(LOG, e, request);
} catch (IOException ex) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,13 +69,13 @@ public static ContainerCommandResponseProto getPutFileResponseSuccess(
* @return Response.
*/
public static ContainerCommandResponseProto getGetSmallFileResponseSuccess(
ContainerCommandRequestProto msg, byte[] data, ChunkInfo info) {
ContainerCommandRequestProto msg, ByteString data, ChunkInfo info) {
Preconditions.checkNotNull(msg);

ContainerProtos.ReadChunkResponseProto.Builder readChunkresponse =
ContainerProtos.ReadChunkResponseProto.newBuilder();
readChunkresponse.setChunkData(info.getProtoBufMessage());
readChunkresponse.setData(ByteString.copyFrom(data));
readChunkresponse.setData((data));
readChunkresponse.setBlockID(msg.getGetSmallFile().getBlock().getBlockID());

ContainerProtos.GetSmallFileResponseProto.Builder getSmallFile =
Expand Down