|
@@ -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;
|
|
@@ -36,12 +37,11 @@ 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.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
import java.util.UUID;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
@@ -54,16 +54,16 @@ import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTER
|
|
|
*/
|
|
|
public class TestFailureHandlingByClient {
|
|
|
|
|
|
- private static MiniOzoneCluster cluster;
|
|
|
- private static OzoneConfiguration conf;
|
|
|
- private static OzoneClient client;
|
|
|
- private static ObjectStore objectStore;
|
|
|
- private static int chunkSize;
|
|
|
- private static int blockSize;
|
|
|
- private static String volumeName;
|
|
|
- private static String bucketName;
|
|
|
- private static String keyString;
|
|
|
- private static int maxRetries;
|
|
|
+ private MiniOzoneCluster cluster;
|
|
|
+ private OzoneConfiguration conf;
|
|
|
+ private OzoneClient client;
|
|
|
+ private ObjectStore objectStore;
|
|
|
+ private int chunkSize;
|
|
|
+ private int blockSize;
|
|
|
+ private String volumeName;
|
|
|
+ private String bucketName;
|
|
|
+ private String keyString;
|
|
|
+ private int maxRetries;
|
|
|
|
|
|
/**
|
|
|
* Create a MiniDFSCluster for testing.
|
|
@@ -72,8 +72,7 @@ public class TestFailureHandlingByClient {
|
|
|
*
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- @Before
|
|
|
- public void init() throws Exception {
|
|
|
+ private void init() throws Exception {
|
|
|
conf = new OzoneConfiguration();
|
|
|
maxRetries = 100;
|
|
|
chunkSize = (int) OzoneConsts.MB;
|
|
@@ -101,11 +100,14 @@ public class TestFailureHandlingByClient {
|
|
|
objectStore.getVolume(volumeName).createBucket(bucketName);
|
|
|
}
|
|
|
|
|
|
+ private void startCluster() throws Exception {
|
|
|
+ init();
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Shutdown MiniDFSCluster.
|
|
|
*/
|
|
|
- @After
|
|
|
- public void shutdown() {
|
|
|
+ private void shutdown() {
|
|
|
if (cluster != null) {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
@@ -113,6 +115,7 @@ public class TestFailureHandlingByClient {
|
|
|
|
|
|
@Test
|
|
|
public void testBlockWritesWithDnFailures() throws Exception {
|
|
|
+ startCluster();
|
|
|
String keyName = UUID.randomUUID().toString();
|
|
|
OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0);
|
|
|
byte[] data =
|
|
@@ -148,10 +151,12 @@ public class TestFailureHandlingByClient {
|
|
|
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
|
|
|
Assert.assertEquals(data.length, keyInfo.getDataSize());
|
|
|
validateData(keyName, data);
|
|
|
+ shutdown();
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testMultiBlockWritesWithDnFailures() throws Exception {
|
|
|
+ startCluster();
|
|
|
String keyName = "ratis3";
|
|
|
OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0);
|
|
|
String data =
|
|
@@ -188,11 +193,13 @@ public class TestFailureHandlingByClient {
|
|
|
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();
|
|
|
String keyName = UUID.randomUUID().toString();
|
|
|
OzoneOutputStream key =
|
|
|
createKey(keyName, ReplicationType.RATIS, 6 * blockSize);
|
|
@@ -232,8 +239,235 @@ public class TestFailureHandlingByClient {
|
|
|
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
|
|
|
Assert.assertEquals(3 * data.getBytes().length, keyInfo.getDataSize());
|
|
|
validateData(keyName, data.concat(data).concat(data).getBytes());
|
|
|
+ shutdown();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testWriteSmallFile() throws Exception {
|
|
|
+ startCluster();
|
|
|
+ String keyName = UUID.randomUUID().toString();
|
|
|
+ OzoneOutputStream key =
|
|
|
+ createKey(keyName, ReplicationType.RATIS, 0);
|
|
|
+ String data = ContainerTestHelper
|
|
|
+ .getFixedLengthString(keyString, chunkSize/2);
|
|
|
+ key.write(data.getBytes());
|
|
|
+ // get the name of a valid container
|
|
|
+ Assert.assertTrue(key.getOutputStream() instanceof KeyOutputStream);
|
|
|
+ KeyOutputStream keyOutputStream =
|
|
|
+ (KeyOutputStream) key.getOutputStream();
|
|
|
+ List<OmKeyLocationInfo> locationInfoList =
|
|
|
+ keyOutputStream.getLocationInfoList();
|
|
|
+ long containerId = locationInfoList.get(0).getContainerID();
|
|
|
+ BlockID blockId = locationInfoList.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));
|
|
|
+ cluster.shutdownHddsDatanode(datanodes.get(1));
|
|
|
+ key.close();
|
|
|
+ // this will throw AlreadyClosedException and and current stream
|
|
|
+ // will be discarded and write a new block
|
|
|
+ OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
|
|
|
+ .setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
|
|
|
+ .setFactor(HddsProtos.ReplicationFactor.THREE).setKeyName(keyName)
|
|
|
+ .build();
|
|
|
+ OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
|
|
|
+
|
|
|
+ // Make sure a new block is written
|
|
|
+ Assert.assertNotEquals(
|
|
|
+ keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly().get(0)
|
|
|
+ .getBlockID(), blockId);
|
|
|
+ Assert.assertEquals(data.getBytes().length, keyInfo.getDataSize());
|
|
|
+ validateData(keyName, data.getBytes());
|
|
|
+ shutdown();
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testContainerExclusionWithClosedContainerException()
|
|
|
+ throws Exception {
|
|
|
+ startCluster();
|
|
|
+ String keyName = UUID.randomUUID().toString();
|
|
|
+ OzoneOutputStream key =
|
|
|
+ createKey(keyName, ReplicationType.RATIS, blockSize);
|
|
|
+ String data = ContainerTestHelper
|
|
|
+ .getFixedLengthString(keyString, chunkSize);
|
|
|
+
|
|
|
+ // get the name of a valid container
|
|
|
+ Assert.assertTrue(key.getOutputStream() instanceof KeyOutputStream);
|
|
|
+ KeyOutputStream keyOutputStream =
|
|
|
+ (KeyOutputStream) key.getOutputStream();
|
|
|
+ List<OmKeyLocationInfo> locationInfoList =
|
|
|
+ keyOutputStream.getLocationInfoList();
|
|
|
+
|
|
|
+ // Assert that 1 block will be preallocated
|
|
|
+ Assert.assertEquals(1, locationInfoList.size());
|
|
|
+ key.write(data.getBytes());
|
|
|
+ key.flush();
|
|
|
+ long containerId = locationInfoList.get(0).getContainerID();
|
|
|
+ BlockID blockId = locationInfoList.get(0).getBlockID();
|
|
|
+ List<Long> containerIdList = new ArrayList<>();
|
|
|
+ containerIdList.add(containerId);
|
|
|
+
|
|
|
+ // below check will assert if the container does not get closed
|
|
|
+ ContainerTestHelper
|
|
|
+ .waitForContainerClose(cluster, containerIdList.toArray(new Long[0]));
|
|
|
+
|
|
|
+ // This write will hit ClosedContainerException and this container should
|
|
|
+ // will be added in the excludelist
|
|
|
+ key.write(data.getBytes());
|
|
|
+ key.flush();
|
|
|
+
|
|
|
+ Assert.assertTrue(keyOutputStream.getExcludeList().getContainerIds()
|
|
|
+ .contains(ContainerID.valueof(containerId)));
|
|
|
+ Assert.assertTrue(
|
|
|
+ keyOutputStream.getExcludeList().getDatanodes().isEmpty());
|
|
|
+ Assert.assertTrue(
|
|
|
+ keyOutputStream.getExcludeList().getPipelineIds().isEmpty());
|
|
|
+
|
|
|
+ // The close will just write to the buffer
|
|
|
+ key.close();
|
|
|
+ OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
|
|
|
+ .setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
|
|
|
+ .setFactor(HddsProtos.ReplicationFactor.THREE).setKeyName(keyName)
|
|
|
+ .build();
|
|
|
+ OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
|
|
|
+
|
|
|
+ // Make sure a new block is written
|
|
|
+ Assert.assertNotEquals(
|
|
|
+ keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly().get(0)
|
|
|
+ .getBlockID(), blockId);
|
|
|
+ Assert.assertEquals(2 * data.getBytes().length, keyInfo.getDataSize());
|
|
|
+ validateData(keyName, data.concat(data).getBytes());
|
|
|
+ shutdown();
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testDatanodeExclusionWithMajorityCommit() throws Exception {
|
|
|
+ startCluster();
|
|
|
+ String keyName = UUID.randomUUID().toString();
|
|
|
+ OzoneOutputStream key =
|
|
|
+ createKey(keyName, ReplicationType.RATIS, blockSize);
|
|
|
+ String data = ContainerTestHelper
|
|
|
+ .getFixedLengthString(keyString, chunkSize);
|
|
|
+
|
|
|
+ // get the name of a valid container
|
|
|
+ Assert.assertTrue(key.getOutputStream() instanceof KeyOutputStream);
|
|
|
+ KeyOutputStream keyOutputStream =
|
|
|
+ (KeyOutputStream) key.getOutputStream();
|
|
|
+ List<OmKeyLocationInfo> locationInfoList =
|
|
|
+ keyOutputStream.getLocationInfoList();
|
|
|
+
|
|
|
+ // Assert that 1 block will be preallocated
|
|
|
+ Assert.assertEquals(1, locationInfoList.size());
|
|
|
+ key.write(data.getBytes());
|
|
|
+ key.flush();
|
|
|
+ long containerId = locationInfoList.get(0).getContainerID();
|
|
|
+ BlockID blockId = locationInfoList.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();
|
|
|
+
|
|
|
+ // shutdown 1 datanode. This will make sure the 2 way commit happens for
|
|
|
+ // next write ops.
|
|
|
+ cluster.shutdownHddsDatanode(datanodes.get(0));
|
|
|
+
|
|
|
+ key.write(data.getBytes());
|
|
|
+ key.write(data.getBytes());
|
|
|
+ // The close will just write to the buffer
|
|
|
+ key.close();
|
|
|
+ Assert.assertTrue(keyOutputStream.getExcludeList().getDatanodes()
|
|
|
+ .contains(datanodes.get(0)));
|
|
|
+ Assert.assertTrue(
|
|
|
+ keyOutputStream.getExcludeList().getContainerIds().isEmpty());
|
|
|
+ Assert.assertTrue(
|
|
|
+ keyOutputStream.getExcludeList().getPipelineIds().isEmpty());
|
|
|
+
|
|
|
+ OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
|
|
|
+ .setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
|
|
|
+ .setFactor(HddsProtos.ReplicationFactor.THREE).setKeyName(keyName)
|
|
|
+ .build();
|
|
|
+ OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
|
|
|
+
|
|
|
+ // Make sure a new block is written
|
|
|
+ Assert.assertNotEquals(
|
|
|
+ keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly().get(0)
|
|
|
+ .getBlockID(), blockId);
|
|
|
+ Assert.assertEquals(3 * data.getBytes().length, keyInfo.getDataSize());
|
|
|
+ validateData(keyName, data.concat(data).concat(data).getBytes());
|
|
|
+ shutdown();
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testPipelineExclusionWithPipelineFailure() throws Exception {
|
|
|
+ startCluster();
|
|
|
+ String keyName = UUID.randomUUID().toString();
|
|
|
+ OzoneOutputStream key =
|
|
|
+ createKey(keyName, ReplicationType.RATIS, blockSize);
|
|
|
+ String data = ContainerTestHelper
|
|
|
+ .getFixedLengthString(keyString, chunkSize);
|
|
|
+
|
|
|
+ // get the name of a valid container
|
|
|
+ Assert.assertTrue(key.getOutputStream() instanceof KeyOutputStream);
|
|
|
+ KeyOutputStream keyOutputStream =
|
|
|
+ (KeyOutputStream) key.getOutputStream();
|
|
|
+ List<OmKeyLocationInfo> locationInfoList =
|
|
|
+ keyOutputStream.getLocationInfoList();
|
|
|
+
|
|
|
+ // Assert that 1 block will be preallocated
|
|
|
+ Assert.assertEquals(1, locationInfoList.size());
|
|
|
+ key.write(data.getBytes());
|
|
|
+ key.flush();
|
|
|
+ long containerId = locationInfoList.get(0).getContainerID();
|
|
|
+ BlockID blockId = locationInfoList.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();
|
|
|
+
|
|
|
+ // Two nodes, next write will hit AlraedyClosedException , the pipeline
|
|
|
+ // will be added in the exclude list
|
|
|
+ cluster.shutdownHddsDatanode(datanodes.get(0));
|
|
|
+ cluster.shutdownHddsDatanode(datanodes.get(1));
|
|
|
+
|
|
|
+ key.write(data.getBytes());
|
|
|
+ key.write(data.getBytes());
|
|
|
+ // The close will just write to the buffer
|
|
|
+ key.close();
|
|
|
+ Assert.assertTrue(keyOutputStream.getExcludeList().getPipelineIds()
|
|
|
+ .contains(pipeline.getId()));
|
|
|
+ Assert.assertTrue(
|
|
|
+ keyOutputStream.getExcludeList().getContainerIds().isEmpty());
|
|
|
+ Assert.assertTrue(
|
|
|
+ keyOutputStream.getExcludeList().getDatanodes().isEmpty());
|
|
|
+
|
|
|
+ OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
|
|
|
+ .setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
|
|
|
+ .setFactor(HddsProtos.ReplicationFactor.THREE).setKeyName(keyName)
|
|
|
+ .build();
|
|
|
+ OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
|
|
|
+
|
|
|
+ // Make sure a new block is written
|
|
|
+ Assert.assertNotEquals(
|
|
|
+ keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly().get(0)
|
|
|
+ .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,
|
|
|
long size) throws Exception {
|