|
@@ -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;
|
|
@@ -30,12 +31,14 @@ import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
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;
|
|
|
|
|
@@ -61,7 +64,6 @@ public class TestMultiBlockWritesWithDnFailures {
|
|
|
private String volumeName;
|
|
|
private String bucketName;
|
|
|
private String keyString;
|
|
|
- private int maxRetries;
|
|
|
|
|
|
/**
|
|
|
* Create a MiniDFSCluster for testing.
|
|
@@ -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);
|
|
@@ -98,14 +102,11 @@ public class TestMultiBlockWritesWithDnFailures {
|
|
|
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();
|
|
|
}
|
|
@@ -113,7 +114,7 @@ public class TestMultiBlockWritesWithDnFailures {
|
|
|
|
|
|
@Test
|
|
|
public void testMultiBlockWritesWithDnFailures() throws Exception {
|
|
|
- startCluster();
|
|
|
+ startCluster(6);
|
|
|
String keyName = "ratis3";
|
|
|
OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0);
|
|
|
String data =
|
|
@@ -151,7 +152,58 @@ public class TestMultiBlockWritesWithDnFailures {
|
|
|
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,
|