HDFS-6948. DN rejects blocks if it has older UC block. Contributed by
Eric Payne.
(cherry picked from commit f02d934fed
)
This commit is contained in:
parent
9333ee3068
commit
2a6c9f0725
|
@ -227,6 +227,9 @@ Release 2.6.0 - UNRELEASED
|
|||
|
||||
HDFS-6970. Move startFile EDEK retries to the DFSClient. (wang)
|
||||
|
||||
HDFS-6948. DN rejects blocks if it has older UC block
|
||||
(Eric Payne via kihwal)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-6690. Deduplicate xattr names in memory. (wang)
|
||||
|
|
|
@ -1090,9 +1090,17 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
ExtendedBlock b) throws IOException {
|
||||
ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(), b.getBlockId());
|
||||
if (replicaInfo != null) {
|
||||
throw new ReplicaAlreadyExistsException("Block " + b +
|
||||
" already exists in state " + replicaInfo.getState() +
|
||||
" and thus cannot be created.");
|
||||
if (replicaInfo.getGenerationStamp() < b.getGenerationStamp()
|
||||
&& replicaInfo instanceof ReplicaInPipeline) {
|
||||
// Stop the previous writer
|
||||
((ReplicaInPipeline)replicaInfo)
|
||||
.stopWriter(datanode.getDnConf().getXceiverStopTimeout());
|
||||
invalidate(b.getBlockPoolId(), new Block[]{replicaInfo});
|
||||
} else {
|
||||
throw new ReplicaAlreadyExistsException("Block " + b +
|
||||
" already exists in state " + replicaInfo.getState() +
|
||||
" and thus cannot be created.");
|
||||
}
|
||||
}
|
||||
|
||||
FsVolumeImpl v = volumes.getNextVolume(storageType, b.getNumBytes());
|
||||
|
|
|
@ -111,7 +111,7 @@ public class TestWriteToReplica {
|
|||
|
||||
// test writeToTemporary
|
||||
@Test
|
||||
public void testWriteToTempoary() throws Exception {
|
||||
public void testWriteToTemporary() throws Exception {
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()).build();
|
||||
try {
|
||||
cluster.waitActive();
|
||||
|
@ -475,5 +475,28 @@ public class TestWriteToReplica {
|
|||
}
|
||||
|
||||
dataSet.createTemporary(StorageType.DEFAULT, blocks[NON_EXISTENT]);
|
||||
|
||||
try {
|
||||
dataSet.createTemporary(StorageType.DEFAULT, blocks[NON_EXISTENT]);
|
||||
Assert.fail("Should not have created a replica that had already been "
|
||||
+ "created " + blocks[NON_EXISTENT]);
|
||||
} catch (Exception e) {
|
||||
Assert.assertTrue(
|
||||
e.getMessage().contains(blocks[NON_EXISTENT].getBlockName()));
|
||||
Assert.assertTrue(e instanceof ReplicaAlreadyExistsException);
|
||||
}
|
||||
|
||||
long newGenStamp = blocks[NON_EXISTENT].getGenerationStamp() * 10;
|
||||
blocks[NON_EXISTENT].setGenerationStamp(newGenStamp);
|
||||
try {
|
||||
ReplicaInPipeline replicaInfo =
|
||||
dataSet.createTemporary(StorageType.DEFAULT, blocks[NON_EXISTENT]);
|
||||
Assert.assertTrue(replicaInfo.getGenerationStamp() == newGenStamp);
|
||||
Assert.assertTrue(
|
||||
replicaInfo.getBlockId() == blocks[NON_EXISTENT].getBlockId());
|
||||
} catch (ReplicaAlreadyExistsException e) {
|
||||
Assert.fail("createRbw() Should have removed the block with the older "
|
||||
+ "genstamp and replaced it with the newer one: " + blocks[NON_EXISTENT]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue