|
@@ -58,7 +58,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.StorageType;
|
|
import org.apache.hadoop.fs.StorageType;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
|
+import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.ExtendedBlockId;
|
|
import org.apache.hadoop.hdfs.ExtendedBlockId;
|
|
|
|
+import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
|
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
|
@@ -66,7 +68,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
|
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;
|
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
|
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
|
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
|
@@ -247,6 +248,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
private static final int MAX_BLOCK_EVICTIONS_PER_ITERATION = 3;
|
|
private static final int MAX_BLOCK_EVICTIONS_PER_ITERATION = 3;
|
|
|
|
|
|
|
|
+ private final int smallBufferSize;
|
|
|
|
|
|
// Used for synchronizing access to usage stats
|
|
// Used for synchronizing access to usage stats
|
|
private final Object statsLock = new Object();
|
|
private final Object statsLock = new Object();
|
|
@@ -264,6 +266,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
this.datanode = datanode;
|
|
this.datanode = datanode;
|
|
this.dataStorage = storage;
|
|
this.dataStorage = storage;
|
|
this.conf = conf;
|
|
this.conf = conf;
|
|
|
|
+ this.smallBufferSize = DFSUtil.getSmallBufferSize(conf);
|
|
// The number of volumes required for operation is the total number
|
|
// The number of volumes required for operation is the total number
|
|
// of volumes minus the number of failed volumes we can tolerate.
|
|
// of volumes minus the number of failed volumes we can tolerate.
|
|
final int volFailuresTolerated =
|
|
final int volFailuresTolerated =
|
|
@@ -837,19 +840,21 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
static File[] copyBlockFiles(long blockId, long genStamp, File srcMeta,
|
|
static File[] copyBlockFiles(long blockId, long genStamp, File srcMeta,
|
|
- File srcFile, File destRoot, boolean calculateChecksum)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ File srcFile, File destRoot, boolean calculateChecksum,
|
|
|
|
+ int smallBufferSize) throws IOException {
|
|
final File destDir = DatanodeUtil.idToBlockDir(destRoot, blockId);
|
|
final File destDir = DatanodeUtil.idToBlockDir(destRoot, blockId);
|
|
final File dstFile = new File(destDir, srcFile.getName());
|
|
final File dstFile = new File(destDir, srcFile.getName());
|
|
final File dstMeta = FsDatasetUtil.getMetaFile(dstFile, genStamp);
|
|
final File dstMeta = FsDatasetUtil.getMetaFile(dstFile, genStamp);
|
|
- return copyBlockFiles(srcMeta, srcFile, dstMeta, dstFile, calculateChecksum);
|
|
|
|
|
|
+ return copyBlockFiles(srcMeta, srcFile, dstMeta, dstFile, calculateChecksum,
|
|
|
|
+ smallBufferSize);
|
|
}
|
|
}
|
|
|
|
|
|
static File[] copyBlockFiles(File srcMeta, File srcFile, File dstMeta,
|
|
static File[] copyBlockFiles(File srcMeta, File srcFile, File dstMeta,
|
|
- File dstFile, boolean calculateChecksum)
|
|
|
|
|
|
+ File dstFile, boolean calculateChecksum,
|
|
|
|
+ int smallBufferSize)
|
|
throws IOException {
|
|
throws IOException {
|
|
if (calculateChecksum) {
|
|
if (calculateChecksum) {
|
|
- computeChecksum(srcMeta, dstMeta, srcFile);
|
|
|
|
|
|
+ computeChecksum(srcMeta, dstMeta, srcFile, smallBufferSize);
|
|
} else {
|
|
} else {
|
|
try {
|
|
try {
|
|
Storage.nativeCopyFileUnbuffered(srcMeta, dstMeta, true);
|
|
Storage.nativeCopyFileUnbuffered(srcMeta, dstMeta, true);
|
|
@@ -913,7 +918,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
File[] blockFiles = copyBlockFiles(block.getBlockId(),
|
|
File[] blockFiles = copyBlockFiles(block.getBlockId(),
|
|
block.getGenerationStamp(), oldMetaFile, oldBlockFile,
|
|
block.getGenerationStamp(), oldMetaFile, oldBlockFile,
|
|
targetVolume.getTmpDir(block.getBlockPoolId()),
|
|
targetVolume.getTmpDir(block.getBlockPoolId()),
|
|
- replicaInfo.isOnTransientStorage());
|
|
|
|
|
|
+ replicaInfo.isOnTransientStorage(), smallBufferSize);
|
|
|
|
|
|
ReplicaInfo newReplicaInfo = new ReplicaInPipeline(
|
|
ReplicaInfo newReplicaInfo = new ReplicaInPipeline(
|
|
replicaInfo.getBlockId(), replicaInfo.getGenerationStamp(),
|
|
replicaInfo.getBlockId(), replicaInfo.getGenerationStamp(),
|
|
@@ -941,7 +946,8 @@ 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)
|
|
|
|
|
|
+ private static void computeChecksum(File srcMeta, File dstMeta,
|
|
|
|
+ File blockFile, int smallBufferSize)
|
|
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];
|
|
@@ -957,7 +963,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
metaOut = new DataOutputStream(new BufferedOutputStream(
|
|
metaOut = new DataOutputStream(new BufferedOutputStream(
|
|
- new FileOutputStream(dstMeta), HdfsServerConstants.SMALL_BUFFER_SIZE));
|
|
|
|
|
|
+ new FileOutputStream(dstMeta), smallBufferSize));
|
|
BlockMetadataHeader.writeHeader(metaOut, checksum);
|
|
BlockMetadataHeader.writeHeader(metaOut, checksum);
|
|
|
|
|
|
int offset = 0;
|
|
int offset = 0;
|
|
@@ -2480,7 +2486,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
final File dstMetaFile = FsDatasetUtil.getMetaFile(dstBlockFile, newGS);
|
|
final File dstMetaFile = FsDatasetUtil.getMetaFile(dstBlockFile, newGS);
|
|
return copyBlockFiles(replicaInfo.getMetaFile(),
|
|
return copyBlockFiles(replicaInfo.getMetaFile(),
|
|
replicaInfo.getBlockFile(),
|
|
replicaInfo.getBlockFile(),
|
|
- dstMetaFile, dstBlockFile, true);
|
|
|
|
|
|
+ dstMetaFile, dstBlockFile, true, smallBufferSize);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|