HDDS-1908. TestMultiBlockWritesWithDnFailures is failing (#1282)

This commit is contained in:
Doroszlai, Attila 2019-08-13 12:08:55 +02:00 committed by bshashikant
parent 454420e4f2
commit 0b507d2ddf
2 changed files with 67 additions and 74 deletions

View File

@ -42,6 +42,7 @@
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -71,7 +72,6 @@ public class TestFailureHandlingByClient {
private String volumeName; private String volumeName;
private String bucketName; private String bucketName;
private String keyString; private String keyString;
private int maxRetries;
/** /**
* Create a MiniDFSCluster for testing. * Create a MiniDFSCluster for testing.
@ -82,7 +82,6 @@ public class TestFailureHandlingByClient {
*/ */
private void init() throws Exception { private void init() throws Exception {
conf = new OzoneConfiguration(); conf = new OzoneConfiguration();
maxRetries = 100;
chunkSize = (int) OzoneConsts.MB; chunkSize = (int) OzoneConsts.MB;
blockSize = 4 * chunkSize; blockSize = 4 * chunkSize;
conf.setTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, 5, conf.setTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, 5,
@ -125,7 +124,8 @@ private void startCluster() throws Exception {
/** /**
* Shutdown MiniDFSCluster. * Shutdown MiniDFSCluster.
*/ */
private void shutdown() { @After
public void shutdown() {
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }
@ -170,61 +170,6 @@ public void testBlockWritesWithDnFailures() throws Exception {
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs); OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
Assert.assertEquals(data.length, keyInfo.getDataSize()); Assert.assertEquals(data.length, keyInfo.getDataSize());
validateData(keyName, data); validateData(keyName, data);
shutdown();
}
@Test
public void testMultiBlockWritesWithIntermittentDnFailures()
throws Exception {
startCluster();
String keyName = UUID.randomUUID().toString();
OzoneOutputStream key =
createKey(keyName, ReplicationType.RATIS, 6 * blockSize);
String data = ContainerTestHelper
.getFixedLengthString(keyString, blockSize + chunkSize);
key.write(data.getBytes());
// get the name of a valid container
Assert.assertTrue(key.getOutputStream() instanceof KeyOutputStream);
KeyOutputStream keyOutputStream =
(KeyOutputStream) key.getOutputStream();
List<BlockOutputStreamEntry> streamEntryList =
keyOutputStream.getStreamEntries();
// Assert that 6 block will be preallocated
Assert.assertEquals(6, streamEntryList.size());
key.write(data.getBytes());
key.flush();
long containerId = streamEntryList.get(0).getBlockID().getContainerID();
BlockID blockId = streamEntryList.get(0).getBlockID();
ContainerInfo container =
cluster.getStorageContainerManager().getContainerManager()
.getContainer(ContainerID.valueof(containerId));
Pipeline pipeline =
cluster.getStorageContainerManager().getPipelineManager()
.getPipeline(container.getPipelineID());
List<DatanodeDetails> datanodes = pipeline.getNodes();
cluster.shutdownHddsDatanode(datanodes.get(0));
// The write will fail but exception will be handled and length will be
// updated correctly in OzoneManager once the steam is closed
key.write(data.getBytes());
// shutdown the second datanode
cluster.shutdownHddsDatanode(datanodes.get(1));
key.write(data.getBytes());
key.close();
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
.setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
.setFactor(HddsProtos.ReplicationFactor.THREE).setKeyName(keyName)
.setRefreshPipeline(true)
.build();
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
Assert.assertEquals(4 * data.getBytes().length, keyInfo.getDataSize());
validateData(keyName,
data.concat(data).concat(data).concat(data).getBytes());
shutdown();
} }
@Test @Test
@ -270,7 +215,6 @@ public void testWriteSmallFile() throws Exception {
.getBlockID(), blockId); .getBlockID(), blockId);
Assert.assertEquals(data.getBytes().length, keyInfo.getDataSize()); Assert.assertEquals(data.getBytes().length, keyInfo.getDataSize());
validateData(keyName, data.getBytes()); validateData(keyName, data.getBytes());
shutdown();
} }
@ -331,7 +275,6 @@ public void testContainerExclusionWithClosedContainerException()
.getBlockID(), blockId); .getBlockID(), blockId);
Assert.assertEquals(2 * data.getBytes().length, keyInfo.getDataSize()); Assert.assertEquals(2 * data.getBytes().length, keyInfo.getDataSize());
validateData(keyName, data.concat(data).getBytes()); validateData(keyName, data.concat(data).getBytes());
shutdown();
} }
@Test @Test
@ -394,7 +337,6 @@ public void testDatanodeExclusionWithMajorityCommit() throws Exception {
.getBlockID(), blockId); .getBlockID(), blockId);
Assert.assertEquals(3 * data.getBytes().length, keyInfo.getDataSize()); Assert.assertEquals(3 * data.getBytes().length, keyInfo.getDataSize());
validateData(keyName, data.concat(data).concat(data).getBytes()); validateData(keyName, data.concat(data).concat(data).getBytes());
shutdown();
} }
@ -458,7 +400,6 @@ public void testPipelineExclusionWithPipelineFailure() throws Exception {
.getBlockID(), blockId); .getBlockID(), blockId);
Assert.assertEquals(3 * data.getBytes().length, keyInfo.getDataSize()); Assert.assertEquals(3 * data.getBytes().length, keyInfo.getDataSize());
validateData(keyName, data.concat(data).concat(data).getBytes()); validateData(keyName, data.concat(data).concat(data).getBytes());
shutdown();
} }
private OzoneOutputStream createKey(String keyName, ReplicationType type, private OzoneOutputStream createKey(String keyName, ReplicationType type,

View File

@ -17,6 +17,7 @@
package org.apache.hadoop.ozone.client.rpc; package org.apache.hadoop.ozone.client.rpc;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@ -30,12 +31,14 @@
import org.apache.hadoop.ozone.client.ObjectStore; import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry;
import org.apache.hadoop.ozone.client.io.KeyOutputStream; import org.apache.hadoop.ozone.client.io.KeyOutputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -61,7 +64,6 @@ public class TestMultiBlockWritesWithDnFailures {
private String volumeName; private String volumeName;
private String bucketName; private String bucketName;
private String keyString; private String keyString;
private int maxRetries;
/** /**
* Create a MiniDFSCluster for testing. * Create a MiniDFSCluster for testing.
@ -70,23 +72,25 @@ public class TestMultiBlockWritesWithDnFailures {
* *
* @throws IOException * @throws IOException
*/ */
private void init() throws Exception { private void startCluster(int datanodes) throws Exception {
conf = new OzoneConfiguration(); conf = new OzoneConfiguration();
maxRetries = 100;
chunkSize = (int) OzoneConsts.MB; chunkSize = (int) OzoneConsts.MB;
blockSize = 4 * chunkSize; blockSize = 4 * chunkSize;
conf.setTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, 5, conf.setTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, 5,
TimeUnit.SECONDS); TimeUnit.SECONDS);
conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS); conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 100, TimeUnit.SECONDS); conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 100, TimeUnit.SECONDS);
conf.setInt(OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY, 5); conf.setInt(OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY, 10);
conf.setTimeDuration( conf.setTimeDuration(
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY, OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY,
1, TimeUnit.SECONDS); 1, TimeUnit.SECONDS);
conf.setTimeDuration(
OzoneConfigKeys.DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY,
1, TimeUnit.SECONDS);
conf.setQuietMode(false); conf.setQuietMode(false);
cluster = MiniOzoneCluster.newBuilder(conf) cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(6).build(); .setNumDatanodes(datanodes).build();
cluster.waitForClusterToBeReady(); cluster.waitForClusterToBeReady();
//the easiest way to create an open container is creating a key //the easiest way to create an open container is creating a key
client = OzoneClientFactory.getClient(conf); client = OzoneClientFactory.getClient(conf);
@ -98,14 +102,11 @@ private void init() throws Exception {
objectStore.getVolume(volumeName).createBucket(bucketName); objectStore.getVolume(volumeName).createBucket(bucketName);
} }
private void startCluster() throws Exception {
init();
}
/** /**
* Shutdown MiniDFSCluster. * Shutdown MiniDFSCluster.
*/ */
private void shutdown() { @After
public void shutdown() {
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }
@ -113,7 +114,7 @@ private void shutdown() {
@Test @Test
public void testMultiBlockWritesWithDnFailures() throws Exception { public void testMultiBlockWritesWithDnFailures() throws Exception {
startCluster(); startCluster(6);
String keyName = "ratis3"; String keyName = "ratis3";
OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0); OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0);
String data = String data =
@ -151,7 +152,58 @@ public void testMultiBlockWritesWithDnFailures() throws Exception {
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs); OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
Assert.assertEquals(2 * data.getBytes().length, keyInfo.getDataSize()); Assert.assertEquals(2 * data.getBytes().length, keyInfo.getDataSize());
validateData(keyName, data.concat(data).getBytes()); validateData(keyName, data.concat(data).getBytes());
shutdown(); }
@Test
public void testMultiBlockWritesWithIntermittentDnFailures()
throws Exception {
startCluster(10);
String keyName = UUID.randomUUID().toString();
OzoneOutputStream key =
createKey(keyName, ReplicationType.RATIS, 6 * blockSize);
String data = ContainerTestHelper
.getFixedLengthString(keyString, blockSize + chunkSize);
key.write(data.getBytes());
// get the name of a valid container
Assert.assertTrue(key.getOutputStream() instanceof KeyOutputStream);
KeyOutputStream keyOutputStream =
(KeyOutputStream) key.getOutputStream();
List<BlockOutputStreamEntry> streamEntryList =
keyOutputStream.getStreamEntries();
// Assert that 6 block will be preallocated
Assert.assertEquals(6, streamEntryList.size());
key.write(data.getBytes());
key.flush();
long containerId = streamEntryList.get(0).getBlockID().getContainerID();
BlockID blockId = streamEntryList.get(0).getBlockID();
ContainerInfo container =
cluster.getStorageContainerManager().getContainerManager()
.getContainer(ContainerID.valueof(containerId));
Pipeline pipeline =
cluster.getStorageContainerManager().getPipelineManager()
.getPipeline(container.getPipelineID());
List<DatanodeDetails> datanodes = pipeline.getNodes();
cluster.shutdownHddsDatanode(datanodes.get(0));
// The write will fail but exception will be handled and length will be
// updated correctly in OzoneManager once the steam is closed
key.write(data.getBytes());
// shutdown the second datanode
cluster.shutdownHddsDatanode(datanodes.get(1));
key.write(data.getBytes());
key.close();
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
.setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
.setFactor(HddsProtos.ReplicationFactor.THREE).setKeyName(keyName)
.setRefreshPipeline(true)
.build();
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
Assert.assertEquals(4 * data.getBytes().length, keyInfo.getDataSize());
validateData(keyName,
data.concat(data).concat(data).concat(data).getBytes());
} }
private OzoneOutputStream createKey(String keyName, ReplicationType type, private OzoneOutputStream createKey(String keyName, ReplicationType type,