diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 8f6b45646c9..324ebf407d5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -761,6 +761,9 @@ Release 0.23.0 - Unreleased HDFS-2512. Add textual error message to data transfer protocol responses (todd) + HDFS-2521. Remove custom checksum headers from data transfer protocol + (todd) + OPTIMIZATIONS HDFS-1458. Improve checkpoint performance by avoiding unnecessary image diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 3c64fae0bbf..174da6841d7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -1033,9 +1033,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable { // send the request new Sender(out).writeBlock(block, accessToken, dfsClient.clientName, nodes, null, recoveryFlag? stage.getRecoveryStage() : stage, - nodes.length, block.getNumBytes(), bytesSent, newGS); - checksum.writeHeader(out); - out.flush(); + nodes.length, block.getNumBytes(), bytesSent, newGS, checksum); // receive ack for connect BlockOpResponseProto resp = BlockOpResponseProto.parseFrom( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java index 0be0bb9fb9f..8ffd56b9137 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java @@ -33,10 +33,13 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.FSInputChecker; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; @@ -408,11 +411,14 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader { BlockOpResponseProto status = BlockOpResponseProto.parseFrom( vintPrefixed(in)); checkSuccess(status, sock, block, file); - DataChecksum checksum = DataChecksum.newDataChecksum( in ); + ReadOpChecksumInfoProto checksumInfo = + status.getReadOpChecksumInfo(); + DataChecksum checksum = DataTransferProtoUtil.fromProto( + checksumInfo.getChecksum()); //Warning when we get CHECKSUM_NULL? // Read the first chunk offset. - long firstChunkOffset = in.readLong(); + long firstChunkOffset = checksumInfo.getChunkOffset(); if ( firstChunkOffset < 0 || firstChunkOffset > startOffset || firstChunkOffset >= (startOffset + checksum.getBytesPerChecksum())) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java index b6d31cbbcb7..598d41e472b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java @@ -23,10 +23,16 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.DataChecksum; + +import com.google.common.collect.BiMap; +import com.google.common.collect.ImmutableBiMap; /** @@ -35,8 +41,20 @@ import org.apache.hadoop.security.token.Token; */ @InterfaceAudience.Private @InterfaceStability.Evolving -abstract class DataTransferProtoUtil { +public abstract class DataTransferProtoUtil { + /** + * Map between the internal DataChecksum identifiers and the protobuf- + * generated identifiers on the wire. + */ + static BiMap checksumTypeMap = + ImmutableBiMap.builder() + .put(DataChecksum.CHECKSUM_CRC32, ChecksumProto.ChecksumType.CRC32) + .put(DataChecksum.CHECKSUM_CRC32C, ChecksumProto.ChecksumType.CRC32C) + .put(DataChecksum.CHECKSUM_NULL, ChecksumProto.ChecksumType.NULL) + .build(); + + static BlockConstructionStage fromProto( OpWriteBlockProto.BlockConstructionStage stage) { return BlockConstructionStage.valueOf(BlockConstructionStage.class, @@ -49,6 +67,28 @@ abstract class DataTransferProtoUtil { stage.name()); } + public static ChecksumProto toProto(DataChecksum checksum) { + ChecksumType type = checksumTypeMap.get(checksum.getChecksumType()); + if (type == null) { + throw new IllegalArgumentException( + "Can't convert checksum to protobuf: " + checksum); + } + + return ChecksumProto.newBuilder() + .setBytesPerChecksum(checksum.getBytesPerChecksum()) + .setType(type) + .build(); + } + + public static DataChecksum fromProto(ChecksumProto proto) { + if (proto == null) return null; + + int bytesPerChecksum = proto.getBytesPerChecksum(); + int type = checksumTypeMap.inverse().get(proto.getType()); + + return DataChecksum.newDataChecksum(type, bytesPerChecksum); + } + static ClientOperationHeaderProto buildClientHeader(ExtendedBlock blk, String client, Token blockToken) { ClientOperationHeaderProto header = diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java index 4faab3e0453..98094472a73 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.DataChecksum; /** * Transfer data to/from datanode using a streaming protocol. @@ -84,7 +85,8 @@ public interface DataTransferProtocol { final int pipelineSize, final long minBytesRcvd, final long maxBytesRcvd, - final long latestGenerationStamp) throws IOException; + final long latestGenerationStamp, + final DataChecksum requestedChecksum) throws IOException; /** * Transfer a block to another datanode. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index dff31e00c64..6ca9d886944 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -103,7 +103,8 @@ public abstract class Receiver implements DataTransferProtocol { fromProto(proto.getStage()), proto.getPipelineSize(), proto.getMinBytesRcvd(), proto.getMaxBytesRcvd(), - proto.getLatestGenerationStamp()); + proto.getLatestGenerationStamp(), + fromProto(proto.getRequestedChecksum())); } /** Receive {@link Op#TRANSFER_BLOCK} */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java index 88288a2a6de..03e13080612 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java @@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; @@ -38,6 +39,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockP import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.DataChecksum; import com.google.protobuf.Message; @@ -93,10 +95,14 @@ public class Sender implements DataTransferProtocol { final int pipelineSize, final long minBytesRcvd, final long maxBytesRcvd, - final long latestGenerationStamp) throws IOException { + final long latestGenerationStamp, + DataChecksum requestedChecksum) throws IOException { ClientOperationHeaderProto header = DataTransferProtoUtil.buildClientHeader( blk, clientName, blockToken); + ChecksumProto checksumProto = + DataTransferProtoUtil.toProto(requestedChecksum); + OpWriteBlockProto.Builder proto = OpWriteBlockProto.newBuilder() .setHeader(header) .addAllTargets(toProtos(targets, 1)) @@ -104,7 +110,8 @@ public class Sender implements DataTransferProtocol { .setPipelineSize(pipelineSize) .setMinBytesRcvd(minBytesRcvd) .setMaxBytesRcvd(maxBytesRcvd) - .setLatestGenerationStamp(latestGenerationStamp); + .setLatestGenerationStamp(latestGenerationStamp) + .setRequestedChecksum(checksumProto); if (source != null) { proto.setSource(toProto(source)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/DataTransferProtos.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/DataTransferProtos.java index 08e15ef88e2..c50b40d5bbc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/DataTransferProtos.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/DataTransferProtos.java @@ -1951,6 +1951,540 @@ public final class DataTransferProtos { // @@protoc_insertion_point(class_scope:OpReadBlockProto) } + public interface ChecksumProtoOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .ChecksumProto.ChecksumType type = 1; + boolean hasType(); + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType getType(); + + // required uint32 bytesPerChecksum = 2; + boolean hasBytesPerChecksum(); + int getBytesPerChecksum(); + } + public static final class ChecksumProto extends + com.google.protobuf.GeneratedMessage + implements ChecksumProtoOrBuilder { + // Use ChecksumProto.newBuilder() to construct. + private ChecksumProto(Builder builder) { + super(builder); + } + private ChecksumProto(boolean noInit) {} + + private static final ChecksumProto defaultInstance; + public static ChecksumProto getDefaultInstance() { + return defaultInstance; + } + + public ChecksumProto getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ChecksumProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ChecksumProto_fieldAccessorTable; + } + + public enum ChecksumType + implements com.google.protobuf.ProtocolMessageEnum { + NULL(0, 0), + CRC32(1, 1), + CRC32C(2, 2), + ; + + public static final int NULL_VALUE = 0; + public static final int CRC32_VALUE = 1; + public static final int CRC32C_VALUE = 2; + + + public final int getNumber() { return value; } + + public static ChecksumType valueOf(int value) { + switch (value) { + case 0: return NULL; + case 1: return CRC32; + case 2: return CRC32C; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public ChecksumType findValueByNumber(int number) { + return ChecksumType.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDescriptor().getEnumTypes().get(0); + } + + private static final ChecksumType[] VALUES = { + NULL, CRC32, CRC32C, + }; + + public static ChecksumType valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private ChecksumType(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:ChecksumProto.ChecksumType) + } + + private int bitField0_; + // required .ChecksumProto.ChecksumType type = 1; + public static final int TYPE_FIELD_NUMBER = 1; + private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType type_; + public boolean hasType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType getType() { + return type_; + } + + // required uint32 bytesPerChecksum = 2; + public static final int BYTESPERCHECKSUM_FIELD_NUMBER = 2; + private int bytesPerChecksum_; + public boolean hasBytesPerChecksum() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getBytesPerChecksum() { + return bytesPerChecksum_; + } + + private void initFields() { + type_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType.NULL; + bytesPerChecksum_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasType()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasBytesPerChecksum()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, type_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt32(2, bytesPerChecksum_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(1, type_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(2, bytesPerChecksum_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto)) { + return super.equals(obj); + } + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto) obj; + + boolean result = true; + result = result && (hasType() == other.hasType()); + if (hasType()) { + result = result && + (getType() == other.getType()); + } + result = result && (hasBytesPerChecksum() == other.hasBytesPerChecksum()); + if (hasBytesPerChecksum()) { + result = result && (getBytesPerChecksum() + == other.getBytesPerChecksum()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasType()) { + hash = (37 * hash) + TYPE_FIELD_NUMBER; + hash = (53 * hash) + hashEnum(getType()); + } + if (hasBytesPerChecksum()) { + hash = (37 * hash) + BYTESPERCHECKSUM_FIELD_NUMBER; + hash = (53 * hash) + getBytesPerChecksum(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ChecksumProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ChecksumProto_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + type_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType.NULL; + bitField0_ = (bitField0_ & ~0x00000001); + bytesPerChecksum_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDescriptor(); + } + + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto getDefaultInstanceForType() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance(); + } + + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto build() { + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto buildPartial() { + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.type_ = type_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.bytesPerChecksum_ = bytesPerChecksum_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto) { + return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto other) { + if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance()) return this; + if (other.hasType()) { + setType(other.getType()); + } + if (other.hasBytesPerChecksum()) { + setBytesPerChecksum(other.getBytesPerChecksum()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasType()) { + + return false; + } + if (!hasBytesPerChecksum()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 8: { + int rawValue = input.readEnum(); + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType value = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(1, rawValue); + } else { + bitField0_ |= 0x00000001; + type_ = value; + } + break; + } + case 16: { + bitField0_ |= 0x00000002; + bytesPerChecksum_ = input.readUInt32(); + break; + } + } + } + } + + private int bitField0_; + + // required .ChecksumProto.ChecksumType type = 1; + private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType type_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType.NULL; + public boolean hasType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType getType() { + return type_; + } + public Builder setType(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + type_ = value; + onChanged(); + return this; + } + public Builder clearType() { + bitField0_ = (bitField0_ & ~0x00000001); + type_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.ChecksumType.NULL; + onChanged(); + return this; + } + + // required uint32 bytesPerChecksum = 2; + private int bytesPerChecksum_ ; + public boolean hasBytesPerChecksum() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getBytesPerChecksum() { + return bytesPerChecksum_; + } + public Builder setBytesPerChecksum(int value) { + bitField0_ |= 0x00000002; + bytesPerChecksum_ = value; + onChanged(); + return this; + } + public Builder clearBytesPerChecksum() { + bitField0_ = (bitField0_ & ~0x00000002); + bytesPerChecksum_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:ChecksumProto) + } + + static { + defaultInstance = new ChecksumProto(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:ChecksumProto) + } + public interface OpWriteBlockProtoOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -1993,6 +2527,11 @@ public final class DataTransferProtos { // required uint64 latestGenerationStamp = 8; boolean hasLatestGenerationStamp(); long getLatestGenerationStamp(); + + // required .ChecksumProto requestedChecksum = 9; + boolean hasRequestedChecksum(); + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto getRequestedChecksum(); + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder getRequestedChecksumOrBuilder(); } public static final class OpWriteBlockProto extends com.google.protobuf.GeneratedMessage @@ -2210,6 +2749,19 @@ public final class DataTransferProtos { return latestGenerationStamp_; } + // required .ChecksumProto requestedChecksum = 9; + public static final int REQUESTEDCHECKSUM_FIELD_NUMBER = 9; + private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto requestedChecksum_; + public boolean hasRequestedChecksum() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto getRequestedChecksum() { + return requestedChecksum_; + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder getRequestedChecksumOrBuilder() { + return requestedChecksum_; + } + private void initFields() { header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance(); targets_ = java.util.Collections.emptyList(); @@ -2219,6 +2771,7 @@ public final class DataTransferProtos { minBytesRcvd_ = 0L; maxBytesRcvd_ = 0L; latestGenerationStamp_ = 0L; + requestedChecksum_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -2249,6 +2802,10 @@ public final class DataTransferProtos { memoizedIsInitialized = 0; return false; } + if (!hasRequestedChecksum()) { + memoizedIsInitialized = 0; + return false; + } if (!getHeader().isInitialized()) { memoizedIsInitialized = 0; return false; @@ -2265,6 +2822,10 @@ public final class DataTransferProtos { return false; } } + if (!getRequestedChecksum().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } memoizedIsInitialized = 1; return true; } @@ -2296,6 +2857,9 @@ public final class DataTransferProtos { if (((bitField0_ & 0x00000040) == 0x00000040)) { output.writeUInt64(8, latestGenerationStamp_); } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeMessage(9, requestedChecksum_); + } getUnknownFields().writeTo(output); } @@ -2337,6 +2901,10 @@ public final class DataTransferProtos { size += com.google.protobuf.CodedOutputStream .computeUInt64Size(8, latestGenerationStamp_); } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(9, requestedChecksum_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -2397,6 +2965,11 @@ public final class DataTransferProtos { result = result && (getLatestGenerationStamp() == other.getLatestGenerationStamp()); } + result = result && (hasRequestedChecksum() == other.hasRequestedChecksum()); + if (hasRequestedChecksum()) { + result = result && getRequestedChecksum() + .equals(other.getRequestedChecksum()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -2438,6 +3011,10 @@ public final class DataTransferProtos { hash = (37 * hash) + LATESTGENERATIONSTAMP_FIELD_NUMBER; hash = (53 * hash) + hashLong(getLatestGenerationStamp()); } + if (hasRequestedChecksum()) { + hash = (37 * hash) + REQUESTEDCHECKSUM_FIELD_NUMBER; + hash = (53 * hash) + getRequestedChecksum().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; } @@ -2549,6 +3126,7 @@ public final class DataTransferProtos { getHeaderFieldBuilder(); getTargetsFieldBuilder(); getSourceFieldBuilder(); + getRequestedChecksumFieldBuilder(); } } private static Builder create() { @@ -2585,6 +3163,12 @@ public final class DataTransferProtos { bitField0_ = (bitField0_ & ~0x00000040); latestGenerationStamp_ = 0L; bitField0_ = (bitField0_ & ~0x00000080); + if (requestedChecksumBuilder_ == null) { + requestedChecksum_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance(); + } else { + requestedChecksumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000100); return this; } @@ -2668,6 +3252,14 @@ public final class DataTransferProtos { to_bitField0_ |= 0x00000040; } result.latestGenerationStamp_ = latestGenerationStamp_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000080; + } + if (requestedChecksumBuilder_ == null) { + result.requestedChecksum_ = requestedChecksum_; + } else { + result.requestedChecksum_ = requestedChecksumBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -2731,6 +3323,9 @@ public final class DataTransferProtos { if (other.hasLatestGenerationStamp()) { setLatestGenerationStamp(other.getLatestGenerationStamp()); } + if (other.hasRequestedChecksum()) { + mergeRequestedChecksum(other.getRequestedChecksum()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -2760,6 +3355,10 @@ public final class DataTransferProtos { return false; } + if (!hasRequestedChecksum()) { + + return false; + } if (!getHeader().isInitialized()) { return false; @@ -2776,6 +3375,10 @@ public final class DataTransferProtos { return false; } } + if (!getRequestedChecksum().isInitialized()) { + + return false; + } return true; } @@ -2857,6 +3460,15 @@ public final class DataTransferProtos { latestGenerationStamp_ = input.readUInt64(); break; } + case 74: { + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.newBuilder(); + if (hasRequestedChecksum()) { + subBuilder.mergeFrom(getRequestedChecksum()); + } + input.readMessage(subBuilder, extensionRegistry); + setRequestedChecksum(subBuilder.buildPartial()); + break; + } } } } @@ -3337,6 +3949,96 @@ public final class DataTransferProtos { return this; } + // required .ChecksumProto requestedChecksum = 9; + private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto requestedChecksum_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder> requestedChecksumBuilder_; + public boolean hasRequestedChecksum() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto getRequestedChecksum() { + if (requestedChecksumBuilder_ == null) { + return requestedChecksum_; + } else { + return requestedChecksumBuilder_.getMessage(); + } + } + public Builder setRequestedChecksum(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto value) { + if (requestedChecksumBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + requestedChecksum_ = value; + onChanged(); + } else { + requestedChecksumBuilder_.setMessage(value); + } + bitField0_ |= 0x00000100; + return this; + } + public Builder setRequestedChecksum( + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder builderForValue) { + if (requestedChecksumBuilder_ == null) { + requestedChecksum_ = builderForValue.build(); + onChanged(); + } else { + requestedChecksumBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000100; + return this; + } + public Builder mergeRequestedChecksum(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto value) { + if (requestedChecksumBuilder_ == null) { + if (((bitField0_ & 0x00000100) == 0x00000100) && + requestedChecksum_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance()) { + requestedChecksum_ = + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.newBuilder(requestedChecksum_).mergeFrom(value).buildPartial(); + } else { + requestedChecksum_ = value; + } + onChanged(); + } else { + requestedChecksumBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000100; + return this; + } + public Builder clearRequestedChecksum() { + if (requestedChecksumBuilder_ == null) { + requestedChecksum_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance(); + onChanged(); + } else { + requestedChecksumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000100); + return this; + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder getRequestedChecksumBuilder() { + bitField0_ |= 0x00000100; + onChanged(); + return getRequestedChecksumFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder getRequestedChecksumOrBuilder() { + if (requestedChecksumBuilder_ != null) { + return requestedChecksumBuilder_.getMessageOrBuilder(); + } else { + return requestedChecksum_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder> + getRequestedChecksumFieldBuilder() { + if (requestedChecksumBuilder_ == null) { + requestedChecksumBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder>( + requestedChecksum_, + getParentForChildren(), + isClean()); + requestedChecksum_ = null; + } + return requestedChecksumBuilder_; + } + // @@protoc_insertion_point(builder_scope:OpWriteBlockProto) } @@ -6920,6 +7622,553 @@ public final class DataTransferProtos { // @@protoc_insertion_point(class_scope:PipelineAckProto) } + public interface ReadOpChecksumInfoProtoOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .ChecksumProto checksum = 1; + boolean hasChecksum(); + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto getChecksum(); + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder getChecksumOrBuilder(); + + // required uint64 chunkOffset = 2; + boolean hasChunkOffset(); + long getChunkOffset(); + } + public static final class ReadOpChecksumInfoProto extends + com.google.protobuf.GeneratedMessage + implements ReadOpChecksumInfoProtoOrBuilder { + // Use ReadOpChecksumInfoProto.newBuilder() to construct. + private ReadOpChecksumInfoProto(Builder builder) { + super(builder); + } + private ReadOpChecksumInfoProto(boolean noInit) {} + + private static final ReadOpChecksumInfoProto defaultInstance; + public static ReadOpChecksumInfoProto getDefaultInstance() { + return defaultInstance; + } + + public ReadOpChecksumInfoProto getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ReadOpChecksumInfoProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ReadOpChecksumInfoProto_fieldAccessorTable; + } + + private int bitField0_; + // required .ChecksumProto checksum = 1; + public static final int CHECKSUM_FIELD_NUMBER = 1; + private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto checksum_; + public boolean hasChecksum() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto getChecksum() { + return checksum_; + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder getChecksumOrBuilder() { + return checksum_; + } + + // required uint64 chunkOffset = 2; + public static final int CHUNKOFFSET_FIELD_NUMBER = 2; + private long chunkOffset_; + public boolean hasChunkOffset() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getChunkOffset() { + return chunkOffset_; + } + + private void initFields() { + checksum_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance(); + chunkOffset_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasChecksum()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasChunkOffset()) { + memoizedIsInitialized = 0; + return false; + } + if (!getChecksum().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, checksum_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, chunkOffset_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, checksum_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(2, chunkOffset_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto)) { + return super.equals(obj); + } + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto) obj; + + boolean result = true; + result = result && (hasChecksum() == other.hasChecksum()); + if (hasChecksum()) { + result = result && getChecksum() + .equals(other.getChecksum()); + } + result = result && (hasChunkOffset() == other.hasChunkOffset()); + if (hasChunkOffset()) { + result = result && (getChunkOffset() + == other.getChunkOffset()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasChecksum()) { + hash = (37 * hash) + CHECKSUM_FIELD_NUMBER; + hash = (53 * hash) + getChecksum().hashCode(); + } + if (hasChunkOffset()) { + hash = (37 * hash) + CHUNKOFFSET_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getChunkOffset()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProtoOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ReadOpChecksumInfoProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ReadOpChecksumInfoProto_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getChecksumFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (checksumBuilder_ == null) { + checksum_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance(); + } else { + checksumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + chunkOffset_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.getDescriptor(); + } + + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto getDefaultInstanceForType() { + return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.getDefaultInstance(); + } + + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto build() { + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto buildPartial() { + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (checksumBuilder_ == null) { + result.checksum_ = checksum_; + } else { + result.checksum_ = checksumBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.chunkOffset_ = chunkOffset_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto) { + return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto other) { + if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.getDefaultInstance()) return this; + if (other.hasChecksum()) { + mergeChecksum(other.getChecksum()); + } + if (other.hasChunkOffset()) { + setChunkOffset(other.getChunkOffset()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasChecksum()) { + + return false; + } + if (!hasChunkOffset()) { + + return false; + } + if (!getChecksum().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.newBuilder(); + if (hasChecksum()) { + subBuilder.mergeFrom(getChecksum()); + } + input.readMessage(subBuilder, extensionRegistry); + setChecksum(subBuilder.buildPartial()); + break; + } + case 16: { + bitField0_ |= 0x00000002; + chunkOffset_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required .ChecksumProto checksum = 1; + private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto checksum_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder> checksumBuilder_; + public boolean hasChecksum() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto getChecksum() { + if (checksumBuilder_ == null) { + return checksum_; + } else { + return checksumBuilder_.getMessage(); + } + } + public Builder setChecksum(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto value) { + if (checksumBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + checksum_ = value; + onChanged(); + } else { + checksumBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder setChecksum( + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder builderForValue) { + if (checksumBuilder_ == null) { + checksum_ = builderForValue.build(); + onChanged(); + } else { + checksumBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder mergeChecksum(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto value) { + if (checksumBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + checksum_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance()) { + checksum_ = + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.newBuilder(checksum_).mergeFrom(value).buildPartial(); + } else { + checksum_ = value; + } + onChanged(); + } else { + checksumBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder clearChecksum() { + if (checksumBuilder_ == null) { + checksum_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.getDefaultInstance(); + onChanged(); + } else { + checksumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder getChecksumBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getChecksumFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder getChecksumOrBuilder() { + if (checksumBuilder_ != null) { + return checksumBuilder_.getMessageOrBuilder(); + } else { + return checksum_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder> + getChecksumFieldBuilder() { + if (checksumBuilder_ == null) { + checksumBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProtoOrBuilder>( + checksum_, + getParentForChildren(), + isClean()); + checksum_ = null; + } + return checksumBuilder_; + } + + // required uint64 chunkOffset = 2; + private long chunkOffset_ ; + public boolean hasChunkOffset() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getChunkOffset() { + return chunkOffset_; + } + public Builder setChunkOffset(long value) { + bitField0_ |= 0x00000002; + chunkOffset_ = value; + onChanged(); + return this; + } + public Builder clearChunkOffset() { + bitField0_ = (bitField0_ & ~0x00000002); + chunkOffset_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:ReadOpChecksumInfoProto) + } + + static { + defaultInstance = new ReadOpChecksumInfoProto(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:ReadOpChecksumInfoProto) + } + public interface BlockOpResponseProtoOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -6936,7 +8185,12 @@ public final class DataTransferProtos { org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto getChecksumResponse(); org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProtoOrBuilder getChecksumResponseOrBuilder(); - // optional string message = 4; + // optional .ReadOpChecksumInfoProto readOpChecksumInfo = 4; + boolean hasReadOpChecksumInfo(); + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto getReadOpChecksumInfo(); + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProtoOrBuilder getReadOpChecksumInfoOrBuilder(); + + // optional string message = 5; boolean hasMessage(); String getMessage(); } @@ -7024,11 +8278,24 @@ public final class DataTransferProtos { return checksumResponse_; } - // optional string message = 4; - public static final int MESSAGE_FIELD_NUMBER = 4; + // optional .ReadOpChecksumInfoProto readOpChecksumInfo = 4; + public static final int READOPCHECKSUMINFO_FIELD_NUMBER = 4; + private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto readOpChecksumInfo_; + public boolean hasReadOpChecksumInfo() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto getReadOpChecksumInfo() { + return readOpChecksumInfo_; + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProtoOrBuilder getReadOpChecksumInfoOrBuilder() { + return readOpChecksumInfo_; + } + + // optional string message = 5; + public static final int MESSAGE_FIELD_NUMBER = 5; private java.lang.Object message_; public boolean hasMessage() { - return ((bitField0_ & 0x00000008) == 0x00000008); + return ((bitField0_ & 0x00000010) == 0x00000010); } public String getMessage() { java.lang.Object ref = message_; @@ -7060,6 +8327,7 @@ public final class DataTransferProtos { status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS; firstBadLink_ = ""; checksumResponse_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.getDefaultInstance(); + readOpChecksumInfo_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.getDefaultInstance(); message_ = ""; } private byte memoizedIsInitialized = -1; @@ -7077,6 +8345,12 @@ public final class DataTransferProtos { return false; } } + if (hasReadOpChecksumInfo()) { + if (!getReadOpChecksumInfo().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } memoizedIsInitialized = 1; return true; } @@ -7094,7 +8368,10 @@ public final class DataTransferProtos { output.writeMessage(3, checksumResponse_); } if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBytes(4, getMessageBytes()); + output.writeMessage(4, readOpChecksumInfo_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getMessageBytes()); } getUnknownFields().writeTo(output); } @@ -7119,7 +8396,11 @@ public final class DataTransferProtos { } if (((bitField0_ & 0x00000008) == 0x00000008)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(4, getMessageBytes()); + .computeMessageSize(4, readOpChecksumInfo_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(5, getMessageBytes()); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -7159,6 +8440,11 @@ public final class DataTransferProtos { result = result && getChecksumResponse() .equals(other.getChecksumResponse()); } + result = result && (hasReadOpChecksumInfo() == other.hasReadOpChecksumInfo()); + if (hasReadOpChecksumInfo()) { + result = result && getReadOpChecksumInfo() + .equals(other.getReadOpChecksumInfo()); + } result = result && (hasMessage() == other.hasMessage()); if (hasMessage()) { result = result && getMessage() @@ -7185,6 +8471,10 @@ public final class DataTransferProtos { hash = (37 * hash) + CHECKSUMRESPONSE_FIELD_NUMBER; hash = (53 * hash) + getChecksumResponse().hashCode(); } + if (hasReadOpChecksumInfo()) { + hash = (37 * hash) + READOPCHECKSUMINFO_FIELD_NUMBER; + hash = (53 * hash) + getReadOpChecksumInfo().hashCode(); + } if (hasMessage()) { hash = (37 * hash) + MESSAGE_FIELD_NUMBER; hash = (53 * hash) + getMessage().hashCode(); @@ -7298,6 +8588,7 @@ public final class DataTransferProtos { private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { getChecksumResponseFieldBuilder(); + getReadOpChecksumInfoFieldBuilder(); } } private static Builder create() { @@ -7316,8 +8607,14 @@ public final class DataTransferProtos { checksumResponseBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000004); - message_ = ""; + if (readOpChecksumInfoBuilder_ == null) { + readOpChecksumInfo_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.getDefaultInstance(); + } else { + readOpChecksumInfoBuilder_.clear(); + } bitField0_ = (bitField0_ & ~0x00000008); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); return this; } @@ -7375,6 +8672,14 @@ public final class DataTransferProtos { if (((from_bitField0_ & 0x00000008) == 0x00000008)) { to_bitField0_ |= 0x00000008; } + if (readOpChecksumInfoBuilder_ == null) { + result.readOpChecksumInfo_ = readOpChecksumInfo_; + } else { + result.readOpChecksumInfo_ = readOpChecksumInfoBuilder_.build(); + } + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } result.message_ = message_; result.bitField0_ = to_bitField0_; onBuilt(); @@ -7401,6 +8706,9 @@ public final class DataTransferProtos { if (other.hasChecksumResponse()) { mergeChecksumResponse(other.getChecksumResponse()); } + if (other.hasReadOpChecksumInfo()) { + mergeReadOpChecksumInfo(other.getReadOpChecksumInfo()); + } if (other.hasMessage()) { setMessage(other.getMessage()); } @@ -7419,6 +8727,12 @@ public final class DataTransferProtos { return false; } } + if (hasReadOpChecksumInfo()) { + if (!getReadOpChecksumInfo().isInitialized()) { + + return false; + } + } return true; } @@ -7471,7 +8785,16 @@ public final class DataTransferProtos { break; } case 34: { - bitField0_ |= 0x00000008; + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.newBuilder(); + if (hasReadOpChecksumInfo()) { + subBuilder.mergeFrom(getReadOpChecksumInfo()); + } + input.readMessage(subBuilder, extensionRegistry); + setReadOpChecksumInfo(subBuilder.buildPartial()); + break; + } + case 42: { + bitField0_ |= 0x00000010; message_ = input.readBytes(); break; } @@ -7631,10 +8954,100 @@ public final class DataTransferProtos { return checksumResponseBuilder_; } - // optional string message = 4; + // optional .ReadOpChecksumInfoProto readOpChecksumInfo = 4; + private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto readOpChecksumInfo_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProtoOrBuilder> readOpChecksumInfoBuilder_; + public boolean hasReadOpChecksumInfo() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto getReadOpChecksumInfo() { + if (readOpChecksumInfoBuilder_ == null) { + return readOpChecksumInfo_; + } else { + return readOpChecksumInfoBuilder_.getMessage(); + } + } + public Builder setReadOpChecksumInfo(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto value) { + if (readOpChecksumInfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + readOpChecksumInfo_ = value; + onChanged(); + } else { + readOpChecksumInfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + return this; + } + public Builder setReadOpChecksumInfo( + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.Builder builderForValue) { + if (readOpChecksumInfoBuilder_ == null) { + readOpChecksumInfo_ = builderForValue.build(); + onChanged(); + } else { + readOpChecksumInfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + return this; + } + public Builder mergeReadOpChecksumInfo(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto value) { + if (readOpChecksumInfoBuilder_ == null) { + if (((bitField0_ & 0x00000008) == 0x00000008) && + readOpChecksumInfo_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.getDefaultInstance()) { + readOpChecksumInfo_ = + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.newBuilder(readOpChecksumInfo_).mergeFrom(value).buildPartial(); + } else { + readOpChecksumInfo_ = value; + } + onChanged(); + } else { + readOpChecksumInfoBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000008; + return this; + } + public Builder clearReadOpChecksumInfo() { + if (readOpChecksumInfoBuilder_ == null) { + readOpChecksumInfo_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.getDefaultInstance(); + onChanged(); + } else { + readOpChecksumInfoBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.Builder getReadOpChecksumInfoBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getReadOpChecksumInfoFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProtoOrBuilder getReadOpChecksumInfoOrBuilder() { + if (readOpChecksumInfoBuilder_ != null) { + return readOpChecksumInfoBuilder_.getMessageOrBuilder(); + } else { + return readOpChecksumInfo_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProtoOrBuilder> + getReadOpChecksumInfoFieldBuilder() { + if (readOpChecksumInfoBuilder_ == null) { + readOpChecksumInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProtoOrBuilder>( + readOpChecksumInfo_, + getParentForChildren(), + isClean()); + readOpChecksumInfo_ = null; + } + return readOpChecksumInfoBuilder_; + } + + // optional string message = 5; private java.lang.Object message_ = ""; public boolean hasMessage() { - return ((bitField0_ & 0x00000008) == 0x00000008); + return ((bitField0_ & 0x00000010) == 0x00000010); } public String getMessage() { java.lang.Object ref = message_; @@ -7650,19 +9063,19 @@ public final class DataTransferProtos { if (value == null) { throw new NullPointerException(); } - bitField0_ |= 0x00000008; + bitField0_ |= 0x00000010; message_ = value; onChanged(); return this; } public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000008); + bitField0_ = (bitField0_ & ~0x00000010); message_ = getDefaultInstance().getMessage(); onChanged(); return this; } void setMessage(com.google.protobuf.ByteString value) { - bitField0_ |= 0x00000008; + bitField0_ |= 0x00000010; message_ = value; onChanged(); } @@ -8999,6 +10412,11 @@ public final class DataTransferProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_OpReadBlockProto_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_ChecksumProto_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_ChecksumProto_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor internal_static_OpWriteBlockProto_descriptor; private static @@ -9034,6 +10452,11 @@ public final class DataTransferProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_PipelineAckProto_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_ReadOpChecksumInfoProto_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_ReadOpChecksumInfoProto_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor internal_static_BlockOpResponseProto_descriptor; private static @@ -9070,47 +10493,55 @@ public final class DataTransferProtos { "\022$\n\nbaseHeader\030\001 \002(\0132\020.BaseHeaderProto\022\022" + "\n\nclientName\030\002 \002(\t\"\\\n\020OpReadBlockProto\022+" + "\n\006header\030\001 \002(\0132\033.ClientOperationHeaderPr" + - "oto\022\016\n\006offset\030\002 \002(\004\022\013\n\003len\030\003 \002(\004\"\257\004\n\021OpW" + - "riteBlockProto\022+\n\006header\030\001 \002(\0132\033.ClientO" + - "perationHeaderProto\022#\n\007targets\030\002 \003(\0132\022.D", - "atanodeInfoProto\022\"\n\006source\030\003 \001(\0132\022.Datan" + - "odeInfoProto\0228\n\005stage\030\004 \002(\0162).OpWriteBlo" + - "ckProto.BlockConstructionStage\022\024\n\014pipeli" + - "neSize\030\005 \002(\r\022\024\n\014minBytesRcvd\030\006 \002(\004\022\024\n\014ma" + - "xBytesRcvd\030\007 \002(\004\022\035\n\025latestGenerationStam" + - "p\030\010 \002(\004\"\210\002\n\026BlockConstructionStage\022\031\n\025PI" + - "PELINE_SETUP_APPEND\020\000\022\"\n\036PIPELINE_SETUP_" + - "APPEND_RECOVERY\020\001\022\022\n\016DATA_STREAMING\020\002\022%\n" + - "!PIPELINE_SETUP_STREAMING_RECOVERY\020\003\022\022\n\016" + - "PIPELINE_CLOSE\020\004\022\033\n\027PIPELINE_CLOSE_RECOV", - "ERY\020\005\022\031\n\025PIPELINE_SETUP_CREATE\020\006\022\020\n\014TRAN" + - "SFER_RBW\020\007\022\026\n\022TRANSFER_FINALIZED\020\010\"h\n\024Op" + - "TransferBlockProto\022+\n\006header\030\001 \002(\0132\033.Cli" + - "entOperationHeaderProto\022#\n\007targets\030\002 \003(\013" + - "2\022.DatanodeInfoProto\"l\n\023OpReplaceBlockPr" + - "oto\022 \n\006header\030\001 \002(\0132\020.BaseHeaderProto\022\017\n" + - "\007delHint\030\002 \002(\t\022\"\n\006source\030\003 \002(\0132\022.Datanod" + - "eInfoProto\"4\n\020OpCopyBlockProto\022 \n\006header" + - "\030\001 \002(\0132\020.BaseHeaderProto\"8\n\024OpBlockCheck" + - "sumProto\022 \n\006header\030\001 \002(\0132\020.BaseHeaderPro", - "to\"e\n\021PacketHeaderProto\022\025\n\roffsetInBlock" + - "\030\001 \002(\020\022\r\n\005seqno\030\002 \002(\020\022\031\n\021lastPacketInBlo" + - "ck\030\003 \002(\010\022\017\n\007dataLen\030\004 \002(\017\":\n\020PipelineAck" + - "Proto\022\r\n\005seqno\030\001 \002(\022\022\027\n\006status\030\002 \003(\0162\007.S" + - "tatus\"\217\001\n\024BlockOpResponseProto\022\027\n\006status" + - "\030\001 \002(\0162\007.Status\022\024\n\014firstBadLink\030\002 \001(\t\0227\n" + - "\020checksumResponse\030\003 \001(\0132\035.OpBlockChecksu" + - "mResponseProto\022\017\n\007message\030\004 \001(\t\"0\n\025Clien" + - "tReadStatusProto\022\027\n\006status\030\001 \002(\0162\007.Statu" + - "s\"-\n\022DNTransferAckProto\022\027\n\006status\030\001 \002(\0162", - "\007.Status\"U\n\034OpBlockChecksumResponseProto" + - "\022\023\n\013bytesPerCrc\030\001 \002(\r\022\023\n\013crcPerBlock\030\002 \002" + - "(\004\022\013\n\003md5\030\003 \002(\014*\202\001\n\006Status\022\013\n\007SUCCESS\020\000\022" + - "\t\n\005ERROR\020\001\022\022\n\016ERROR_CHECKSUM\020\002\022\021\n\rERROR_" + - "INVALID\020\003\022\020\n\014ERROR_EXISTS\020\004\022\026\n\022ERROR_ACC" + - "ESS_TOKEN\020\005\022\017\n\013CHECKSUM_OK\020\006B>\n%org.apac" + - "he.hadoop.hdfs.protocol.protoB\022DataTrans" + - "ferProtos\240\001\001" + "oto\022\016\n\006offset\030\002 \002(\004\022\013\n\003len\030\003 \002(\004\"\205\001\n\rChe" + + "cksumProto\022)\n\004type\030\001 \002(\0162\033.ChecksumProto" + + ".ChecksumType\022\030\n\020bytesPerChecksum\030\002 \002(\r\"", + "/\n\014ChecksumType\022\010\n\004NULL\020\000\022\t\n\005CRC32\020\001\022\n\n\006" + + "CRC32C\020\002\"\332\004\n\021OpWriteBlockProto\022+\n\006header" + + "\030\001 \002(\0132\033.ClientOperationHeaderProto\022#\n\007t" + + "argets\030\002 \003(\0132\022.DatanodeInfoProto\022\"\n\006sour" + + "ce\030\003 \001(\0132\022.DatanodeInfoProto\0228\n\005stage\030\004 " + + "\002(\0162).OpWriteBlockProto.BlockConstructio" + + "nStage\022\024\n\014pipelineSize\030\005 \002(\r\022\024\n\014minBytes" + + "Rcvd\030\006 \002(\004\022\024\n\014maxBytesRcvd\030\007 \002(\004\022\035\n\025late" + + "stGenerationStamp\030\010 \002(\004\022)\n\021requestedChec" + + "ksum\030\t \002(\0132\016.ChecksumProto\"\210\002\n\026BlockCons", + "tructionStage\022\031\n\025PIPELINE_SETUP_APPEND\020\000" + + "\022\"\n\036PIPELINE_SETUP_APPEND_RECOVERY\020\001\022\022\n\016" + + "DATA_STREAMING\020\002\022%\n!PIPELINE_SETUP_STREA" + + "MING_RECOVERY\020\003\022\022\n\016PIPELINE_CLOSE\020\004\022\033\n\027P" + + "IPELINE_CLOSE_RECOVERY\020\005\022\031\n\025PIPELINE_SET" + + "UP_CREATE\020\006\022\020\n\014TRANSFER_RBW\020\007\022\026\n\022TRANSFE" + + "R_FINALIZED\020\010\"h\n\024OpTransferBlockProto\022+\n" + + "\006header\030\001 \002(\0132\033.ClientOperationHeaderPro" + + "to\022#\n\007targets\030\002 \003(\0132\022.DatanodeInfoProto\"" + + "l\n\023OpReplaceBlockProto\022 \n\006header\030\001 \002(\0132\020", + ".BaseHeaderProto\022\017\n\007delHint\030\002 \002(\t\022\"\n\006sou" + + "rce\030\003 \002(\0132\022.DatanodeInfoProto\"4\n\020OpCopyB" + + "lockProto\022 \n\006header\030\001 \002(\0132\020.BaseHeaderPr" + + "oto\"8\n\024OpBlockChecksumProto\022 \n\006header\030\001 " + + "\002(\0132\020.BaseHeaderProto\"e\n\021PacketHeaderPro" + + "to\022\025\n\roffsetInBlock\030\001 \002(\020\022\r\n\005seqno\030\002 \002(\020" + + "\022\031\n\021lastPacketInBlock\030\003 \002(\010\022\017\n\007dataLen\030\004" + + " \002(\017\":\n\020PipelineAckProto\022\r\n\005seqno\030\001 \002(\022\022" + + "\027\n\006status\030\002 \003(\0162\007.Status\"P\n\027ReadOpChecks" + + "umInfoProto\022 \n\010checksum\030\001 \002(\0132\016.Checksum", + "Proto\022\023\n\013chunkOffset\030\002 \002(\004\"\305\001\n\024BlockOpRe" + + "sponseProto\022\027\n\006status\030\001 \002(\0162\007.Status\022\024\n\014" + + "firstBadLink\030\002 \001(\t\0227\n\020checksumResponse\030\003" + + " \001(\0132\035.OpBlockChecksumResponseProto\0224\n\022r" + + "eadOpChecksumInfo\030\004 \001(\0132\030.ReadOpChecksum" + + "InfoProto\022\017\n\007message\030\005 \001(\t\"0\n\025ClientRead" + + "StatusProto\022\027\n\006status\030\001 \002(\0162\007.Status\"-\n\022" + + "DNTransferAckProto\022\027\n\006status\030\001 \002(\0162\007.Sta" + + "tus\"U\n\034OpBlockChecksumResponseProto\022\023\n\013b" + + "ytesPerCrc\030\001 \002(\r\022\023\n\013crcPerBlock\030\002 \002(\004\022\013\n", + "\003md5\030\003 \002(\014*\202\001\n\006Status\022\013\n\007SUCCESS\020\000\022\t\n\005ER" + + "ROR\020\001\022\022\n\016ERROR_CHECKSUM\020\002\022\021\n\rERROR_INVAL" + + "ID\020\003\022\020\n\014ERROR_EXISTS\020\004\022\026\n\022ERROR_ACCESS_T" + + "OKEN\020\005\022\017\n\013CHECKSUM_OK\020\006B>\n%org.apache.ha" + + "doop.hdfs.protocol.protoB\022DataTransferPr" + + "otos\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -9141,16 +10572,24 @@ public final class DataTransferProtos { new java.lang.String[] { "Header", "Offset", "Len", }, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto.Builder.class); - internal_static_OpWriteBlockProto_descriptor = + internal_static_ChecksumProto_descriptor = getDescriptor().getMessageTypes().get(3); + internal_static_ChecksumProto_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_ChecksumProto_descriptor, + new java.lang.String[] { "Type", "BytesPerChecksum", }, + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.class, + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto.Builder.class); + internal_static_OpWriteBlockProto_descriptor = + getDescriptor().getMessageTypes().get(4); internal_static_OpWriteBlockProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_OpWriteBlockProto_descriptor, - new java.lang.String[] { "Header", "Targets", "Source", "Stage", "PipelineSize", "MinBytesRcvd", "MaxBytesRcvd", "LatestGenerationStamp", }, + new java.lang.String[] { "Header", "Targets", "Source", "Stage", "PipelineSize", "MinBytesRcvd", "MaxBytesRcvd", "LatestGenerationStamp", "RequestedChecksum", }, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.Builder.class); internal_static_OpTransferBlockProto_descriptor = - getDescriptor().getMessageTypes().get(4); + getDescriptor().getMessageTypes().get(5); internal_static_OpTransferBlockProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_OpTransferBlockProto_descriptor, @@ -9158,7 +10597,7 @@ public final class DataTransferProtos { org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto.Builder.class); internal_static_OpReplaceBlockProto_descriptor = - getDescriptor().getMessageTypes().get(5); + getDescriptor().getMessageTypes().get(6); internal_static_OpReplaceBlockProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_OpReplaceBlockProto_descriptor, @@ -9166,7 +10605,7 @@ public final class DataTransferProtos { org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto.Builder.class); internal_static_OpCopyBlockProto_descriptor = - getDescriptor().getMessageTypes().get(6); + getDescriptor().getMessageTypes().get(7); internal_static_OpCopyBlockProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_OpCopyBlockProto_descriptor, @@ -9174,7 +10613,7 @@ public final class DataTransferProtos { org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto.Builder.class); internal_static_OpBlockChecksumProto_descriptor = - getDescriptor().getMessageTypes().get(7); + getDescriptor().getMessageTypes().get(8); internal_static_OpBlockChecksumProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_OpBlockChecksumProto_descriptor, @@ -9182,7 +10621,7 @@ public final class DataTransferProtos { org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto.Builder.class); internal_static_PacketHeaderProto_descriptor = - getDescriptor().getMessageTypes().get(8); + getDescriptor().getMessageTypes().get(9); internal_static_PacketHeaderProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_PacketHeaderProto_descriptor, @@ -9190,23 +10629,31 @@ public final class DataTransferProtos { org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto.Builder.class); internal_static_PipelineAckProto_descriptor = - getDescriptor().getMessageTypes().get(9); + getDescriptor().getMessageTypes().get(10); internal_static_PipelineAckProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_PipelineAckProto_descriptor, new java.lang.String[] { "Seqno", "Status", }, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto.Builder.class); + internal_static_ReadOpChecksumInfoProto_descriptor = + getDescriptor().getMessageTypes().get(11); + internal_static_ReadOpChecksumInfoProto_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_ReadOpChecksumInfoProto_descriptor, + new java.lang.String[] { "Checksum", "ChunkOffset", }, + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.class, + org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto.Builder.class); internal_static_BlockOpResponseProto_descriptor = - getDescriptor().getMessageTypes().get(10); + getDescriptor().getMessageTypes().get(12); internal_static_BlockOpResponseProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_BlockOpResponseProto_descriptor, - new java.lang.String[] { "Status", "FirstBadLink", "ChecksumResponse", "Message", }, + new java.lang.String[] { "Status", "FirstBadLink", "ChecksumResponse", "ReadOpChecksumInfo", "Message", }, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.Builder.class); internal_static_ClientReadStatusProto_descriptor = - getDescriptor().getMessageTypes().get(11); + getDescriptor().getMessageTypes().get(13); internal_static_ClientReadStatusProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ClientReadStatusProto_descriptor, @@ -9214,7 +10661,7 @@ public final class DataTransferProtos { org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto.Builder.class); internal_static_DNTransferAckProto_descriptor = - getDescriptor().getMessageTypes().get(12); + getDescriptor().getMessageTypes().get(14); internal_static_DNTransferAckProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DNTransferAckProto_descriptor, @@ -9222,7 +10669,7 @@ public final class DataTransferProtos { org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto.class, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto.Builder.class); internal_static_OpBlockChecksumResponseProto_descriptor = - getDescriptor().getMessageTypes().get(13); + getDescriptor().getMessageTypes().get(15); internal_static_OpBlockChecksumResponseProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_OpBlockChecksumResponseProto_descriptor, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java index 2a53b3dd78a..579eb8ed1a4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java @@ -403,8 +403,8 @@ class BlockPoolSliceScanner { try { adjustThrottler(); - blockSender = new BlockSender(block, 0, -1, false, false, true, - datanode, null); + blockSender = new BlockSender(block, 0, -1, false, true, datanode, + null); DataOutputStream out = new DataOutputStream(new IOUtils.NullOutputStream()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java index b935aafd412..94920fd5bc3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java @@ -108,7 +108,8 @@ class BlockReceiver implements Closeable { final BlockConstructionStage stage, final long newGs, final long minBytesRcvd, final long maxBytesRcvd, final String clientname, final DatanodeInfo srcDataNode, - final DataNode datanode) throws IOException { + final DataNode datanode, DataChecksum requestedChecksum) + throws IOException { try{ this.block = block; this.in = in; @@ -177,7 +178,7 @@ class BlockReceiver implements Closeable { } } // read checksum meta information - this.checksum = DataChecksum.newDataChecksum(in); + this.checksum = requestedChecksum; this.bytesPerChecksum = checksum.getBytesPerChecksum(); this.checksumSize = checksum.getChecksumSize(); this.dropCacheBehindWrites = datanode.shouldDropCacheBehindWrites(); @@ -687,11 +688,6 @@ class BlockReceiver implements Closeable { } } - void writeChecksumHeader(DataOutputStream mirrorOut) throws IOException { - checksum.writeHeader(mirrorOut); - } - - void receiveBlock( DataOutputStream mirrOut, // output to next datanode DataInputStream mirrIn, // input from next datanode diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java index ca9765ce3ea..f4168ee1c90 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java @@ -134,8 +134,6 @@ class BlockSender implements java.io.Closeable { private final int checksumSize; /** If true, failure to read checksum is ignored */ private final boolean corruptChecksumOk; - /** true if chunk offset is needed to be sent in Checksum header */ - private final boolean chunkOffsetOK; /** Sequence number of packet being sent */ private long seqno; /** Set to true if transferTo is allowed for sending data to the client */ @@ -173,19 +171,17 @@ class BlockSender implements java.io.Closeable { * @param startOffset starting offset to read from * @param length length of data to read * @param corruptChecksumOk - * @param chunkOffsetOK need to send check offset in checksum header * @param verifyChecksum verify checksum while reading the data * @param datanode datanode from which the block is being read * @param clientTraceFmt format string used to print client trace logs * @throws IOException */ BlockSender(ExtendedBlock block, long startOffset, long length, - boolean corruptChecksumOk, boolean chunkOffsetOK, - boolean verifyChecksum, DataNode datanode, String clientTraceFmt) + boolean corruptChecksumOk, boolean verifyChecksum, + DataNode datanode, String clientTraceFmt) throws IOException { try { this.block = block; - this.chunkOffsetOK = chunkOffsetOK; this.corruptChecksumOk = corruptChecksumOk; this.verifyChecksum = verifyChecksum; this.clientTraceFmt = clientTraceFmt; @@ -600,8 +596,6 @@ class BlockSender implements java.io.Closeable { final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0; try { - writeChecksumHeader(out); - int maxChunksPerPacket; int pktSize = PacketHeader.PKT_HEADER_LEN; boolean transferTo = transferToAllowed && !verifyChecksum @@ -691,22 +685,6 @@ class BlockSender implements java.io.Closeable { return (endOffset - offset) > LONG_READ_THRESHOLD_BYTES; } - - /** - * Write checksum header to the output stream - */ - private void writeChecksumHeader(DataOutputStream out) throws IOException { - try { - checksum.writeHeader(out); - if (chunkOffsetOK) { - out.writeLong(offset); - } - out.flush(); - } catch (IOException e) { //socket error - throw ioeToSocketException(e); - } - } - /** * Write packet header into {@code pkt} */ @@ -720,4 +698,19 @@ class BlockSender implements java.io.Closeable { boolean didSendEntireByteRange() { return sentEntireByteRange; } + + /** + * @return the checksum type that will be used with this block transfer. + */ + DataChecksum getChecksum() { + return checksum; + } + + /** + * @return the offset into the block file where the sender is currently + * reading. + */ + long getOffset() { + return offset; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 5be82dd59d8..8f6b0f92aec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -2053,7 +2053,7 @@ public class DataNode extends Configured out = new DataOutputStream(new BufferedOutputStream(baseStream, HdfsConstants.SMALL_BUFFER_SIZE)); blockSender = new BlockSender(b, 0, b.getNumBytes(), - false, false, false, DataNode.this, null); + false, false, DataNode.this, null); DatanodeInfo srcNode = new DatanodeInfo(bpReg); // @@ -2066,7 +2066,7 @@ public class DataNode extends Configured } new Sender(out).writeBlock(b, accessToken, clientname, targets, srcNode, - stage, 0, 0, 0, 0); + stage, 0, 0, 0, 0, blockSender.getChecksum()); // send data & checksum blockSender.sendBlock(out, baseStream, null); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java index efad2f80e8d..d6a3963c0b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; +import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.Op; import org.apache.hadoop.hdfs.protocol.datatransfer.Receiver; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; @@ -52,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProtoOrBuilder; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; @@ -225,7 +227,7 @@ class DataXceiver extends Receiver implements Runnable { try { try { blockSender = new BlockSender(block, blockOffset, length, - true, true, false, datanode, clientTraceFmt); + true, false, datanode, clientTraceFmt); } catch(IOException e) { String msg = "opReadBlock " + block + " received exception " + e; LOG.info(msg); @@ -234,7 +236,8 @@ class DataXceiver extends Receiver implements Runnable { } // send op status - sendResponse(s, SUCCESS, null, datanode.socketWriteTimeout); + writeSuccessWithChecksumInfo(blockSender, + getStreamWithTimeout(s, datanode.socketWriteTimeout)); long read = blockSender.sendBlock(out, baseStream, null); // send data @@ -292,7 +295,8 @@ class DataXceiver extends Receiver implements Runnable { final int pipelineSize, final long minBytesRcvd, final long maxBytesRcvd, - final long latestGenerationStamp) throws IOException { + final long latestGenerationStamp, + DataChecksum requestedChecksum) throws IOException { updateCurrentThreadName("Receiving block " + block + " client=" + clientname); final boolean isDatanode = clientname.length() == 0; final boolean isClient = !isDatanode; @@ -351,7 +355,7 @@ class DataXceiver extends Receiver implements Runnable { s.getRemoteSocketAddress().toString(), s.getLocalSocketAddress().toString(), stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd, - clientname, srcDataNode, datanode); + clientname, srcDataNode, datanode, requestedChecksum); } else { datanode.data.recoverClose(block, latestGenerationStamp, minBytesRcvd); } @@ -381,11 +385,8 @@ class DataXceiver extends Receiver implements Runnable { new Sender(mirrorOut).writeBlock(originalBlock, blockToken, clientname, targets, srcDataNode, stage, pipelineSize, - minBytesRcvd, maxBytesRcvd, latestGenerationStamp); + minBytesRcvd, maxBytesRcvd, latestGenerationStamp, requestedChecksum); - if (blockReceiver != null) { // send checksum header - blockReceiver.writeChecksumHeader(mirrorOut); - } mirrorOut.flush(); // read connect ack (only for clients, not for replication req) @@ -600,8 +601,8 @@ class DataXceiver extends Receiver implements Runnable { try { // check if the block exists or not - blockSender = new BlockSender(block, 0, -1, false, false, false, - datanode, null); + blockSender = new BlockSender(block, 0, -1, false, false, datanode, + null); // set up response stream OutputStream baseStream = NetUtils.getOutputStream( @@ -610,7 +611,7 @@ class DataXceiver extends Receiver implements Runnable { baseStream, HdfsConstants.SMALL_BUFFER_SIZE)); // send status first - writeResponse(SUCCESS, null, reply); + writeSuccessWithChecksumInfo(blockSender, reply); // send block content to the target long read = blockSender.sendBlock(reply, baseStream, dataXceiverServer.balanceThrottler); @@ -709,11 +710,16 @@ class DataXceiver extends Receiver implements Runnable { throw new IOException("Copy block " + block + " from " + proxySock.getRemoteSocketAddress() + " failed"); } + + // get checksum info about the block we're copying + ReadOpChecksumInfoProto checksumInfo = copyResponse.getReadOpChecksumInfo(); + DataChecksum remoteChecksum = DataTransferProtoUtil.fromProto( + checksumInfo.getChecksum()); // open a block receiver and check if the block does not exist blockReceiver = new BlockReceiver( block, proxyReply, proxySock.getRemoteSocketAddress().toString(), proxySock.getLocalSocketAddress().toString(), - null, 0, 0, 0, "", null, datanode); + null, 0, 0, 0, "", null, datanode, remoteChecksum); // receive a block blockReceiver.receiveBlock(null, null, null, null, @@ -767,15 +773,19 @@ class DataXceiver extends Receiver implements Runnable { * @param opStatus status message to write * @param timeout send timeout **/ - private void sendResponse(Socket s, Status status, String message, + private static void sendResponse(Socket s, Status status, String message, long timeout) throws IOException { - DataOutputStream reply = - new DataOutputStream(NetUtils.getOutputStream(s, timeout)); + DataOutputStream reply = getStreamWithTimeout(s, timeout); writeResponse(status, message, reply); } - private void writeResponse(Status status, String message, OutputStream out) + private static DataOutputStream getStreamWithTimeout(Socket s, long timeout) + throws IOException { + return new DataOutputStream(NetUtils.getOutputStream(s, timeout)); + } + + private static void writeResponse(Status status, String message, OutputStream out) throws IOException { BlockOpResponseProto.Builder response = BlockOpResponseProto.newBuilder() .setStatus(status); @@ -786,6 +796,22 @@ class DataXceiver extends Receiver implements Runnable { out.flush(); } + private void writeSuccessWithChecksumInfo(BlockSender blockSender, + DataOutputStream out) throws IOException { + + ReadOpChecksumInfoProto ckInfo = ReadOpChecksumInfoProto.newBuilder() + .setChecksum(DataTransferProtoUtil.toProto(blockSender.getChecksum())) + .setChunkOffset(blockSender.getOffset()) + .build(); + + BlockOpResponseProto response = BlockOpResponseProto.newBuilder() + .setStatus(SUCCESS) + .setReadOpChecksumInfo(ckInfo) + .build(); + response.writeDelimitedTo(out); + out.flush(); + } + private void checkAccess(DataOutputStream out, final boolean reply, final ExtendedBlock blk, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/proto/datatransfer.proto index 1319ed6e744..316c05cea98 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/proto/datatransfer.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/proto/datatransfer.proto @@ -40,6 +40,17 @@ message OpReadBlockProto { required uint64 offset = 2; required uint64 len = 3; } + + +message ChecksumProto { + enum ChecksumType { + NULL = 0; + CRC32 = 1; + CRC32C = 2; + } + required ChecksumType type = 1; + required uint32 bytesPerChecksum = 2; +} message OpWriteBlockProto { required ClientOperationHeaderProto header = 1; @@ -69,6 +80,11 @@ message OpWriteBlockProto { required uint64 minBytesRcvd = 6; required uint64 maxBytesRcvd = 7; required uint64 latestGenerationStamp = 8; + + /** + * The requested checksum mechanism for this block write. + */ + required ChecksumProto requestedChecksum = 9; } message OpTransferBlockProto { @@ -114,14 +130,30 @@ message PipelineAckProto { repeated Status status = 2; } +/** + * Sent as part of the BlockOpResponseProto + * for READ_BLOCK and COPY_BLOCK operations. + */ +message ReadOpChecksumInfoProto { + required ChecksumProto checksum = 1; + + /** + * The offset into the block at which the first packet + * will start. This is necessary since reads will align + * backwards to a checksum chunk boundary. + */ + required uint64 chunkOffset = 2; +} + message BlockOpResponseProto { required Status status = 1; optional string firstBadLink = 2; optional OpBlockChecksumResponseProto checksumResponse = 3; + optional ReadOpChecksumInfoProto readOpChecksumInfo = 4; /** explanatory text which may be useful to log on the client side */ - optional string message = 4; + optional string message = 5; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java index 9ebecd48bdd..d02ae1da353 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java @@ -31,6 +31,7 @@ import java.util.Random; import junit.framework.TestCase; +import org.apache.commons.digester.SetRootRule; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -43,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; +import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol; import org.apache.hadoop.hdfs.protocol.datatransfer.Op; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; @@ -50,6 +52,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.Builder; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; @@ -59,6 +62,7 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.StringUtils; import org.junit.Test; +import org.mockito.Mockito; /** * This tests data transfer protocol handling in the Datanode. It sends @@ -68,6 +72,9 @@ public class TestDataTransferProtocol extends TestCase { private static final Log LOG = LogFactory.getLog( "org.apache.hadoop.hdfs.TestDataTransferProtocol"); + + private static final DataChecksum DEFAULT_CHECKSUM = + DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_CRC32, 512); DatanodeID datanode; InetSocketAddress dnAddr; @@ -149,9 +156,6 @@ public class TestDataTransferProtocol extends TestCase { private void writeZeroLengthPacket(ExtendedBlock block, String description) throws IOException { - sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32); - sendOut.writeInt(512); // checksum size - PacketHeader hdr = new PacketHeader( 8, // size of packet block.getNumBytes(), // OffsetInBlock @@ -188,7 +192,8 @@ public class TestDataTransferProtocol extends TestCase { recvBuf.reset(); sender.writeBlock(block, BlockTokenSecretManager.DUMMY_TOKEN, "cl", new DatanodeInfo[1], null, stage, - 0, block.getNumBytes(), block.getNumBytes(), newGS); + 0, block.getNumBytes(), block.getNumBytes(), newGS, + DEFAULT_CHECKSUM); if (eofExcepted) { sendResponse(Status.ERROR, null, null, recvOut); sendRecvData(description, true); @@ -373,15 +378,16 @@ public class TestDataTransferProtocol extends TestCase { /* Test OP_WRITE_BLOCK */ sendBuf.reset(); + + DataChecksum badChecksum = Mockito.spy(DEFAULT_CHECKSUM); + Mockito.doReturn(-1).when(badChecksum).getBytesPerChecksum(); + sender.writeBlock(new ExtendedBlock(poolId, newBlockId), BlockTokenSecretManager.DUMMY_TOKEN, "cl", new DatanodeInfo[1], null, BlockConstructionStage.PIPELINE_SETUP_CREATE, - 0, 0L, 0L, 0L); - sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32); - - // bad bytes per checksum - sendOut.writeInt(-1-random.nextInt(oneMil)); + 0, 0L, 0L, 0L, + badChecksum); recvBuf.reset(); sendResponse(Status.ERROR, null, null, recvOut); sendRecvData("wrong bytesPerChecksum while writing", true); @@ -391,9 +397,8 @@ public class TestDataTransferProtocol extends TestCase { sender.writeBlock(new ExtendedBlock(poolId, ++newBlockId), BlockTokenSecretManager.DUMMY_TOKEN, "cl", new DatanodeInfo[1], null, - BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0L, 0L, 0L); - sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32); - sendOut.writeInt(512); + BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0L, 0L, 0L, + DEFAULT_CHECKSUM); PacketHeader hdr = new PacketHeader( 4, // size of packet @@ -414,9 +419,8 @@ public class TestDataTransferProtocol extends TestCase { sender.writeBlock(new ExtendedBlock(poolId, ++newBlockId), BlockTokenSecretManager.DUMMY_TOKEN, "cl", new DatanodeInfo[1], null, - BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0L, 0L, 0L); - sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32); - sendOut.writeInt(512); // checksum size + BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0L, 0L, 0L, + DEFAULT_CHECKSUM); hdr = new PacketHeader( 8, // size of packet @@ -462,7 +466,15 @@ public class TestDataTransferProtocol extends TestCase { // negative length is ok. Datanode assumes we want to read the whole block. recvBuf.reset(); - sendResponse(Status.SUCCESS, null, null, recvOut); + + BlockOpResponseProto.newBuilder() + .setStatus(Status.SUCCESS) + .setReadOpChecksumInfo(ReadOpChecksumInfoProto.newBuilder() + .setChecksum(DataTransferProtoUtil.toProto(DEFAULT_CHECKSUM)) + .setChunkOffset(0L)) + .build() + .writeDelimitedTo(recvOut); + sendBuf.reset(); sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl", 0L, -1L-random.nextInt(oneMil)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java index b48f4ef3b3e..74434e5a441 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.apache.hadoop.util.DataChecksum; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -140,14 +141,13 @@ public class TestDiskError { // write the header. DataOutputStream out = new DataOutputStream(s.getOutputStream()); + DataChecksum checksum = DataChecksum.newDataChecksum( + DataChecksum.CHECKSUM_CRC32, 512); new Sender(out).writeBlock(block.getBlock(), BlockTokenSecretManager.DUMMY_TOKEN, "", new DatanodeInfo[0], null, - BlockConstructionStage.PIPELINE_SETUP_CREATE, 1, 0L, 0L, 0L); - - // write check header - out.writeByte( 1 ); - out.writeInt( 512 ); + BlockConstructionStage.PIPELINE_SETUP_CREATE, 1, 0L, 0L, 0L, + checksum); out.flush(); // close the connection before sending the content of the block