|
@@ -942,7 +942,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
* @param blockFile block file for which the checksum will be computed
|
|
* @param blockFile block file for which the checksum will be computed
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- private static void computeChecksum(File srcMeta, File dstMeta, File blockFile)
|
|
|
|
|
|
+ static void computeChecksum(File srcMeta, File dstMeta, File blockFile)
|
|
throws IOException {
|
|
throws IOException {
|
|
final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(srcMeta);
|
|
final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(srcMeta);
|
|
final byte[] data = new byte[1 << 16];
|
|
final byte[] data = new byte[1 << 16];
|
|
@@ -1079,7 +1079,30 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
}
|
|
}
|
|
return new ReplicaHandler(replica, ref);
|
|
return new ReplicaHandler(replica, ref);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ private byte[] loadLastPartialChunkChecksum(
|
|
|
|
+ File blockFile, File metaFile) throws IOException {
|
|
|
|
+ DataChecksum dcs = BlockMetadataHeader.readHeader(metaFile).getChecksum();
|
|
|
|
+ final int checksumSize = dcs.getChecksumSize();
|
|
|
|
+ final long onDiskLen = blockFile.length();
|
|
|
|
+ final int bytesPerChecksum = dcs.getBytesPerChecksum();
|
|
|
|
+
|
|
|
|
+ if (onDiskLen % bytesPerChecksum == 0) {
|
|
|
|
+ // the last chunk is a complete one. No need to preserve its checksum
|
|
|
|
+ // because it will not be modified.
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ int offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
|
|
|
|
+ (int)(onDiskLen / bytesPerChecksum * checksumSize);
|
|
|
|
+ byte[] lastChecksum = new byte[checksumSize];
|
|
|
|
+ RandomAccessFile raf = new RandomAccessFile(metaFile, "r");
|
|
|
|
+ raf.seek(offsetInChecksum);
|
|
|
|
+ raf.read(lastChecksum, 0, checksumSize);
|
|
|
|
+ return lastChecksum;
|
|
|
|
+ }
|
|
|
|
+
|
|
/** Append to a finalized replica
|
|
/** Append to a finalized replica
|
|
* Change a finalized replica to be a RBW replica and
|
|
* Change a finalized replica to be a RBW replica and
|
|
* bump its generation stamp to be the newGS
|
|
* bump its generation stamp to be the newGS
|
|
@@ -1113,6 +1136,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
ReplicaBeingWritten newReplicaInfo = new ReplicaBeingWritten(
|
|
ReplicaBeingWritten newReplicaInfo = new ReplicaBeingWritten(
|
|
replicaInfo.getBlockId(), replicaInfo.getNumBytes(), newGS,
|
|
replicaInfo.getBlockId(), replicaInfo.getNumBytes(), newGS,
|
|
v, newBlkFile.getParentFile(), Thread.currentThread(), bytesReserved);
|
|
v, newBlkFile.getParentFile(), Thread.currentThread(), bytesReserved);
|
|
|
|
+
|
|
|
|
+ // load last checksum and datalen
|
|
|
|
+ byte[] lastChunkChecksum = loadLastPartialChunkChecksum(
|
|
|
|
+ replicaInfo.getBlockFile(), replicaInfo.getMetaFile());
|
|
|
|
+ newReplicaInfo.setLastChecksumAndDataLen(
|
|
|
|
+ replicaInfo.getNumBytes(), lastChunkChecksum);
|
|
|
|
+
|
|
File newmeta = newReplicaInfo.getMetaFile();
|
|
File newmeta = newReplicaInfo.getMetaFile();
|
|
|
|
|
|
// rename meta file to rbw directory
|
|
// rename meta file to rbw directory
|
|
@@ -1435,6 +1465,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
blockId, numBytes, expectedGs,
|
|
blockId, numBytes, expectedGs,
|
|
v, dest.getParentFile(), Thread.currentThread(), 0);
|
|
v, dest.getParentFile(), Thread.currentThread(), 0);
|
|
rbw.setBytesAcked(visible);
|
|
rbw.setBytesAcked(visible);
|
|
|
|
+
|
|
|
|
+ // load last checksum and datalen
|
|
|
|
+ final File destMeta = FsDatasetUtil.getMetaFile(dest,
|
|
|
|
+ b.getGenerationStamp());
|
|
|
|
+ byte[] lastChunkChecksum = loadLastPartialChunkChecksum(dest, destMeta);
|
|
|
|
+ rbw.setLastChecksumAndDataLen(numBytes, lastChunkChecksum);
|
|
// overwrite the RBW in the volume map
|
|
// overwrite the RBW in the volume map
|
|
volumeMap.add(b.getBlockPoolId(), rbw);
|
|
volumeMap.add(b.getBlockPoolId(), rbw);
|
|
return rbw;
|
|
return rbw;
|
|
@@ -2466,6 +2502,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
newBlockId, recoveryId, volume, blockFile.getParentFile(),
|
|
newBlockId, recoveryId, volume, blockFile.getParentFile(),
|
|
newlength);
|
|
newlength);
|
|
newReplicaInfo.setNumBytes(newlength);
|
|
newReplicaInfo.setNumBytes(newlength);
|
|
|
|
+ // In theory, this rbw replica needs to reload last chunk checksum,
|
|
|
|
+ // but it is immediately converted to finalized state within the same
|
|
|
|
+ // lock, so no need to update it.
|
|
volumeMap.add(bpid, newReplicaInfo);
|
|
volumeMap.add(bpid, newReplicaInfo);
|
|
finalizeReplica(bpid, newReplicaInfo);
|
|
finalizeReplica(bpid, newReplicaInfo);
|
|
}
|
|
}
|