|
@@ -17,11 +17,13 @@
|
|
|
|
|
|
package org.apache.hadoop.hdds.scm.container;
|
|
|
|
|
|
+import org.apache.commons.lang3.RandomUtils;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
|
|
+import org.apache.hadoop.hdds.scm.container.common.helpers
|
|
|
+ .ContainerWithPipeline;
|
|
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
|
|
@@ -33,12 +35,12 @@ import org.junit.Test;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
-import java.util.Random;
|
|
|
|
|
|
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATE;
|
|
|
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATED;
|
|
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT;
|
|
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
|
|
|
+import static org.apache.hadoop.hdds.scm.events.SCMEvents.CLOSE_CONTAINER;
|
|
|
|
|
|
/**
|
|
|
* Tests the closeContainerEventHandler class.
|
|
@@ -65,7 +67,7 @@ public class TestCloseContainerEventHandler {
|
|
|
nodeManager = new MockNodeManager(true, 10);
|
|
|
mapping = new ContainerMapping(configuration, nodeManager, 128);
|
|
|
eventQueue = new EventQueue();
|
|
|
- eventQueue.addHandler(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT,
|
|
|
+ eventQueue.addHandler(CLOSE_CONTAINER,
|
|
|
new CloseContainerEventHandler(mapping));
|
|
|
}
|
|
|
|
|
@@ -81,8 +83,8 @@ public class TestCloseContainerEventHandler {
|
|
|
public void testIfCloseContainerEventHadnlerInvoked() {
|
|
|
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
|
|
.captureLogs(CloseContainerEventHandler.LOG);
|
|
|
- eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT,
|
|
|
- new ContainerID(Math.abs(new Random().nextLong())));
|
|
|
+ eventQueue.fireEvent(CLOSE_CONTAINER,
|
|
|
+ new ContainerID(Math.abs(RandomUtils.nextInt())));
|
|
|
eventQueue.processAll(1000);
|
|
|
Assert.assertTrue(logCapturer.getOutput()
|
|
|
.contains("Close container Event triggered for container"));
|
|
@@ -90,10 +92,10 @@ public class TestCloseContainerEventHandler {
|
|
|
|
|
|
@Test
|
|
|
public void testCloseContainerEventWithInvalidContainer() {
|
|
|
- long id = Math.abs(new Random().nextLong());
|
|
|
+ long id = Math.abs(RandomUtils.nextInt());
|
|
|
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
|
|
.captureLogs(CloseContainerEventHandler.LOG);
|
|
|
- eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT,
|
|
|
+ eventQueue.fireEvent(CLOSE_CONTAINER,
|
|
|
new ContainerID(id));
|
|
|
eventQueue.processAll(1000);
|
|
|
Assert.assertTrue(logCapturer.getOutput()
|
|
@@ -112,7 +114,7 @@ public class TestCloseContainerEventHandler {
|
|
|
containerWithPipeline.getContainerInfo().getContainerID());
|
|
|
DatanodeDetails datanode = containerWithPipeline.getPipeline().getLeader();
|
|
|
int closeCount = nodeManager.getCommandCount(datanode);
|
|
|
- eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id);
|
|
|
+ eventQueue.fireEvent(CLOSE_CONTAINER, id);
|
|
|
eventQueue.processAll(1000);
|
|
|
// At this point of time, the allocated container is not in open
|
|
|
// state, so firing close container event should not queue CLOSE
|
|
@@ -125,11 +127,12 @@ public class TestCloseContainerEventHandler {
|
|
|
//Execute these state transitions so that we can close the container.
|
|
|
mapping.updateContainerState(id.getId(), CREATE);
|
|
|
mapping.updateContainerState(id.getId(), CREATED);
|
|
|
- eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT,
|
|
|
+ eventQueue.fireEvent(CLOSE_CONTAINER,
|
|
|
new ContainerID(
|
|
|
containerWithPipeline.getContainerInfo().getContainerID()));
|
|
|
eventQueue.processAll(1000);
|
|
|
- Assert.assertEquals(closeCount + 1, nodeManager.getCommandCount(datanode));
|
|
|
+ Assert.assertEquals(closeCount + 1,
|
|
|
+ nodeManager.getCommandCount(datanode));
|
|
|
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,
|
|
|
mapping.getStateManager().getContainer(id).getState());
|
|
|
}
|
|
@@ -145,7 +148,7 @@ public class TestCloseContainerEventHandler {
|
|
|
ContainerID id = new ContainerID(
|
|
|
containerWithPipeline.getContainerInfo().getContainerID());
|
|
|
int[] closeCount = new int[3];
|
|
|
- eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id);
|
|
|
+ eventQueue.fireEvent(CLOSE_CONTAINER, id);
|
|
|
eventQueue.processAll(1000);
|
|
|
int i = 0;
|
|
|
for (DatanodeDetails details : containerWithPipeline.getPipeline()
|
|
@@ -166,7 +169,7 @@ public class TestCloseContainerEventHandler {
|
|
|
//Execute these state transitions so that we can close the container.
|
|
|
mapping.updateContainerState(id.getId(), CREATE);
|
|
|
mapping.updateContainerState(id.getId(), CREATED);
|
|
|
- eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id);
|
|
|
+ eventQueue.fireEvent(CLOSE_CONTAINER, id);
|
|
|
eventQueue.processAll(1000);
|
|
|
i = 0;
|
|
|
// Make sure close is queued for each datanode on the pipeline
|