HDFS-8626. Reserved RBW space is not released if creation of RBW File fails. (Contributed by kanaka kumar avvaru)

This commit is contained in:
Arpit Agarwal 2015-06-19 18:23:34 -07:00
parent 49f5d20efe
commit 55278c012f
3 changed files with 54 additions and 1 deletions

View File

@ -1064,6 +1064,9 @@ Release 2.7.1 - UNRELEASED
HDFS-8633. Fix setting of dfs.datanode.readahead.bytes in hdfs-default.xml
to match DFSConfigKeys. (Ray Chiang via Yongjun Zhang)
HDFS-8626. Reserved RBW space is not released if creation of RBW File
fails. (kanaka kumar avvaru via Arpit Agarwal)
Release 2.7.0 - 2015-04-20
INCOMPATIBLE CHANGES

View File

@ -780,7 +780,12 @@ public FsDatasetSpi getDataset() {
File createRbwFile(String bpid, Block b) throws IOException {
checkReference();
reserveSpaceForRbw(b.getNumBytes());
return getBlockPoolSlice(bpid).createRbwFile(b);
try {
return getBlockPoolSlice(bpid).createRbwFile(b);
} catch (IOException exception) {
releaseReservedSpace(b.getNumBytes());
throw exception;
}
}
/**

View File

@ -27,10 +27,13 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.*;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
@ -42,9 +45,12 @@
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.mockito.Mockito;
import java.io.IOException;
import java.io.OutputStream;
import java.lang.reflect.Field;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.TimeoutException;
@ -279,6 +285,45 @@ public Boolean get() {
}
}
@SuppressWarnings("unchecked")
@Test(timeout = 30000)
public void testRBWFileCreationError() throws Exception {
final short replication = 1;
startCluster(BLOCK_SIZE, replication, -1);
final FsVolumeImpl fsVolumeImpl = (FsVolumeImpl) cluster.getDataNodes()
.get(0).getFSDataset().getFsVolumeReferences().get(0);
final String methodName = GenericTestUtils.getMethodName();
final Path file = new Path("/" + methodName + ".01.dat");
// Mock BlockPoolSlice so that RBW file creation gives IOExcception
BlockPoolSlice blockPoolSlice = Mockito.mock(BlockPoolSlice.class);
Mockito.when(blockPoolSlice.createRbwFile((Block) Mockito.any()))
.thenThrow(new IOException("Synthetic IO Exception Throgh MOCK"));
Field field = FsVolumeImpl.class.getDeclaredField("bpSlices");
field.setAccessible(true);
Map<String, BlockPoolSlice> bpSlices = (Map<String, BlockPoolSlice>) field
.get(fsVolumeImpl);
bpSlices.put(fsVolumeImpl.getBlockPoolList()[0], blockPoolSlice);
try {
// Write 1 byte to the file
FSDataOutputStream os = fs.create(file, replication);
os.write(new byte[1]);
os.hsync();
os.close();
fail("Expecting IOException file creation failure");
} catch (IOException e) {
// Exception can be ignored (expected)
}
// Ensure RBW space reserved is released
assertTrue("Expected ZERO but got " + fsVolumeImpl.getReservedForRbw(),
fsVolumeImpl.getReservedForRbw() == 0);
}
/**
* Stress test to ensure we are not leaking reserved space.
* @throws IOException