|
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdds.protocol.proto
|
|
import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
|
import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
|
import org.apache.hadoop.hdds.scm.ScmUtils;
|
|
import org.apache.hadoop.hdds.scm.ScmUtils;
|
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
|
|
import org.apache.hadoop.hdds.scm.safemode.SafeModePrecheck;
|
|
import org.apache.hadoop.hdds.scm.safemode.SafeModePrecheck;
|
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
|
|
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
|
|
@@ -68,6 +69,7 @@ import org.slf4j.LoggerFactory;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
|
|
+import java.util.Collections;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
@@ -216,15 +218,14 @@ public class SCMClientProtocolServer implements
|
|
@Override
|
|
@Override
|
|
public ContainerWithPipeline getContainerWithPipeline(long containerID)
|
|
public ContainerWithPipeline getContainerWithPipeline(long containerID)
|
|
throws IOException {
|
|
throws IOException {
|
|
- Map<String, String> auditMap = Maps.newHashMap();
|
|
|
|
- auditMap.put("containerID", String.valueOf(containerID));
|
|
|
|
- boolean auditSuccess = true;
|
|
|
|
|
|
+ final ContainerID cid = ContainerID.valueof(containerID);
|
|
try {
|
|
try {
|
|
|
|
+ final ContainerInfo container = scm.getContainerManager()
|
|
|
|
+ .getContainer(cid);
|
|
|
|
+
|
|
if (safeModePrecheck.isInSafeMode()) {
|
|
if (safeModePrecheck.isInSafeMode()) {
|
|
- ContainerInfo contInfo = scm.getContainerManager()
|
|
|
|
- .getContainer(ContainerID.valueof(containerID));
|
|
|
|
- if (contInfo.isOpen()) {
|
|
|
|
- if (!hasRequiredReplicas(contInfo)) {
|
|
|
|
|
|
+ if (container.isOpen()) {
|
|
|
|
+ if (!hasRequiredReplicas(container)) {
|
|
throw new SCMException("Open container " + containerID + " doesn't"
|
|
throw new SCMException("Open container " + containerID + " doesn't"
|
|
+ " have enough replicas to service this operation in "
|
|
+ " have enough replicas to service this operation in "
|
|
+ "Safe mode.", ResultCodes.SAFE_MODE_EXCEPTION);
|
|
+ "Safe mode.", ResultCodes.SAFE_MODE_EXCEPTION);
|
|
@@ -233,40 +234,35 @@ public class SCMClientProtocolServer implements
|
|
}
|
|
}
|
|
getScm().checkAdminAccess(null);
|
|
getScm().checkAdminAccess(null);
|
|
|
|
|
|
- final ContainerID id = ContainerID.valueof(containerID);
|
|
|
|
- final ContainerInfo container = scm.getContainerManager().
|
|
|
|
- getContainer(id);
|
|
|
|
- final Pipeline pipeline;
|
|
|
|
|
|
+ Pipeline pipeline;
|
|
|
|
+ try {
|
|
|
|
+ pipeline = container.isOpen() ? scm.getPipelineManager()
|
|
|
|
+ .getPipeline(container.getPipelineID()) : null;
|
|
|
|
+ } catch (PipelineNotFoundException ex) {
|
|
|
|
+ // The pipeline is destroyed.
|
|
|
|
+ pipeline = null;
|
|
|
|
+ }
|
|
|
|
|
|
- if (container.isOpenNotClosing()) {
|
|
|
|
- // Ratis pipeline
|
|
|
|
- pipeline = scm.getPipelineManager()
|
|
|
|
- .getPipeline(container.getPipelineID());
|
|
|
|
- } else {
|
|
|
|
|
|
+ if (pipeline == null) {
|
|
pipeline = scm.getPipelineManager().createPipeline(
|
|
pipeline = scm.getPipelineManager().createPipeline(
|
|
HddsProtos.ReplicationType.STAND_ALONE,
|
|
HddsProtos.ReplicationType.STAND_ALONE,
|
|
container.getReplicationFactor(),
|
|
container.getReplicationFactor(),
|
|
scm.getContainerManager()
|
|
scm.getContainerManager()
|
|
- .getContainerReplicas(id).stream()
|
|
|
|
|
|
+ .getContainerReplicas(cid).stream()
|
|
.map(ContainerReplica::getDatanodeDetails)
|
|
.map(ContainerReplica::getDatanodeDetails)
|
|
.collect(Collectors.toList()));
|
|
.collect(Collectors.toList()));
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ AUDIT.logReadSuccess(buildAuditMessageForSuccess(
|
|
|
|
+ SCMAction.GET_CONTAINER_WITH_PIPELINE,
|
|
|
|
+ Collections.singletonMap("containerID", cid.toString())));
|
|
|
|
+
|
|
return new ContainerWithPipeline(container, pipeline);
|
|
return new ContainerWithPipeline(container, pipeline);
|
|
} catch (IOException ex) {
|
|
} catch (IOException ex) {
|
|
- auditSuccess = false;
|
|
|
|
- AUDIT.logReadFailure(
|
|
|
|
- buildAuditMessageForFailure(SCMAction.GET_CONTAINER_WITH_PIPELINE,
|
|
|
|
- auditMap, ex)
|
|
|
|
- );
|
|
|
|
|
|
+ AUDIT.logReadFailure(buildAuditMessageForFailure(
|
|
|
|
+ SCMAction.GET_CONTAINER_WITH_PIPELINE,
|
|
|
|
+ Collections.singletonMap("containerID", cid.toString()), ex));
|
|
throw ex;
|
|
throw ex;
|
|
- } finally {
|
|
|
|
- if(auditSuccess) {
|
|
|
|
- AUDIT.logReadSuccess(
|
|
|
|
- buildAuditMessageForSuccess(SCMAction.GET_CONTAINER_WITH_PIPELINE,
|
|
|
|
- auditMap)
|
|
|
|
- );
|
|
|
|
- }
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|