Skip to content

Commit

Permalink
HDDS-1908. TestMultiBlockWritesWithDnFailures is failing (#1282)
Browse files Browse the repository at this point in the history
  • Loading branch information
adoroszlai authored and bshashikant committed Aug 13, 2019
1 parent 454420e commit 0b507d2
Show file tree
Hide file tree
Showing 2 changed files with 67 additions and 74 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;

Expand Down Expand Up @@ -71,7 +72,6 @@ public class TestFailureHandlingByClient {
private String volumeName;
private String bucketName;
private String keyString;
private int maxRetries;

/**
* Create a MiniDFSCluster for testing.
Expand All @@ -82,7 +82,6 @@ public class TestFailureHandlingByClient {
*/
private void init() throws Exception {
conf = new OzoneConfiguration();
maxRetries = 100;
chunkSize = (int) OzoneConsts.MB;
blockSize = 4 * chunkSize;
conf.setTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, 5,
Expand Down Expand Up @@ -125,7 +124,8 @@ private void startCluster() throws Exception {
/**
* Shutdown MiniDFSCluster.
*/
private void shutdown() {
@After
public void shutdown() {
if (cluster != null) {
cluster.shutdown();
}
Expand Down Expand Up @@ -170,61 +170,6 @@ public void testBlockWritesWithDnFailures() throws Exception {
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
Assert.assertEquals(data.length, keyInfo.getDataSize());
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
Expand Down Expand Up @@ -270,7 +215,6 @@ public void testWriteSmallFile() throws Exception {
.getBlockID(), blockId);
Assert.assertEquals(data.getBytes().length, keyInfo.getDataSize());
validateData(keyName, data.getBytes());
shutdown();
}


Expand Down Expand Up @@ -331,7 +275,6 @@ public void testContainerExclusionWithClosedContainerException()
.getBlockID(), blockId);
Assert.assertEquals(2 * data.getBytes().length, keyInfo.getDataSize());
validateData(keyName, data.concat(data).getBytes());
shutdown();
}

@Test
Expand Down Expand Up @@ -394,7 +337,6 @@ public void testDatanodeExclusionWithMajorityCommit() throws Exception {
.getBlockID(), blockId);
Assert.assertEquals(3 * data.getBytes().length, keyInfo.getDataSize());
validateData(keyName, data.concat(data).concat(data).getBytes());
shutdown();
}


Expand Down Expand Up @@ -458,7 +400,6 @@ public void testPipelineExclusionWithPipelineFailure() throws Exception {
.getBlockID(), blockId);
Assert.assertEquals(3 * data.getBytes().length, keyInfo.getDataSize());
validateData(keyName, data.concat(data).concat(data).getBytes());
shutdown();
}

private OzoneOutputStream createKey(String keyName, ReplicationType type,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

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.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
Expand All @@ -30,12 +31,14 @@
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneClient;
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.OzoneOutputStream;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;

Expand All @@ -61,7 +64,6 @@ public class TestMultiBlockWritesWithDnFailures {
private String volumeName;
private String bucketName;
private String keyString;
private int maxRetries;

/**
* Create a MiniDFSCluster for testing.
Expand All @@ -70,23 +72,25 @@ public class TestMultiBlockWritesWithDnFailures {
*
* @throws IOException
*/
private void init() throws Exception {
private void startCluster(int datanodes) throws Exception {
conf = new OzoneConfiguration();
maxRetries = 100;
chunkSize = (int) OzoneConsts.MB;
blockSize = 4 * chunkSize;
conf.setTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, 5,
TimeUnit.SECONDS);
conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
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(
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY,
1, TimeUnit.SECONDS);
conf.setTimeDuration(
OzoneConfigKeys.DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY,
1, TimeUnit.SECONDS);

conf.setQuietMode(false);
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(6).build();
.setNumDatanodes(datanodes).build();
cluster.waitForClusterToBeReady();
//the easiest way to create an open container is creating a key
client = OzoneClientFactory.getClient(conf);
Expand All @@ -98,22 +102,19 @@ private void init() throws Exception {
objectStore.getVolume(volumeName).createBucket(bucketName);
}

private void startCluster() throws Exception {
init();
}

/**
* Shutdown MiniDFSCluster.
*/
private void shutdown() {
@After
public void shutdown() {
if (cluster != null) {
cluster.shutdown();
}
}

@Test
public void testMultiBlockWritesWithDnFailures() throws Exception {
startCluster();
startCluster(6);
String keyName = "ratis3";
OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0);
String data =
Expand Down Expand Up @@ -151,7 +152,58 @@ public void testMultiBlockWritesWithDnFailures() throws Exception {
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
Assert.assertEquals(2 * data.getBytes().length, keyInfo.getDataSize());
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,
Expand Down

0 comments on commit 0b507d2

Please sign in to comment.