HDFS-12980. Ozone: SCM: Restructuring container state transition and event. Contributed by Nanda kumar.
This commit is contained in:
parent
87b7774488
commit
5f10c2d8ce
|
@ -104,16 +104,16 @@ enum LifeCycleState {
|
||||||
ALLOCATED = 1;
|
ALLOCATED = 1;
|
||||||
CREATING = 2; // Used for container allocated/created by different client.
|
CREATING = 2; // Used for container allocated/created by different client.
|
||||||
OPEN =3; // Mostly an update to SCM via HB or client call.
|
OPEN =3; // Mostly an update to SCM via HB or client call.
|
||||||
PENDING_CLOSE = 4;
|
CLOSING = 4;
|
||||||
CLOSED = 5; // !!State after this has not been used yet.
|
CLOSED = 5; // !!State after this has not been used yet.
|
||||||
DELETING = 6;
|
DELETING = 6;
|
||||||
DELETED = 7; // object is deleted.
|
DELETED = 7; // object is deleted.
|
||||||
}
|
}
|
||||||
|
|
||||||
enum LifeCycleEvent {
|
enum LifeCycleEvent {
|
||||||
BEGIN_CREATE = 1; // A request to client to create this object
|
CREATE = 1; // A request to client to create this object
|
||||||
COMPLETE_CREATE = 2;
|
CREATED = 2;
|
||||||
FULL_CONTAINER = 3;
|
FINALIZE = 3;
|
||||||
CLOSE = 4; // !!Event after this has not been used yet.
|
CLOSE = 4; // !!Event after this has not been used yet.
|
||||||
UPDATE = 5;
|
UPDATE = 5;
|
||||||
TIMEOUT = 6; // creation has timed out from SCM's View.
|
TIMEOUT = 6; // creation has timed out from SCM's View.
|
||||||
|
|
|
@ -688,10 +688,10 @@ public class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
info.getState(), stage);
|
info.getState(), stage);
|
||||||
if (stage == NotifyObjectCreationStageRequestProto.Stage.begin) {
|
if (stage == NotifyObjectCreationStageRequestProto.Stage.begin) {
|
||||||
scmContainerManager.updateContainerState(name,
|
scmContainerManager.updateContainerState(name,
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
} else {
|
} else {
|
||||||
scmContainerManager.updateContainerState(name,
|
scmContainerManager.updateContainerState(name,
|
||||||
OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATED);
|
||||||
}
|
}
|
||||||
} else if (type == NotifyObjectCreationStageRequestProto.Type.pipeline) {
|
} else if (type == NotifyObjectCreationStageRequestProto.Type.pipeline) {
|
||||||
// TODO: pipeline state update will be addressed in future patch.
|
// TODO: pipeline state update will be addressed in future patch.
|
||||||
|
|
|
@ -270,7 +270,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
|
||||||
.ALLOCATED);
|
.ALLOCATED);
|
||||||
if (containerInfo != null) {
|
if (containerInfo != null) {
|
||||||
containerManager.updateContainerState(containerInfo.getContainerName(),
|
containerManager.updateContainerState(containerInfo.getContainerName(),
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
return newBlock(containerInfo, OzoneProtos.LifeCycleState.ALLOCATED);
|
return newBlock(containerInfo, OzoneProtos.LifeCycleState.ALLOCATED);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -300,7 +300,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
|
||||||
.ALLOCATED);
|
.ALLOCATED);
|
||||||
if (containerInfo != null) {
|
if (containerInfo != null) {
|
||||||
containerManager.updateContainerState(containerInfo.getContainerName(),
|
containerManager.updateContainerState(containerInfo.getContainerName(),
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
return newBlock(containerInfo, OzoneProtos.LifeCycleState.ALLOCATED);
|
return newBlock(containerInfo, OzoneProtos.LifeCycleState.ALLOCATED);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -298,7 +298,7 @@ public class ContainerMapping implements Mapping {
|
||||||
|
|
||||||
Preconditions.checkNotNull(containerInfo);
|
Preconditions.checkNotNull(containerInfo);
|
||||||
switch (event) {
|
switch (event) {
|
||||||
case BEGIN_CREATE:
|
case CREATE:
|
||||||
// Acquire lease on container
|
// Acquire lease on container
|
||||||
Lease<ContainerInfo> containerLease =
|
Lease<ContainerInfo> containerLease =
|
||||||
containerLeaseManager.acquire(containerInfo);
|
containerLeaseManager.acquire(containerInfo);
|
||||||
|
@ -309,15 +309,11 @@ public class ContainerMapping implements Mapping {
|
||||||
return null;
|
return null;
|
||||||
});
|
});
|
||||||
break;
|
break;
|
||||||
case COMPLETE_CREATE:
|
case CREATED:
|
||||||
// Release the lease on container
|
// Release the lease on container
|
||||||
containerLeaseManager.release(containerInfo);
|
containerLeaseManager.release(containerInfo);
|
||||||
break;
|
break;
|
||||||
case TIMEOUT:
|
case FINALIZE:
|
||||||
break;
|
|
||||||
case CLEANUP:
|
|
||||||
break;
|
|
||||||
case FULL_CONTAINER:
|
|
||||||
break;
|
break;
|
||||||
case CLOSE:
|
case CLOSE:
|
||||||
break;
|
break;
|
||||||
|
@ -325,6 +321,10 @@ public class ContainerMapping implements Mapping {
|
||||||
break;
|
break;
|
||||||
case DELETE:
|
case DELETE:
|
||||||
break;
|
break;
|
||||||
|
case TIMEOUT:
|
||||||
|
break;
|
||||||
|
case CLEANUP:
|
||||||
|
break;
|
||||||
default:
|
default:
|
||||||
throw new SCMException("Unsupported container LifeCycleEvent.",
|
throw new SCMException("Unsupported container LifeCycleEvent.",
|
||||||
FAILED_TO_CHANGE_CONTAINER_STATE);
|
FAILED_TO_CHANGE_CONTAINER_STATE);
|
||||||
|
@ -406,8 +406,8 @@ public class ContainerMapping implements Mapping {
|
||||||
// have to add the containers to that list.
|
// have to add the containers to that list.
|
||||||
OzoneProtos.LifeCycleState state = updateContainerState(
|
OzoneProtos.LifeCycleState state = updateContainerState(
|
||||||
ContainerInfo.fromProtobuf(newContainerInfo).getContainerName(),
|
ContainerInfo.fromProtobuf(newContainerInfo).getContainerName(),
|
||||||
OzoneProtos.LifeCycleEvent.FULL_CONTAINER);
|
OzoneProtos.LifeCycleEvent.FINALIZE);
|
||||||
if (state != OzoneProtos.LifeCycleState.PENDING_CLOSE) {
|
if (state != OzoneProtos.LifeCycleState.CLOSING) {
|
||||||
LOG.error("Failed to close container {}, reason : Not able to " +
|
LOG.error("Failed to close container {}, reason : Not able to " +
|
||||||
"update container state, current container state: {}." +
|
"update container state, current container state: {}." +
|
||||||
"in state {}", containerInfo.getContainerName(), state);
|
"in state {}", containerInfo.getContainerName(), state);
|
||||||
|
|
|
@ -234,60 +234,75 @@ public class ContainerStateManager implements Closeable {
|
||||||
return list;
|
return list;
|
||||||
}
|
}
|
||||||
|
|
||||||
// 1. Client -> SCM: Begin_create
|
/*
|
||||||
// 2. Client -> Datanode: create
|
*
|
||||||
// 3. Client -> SCM: complete {SCM:Creating ->OK}
|
* Event and State Transition Mapping:
|
||||||
|
*
|
||||||
// 3. Client -> SCM: complete {SCM:DELETING -> INVALID}
|
* State: ALLOCATED ---------------> CREATING
|
||||||
|
* Event: CREATE
|
||||||
// 4. Client->Datanode: write data.
|
*
|
||||||
|
* State: CREATING ---------------> OPEN
|
||||||
// Client-driven Create State Machine
|
* Event: CREATED
|
||||||
// States: <ALLOCATED>------------->CREATING----------------->[OPEN]
|
*
|
||||||
// Events: (BEGIN_CREATE) | (COMPLETE_CREATE)
|
* State: OPEN ---------------> CLOSING
|
||||||
// |
|
* Event: FINALIZE
|
||||||
// |(TIMEOUT)
|
*
|
||||||
// V
|
* State: CLOSING ---------------> CLOSED
|
||||||
// DELETING----------------->[DELETED]
|
* Event: CLOSE
|
||||||
// (CLEANUP)
|
*
|
||||||
// SCM Open/Close State Machine
|
* State: CLOSED ----------------> DELETING
|
||||||
// States: OPEN------------------>PENDING_CLOSE---------->[CLOSED]
|
* Event: DELETE
|
||||||
// Events: (FULL_CONTAINER) (CLOSE)
|
*
|
||||||
// Delete State Machine
|
* State: DELETING ----------------> DELETED
|
||||||
// States: OPEN------------------>DELETING------------------>[DELETED]
|
* Event: CLEANUP
|
||||||
// Events: (DELETE) (CLEANUP)
|
*
|
||||||
|
* State: CREATING ---------------> DELETING
|
||||||
// Should we allow DELETING of OPEN containers? we can always have
|
* Event: TIMEOUT
|
||||||
// OPEN--------->PENDING_CLOSE----->CLOSE---->DELETING---->[DELETED]
|
*
|
||||||
|
*
|
||||||
|
* Container State Flow:
|
||||||
|
*
|
||||||
|
* [ALLOCATED]------->[CREATING]--------->[OPEN]---------->[CLOSING]------->[CLOSED]
|
||||||
|
* (CREATE) | (CREATED) (FINALIZE) (CLOSE) |
|
||||||
|
* | |
|
||||||
|
* | |
|
||||||
|
* |(TIMEOUT) (DELETE)|
|
||||||
|
* | |
|
||||||
|
* +------------------> [DELETING] <-------------------+
|
||||||
|
* |
|
||||||
|
* |
|
||||||
|
* (CLEANUP)|
|
||||||
|
* |
|
||||||
|
* [DELETED]
|
||||||
|
*/
|
||||||
private void initializeStateMachine() {
|
private void initializeStateMachine() {
|
||||||
stateMachine.addTransition(LifeCycleState.ALLOCATED,
|
stateMachine.addTransition(LifeCycleState.ALLOCATED,
|
||||||
LifeCycleState.CREATING,
|
LifeCycleState.CREATING,
|
||||||
LifeCycleEvent.BEGIN_CREATE);
|
LifeCycleEvent.CREATE);
|
||||||
|
|
||||||
stateMachine.addTransition(LifeCycleState.CREATING,
|
stateMachine.addTransition(LifeCycleState.CREATING,
|
||||||
LifeCycleState.OPEN,
|
LifeCycleState.OPEN,
|
||||||
LifeCycleEvent.COMPLETE_CREATE);
|
LifeCycleEvent.CREATED);
|
||||||
|
|
||||||
stateMachine.addTransition(LifeCycleState.OPEN,
|
stateMachine.addTransition(LifeCycleState.OPEN,
|
||||||
LifeCycleState.PENDING_CLOSE,
|
LifeCycleState.CLOSING,
|
||||||
LifeCycleEvent.FULL_CONTAINER);
|
LifeCycleEvent.FINALIZE);
|
||||||
|
|
||||||
stateMachine.addTransition(LifeCycleState.PENDING_CLOSE,
|
stateMachine.addTransition(LifeCycleState.CLOSING,
|
||||||
LifeCycleState.CLOSED,
|
LifeCycleState.CLOSED,
|
||||||
LifeCycleEvent.CLOSE);
|
LifeCycleEvent.CLOSE);
|
||||||
|
|
||||||
stateMachine.addTransition(LifeCycleState.OPEN,
|
stateMachine.addTransition(LifeCycleState.CLOSED,
|
||||||
LifeCycleState.DELETING,
|
LifeCycleState.DELETING,
|
||||||
LifeCycleEvent.DELETE);
|
LifeCycleEvent.DELETE);
|
||||||
|
|
||||||
|
stateMachine.addTransition(LifeCycleState.CREATING,
|
||||||
|
LifeCycleState.DELETING,
|
||||||
|
LifeCycleEvent.TIMEOUT);
|
||||||
|
|
||||||
stateMachine.addTransition(LifeCycleState.DELETING,
|
stateMachine.addTransition(LifeCycleState.DELETING,
|
||||||
LifeCycleState.DELETED,
|
LifeCycleState.DELETED,
|
||||||
LifeCycleEvent.CLEANUP);
|
LifeCycleEvent.CLEANUP);
|
||||||
|
|
||||||
// Creating timeout -> Deleting
|
|
||||||
stateMachine.addTransition(LifeCycleState.CREATING,
|
|
||||||
LifeCycleState.DELETING,
|
|
||||||
LifeCycleEvent.TIMEOUT);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -183,7 +183,7 @@ public class TestContainerMapping {
|
||||||
containerName,
|
containerName,
|
||||||
OzoneProtos.Owner.OZONE);
|
OzoneProtos.Owner.OZONE);
|
||||||
mapping.updateContainerState(containerInfo.getContainerName(),
|
mapping.updateContainerState(containerInfo.getContainerName(),
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
Thread.sleep(TIMEOUT + 1000);
|
Thread.sleep(TIMEOUT + 1000);
|
||||||
|
|
||||||
List<ContainerInfo> deleteContainers = mapping.getStateManager()
|
List<ContainerInfo> deleteContainers = mapping.getStateManager()
|
||||||
|
@ -199,7 +199,7 @@ public class TestContainerMapping {
|
||||||
thrown.expect(IOException.class);
|
thrown.expect(IOException.class);
|
||||||
thrown.expectMessage("Lease Exception");
|
thrown.expectMessage("Lease Exception");
|
||||||
mapping.updateContainerState(containerInfo.getContainerName(),
|
mapping.updateContainerState(containerInfo.getContainerName(),
|
||||||
OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATED);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -234,8 +234,7 @@ public class TestContainerMapping {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testContainerCloseWithContainerReport() throws IOException,
|
public void testContainerCloseWithContainerReport() throws IOException {
|
||||||
InterruptedException {
|
|
||||||
String containerName = UUID.randomUUID().toString();
|
String containerName = UUID.randomUUID().toString();
|
||||||
createContainer(containerName);
|
createContainer(containerName);
|
||||||
DatanodeID datanodeID = SCMTestUtils.getDatanodeID();
|
DatanodeID datanodeID = SCMTestUtils.getDatanodeID();
|
||||||
|
@ -269,7 +268,7 @@ public class TestContainerMapping {
|
||||||
OzoneProtos.Owner.OZONE,
|
OzoneProtos.Owner.OZONE,
|
||||||
xceiverClientManager.getType(),
|
xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(),
|
xceiverClientManager.getFactor(),
|
||||||
OzoneProtos.LifeCycleState.PENDING_CLOSE);
|
OzoneProtos.LifeCycleState.CLOSING);
|
||||||
Assert.assertTrue(pendingCloseContainers.stream().map(
|
Assert.assertTrue(pendingCloseContainers.stream().map(
|
||||||
container -> container.getContainerName()).collect(
|
container -> container.getContainerName()).collect(
|
||||||
Collectors.toList()).contains(containerName));
|
Collectors.toList()).contains(containerName));
|
||||||
|
@ -280,13 +279,13 @@ public class TestContainerMapping {
|
||||||
String containerName = UUID.randomUUID().toString();
|
String containerName = UUID.randomUUID().toString();
|
||||||
createContainer(containerName);
|
createContainer(containerName);
|
||||||
mapping.updateContainerState(containerName,
|
mapping.updateContainerState(containerName,
|
||||||
OzoneProtos.LifeCycleEvent.FULL_CONTAINER);
|
OzoneProtos.LifeCycleEvent.FINALIZE);
|
||||||
List<ContainerInfo> pendingCloseContainers = mapping.getStateManager()
|
List<ContainerInfo> pendingCloseContainers = mapping.getStateManager()
|
||||||
.getMatchingContainers(
|
.getMatchingContainers(
|
||||||
OzoneProtos.Owner.OZONE,
|
OzoneProtos.Owner.OZONE,
|
||||||
xceiverClientManager.getType(),
|
xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(),
|
xceiverClientManager.getFactor(),
|
||||||
OzoneProtos.LifeCycleState.PENDING_CLOSE);
|
OzoneProtos.LifeCycleState.CLOSING);
|
||||||
Assert.assertTrue(pendingCloseContainers.stream().map(
|
Assert.assertTrue(pendingCloseContainers.stream().map(
|
||||||
container -> container.getContainerName()).collect(
|
container -> container.getContainerName()).collect(
|
||||||
Collectors.toList()).contains(containerName));
|
Collectors.toList()).contains(containerName));
|
||||||
|
@ -317,9 +316,9 @@ public class TestContainerMapping {
|
||||||
containerName,
|
containerName,
|
||||||
OzoneProtos.Owner.OZONE);
|
OzoneProtos.Owner.OZONE);
|
||||||
mapping.updateContainerState(containerInfo.getContainerName(),
|
mapping.updateContainerState(containerInfo.getContainerName(),
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
mapping.updateContainerState(containerInfo.getContainerName(),
|
mapping.updateContainerState(containerInfo.getContainerName(),
|
||||||
OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATED);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -108,7 +108,7 @@ public class TestContainerStateManager {
|
||||||
xceiverClientManager.getFactor(), cname + i);
|
xceiverClientManager.getFactor(), cname + i);
|
||||||
if (i >= 5) {
|
if (i >= 5) {
|
||||||
scm.getScmContainerManager().updateContainerState(cname + i,
|
scm.getScmContainerManager().updateContainerState(cname + i,
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -134,9 +134,9 @@ public class TestContainerStateManager {
|
||||||
scm.allocateContainer(xceiverClientManager.getType(),
|
scm.allocateContainer(xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), container1);
|
xceiverClientManager.getFactor(), container1);
|
||||||
scmContainerMapping.updateContainerState(container1,
|
scmContainerMapping.updateContainerState(container1,
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
scmContainerMapping.updateContainerState(container1,
|
scmContainerMapping.updateContainerState(container1,
|
||||||
OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATED);
|
||||||
|
|
||||||
String container2 = "container" + RandomStringUtils.randomNumeric(5);
|
String container2 = "container" + RandomStringUtils.randomNumeric(5);
|
||||||
scm.allocateContainer(xceiverClientManager.getType(),
|
scm.allocateContainer(xceiverClientManager.getType(),
|
||||||
|
@ -161,9 +161,9 @@ public class TestContainerStateManager {
|
||||||
Assert.assertEquals(container2, info.getContainerName());
|
Assert.assertEquals(container2, info.getContainerName());
|
||||||
|
|
||||||
scmContainerMapping.updateContainerState(container2,
|
scmContainerMapping.updateContainerState(container2,
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
scmContainerMapping.updateContainerState(container2,
|
scmContainerMapping.updateContainerState(container2,
|
||||||
OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATED);
|
||||||
info = stateManager
|
info = stateManager
|
||||||
.getMatchingContainer(OzoneConsts.GB * 3, OzoneProtos.Owner.OZONE,
|
.getMatchingContainer(OzoneConsts.GB * 3, OzoneProtos.Owner.OZONE,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
@ -180,7 +180,7 @@ public class TestContainerStateManager {
|
||||||
Assert.assertEquals(0, containers);
|
Assert.assertEquals(0, containers);
|
||||||
|
|
||||||
// Allocate container1 and update its state from ALLOCATED -> CREATING ->
|
// Allocate container1 and update its state from ALLOCATED -> CREATING ->
|
||||||
// OPEN -> DELETING -> DELETED
|
// OPEN -> CLOSING -> CLOSED -> DELETING -> DELETED
|
||||||
String container1 = "container" + RandomStringUtils.randomNumeric(5);
|
String container1 = "container" + RandomStringUtils.randomNumeric(5);
|
||||||
scm.allocateContainer(xceiverClientManager.getType(),
|
scm.allocateContainer(xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), container1);
|
xceiverClientManager.getFactor(), container1);
|
||||||
|
@ -190,19 +190,33 @@ public class TestContainerStateManager {
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
scmContainerMapping.updateContainerState(container1,
|
scmContainerMapping.updateContainerState(container1,
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
OzoneProtos.LifeCycleState.CREATING).size();
|
OzoneProtos.LifeCycleState.CREATING).size();
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
scmContainerMapping.updateContainerState(container1,
|
scmContainerMapping.updateContainerState(container1,
|
||||||
OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATED);
|
||||||
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
OzoneProtos.LifeCycleState.OPEN).size();
|
OzoneProtos.LifeCycleState.OPEN).size();
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
|
scmContainerMapping
|
||||||
|
.updateContainerState(container1, OzoneProtos.LifeCycleEvent.FINALIZE);
|
||||||
|
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
||||||
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
OzoneProtos.LifeCycleState.CLOSING).size();
|
||||||
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
|
scmContainerMapping
|
||||||
|
.updateContainerState(container1, OzoneProtos.LifeCycleEvent.CLOSE);
|
||||||
|
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
||||||
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
OzoneProtos.LifeCycleState.CLOSED).size();
|
||||||
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
scmContainerMapping
|
scmContainerMapping
|
||||||
.updateContainerState(container1, OzoneProtos.LifeCycleEvent.DELETE);
|
.updateContainerState(container1, OzoneProtos.LifeCycleEvent.DELETE);
|
||||||
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
||||||
|
@ -223,7 +237,7 @@ public class TestContainerStateManager {
|
||||||
scm.allocateContainer(xceiverClientManager.getType(),
|
scm.allocateContainer(xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), container2);
|
xceiverClientManager.getFactor(), container2);
|
||||||
scmContainerMapping.updateContainerState(container2,
|
scmContainerMapping.updateContainerState(container2,
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
scmContainerMapping
|
scmContainerMapping
|
||||||
.updateContainerState(container2, OzoneProtos.LifeCycleEvent.TIMEOUT);
|
.updateContainerState(container2, OzoneProtos.LifeCycleEvent.TIMEOUT);
|
||||||
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
||||||
|
@ -232,14 +246,16 @@ public class TestContainerStateManager {
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
// Allocate container1 and update its state from ALLOCATED -> CREATING ->
|
// Allocate container1 and update its state from ALLOCATED -> CREATING ->
|
||||||
// OPEN -> CLOSED
|
// OPEN -> CLOSING -> CLOSED
|
||||||
String container3 = "container" + RandomStringUtils.randomNumeric(5);
|
String container3 = "container" + RandomStringUtils.randomNumeric(5);
|
||||||
scm.allocateContainer(xceiverClientManager.getType(),
|
scm.allocateContainer(xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), container3);
|
xceiverClientManager.getFactor(), container3);
|
||||||
scmContainerMapping.updateContainerState(container3,
|
scmContainerMapping.updateContainerState(container3,
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
scmContainerMapping.updateContainerState(container3,
|
scmContainerMapping.updateContainerState(container3,
|
||||||
OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATED);
|
||||||
|
scmContainerMapping.updateContainerState(container3,
|
||||||
|
OzoneProtos.LifeCycleEvent.FINALIZE);
|
||||||
scmContainerMapping
|
scmContainerMapping
|
||||||
.updateContainerState(container3, OzoneProtos.LifeCycleEvent.CLOSE);
|
.updateContainerState(container3, OzoneProtos.LifeCycleEvent.CLOSE);
|
||||||
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
|
||||||
|
@ -254,9 +270,9 @@ public class TestContainerStateManager {
|
||||||
scm.allocateContainer(xceiverClientManager.getType(),
|
scm.allocateContainer(xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), container1);
|
xceiverClientManager.getFactor(), container1);
|
||||||
scmContainerMapping.updateContainerState(container1,
|
scmContainerMapping.updateContainerState(container1,
|
||||||
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATE);
|
||||||
scmContainerMapping.updateContainerState(container1,
|
scmContainerMapping.updateContainerState(container1,
|
||||||
OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
|
OzoneProtos.LifeCycleEvent.CREATED);
|
||||||
|
|
||||||
Random ran = new Random();
|
Random ran = new Random();
|
||||||
long allocatedSize = 0;
|
long allocatedSize = 0;
|
||||||
|
|
Loading…
Reference in New Issue