|
@@ -23,6 +23,7 @@ import java.io.FileOutputStream;
|
|
import java.io.FilenameFilter;
|
|
import java.io.FilenameFilter;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.OutputStreamWriter;
|
|
import java.io.OutputStreamWriter;
|
|
|
|
+import java.io.RandomAccessFile;
|
|
import java.net.URI;
|
|
import java.net.URI;
|
|
import java.nio.channels.ClosedChannelException;
|
|
import java.nio.channels.ClosedChannelException;
|
|
import java.nio.file.Files;
|
|
import java.nio.file.Files;
|
|
@@ -47,6 +48,7 @@ import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.DF;
|
|
import org.apache.hadoop.fs.DF;
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
import org.apache.hadoop.fs.StorageType;
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
|
import org.apache.hadoop.util.AutoCloseableLock;
|
|
import org.apache.hadoop.util.AutoCloseableLock;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
@@ -59,6 +61,7 @@ import org.apache.hadoop.hdfs.server.datanode.LocalReplica;
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
|
|
+import org.apache.hadoop.util.DataChecksum;
|
|
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
|
|
import org.apache.hadoop.hdfs.server.datanode.LocalReplicaInPipeline;
|
|
import org.apache.hadoop.hdfs.server.datanode.LocalReplicaInPipeline;
|
|
@@ -1102,6 +1105,28 @@ public class FsVolumeImpl implements FsVolumeSpi {
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
+ 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;
|
|
|
|
+ }
|
|
|
|
+
|
|
public ReplicaInPipeline append(String bpid, ReplicaInfo replicaInfo,
|
|
public ReplicaInPipeline append(String bpid, ReplicaInfo replicaInfo,
|
|
long newGS, long estimateBlockLen) throws IOException {
|
|
long newGS, long estimateBlockLen) throws IOException {
|
|
|
|
|
|
@@ -1126,6 +1151,13 @@ public class FsVolumeImpl implements FsVolumeSpi {
|
|
.setBytesToReserve(bytesReserved)
|
|
.setBytesToReserve(bytesReserved)
|
|
.buildLocalReplicaInPipeline();
|
|
.buildLocalReplicaInPipeline();
|
|
|
|
|
|
|
|
+ // load last checksum and datalen
|
|
|
|
+ LocalReplica localReplica = (LocalReplica)replicaInfo;
|
|
|
|
+ byte[] lastChunkChecksum = loadLastPartialChunkChecksum(
|
|
|
|
+ localReplica.getBlockFile(), localReplica.getMetaFile());
|
|
|
|
+ newReplicaInfo.setLastChecksumAndDataLen(
|
|
|
|
+ replicaInfo.getNumBytes(), lastChunkChecksum);
|
|
|
|
+
|
|
// rename meta file to rbw directory
|
|
// rename meta file to rbw directory
|
|
// rename block file to rbw directory
|
|
// rename block file to rbw directory
|
|
newReplicaInfo.moveReplicaFrom(replicaInfo, newBlkFile);
|
|
newReplicaInfo.moveReplicaFrom(replicaInfo, newBlkFile);
|
|
@@ -1170,6 +1202,12 @@ public class FsVolumeImpl implements FsVolumeSpi {
|
|
.setBytesToReserve(0)
|
|
.setBytesToReserve(0)
|
|
.buildLocalReplicaInPipeline();
|
|
.buildLocalReplicaInPipeline();
|
|
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);
|
|
return rbw;
|
|
return rbw;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1206,6 +1244,9 @@ public class FsVolumeImpl implements FsVolumeSpi {
|
|
.setDirectoryToUse(blockFile.getParentFile())
|
|
.setDirectoryToUse(blockFile.getParentFile())
|
|
.setBytesToReserve(newlength)
|
|
.setBytesToReserve(newlength)
|
|
.buildLocalReplicaInPipeline();
|
|
.buildLocalReplicaInPipeline();
|
|
|
|
+ // 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.
|
|
return newReplicaInfo;
|
|
return newReplicaInfo;
|
|
}
|
|
}
|
|
|
|
|