|
@@ -28,6 +28,7 @@ import org.apache.hadoop.ozone.OzoneConsts;
|
|
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.OzoneKeyDetails;
|
|
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.common.impl.ContainerData;
|
|
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
|
|
@@ -58,6 +59,8 @@ import static org.apache.hadoop.hdds.scm.ScmConfigKeys.
|
|
HDDS_SCM_WATCHER_TIMEOUT;
|
|
HDDS_SCM_WATCHER_TIMEOUT;
|
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.
|
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.
|
|
OZONE_SCM_STALENODE_INTERVAL;
|
|
OZONE_SCM_STALENODE_INTERVAL;
|
|
|
|
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.
|
|
|
|
+ OZONE_SCM_PIPELINE_DESTROY_TIMEOUT;
|
|
import static org.hamcrest.core.Is.is;
|
|
import static org.hamcrest.core.Is.is;
|
|
import static org.junit.Assert.assertThat;
|
|
import static org.junit.Assert.assertThat;
|
|
|
|
|
|
@@ -96,6 +99,8 @@ public class TestContainerStateMachineFailures {
|
|
TimeUnit.MILLISECONDS);
|
|
TimeUnit.MILLISECONDS);
|
|
conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
|
|
conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
|
|
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
|
|
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
|
|
|
|
+ conf.setTimeDuration(OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, 10,
|
|
|
|
+ TimeUnit.SECONDS);
|
|
conf.setQuietMode(false);
|
|
conf.setQuietMode(false);
|
|
cluster =
|
|
cluster =
|
|
MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).setHbInterval(200)
|
|
MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).setHbInterval(200)
|
|
@@ -126,10 +131,14 @@ public class TestContainerStateMachineFailures {
|
|
objectStore.getVolume(volumeName).getBucket(bucketName)
|
|
objectStore.getVolume(volumeName).getBucket(bucketName)
|
|
.createKey("ratis", 1024, ReplicationType.RATIS,
|
|
.createKey("ratis", 1024, ReplicationType.RATIS,
|
|
ReplicationFactor.ONE, new HashMap<>());
|
|
ReplicationFactor.ONE, new HashMap<>());
|
|
|
|
+ byte[] testData = "ratis".getBytes();
|
|
|
|
+ long written = 0;
|
|
// First write and flush creates a container in the datanode
|
|
// First write and flush creates a container in the datanode
|
|
- key.write("ratis".getBytes());
|
|
|
|
|
|
+ key.write(testData);
|
|
|
|
+ written += testData.length;
|
|
key.flush();
|
|
key.flush();
|
|
- key.write("ratis".getBytes());
|
|
|
|
|
|
+ key.write(testData);
|
|
|
|
+ written += testData.length;
|
|
|
|
|
|
//get the name of a valid container
|
|
//get the name of a valid container
|
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName).
|
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName).
|
|
@@ -148,13 +157,7 @@ public class TestContainerStateMachineFailures {
|
|
.getContainer().getContainerSet()
|
|
.getContainer().getContainerSet()
|
|
.getContainer(omKeyLocationInfo.getContainerID()).getContainerData()
|
|
.getContainer(omKeyLocationInfo.getContainerID()).getContainerData()
|
|
.getContainerPath()));
|
|
.getContainerPath()));
|
|
- try {
|
|
|
|
- key.close();
|
|
|
|
- Assert.fail();
|
|
|
|
- } catch (IOException ioe) {
|
|
|
|
- Assert.assertTrue(ioe.getMessage().contains(
|
|
|
|
- "Requested operation not allowed as ContainerState is UNHEALTHY"));
|
|
|
|
- }
|
|
|
|
|
|
+ key.close();
|
|
long containerID = omKeyLocationInfo.getContainerID();
|
|
long containerID = omKeyLocationInfo.getContainerID();
|
|
|
|
|
|
// Make sure the container is marked unhealthy
|
|
// Make sure the container is marked unhealthy
|
|
@@ -170,27 +173,28 @@ public class TestContainerStateMachineFailures {
|
|
HddsDispatcher dispatcher = (HddsDispatcher) ozoneContainer.getDispatcher();
|
|
HddsDispatcher dispatcher = (HddsDispatcher) ozoneContainer.getDispatcher();
|
|
Assert.assertTrue(dispatcher.getMissingContainerSet().isEmpty());
|
|
Assert.assertTrue(dispatcher.getMissingContainerSet().isEmpty());
|
|
|
|
|
|
- // restart the hdds datanode and see if the container is listed in the
|
|
|
|
- // in the missing container set and not in the regular set
|
|
|
|
|
|
+ // restart the hdds datanode, container should not in the regular set
|
|
cluster.restartHddsDatanode(0, true);
|
|
cluster.restartHddsDatanode(0, true);
|
|
ozoneContainer = cluster.getHddsDatanodes().get(0)
|
|
ozoneContainer = cluster.getHddsDatanodes().get(0)
|
|
.getDatanodeStateMachine().getContainer();
|
|
.getDatanodeStateMachine().getContainer();
|
|
- dispatcher = (HddsDispatcher) ozoneContainer.getDispatcher();
|
|
|
|
-
|
|
|
|
Assert
|
|
Assert
|
|
.assertNull(ozoneContainer.getContainerSet().getContainer(containerID));
|
|
.assertNull(ozoneContainer.getContainerSet().getContainer(containerID));
|
|
- Assert.assertTrue(dispatcher.getMissingContainerSet()
|
|
|
|
- .contains(containerID));
|
|
|
|
- ContainerProtos.ContainerCommandRequestProto.Builder request =
|
|
|
|
- ContainerProtos.ContainerCommandRequestProto.newBuilder();
|
|
|
|
- request.setCmdType(ContainerProtos.Type.CreateContainer);
|
|
|
|
- request.setContainerID(containerID);
|
|
|
|
- request.setCreateContainer(
|
|
|
|
- ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
|
|
|
|
- request.setDatanodeUuid(
|
|
|
|
- cluster.getHddsDatanodes().get(0).getDatanodeDetails().getUuidString());
|
|
|
|
- Assert.assertEquals(ContainerProtos.Result.CONTAINER_MISSING,
|
|
|
|
- dispatcher.dispatch(request.build(), null).getResult());
|
|
|
|
|
|
+
|
|
|
|
+ OzoneKeyDetails keyDetails = objectStore.getVolume(volumeName)
|
|
|
|
+ .getBucket(bucketName).getKey("ratis");
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Ensure length of data stored in key is equal to number of bytes written.
|
|
|
|
+ */
|
|
|
|
+ Assert.assertTrue("Number of bytes stored in the key is not equal " +
|
|
|
|
+ "to number of bytes written.", keyDetails.getDataSize() == written);
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Pending data from the second write should get written to a new container
|
|
|
|
+ * during key.close() because the first container is UNHEALTHY by that time
|
|
|
|
+ */
|
|
|
|
+ Assert.assertTrue("Expect Key to be stored in 2 separate containers",
|
|
|
|
+ keyDetails.getOzoneKeyLocations().size() == 2);
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
@@ -224,13 +228,9 @@ public class TestContainerStateMachineFailures {
|
|
(KeyValueContainerData) containerData;
|
|
(KeyValueContainerData) containerData;
|
|
// delete the container db file
|
|
// delete the container db file
|
|
FileUtil.fullyDelete(new File(keyValueContainerData.getChunksPath()));
|
|
FileUtil.fullyDelete(new File(keyValueContainerData.getChunksPath()));
|
|
- try {
|
|
|
|
- key.close();
|
|
|
|
- Assert.fail();
|
|
|
|
- } catch (IOException ioe) {
|
|
|
|
- Assert.assertTrue(ioe.getMessage().contains(
|
|
|
|
- "Requested operation not allowed as ContainerState is UNHEALTHY"));
|
|
|
|
- }
|
|
|
|
|
|
+
|
|
|
|
+ key.close();
|
|
|
|
+
|
|
long containerID = omKeyLocationInfo.getContainerID();
|
|
long containerID = omKeyLocationInfo.getContainerID();
|
|
|
|
|
|
// Make sure the container is marked unhealthy
|
|
// Make sure the container is marked unhealthy
|