|
@@ -21,7 +21,7 @@ import com.google.common.base.Preconditions;
|
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerData;
|
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ReadContainerResponseProto;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.NotifyObjectCreationStageRequestProto;
|
|
|
+import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto;
|
|
|
import org.apache.hadoop.scm.XceiverClientManager;
|
|
|
import org.apache.hadoop.scm.XceiverClientSpi;
|
|
|
import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
|
|
@@ -122,15 +122,17 @@ public class ContainerOperationClient implements ScmClient {
|
|
|
public void createContainer(String containerId, XceiverClientSpi client,
|
|
|
Pipeline pipeline) throws IOException {
|
|
|
String traceID = UUID.randomUUID().toString();
|
|
|
- storageContainerLocationClient.notifyObjectCreationStage(
|
|
|
- NotifyObjectCreationStageRequestProto.Type.container,
|
|
|
+ storageContainerLocationClient.notifyObjectStageChange(
|
|
|
+ ObjectStageChangeRequestProto.Type.container,
|
|
|
containerId,
|
|
|
- NotifyObjectCreationStageRequestProto.Stage.begin);
|
|
|
+ ObjectStageChangeRequestProto.Op.create,
|
|
|
+ ObjectStageChangeRequestProto.Stage.begin);
|
|
|
ContainerProtocolCalls.createContainer(client, traceID);
|
|
|
- storageContainerLocationClient.notifyObjectCreationStage(
|
|
|
- NotifyObjectCreationStageRequestProto.Type.container,
|
|
|
+ storageContainerLocationClient.notifyObjectStageChange(
|
|
|
+ ObjectStageChangeRequestProto.Type.container,
|
|
|
containerId,
|
|
|
- NotifyObjectCreationStageRequestProto.Stage.complete);
|
|
|
+ ObjectStageChangeRequestProto.Op.create,
|
|
|
+ ObjectStageChangeRequestProto.Stage.complete);
|
|
|
|
|
|
// Let us log this info after we let SCM know that we have completed the
|
|
|
// creation state.
|
|
@@ -162,18 +164,20 @@ public class ContainerOperationClient implements ScmClient {
|
|
|
// 2. Talk to Datanodes to create the pipeline.
|
|
|
//
|
|
|
// 3. update SCM that pipeline creation was successful.
|
|
|
- storageContainerLocationClient.notifyObjectCreationStage(
|
|
|
- NotifyObjectCreationStageRequestProto.Type.pipeline,
|
|
|
+ storageContainerLocationClient.notifyObjectStageChange(
|
|
|
+ ObjectStageChangeRequestProto.Type.pipeline,
|
|
|
pipeline.getPipelineName(),
|
|
|
- NotifyObjectCreationStageRequestProto.Stage.begin);
|
|
|
+ ObjectStageChangeRequestProto.Op.create,
|
|
|
+ ObjectStageChangeRequestProto.Stage.begin);
|
|
|
|
|
|
client.createPipeline(pipeline.getPipelineName(),
|
|
|
pipeline.getMachines());
|
|
|
|
|
|
- storageContainerLocationClient.notifyObjectCreationStage(
|
|
|
- NotifyObjectCreationStageRequestProto.Type.pipeline,
|
|
|
+ storageContainerLocationClient.notifyObjectStageChange(
|
|
|
+ ObjectStageChangeRequestProto.Type.pipeline,
|
|
|
pipeline.getPipelineName(),
|
|
|
- NotifyObjectCreationStageRequestProto.Stage.complete);
|
|
|
+ ObjectStageChangeRequestProto.Op.create,
|
|
|
+ ObjectStageChangeRequestProto.Stage.complete);
|
|
|
|
|
|
// TODO : Should we change the state on the client side ??
|
|
|
// That makes sense, but it is not needed for the client to work.
|
|
@@ -358,10 +362,22 @@ public class ContainerOperationClient implements ScmClient {
|
|
|
// Actually close the container on Datanode
|
|
|
client = xceiverClientManager.acquireClient(pipeline);
|
|
|
String traceID = UUID.randomUUID().toString();
|
|
|
+
|
|
|
+ String containerId = pipeline.getContainerName();
|
|
|
+
|
|
|
+ storageContainerLocationClient.notifyObjectStageChange(
|
|
|
+ ObjectStageChangeRequestProto.Type.container,
|
|
|
+ containerId,
|
|
|
+ ObjectStageChangeRequestProto.Op.close,
|
|
|
+ ObjectStageChangeRequestProto.Stage.begin);
|
|
|
+
|
|
|
ContainerProtocolCalls.closeContainer(client, traceID);
|
|
|
// Notify SCM to close the container
|
|
|
- String containerId = pipeline.getContainerName();
|
|
|
- storageContainerLocationClient.closeContainer(containerId);
|
|
|
+ storageContainerLocationClient.notifyObjectStageChange(
|
|
|
+ ObjectStageChangeRequestProto.Type.container,
|
|
|
+ containerId,
|
|
|
+ ObjectStageChangeRequestProto.Op.close,
|
|
|
+ ObjectStageChangeRequestProto.Stage.complete);
|
|
|
} finally {
|
|
|
if (client != null) {
|
|
|
xceiverClientManager.releaseClient(client);
|