|
@@ -21,6 +21,8 @@ import com.google.common.base.Preconditions;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
|
+import org.apache.hadoop.hdfs.protocol.BlockChecksumOptions;
|
|
|
+import org.apache.hadoop.hdfs.protocol.BlockChecksumType;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
@@ -32,6 +34,8 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos;
|
|
|
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedBlockChecksumCompositeCrcReconstructor;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedBlockChecksumMd5CrcReconstructor;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedBlockChecksumReconstructor;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedReconstructionInfo;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
|
|
@@ -40,6 +44,8 @@ import org.apache.hadoop.io.DataOutputBuffer;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
+import org.apache.hadoop.util.CrcComposer;
|
|
|
+import org.apache.hadoop.util.CrcUtil;
|
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -70,6 +76,7 @@ final class BlockChecksumHelper {
|
|
|
*/
|
|
|
static abstract class AbstractBlockChecksumComputer {
|
|
|
private final DataNode datanode;
|
|
|
+ private final BlockChecksumOptions blockChecksumOptions;
|
|
|
|
|
|
private byte[] outBytes;
|
|
|
private int bytesPerCRC = -1;
|
|
@@ -77,8 +84,11 @@ final class BlockChecksumHelper {
|
|
|
private long crcPerBlock = -1;
|
|
|
private int checksumSize = -1;
|
|
|
|
|
|
- AbstractBlockChecksumComputer(DataNode datanode) throws IOException {
|
|
|
+ AbstractBlockChecksumComputer(
|
|
|
+ DataNode datanode,
|
|
|
+ BlockChecksumOptions blockChecksumOptions) throws IOException {
|
|
|
this.datanode = datanode;
|
|
|
+ this.blockChecksumOptions = blockChecksumOptions;
|
|
|
}
|
|
|
|
|
|
abstract void compute() throws IOException;
|
|
@@ -92,6 +102,10 @@ final class BlockChecksumHelper {
|
|
|
return datanode;
|
|
|
}
|
|
|
|
|
|
+ BlockChecksumOptions getBlockChecksumOptions() {
|
|
|
+ return blockChecksumOptions;
|
|
|
+ }
|
|
|
+
|
|
|
InputStream getBlockInputStream(ExtendedBlock block, long seekOffset)
|
|
|
throws IOException {
|
|
|
return datanode.data.getBlockInputStream(block, seekOffset);
|
|
@@ -155,8 +169,10 @@ final class BlockChecksumHelper {
|
|
|
private DataChecksum checksum;
|
|
|
|
|
|
BlockChecksumComputer(DataNode datanode,
|
|
|
- ExtendedBlock block) throws IOException {
|
|
|
- super(datanode);
|
|
|
+ ExtendedBlock block,
|
|
|
+ BlockChecksumOptions blockChecksumOptions)
|
|
|
+ throws IOException {
|
|
|
+ super(datanode, blockChecksumOptions);
|
|
|
this.block = block;
|
|
|
this.requestLength = block.getNumBytes();
|
|
|
Preconditions.checkArgument(requestLength >= 0);
|
|
@@ -268,8 +284,10 @@ final class BlockChecksumHelper {
|
|
|
static class ReplicatedBlockChecksumComputer extends BlockChecksumComputer {
|
|
|
|
|
|
ReplicatedBlockChecksumComputer(DataNode datanode,
|
|
|
- ExtendedBlock block) throws IOException {
|
|
|
- super(datanode, block);
|
|
|
+ ExtendedBlock block,
|
|
|
+ BlockChecksumOptions blockChecksumOptions)
|
|
|
+ throws IOException {
|
|
|
+ super(datanode, block, blockChecksumOptions);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -277,22 +295,38 @@ final class BlockChecksumHelper {
|
|
|
try {
|
|
|
readHeader();
|
|
|
|
|
|
- MD5Hash md5out;
|
|
|
- if (isPartialBlk() && getCrcPerBlock() > 0) {
|
|
|
- md5out = checksumPartialBlock();
|
|
|
- } else {
|
|
|
- md5out = checksumWholeBlock();
|
|
|
+ BlockChecksumType type =
|
|
|
+ getBlockChecksumOptions().getBlockChecksumType();
|
|
|
+ switch (type) {
|
|
|
+ case MD5CRC:
|
|
|
+ computeMd5Crc();
|
|
|
+ break;
|
|
|
+ case COMPOSITE_CRC:
|
|
|
+ computeCompositeCrc(getBlockChecksumOptions().getStripeLength());
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ throw new IOException(String.format(
|
|
|
+ "Unrecognized BlockChecksumType: %s", type));
|
|
|
}
|
|
|
- setOutBytes(md5out.getDigest());
|
|
|
-
|
|
|
- LOG.debug("block={}, bytesPerCRC={}, crcPerBlock={}, md5out={}",
|
|
|
- getBlock(), getBytesPerCRC(), getCrcPerBlock(), md5out);
|
|
|
} finally {
|
|
|
IOUtils.closeStream(getChecksumIn());
|
|
|
IOUtils.closeStream(getMetadataIn());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void computeMd5Crc() throws IOException {
|
|
|
+ MD5Hash md5out;
|
|
|
+ if (isPartialBlk() && getCrcPerBlock() > 0) {
|
|
|
+ md5out = checksumPartialBlock();
|
|
|
+ } else {
|
|
|
+ md5out = checksumWholeBlock();
|
|
|
+ }
|
|
|
+ setOutBytes(md5out.getDigest());
|
|
|
+
|
|
|
+ LOG.debug("block={}, bytesPerCRC={}, crcPerBlock={}, md5out={}",
|
|
|
+ getBlock(), getBytesPerCRC(), getCrcPerBlock(), md5out);
|
|
|
+ }
|
|
|
+
|
|
|
private MD5Hash checksumWholeBlock() throws IOException {
|
|
|
MD5Hash md5out = MD5Hash.digest(getChecksumIn());
|
|
|
return md5out;
|
|
@@ -320,6 +354,56 @@ final class BlockChecksumHelper {
|
|
|
|
|
|
return new MD5Hash(digester.digest());
|
|
|
}
|
|
|
+
|
|
|
+ private void computeCompositeCrc(long stripeLength) throws IOException {
|
|
|
+ long checksumDataLength =
|
|
|
+ Math.min(getVisibleLength(), getRequestLength());
|
|
|
+ if (stripeLength <= 0 || stripeLength > checksumDataLength) {
|
|
|
+ stripeLength = checksumDataLength;
|
|
|
+ }
|
|
|
+
|
|
|
+ CrcComposer crcComposer = CrcComposer.newStripedCrcComposer(
|
|
|
+ getCrcType(), getBytesPerCRC(), stripeLength);
|
|
|
+ DataInputStream checksumIn = getChecksumIn();
|
|
|
+
|
|
|
+ // Whether getting the checksum for the entire block (which itself may
|
|
|
+ // not be a full block size and may have a final chunk smaller than
|
|
|
+ // getBytesPerCRC()), we begin with a number of full chunks, all of size
|
|
|
+ // getBytesPerCRC().
|
|
|
+ long numFullChunks = checksumDataLength / getBytesPerCRC();
|
|
|
+ crcComposer.update(checksumIn, numFullChunks, getBytesPerCRC());
|
|
|
+
|
|
|
+ // There may be a final partial chunk that is not full-sized. Unlike the
|
|
|
+ // MD5 case, we still consider this a "partial chunk" even if
|
|
|
+ // getRequestLength() == getVisibleLength(), since the CRC composition
|
|
|
+ // depends on the byte size of that final chunk, even if it already has a
|
|
|
+ // precomputed CRC stored in metadata. So there are two cases:
|
|
|
+ // 1. Reading only part of a block via getRequestLength(); we get the
|
|
|
+ // crcPartialBlock() explicitly.
|
|
|
+ // 2. Reading full visible length; the partial chunk already has a CRC
|
|
|
+ // stored in block metadata, so we just continue reading checksumIn.
|
|
|
+ long partialChunkSize = checksumDataLength % getBytesPerCRC();
|
|
|
+ if (partialChunkSize > 0) {
|
|
|
+ if (isPartialBlk()) {
|
|
|
+ byte[] partialChunkCrcBytes = crcPartialBlock();
|
|
|
+ crcComposer.update(
|
|
|
+ partialChunkCrcBytes, 0, partialChunkCrcBytes.length,
|
|
|
+ partialChunkSize);
|
|
|
+ } else {
|
|
|
+ int partialChunkCrc = checksumIn.readInt();
|
|
|
+ crcComposer.update(partialChunkCrc, partialChunkSize);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ byte[] composedCrcs = crcComposer.digest();
|
|
|
+ setOutBytes(composedCrcs);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug(
|
|
|
+ "block={}, getBytesPerCRC={}, crcPerBlock={}, compositeCrc={}",
|
|
|
+ getBlock(), getBytesPerCRC(), getCrcPerBlock(),
|
|
|
+ CrcUtil.toMultiCrcString(composedCrcs));
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -335,19 +419,29 @@ final class BlockChecksumHelper {
|
|
|
private final byte[] blockIndices;
|
|
|
private final long requestedNumBytes;
|
|
|
|
|
|
- private final DataOutputBuffer md5writer = new DataOutputBuffer();
|
|
|
+ private final DataOutputBuffer blockChecksumBuf = new DataOutputBuffer();
|
|
|
+
|
|
|
+ // Keeps track of the positions within blockChecksumBuf where each data
|
|
|
+ // block's checksum begins; for fixed-size block checksums this is easily
|
|
|
+ // calculated as a multiple of the checksum size, but for striped block
|
|
|
+ // CRCs, it's less error-prone to simply keep track of exact byte offsets
|
|
|
+ // before each block checksum is populated into the buffer.
|
|
|
+ private final int[] blockChecksumPositions;
|
|
|
|
|
|
- BlockGroupNonStripedChecksumComputer(DataNode datanode,
|
|
|
- StripedBlockInfo stripedBlockInfo,
|
|
|
- long requestedNumBytes)
|
|
|
+ BlockGroupNonStripedChecksumComputer(
|
|
|
+ DataNode datanode,
|
|
|
+ StripedBlockInfo stripedBlockInfo,
|
|
|
+ long requestedNumBytes,
|
|
|
+ BlockChecksumOptions blockChecksumOptions)
|
|
|
throws IOException {
|
|
|
- super(datanode);
|
|
|
+ super(datanode, blockChecksumOptions);
|
|
|
this.blockGroup = stripedBlockInfo.getBlock();
|
|
|
this.ecPolicy = stripedBlockInfo.getErasureCodingPolicy();
|
|
|
this.datanodes = stripedBlockInfo.getDatanodes();
|
|
|
this.blockTokens = stripedBlockInfo.getBlockTokens();
|
|
|
this.blockIndices = stripedBlockInfo.getBlockIndices();
|
|
|
this.requestedNumBytes = requestedNumBytes;
|
|
|
+ this.blockChecksumPositions = new int[this.ecPolicy.getNumDataUnits()];
|
|
|
}
|
|
|
|
|
|
private static class LiveBlockInfo {
|
|
@@ -383,6 +477,9 @@ final class BlockChecksumHelper {
|
|
|
}
|
|
|
long checksumLen = 0;
|
|
|
for (int idx = 0; idx < numDataUnits && idx < blkIndxLen; idx++) {
|
|
|
+ // Before populating the blockChecksum at this index, record the byte
|
|
|
+ // offset where it will begin.
|
|
|
+ blockChecksumPositions[idx] = blockChecksumBuf.getLength();
|
|
|
try {
|
|
|
ExtendedBlock block = getInternalBlock(numDataUnits, idx);
|
|
|
|
|
@@ -409,8 +506,75 @@ final class BlockChecksumHelper {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- MD5Hash md5out = MD5Hash.digest(md5writer.getData());
|
|
|
- setOutBytes(md5out.getDigest());
|
|
|
+ BlockChecksumType type = getBlockChecksumOptions().getBlockChecksumType();
|
|
|
+ switch (type) {
|
|
|
+ case MD5CRC:
|
|
|
+ MD5Hash md5out = MD5Hash.digest(blockChecksumBuf.getData());
|
|
|
+ setOutBytes(md5out.getDigest());
|
|
|
+ break;
|
|
|
+ case COMPOSITE_CRC:
|
|
|
+ byte[] digest = reassembleNonStripedCompositeCrc(checksumLen);
|
|
|
+ setOutBytes(digest);
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ throw new IOException(String.format(
|
|
|
+ "Unrecognized BlockChecksumType: %s", type));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * @param checksumLen The sum of bytes associated with the block checksum
|
|
|
+ * data being digested into a block-group level checksum.
|
|
|
+ */
|
|
|
+ private byte[] reassembleNonStripedCompositeCrc(long checksumLen)
|
|
|
+ throws IOException {
|
|
|
+ int numDataUnits = ecPolicy.getNumDataUnits();
|
|
|
+ CrcComposer crcComposer = CrcComposer.newCrcComposer(
|
|
|
+ getCrcType(), ecPolicy.getCellSize());
|
|
|
+
|
|
|
+ // This should hold all the cell-granularity checksums of blk0
|
|
|
+ // followed by all cell checksums of blk1, etc. We must unstripe the
|
|
|
+ // cell checksums in order of logical file bytes. Also, note that the
|
|
|
+ // length of this array may not equal the the number of actually valid
|
|
|
+ // bytes in the buffer (blockChecksumBuf.getLength()).
|
|
|
+ byte[] flatBlockChecksumData = blockChecksumBuf.getData();
|
|
|
+
|
|
|
+ // Initialize byte-level cursors to where each block's checksum begins
|
|
|
+ // inside the combined flattened buffer.
|
|
|
+ int[] blockChecksumCursors = new int[numDataUnits];
|
|
|
+ for (int idx = 0; idx < numDataUnits; ++idx) {
|
|
|
+ blockChecksumCursors[idx] = blockChecksumPositions[idx];
|
|
|
+ }
|
|
|
+
|
|
|
+ // Reassemble cell-level CRCs in the right order.
|
|
|
+ long numFullCells = checksumLen / ecPolicy.getCellSize();
|
|
|
+ for (long cellIndex = 0; cellIndex < numFullCells; ++cellIndex) {
|
|
|
+ int blockIndex = (int) (cellIndex % numDataUnits);
|
|
|
+ int checksumCursor = blockChecksumCursors[blockIndex];
|
|
|
+ int cellCrc = CrcUtil.readInt(
|
|
|
+ flatBlockChecksumData, checksumCursor);
|
|
|
+ blockChecksumCursors[blockIndex] += 4;
|
|
|
+ crcComposer.update(cellCrc, ecPolicy.getCellSize());
|
|
|
+ }
|
|
|
+ if (checksumLen % ecPolicy.getCellSize() != 0) {
|
|
|
+ // Final partial cell.
|
|
|
+ int blockIndex = (int) (numFullCells % numDataUnits);
|
|
|
+ int checksumCursor = blockChecksumCursors[blockIndex];
|
|
|
+ int cellCrc = CrcUtil.readInt(
|
|
|
+ flatBlockChecksumData, checksumCursor);
|
|
|
+ blockChecksumCursors[blockIndex] += 4;
|
|
|
+ crcComposer.update(cellCrc, checksumLen % ecPolicy.getCellSize());
|
|
|
+ }
|
|
|
+ byte[] digest = crcComposer.digest();
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("flatBlockChecksumData.length={}, numDataUnits={}, "
|
|
|
+ + "checksumLen={}, digest={}",
|
|
|
+ flatBlockChecksumData.length,
|
|
|
+ numDataUnits,
|
|
|
+ checksumLen,
|
|
|
+ CrcUtil.toSingleCrcString(digest));
|
|
|
+ }
|
|
|
+ return digest;
|
|
|
}
|
|
|
|
|
|
private ExtendedBlock getInternalBlock(int numDataUnits, int idx) {
|
|
@@ -437,8 +601,26 @@ final class BlockChecksumHelper {
|
|
|
LOG.debug("write to {}: {}, block={}",
|
|
|
getDatanode(), Op.BLOCK_CHECKSUM, block);
|
|
|
|
|
|
- // get block MD5
|
|
|
- createSender(pair).blockChecksum(block, blockToken);
|
|
|
+ // get block checksum
|
|
|
+ // A BlockGroupCheckum of type COMPOSITE_CRC uses underlying
|
|
|
+ // BlockChecksums also of type COMPOSITE_CRC but with
|
|
|
+ // stripeLength == ecPolicy.getCellSize().
|
|
|
+ BlockChecksumOptions childOptions;
|
|
|
+ BlockChecksumType groupChecksumType =
|
|
|
+ getBlockChecksumOptions().getBlockChecksumType();
|
|
|
+ switch (groupChecksumType) {
|
|
|
+ case MD5CRC:
|
|
|
+ childOptions = getBlockChecksumOptions();
|
|
|
+ break;
|
|
|
+ case COMPOSITE_CRC:
|
|
|
+ childOptions = new BlockChecksumOptions(
|
|
|
+ BlockChecksumType.COMPOSITE_CRC, ecPolicy.getCellSize());
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ throw new IOException(
|
|
|
+ "Unknown BlockChecksumType: " + groupChecksumType);
|
|
|
+ }
|
|
|
+ createSender(pair).blockChecksum(block, blockToken, childOptions);
|
|
|
|
|
|
final DataTransferProtos.BlockOpResponseProto reply =
|
|
|
DataTransferProtos.BlockOpResponseProto.parseFrom(
|
|
@@ -463,10 +645,37 @@ final class BlockChecksumHelper {
|
|
|
|
|
|
setOrVerifyChecksumProperties(blockIdx, checksumData.getBytesPerCrc(),
|
|
|
checksumData.getCrcPerBlock(), ct);
|
|
|
- //read md5
|
|
|
- final MD5Hash md5 = new MD5Hash(checksumData.getMd5().toByteArray());
|
|
|
- md5.write(md5writer);
|
|
|
- LOG.debug("got reply from datanode:{}, md5={}", targetDatanode, md5);
|
|
|
+
|
|
|
+ switch (groupChecksumType) {
|
|
|
+ case MD5CRC:
|
|
|
+ //read md5
|
|
|
+ final MD5Hash md5 =
|
|
|
+ new MD5Hash(checksumData.getBlockChecksum().toByteArray());
|
|
|
+ md5.write(blockChecksumBuf);
|
|
|
+ LOG.debug("got reply from datanode:{}, md5={}",
|
|
|
+ targetDatanode, md5);
|
|
|
+ break;
|
|
|
+ case COMPOSITE_CRC:
|
|
|
+ BlockChecksumType returnedType = PBHelperClient.convert(
|
|
|
+ checksumData.getBlockChecksumOptions().getBlockChecksumType());
|
|
|
+ if (returnedType != BlockChecksumType.COMPOSITE_CRC) {
|
|
|
+ throw new IOException(String.format(
|
|
|
+ "Unexpected blockChecksumType '%s', expecting COMPOSITE_CRC",
|
|
|
+ returnedType));
|
|
|
+ }
|
|
|
+ byte[] checksumBytes =
|
|
|
+ checksumData.getBlockChecksum().toByteArray();
|
|
|
+ blockChecksumBuf.write(checksumBytes, 0, checksumBytes.length);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("got reply from datanode:{} for blockIdx:{}, checksum:{}",
|
|
|
+ targetDatanode, blockIdx,
|
|
|
+ CrcUtil.toMultiCrcString(checksumBytes));
|
|
|
+ }
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ throw new IOException(
|
|
|
+ "Unknown BlockChecksumType: " + groupChecksumType);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -489,10 +698,16 @@ final class BlockChecksumHelper {
|
|
|
StripedReconstructionInfo stripedReconInfo =
|
|
|
new StripedReconstructionInfo(
|
|
|
blockGroup, ecPolicy, blockIndices, datanodes, errIndices);
|
|
|
+ BlockChecksumType groupChecksumType =
|
|
|
+ getBlockChecksumOptions().getBlockChecksumType();
|
|
|
final StripedBlockChecksumReconstructor checksumRecon =
|
|
|
- new StripedBlockChecksumReconstructor(
|
|
|
+ groupChecksumType == BlockChecksumType.COMPOSITE_CRC ?
|
|
|
+ new StripedBlockChecksumCompositeCrcReconstructor(
|
|
|
getDatanode().getErasureCodingWorker(), stripedReconInfo,
|
|
|
- md5writer, blockLength);
|
|
|
+ blockChecksumBuf, blockLength) :
|
|
|
+ new StripedBlockChecksumMd5CrcReconstructor(
|
|
|
+ getDatanode().getErasureCodingWorker(), stripedReconInfo,
|
|
|
+ blockChecksumBuf, blockLength);
|
|
|
checksumRecon.reconstruct();
|
|
|
|
|
|
DataChecksum checksum = checksumRecon.getChecksum();
|
|
@@ -501,8 +716,8 @@ final class BlockChecksumHelper {
|
|
|
setOrVerifyChecksumProperties(errBlkIndex,
|
|
|
checksum.getBytesPerChecksum(), crcPerBlock,
|
|
|
checksum.getChecksumType());
|
|
|
- LOG.debug("Recalculated checksum for the block index:{}, md5={}",
|
|
|
- errBlkIndex, checksumRecon.getMD5());
|
|
|
+ LOG.debug("Recalculated checksum for the block index:{}, checksum={}",
|
|
|
+ errBlkIndex, checksumRecon.getDigestObject());
|
|
|
}
|
|
|
|
|
|
private void setOrVerifyChecksumProperties(int blockIdx, int bpc,
|
|
@@ -524,8 +739,16 @@ final class BlockChecksumHelper {
|
|
|
setCrcType(ct);
|
|
|
} else if (getCrcType() != DataChecksum.Type.MIXED &&
|
|
|
getCrcType() != ct) {
|
|
|
- // if crc types are mixed in a file
|
|
|
- setCrcType(DataChecksum.Type.MIXED);
|
|
|
+ BlockChecksumType groupChecksumType =
|
|
|
+ getBlockChecksumOptions().getBlockChecksumType();
|
|
|
+ if (groupChecksumType == BlockChecksumType.COMPOSITE_CRC) {
|
|
|
+ throw new IOException(String.format(
|
|
|
+ "BlockChecksumType COMPOSITE_CRC doesn't support MIXED "
|
|
|
+ + "underlying types; previous block was %s, next block is %s",
|
|
|
+ getCrcType(), ct));
|
|
|
+ } else {
|
|
|
+ setCrcType(DataChecksum.Type.MIXED);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
if (blockIdx == 0) {
|