You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2018/01/03 01:44:25 UTC
hadoop git commit: HDFS-12980. Ozone: SCM: Restructuring container
state transition and event. Contributed by Nanda kumar.
Repository: hadoop
Updated Branches:
refs/heads/HDFS-7240 c04492ac8 -> ace9aece2
HDFS-12980. Ozone: SCM: Restructuring container state transition and event. Contributed by Nanda kumar.
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ace9aece
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ace9aece
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ace9aece
Branch: refs/heads/HDFS-7240
Commit: ace9aece23ac987b616761a276ee218b8ee94be9
Parents: c04492a
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Jan 2 17:35:08 2018 -0800
Committer: Anu Engineer <ae...@apache.org>
Committed: Tue Jan 2 17:35:08 2018 -0800
----------------------------------------------------------------------
.../src/main/proto/Ozone.proto | 8 +-
.../ozone/scm/StorageContainerManager.java | 4 +-
.../ozone/scm/block/BlockManagerImpl.java | 4 +-
.../ozone/scm/container/ContainerMapping.java | 18 ++--
.../scm/container/ContainerStateManager.java | 87 ++++++++++++--------
.../scm/container/TestContainerMapping.java | 17 ++--
.../container/TestContainerStateManager.java | 44 ++++++----
7 files changed, 106 insertions(+), 76 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace9aece/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/Ozone.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/Ozone.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/Ozone.proto
index 9be36dd..4941b62 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/Ozone.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/Ozone.proto
@@ -104,16 +104,16 @@ enum LifeCycleState {
ALLOCATED = 1;
CREATING = 2; // Used for container allocated/created by different client.
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.
DELETING = 6;
DELETED = 7; // object is deleted.
}
enum LifeCycleEvent {
- BEGIN_CREATE = 1; // A request to client to create this object
- COMPLETE_CREATE = 2;
- FULL_CONTAINER = 3;
+ CREATE = 1; // A request to client to create this object
+ CREATED = 2;
+ FINALIZE = 3;
CLOSE = 4; // !!Event after this has not been used yet.
UPDATE = 5;
TIMEOUT = 6; // creation has timed out from SCM's View.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace9aece/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
index b547a30..42bb082 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
@@ -688,10 +688,10 @@ public class StorageContainerManager extends ServiceRuntimeInfoImpl
info.getState(), stage);
if (stage == NotifyObjectCreationStageRequestProto.Stage.begin) {
scmContainerManager.updateContainerState(name,
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
} else {
scmContainerManager.updateContainerState(name,
- OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATED);
}
} else if (type == NotifyObjectCreationStageRequestProto.Type.pipeline) {
// TODO: pipeline state update will be addressed in future patch.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace9aece/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java
index 521adf8..345db1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java
@@ -270,7 +270,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
.ALLOCATED);
if (containerInfo != null) {
containerManager.updateContainerState(containerInfo.getContainerName(),
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
return newBlock(containerInfo, OzoneProtos.LifeCycleState.ALLOCATED);
}
@@ -300,7 +300,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
.ALLOCATED);
if (containerInfo != null) {
containerManager.updateContainerState(containerInfo.getContainerName(),
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
return newBlock(containerInfo, OzoneProtos.LifeCycleState.ALLOCATED);
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace9aece/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java
index 47b84f9..4ae1883 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java
@@ -298,7 +298,7 @@ public class ContainerMapping implements Mapping {
Preconditions.checkNotNull(containerInfo);
switch (event) {
- case BEGIN_CREATE:
+ case CREATE:
// Acquire lease on container
Lease<ContainerInfo> containerLease =
containerLeaseManager.acquire(containerInfo);
@@ -309,15 +309,11 @@ public class ContainerMapping implements Mapping {
return null;
});
break;
- case COMPLETE_CREATE:
+ case CREATED:
// Release the lease on container
containerLeaseManager.release(containerInfo);
break;
- case TIMEOUT:
- break;
- case CLEANUP:
- break;
- case FULL_CONTAINER:
+ case FINALIZE:
break;
case CLOSE:
break;
@@ -325,6 +321,10 @@ public class ContainerMapping implements Mapping {
break;
case DELETE:
break;
+ case TIMEOUT:
+ break;
+ case CLEANUP:
+ break;
default:
throw new SCMException("Unsupported container LifeCycleEvent.",
FAILED_TO_CHANGE_CONTAINER_STATE);
@@ -406,8 +406,8 @@ public class ContainerMapping implements Mapping {
// have to add the containers to that list.
OzoneProtos.LifeCycleState state = updateContainerState(
ContainerInfo.fromProtobuf(newContainerInfo).getContainerName(),
- OzoneProtos.LifeCycleEvent.FULL_CONTAINER);
- if (state != OzoneProtos.LifeCycleState.PENDING_CLOSE) {
+ OzoneProtos.LifeCycleEvent.FINALIZE);
+ if (state != OzoneProtos.LifeCycleState.CLOSING) {
LOG.error("Failed to close container {}, reason : Not able to " +
"update container state, current container state: {}." +
"in state {}", containerInfo.getContainerName(), state);
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace9aece/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerStateManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerStateManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerStateManager.java
index 57790d6..9b67473 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerStateManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerStateManager.java
@@ -234,60 +234,75 @@ public class ContainerStateManager implements Closeable {
return list;
}
- // 1. Client -> SCM: Begin_create
- // 2. Client -> Datanode: create
- // 3. Client -> SCM: complete {SCM:Creating ->OK}
-
- // 3. Client -> SCM: complete {SCM:DELETING -> INVALID}
-
- // 4. Client->Datanode: write data.
-
- // Client-driven Create State Machine
- // States: <ALLOCATED>------------->CREATING----------------->[OPEN]
- // Events: (BEGIN_CREATE) | (COMPLETE_CREATE)
- // |
- // |(TIMEOUT)
- // V
- // DELETING----------------->[DELETED]
- // (CLEANUP)
- // SCM Open/Close State Machine
- // States: OPEN------------------>PENDING_CLOSE---------->[CLOSED]
- // Events: (FULL_CONTAINER) (CLOSE)
- // Delete State Machine
- // States: OPEN------------------>DELETING------------------>[DELETED]
- // Events: (DELETE) (CLEANUP)
-
- // Should we allow DELETING of OPEN containers? we can always have
- // OPEN--------->PENDING_CLOSE----->CLOSE---->DELETING---->[DELETED]
+ /*
+ *
+ * Event and State Transition Mapping:
+ *
+ * State: ALLOCATED ---------------> CREATING
+ * Event: CREATE
+ *
+ * State: CREATING ---------------> OPEN
+ * Event: CREATED
+ *
+ * State: OPEN ---------------> CLOSING
+ * Event: FINALIZE
+ *
+ * State: CLOSING ---------------> CLOSED
+ * Event: CLOSE
+ *
+ * State: CLOSED ----------------> DELETING
+ * Event: DELETE
+ *
+ * State: DELETING ----------------> DELETED
+ * Event: CLEANUP
+ *
+ * State: CREATING ---------------> DELETING
+ * Event: TIMEOUT
+ *
+ *
+ * Container State Flow:
+ *
+ * [ALLOCATED]------->[CREATING]--------->[OPEN]---------->[CLOSING]------->[CLOSED]
+ * (CREATE) | (CREATED) (FINALIZE) (CLOSE) |
+ * | |
+ * | |
+ * |(TIMEOUT) (DELETE)|
+ * | |
+ * +------------------> [DELETING] <-------------------+
+ * |
+ * |
+ * (CLEANUP)|
+ * |
+ * [DELETED]
+ */
private void initializeStateMachine() {
stateMachine.addTransition(LifeCycleState.ALLOCATED,
LifeCycleState.CREATING,
- LifeCycleEvent.BEGIN_CREATE);
+ LifeCycleEvent.CREATE);
stateMachine.addTransition(LifeCycleState.CREATING,
LifeCycleState.OPEN,
- LifeCycleEvent.COMPLETE_CREATE);
+ LifeCycleEvent.CREATED);
stateMachine.addTransition(LifeCycleState.OPEN,
- LifeCycleState.PENDING_CLOSE,
- LifeCycleEvent.FULL_CONTAINER);
+ LifeCycleState.CLOSING,
+ LifeCycleEvent.FINALIZE);
- stateMachine.addTransition(LifeCycleState.PENDING_CLOSE,
+ stateMachine.addTransition(LifeCycleState.CLOSING,
LifeCycleState.CLOSED,
LifeCycleEvent.CLOSE);
- stateMachine.addTransition(LifeCycleState.OPEN,
+ stateMachine.addTransition(LifeCycleState.CLOSED,
LifeCycleState.DELETING,
LifeCycleEvent.DELETE);
- stateMachine.addTransition(LifeCycleState.DELETING,
- LifeCycleState.DELETED,
- LifeCycleEvent.CLEANUP);
-
- // Creating timeout -> Deleting
stateMachine.addTransition(LifeCycleState.CREATING,
LifeCycleState.DELETING,
LifeCycleEvent.TIMEOUT);
+
+ stateMachine.addTransition(LifeCycleState.DELETING,
+ LifeCycleState.DELETED,
+ LifeCycleEvent.CLEANUP);
}
/**
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace9aece/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerMapping.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerMapping.java
index 25cb75e..5907b2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerMapping.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerMapping.java
@@ -183,7 +183,7 @@ public class TestContainerMapping {
containerName,
OzoneProtos.Owner.OZONE);
mapping.updateContainerState(containerInfo.getContainerName(),
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
Thread.sleep(TIMEOUT + 1000);
List<ContainerInfo> deleteContainers = mapping.getStateManager()
@@ -199,7 +199,7 @@ public class TestContainerMapping {
thrown.expect(IOException.class);
thrown.expectMessage("Lease Exception");
mapping.updateContainerState(containerInfo.getContainerName(),
- OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATED);
}
@Test
@@ -234,8 +234,7 @@ public class TestContainerMapping {
}
@Test
- public void testContainerCloseWithContainerReport() throws IOException,
- InterruptedException {
+ public void testContainerCloseWithContainerReport() throws IOException {
String containerName = UUID.randomUUID().toString();
createContainer(containerName);
DatanodeID datanodeID = SCMTestUtils.getDatanodeID();
@@ -269,7 +268,7 @@ public class TestContainerMapping {
OzoneProtos.Owner.OZONE,
xceiverClientManager.getType(),
xceiverClientManager.getFactor(),
- OzoneProtos.LifeCycleState.PENDING_CLOSE);
+ OzoneProtos.LifeCycleState.CLOSING);
Assert.assertTrue(pendingCloseContainers.stream().map(
container -> container.getContainerName()).collect(
Collectors.toList()).contains(containerName));
@@ -280,13 +279,13 @@ public class TestContainerMapping {
String containerName = UUID.randomUUID().toString();
createContainer(containerName);
mapping.updateContainerState(containerName,
- OzoneProtos.LifeCycleEvent.FULL_CONTAINER);
+ OzoneProtos.LifeCycleEvent.FINALIZE);
List<ContainerInfo> pendingCloseContainers = mapping.getStateManager()
.getMatchingContainers(
OzoneProtos.Owner.OZONE,
xceiverClientManager.getType(),
xceiverClientManager.getFactor(),
- OzoneProtos.LifeCycleState.PENDING_CLOSE);
+ OzoneProtos.LifeCycleState.CLOSING);
Assert.assertTrue(pendingCloseContainers.stream().map(
container -> container.getContainerName()).collect(
Collectors.toList()).contains(containerName));
@@ -317,9 +316,9 @@ public class TestContainerMapping {
containerName,
OzoneProtos.Owner.OZONE);
mapping.updateContainerState(containerInfo.getContainerName(),
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
mapping.updateContainerState(containerInfo.getContainerName(),
- OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATED);
}
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace9aece/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerStateManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerStateManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerStateManager.java
index 675e73a..30ba227 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerStateManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerStateManager.java
@@ -108,7 +108,7 @@ public class TestContainerStateManager {
xceiverClientManager.getFactor(), cname + i);
if (i >= 5) {
scm.getScmContainerManager().updateContainerState(cname + i,
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
}
}
@@ -134,9 +134,9 @@ public class TestContainerStateManager {
scm.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), container1);
scmContainerMapping.updateContainerState(container1,
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
scmContainerMapping.updateContainerState(container1,
- OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATED);
String container2 = "container" + RandomStringUtils.randomNumeric(5);
scm.allocateContainer(xceiverClientManager.getType(),
@@ -161,9 +161,9 @@ public class TestContainerStateManager {
Assert.assertEquals(container2, info.getContainerName());
scmContainerMapping.updateContainerState(container2,
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
scmContainerMapping.updateContainerState(container2,
- OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATED);
info = stateManager
.getMatchingContainer(OzoneConsts.GB * 3, OzoneProtos.Owner.OZONE,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@@ -180,7 +180,7 @@ public class TestContainerStateManager {
Assert.assertEquals(0, containers);
// Allocate container1 and update its state from ALLOCATED -> CREATING ->
- // OPEN -> DELETING -> DELETED
+ // OPEN -> CLOSING -> CLOSED -> DELETING -> DELETED
String container1 = "container" + RandomStringUtils.randomNumeric(5);
scm.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), container1);
@@ -190,20 +190,34 @@ public class TestContainerStateManager {
Assert.assertEquals(1, containers);
scmContainerMapping.updateContainerState(container1,
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
OzoneProtos.LifeCycleState.CREATING).size();
Assert.assertEquals(1, containers);
scmContainerMapping.updateContainerState(container1,
- OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATED);
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
OzoneProtos.LifeCycleState.OPEN).size();
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
.updateContainerState(container1, OzoneProtos.LifeCycleEvent.DELETE);
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@@ -223,7 +237,7 @@ public class TestContainerStateManager {
scm.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), container2);
scmContainerMapping.updateContainerState(container2,
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
scmContainerMapping
.updateContainerState(container2, OzoneProtos.LifeCycleEvent.TIMEOUT);
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
@@ -232,14 +246,16 @@ public class TestContainerStateManager {
Assert.assertEquals(1, containers);
// Allocate container1 and update its state from ALLOCATED -> CREATING ->
- // OPEN -> CLOSED
+ // OPEN -> CLOSING -> CLOSED
String container3 = "container" + RandomStringUtils.randomNumeric(5);
scm.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), container3);
scmContainerMapping.updateContainerState(container3,
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
+ scmContainerMapping.updateContainerState(container3,
+ OzoneProtos.LifeCycleEvent.CREATED);
scmContainerMapping.updateContainerState(container3,
- OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
+ OzoneProtos.LifeCycleEvent.FINALIZE);
scmContainerMapping
.updateContainerState(container3, OzoneProtos.LifeCycleEvent.CLOSE);
containers = stateManager.getMatchingContainers(OzoneProtos.Owner.OZONE,
@@ -254,9 +270,9 @@ public class TestContainerStateManager {
scm.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), container1);
scmContainerMapping.updateContainerState(container1,
- OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATE);
scmContainerMapping.updateContainerState(container1,
- OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
+ OzoneProtos.LifeCycleEvent.CREATED);
Random ran = new Random();
long allocatedSize = 0;
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org