HDFS-8626. Add test case for branch-2.7

This commit is contained in:
Arpit Agarwal 2015-06-23 10:03:44 -07:00
parent 6b1a156e27
commit 4a8db40f3b
1 changed files with 45 additions and 0 deletions

View File

@ -27,10 +27,13 @@ import org.apache.hadoop.conf.Configuration;
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.FsVolumeSpi;
import org.apache.hadoop.ipc.RemoteException;
@ -41,10 +44,13 @@ import org.junit.After;
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.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.TimeoutException;
@ -276,6 +282,45 @@ public class TestRbwSpaceReservation {
}
}
@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().getVolumes().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