|
@@ -23,6 +23,7 @@ package org.apache.hadoop.hdds.scm.server;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
|
|
+import com.google.common.collect.Maps;
|
|
import com.google.protobuf.BlockingService;
|
|
import com.google.protobuf.BlockingService;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
@@ -52,6 +53,14 @@ import org.apache.hadoop.hdds.server.events.EventPublisher;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
|
|
+import org.apache.hadoop.ipc.Server;
|
|
|
|
+import org.apache.hadoop.ozone.audit.AuditAction;
|
|
|
|
+import org.apache.hadoop.ozone.audit.AuditEventStatus;
|
|
|
|
+import org.apache.hadoop.ozone.audit.AuditLogger;
|
|
|
|
+import org.apache.hadoop.ozone.audit.AuditLoggerType;
|
|
|
|
+import org.apache.hadoop.ozone.audit.AuditMessage;
|
|
|
|
+import org.apache.hadoop.ozone.audit.Auditor;
|
|
|
|
+import org.apache.hadoop.ozone.audit.SCMAction;
|
|
import org.apache.hadoop.ozone.protocolPB
|
|
import org.apache.hadoop.ozone.protocolPB
|
|
.StorageContainerLocationProtocolServerSideTranslatorPB;
|
|
.StorageContainerLocationProtocolServerSideTranslatorPB;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
@@ -62,6 +71,7 @@ import java.io.IOException;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
import java.util.TreeSet;
|
|
import java.util.TreeSet;
|
|
import java.util.stream.Collectors;
|
|
import java.util.stream.Collectors;
|
|
@@ -84,9 +94,11 @@ import static org.apache.hadoop.hdds.scm.server.StorageContainerManager
|
|
* The RPC server that listens to requests from clients.
|
|
* The RPC server that listens to requests from clients.
|
|
*/
|
|
*/
|
|
public class SCMClientProtocolServer implements
|
|
public class SCMClientProtocolServer implements
|
|
- StorageContainerLocationProtocol, EventHandler<Boolean> {
|
|
|
|
|
|
+ StorageContainerLocationProtocol, EventHandler<Boolean>, Auditor {
|
|
private static final Logger LOG =
|
|
private static final Logger LOG =
|
|
LoggerFactory.getLogger(SCMClientProtocolServer.class);
|
|
LoggerFactory.getLogger(SCMClientProtocolServer.class);
|
|
|
|
+ private static final AuditLogger AUDIT =
|
|
|
|
+ new AuditLogger(AuditLoggerType.SCMLOGGER);
|
|
private final RPC.Server clientRpcServer;
|
|
private final RPC.Server clientRpcServer;
|
|
private final InetSocketAddress clientRpcAddress;
|
|
private final InetSocketAddress clientRpcAddress;
|
|
private final StorageContainerManager scm;
|
|
private final StorageContainerManager scm;
|
|
@@ -177,46 +189,84 @@ public class SCMClientProtocolServer implements
|
|
@Override
|
|
@Override
|
|
public ContainerInfo getContainer(long containerID) throws IOException {
|
|
public ContainerInfo getContainer(long containerID) throws IOException {
|
|
String remoteUser = getRpcRemoteUsername();
|
|
String remoteUser = getRpcRemoteUsername();
|
|
|
|
+ boolean auditSuccess = true;
|
|
|
|
+ Map<String, String> auditMap = Maps.newHashMap();
|
|
|
|
+ auditMap.put("containerID", String.valueOf(containerID));
|
|
getScm().checkAdminAccess(remoteUser);
|
|
getScm().checkAdminAccess(remoteUser);
|
|
- return scm.getContainerManager()
|
|
|
|
- .getContainer(ContainerID.valueof(containerID));
|
|
|
|
|
|
+ try {
|
|
|
|
+ return scm.getContainerManager()
|
|
|
|
+ .getContainer(ContainerID.valueof(containerID));
|
|
|
|
+ } catch (IOException ex) {
|
|
|
|
+ auditSuccess = false;
|
|
|
|
+ AUDIT.logReadFailure(
|
|
|
|
+ buildAuditMessageForFailure(SCMAction.GET_CONTAINER, auditMap, ex)
|
|
|
|
+ );
|
|
|
|
+ throw ex;
|
|
|
|
+ } finally {
|
|
|
|
+ if(auditSuccess) {
|
|
|
|
+ AUDIT.logReadSuccess(
|
|
|
|
+ buildAuditMessageForSuccess(SCMAction.GET_CONTAINER, auditMap)
|
|
|
|
+ );
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public ContainerWithPipeline getContainerWithPipeline(long containerID)
|
|
public ContainerWithPipeline getContainerWithPipeline(long containerID)
|
|
throws IOException {
|
|
throws IOException {
|
|
- if (chillModePrecheck.isInChillMode()) {
|
|
|
|
- ContainerInfo contInfo = scm.getContainerManager()
|
|
|
|
- .getContainer(ContainerID.valueof(containerID));
|
|
|
|
- if (contInfo.isOpen()) {
|
|
|
|
- if (!hasRequiredReplicas(contInfo)) {
|
|
|
|
- throw new SCMException("Open container " + containerID + " doesn't"
|
|
|
|
- + " have enough replicas to service this operation in "
|
|
|
|
- + "Chill mode.", ResultCodes.CHILL_MODE_EXCEPTION);
|
|
|
|
|
|
+ Map<String, String> auditMap = Maps.newHashMap();
|
|
|
|
+ auditMap.put("containerID", String.valueOf(containerID));
|
|
|
|
+ boolean auditSuccess = true;
|
|
|
|
+ try {
|
|
|
|
+ if (chillModePrecheck.isInChillMode()) {
|
|
|
|
+ ContainerInfo contInfo = scm.getContainerManager()
|
|
|
|
+ .getContainer(ContainerID.valueof(containerID));
|
|
|
|
+ if (contInfo.isOpen()) {
|
|
|
|
+ if (!hasRequiredReplicas(contInfo)) {
|
|
|
|
+ throw new SCMException("Open container " + containerID + " doesn't"
|
|
|
|
+ + " have enough replicas to service this operation in "
|
|
|
|
+ + "Chill mode.", ResultCodes.CHILL_MODE_EXCEPTION);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- }
|
|
|
|
- getScm().checkAdminAccess(null);
|
|
|
|
-
|
|
|
|
- final ContainerID id = ContainerID.valueof(containerID);
|
|
|
|
- final ContainerInfo container = scm.getContainerManager().getContainer(id);
|
|
|
|
- final Pipeline pipeline;
|
|
|
|
-
|
|
|
|
- if (container.isOpen()) {
|
|
|
|
- // Ratis pipeline
|
|
|
|
- pipeline = scm.getPipelineManager()
|
|
|
|
- .getPipeline(container.getPipelineID());
|
|
|
|
- } else {
|
|
|
|
- pipeline = scm.getPipelineManager().createPipeline(
|
|
|
|
- HddsProtos.ReplicationType.STAND_ALONE,
|
|
|
|
- container.getReplicationFactor(),
|
|
|
|
- scm.getContainerManager()
|
|
|
|
- .getContainerReplicas(id).stream()
|
|
|
|
- .map(ContainerReplica::getDatanodeDetails)
|
|
|
|
- .collect(Collectors.toList()));
|
|
|
|
- }
|
|
|
|
|
|
+ getScm().checkAdminAccess(null);
|
|
|
|
+
|
|
|
|
+ final ContainerID id = ContainerID.valueof(containerID);
|
|
|
|
+ final ContainerInfo container = scm.getContainerManager().
|
|
|
|
+ getContainer(id);
|
|
|
|
+ final Pipeline pipeline;
|
|
|
|
+
|
|
|
|
+ if (container.isOpen()) {
|
|
|
|
+ // Ratis pipeline
|
|
|
|
+ pipeline = scm.getPipelineManager()
|
|
|
|
+ .getPipeline(container.getPipelineID());
|
|
|
|
+ } else {
|
|
|
|
+ pipeline = scm.getPipelineManager().createPipeline(
|
|
|
|
+ HddsProtos.ReplicationType.STAND_ALONE,
|
|
|
|
+ container.getReplicationFactor(),
|
|
|
|
+ scm.getContainerManager()
|
|
|
|
+ .getContainerReplicas(id).stream()
|
|
|
|
+ .map(ContainerReplica::getDatanodeDetails)
|
|
|
|
+ .collect(Collectors.toList()));
|
|
|
|
+ }
|
|
|
|
|
|
- return new ContainerWithPipeline(container, pipeline);
|
|
|
|
|
|
+ return new ContainerWithPipeline(container, pipeline);
|
|
|
|
+ } catch (IOException ex) {
|
|
|
|
+ auditSuccess = false;
|
|
|
|
+ AUDIT.logReadFailure(
|
|
|
|
+ buildAuditMessageForFailure(SCMAction.GET_CONTAINER_WITH_PIPELINE,
|
|
|
|
+ auditMap, ex)
|
|
|
|
+ );
|
|
|
|
+ throw ex;
|
|
|
|
+ } finally {
|
|
|
|
+ if(auditSuccess) {
|
|
|
|
+ AUDIT.logReadSuccess(
|
|
|
|
+ buildAuditMessageForSuccess(SCMAction.GET_CONTAINER_WITH_PIPELINE,
|
|
|
|
+ auditMap)
|
|
|
|
+ );
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -238,16 +288,51 @@ public class SCMClientProtocolServer implements
|
|
@Override
|
|
@Override
|
|
public List<ContainerInfo> listContainer(long startContainerID,
|
|
public List<ContainerInfo> listContainer(long startContainerID,
|
|
int count) throws IOException {
|
|
int count) throws IOException {
|
|
- return scm.getContainerManager().
|
|
|
|
- listContainer(ContainerID.valueof(startContainerID), count);
|
|
|
|
|
|
+ boolean auditSuccess = true;
|
|
|
|
+ Map<String, String> auditMap = Maps.newHashMap();
|
|
|
|
+ auditMap.put("startContainerID", String.valueOf(startContainerID));
|
|
|
|
+ auditMap.put("count", String.valueOf(count));
|
|
|
|
+ try {
|
|
|
|
+ return scm.getContainerManager().
|
|
|
|
+ listContainer(ContainerID.valueof(startContainerID), count);
|
|
|
|
+ } catch (Exception ex) {
|
|
|
|
+ auditSuccess = false;
|
|
|
|
+ AUDIT.logReadFailure(
|
|
|
|
+ buildAuditMessageForFailure(SCMAction.LIST_CONTAINER, auditMap, ex));
|
|
|
|
+ throw ex;
|
|
|
|
+ } finally {
|
|
|
|
+ if(auditSuccess) {
|
|
|
|
+ AUDIT.logReadSuccess(
|
|
|
|
+ buildAuditMessageForSuccess(SCMAction.LIST_CONTAINER, auditMap));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void deleteContainer(long containerID) throws IOException {
|
|
public void deleteContainer(long containerID) throws IOException {
|
|
String remoteUser = getRpcRemoteUsername();
|
|
String remoteUser = getRpcRemoteUsername();
|
|
- getScm().checkAdminAccess(remoteUser);
|
|
|
|
- scm.getContainerManager().deleteContainer(ContainerID.valueof(containerID));
|
|
|
|
-
|
|
|
|
|
|
+ boolean auditSuccess = true;
|
|
|
|
+ Map<String, String> auditMap = Maps.newHashMap();
|
|
|
|
+ auditMap.put("containerID", String.valueOf(containerID));
|
|
|
|
+ auditMap.put("remoteUser", remoteUser);
|
|
|
|
+ try {
|
|
|
|
+ getScm().checkAdminAccess(remoteUser);
|
|
|
|
+ scm.getContainerManager().deleteContainer(
|
|
|
|
+ ContainerID.valueof(containerID));
|
|
|
|
+ } catch (Exception ex) {
|
|
|
|
+ auditSuccess = false;
|
|
|
|
+ AUDIT.logWriteFailure(
|
|
|
|
+ buildAuditMessageForFailure(SCMAction.DELETE_CONTAINER, auditMap, ex)
|
|
|
|
+ );
|
|
|
|
+ throw ex;
|
|
|
|
+ } finally {
|
|
|
|
+ if(auditSuccess) {
|
|
|
|
+ AUDIT.logWriteSuccess(
|
|
|
|
+ buildAuditMessageForSuccess(SCMAction.DELETE_CONTAINER, auditMap)
|
|
|
|
+ );
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -311,26 +396,48 @@ public class SCMClientProtocolServer implements
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public List<Pipeline> listPipelines() {
|
|
public List<Pipeline> listPipelines() {
|
|
|
|
+ AUDIT.logReadSuccess(
|
|
|
|
+ buildAuditMessageForSuccess(SCMAction.LIST_PIPELINE, null));
|
|
return scm.getPipelineManager().getPipelines();
|
|
return scm.getPipelineManager().getPipelines();
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void closePipeline(HddsProtos.PipelineID pipelineID)
|
|
public void closePipeline(HddsProtos.PipelineID pipelineID)
|
|
throws IOException {
|
|
throws IOException {
|
|
|
|
+ Map<String, String> auditMap = Maps.newHashMap();
|
|
|
|
+ auditMap.put("pipelineID", pipelineID.getId());
|
|
PipelineManager pipelineManager = scm.getPipelineManager();
|
|
PipelineManager pipelineManager = scm.getPipelineManager();
|
|
Pipeline pipeline =
|
|
Pipeline pipeline =
|
|
pipelineManager.getPipeline(PipelineID.getFromProtobuf(pipelineID));
|
|
pipelineManager.getPipeline(PipelineID.getFromProtobuf(pipelineID));
|
|
RatisPipelineUtils
|
|
RatisPipelineUtils
|
|
.finalizeAndDestroyPipeline(pipelineManager, pipeline, conf, false);
|
|
.finalizeAndDestroyPipeline(pipelineManager, pipeline, conf, false);
|
|
|
|
+ AUDIT.logWriteSuccess(
|
|
|
|
+ buildAuditMessageForSuccess(SCMAction.CLOSE_PIPELINE, null)
|
|
|
|
+ );
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public ScmInfo getScmInfo() throws IOException {
|
|
public ScmInfo getScmInfo() throws IOException {
|
|
- ScmInfo.Builder builder =
|
|
|
|
- new ScmInfo.Builder()
|
|
|
|
- .setClusterId(scm.getScmStorage().getClusterID())
|
|
|
|
- .setScmId(scm.getScmStorage().getScmId());
|
|
|
|
- return builder.build();
|
|
|
|
|
|
+ boolean auditSuccess = true;
|
|
|
|
+ try{
|
|
|
|
+ ScmInfo.Builder builder =
|
|
|
|
+ new ScmInfo.Builder()
|
|
|
|
+ .setClusterId(scm.getScmStorage().getClusterID())
|
|
|
|
+ .setScmId(scm.getScmStorage().getScmId());
|
|
|
|
+ return builder.build();
|
|
|
|
+ } catch (Exception ex) {
|
|
|
|
+ auditSuccess = false;
|
|
|
|
+ AUDIT.logReadFailure(
|
|
|
|
+ buildAuditMessageForFailure(SCMAction.GET_SCM_INFO, null, ex)
|
|
|
|
+ );
|
|
|
|
+ throw ex;
|
|
|
|
+ } finally {
|
|
|
|
+ if(auditSuccess) {
|
|
|
|
+ AUDIT.logReadSuccess(
|
|
|
|
+ buildAuditMessageForSuccess(SCMAction.GET_SCM_INFO, null)
|
|
|
|
+ );
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -341,6 +448,9 @@ public class SCMClientProtocolServer implements
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public boolean inChillMode() throws IOException {
|
|
public boolean inChillMode() throws IOException {
|
|
|
|
+ AUDIT.logReadSuccess(
|
|
|
|
+ buildAuditMessageForSuccess(SCMAction.IN_CHILL_MODE, null)
|
|
|
|
+ );
|
|
return scm.isInChillMode();
|
|
return scm.isInChillMode();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -352,6 +462,9 @@ public class SCMClientProtocolServer implements
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public boolean forceExitChillMode() throws IOException {
|
|
public boolean forceExitChillMode() throws IOException {
|
|
|
|
+ AUDIT.logWriteSuccess(
|
|
|
|
+ buildAuditMessageForSuccess(SCMAction.FORCE_EXIT_CHILL_MODE, null)
|
|
|
|
+ );
|
|
return scm.exitChillMode();
|
|
return scm.exitChillMode();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -409,4 +522,34 @@ public class SCMClientProtocolServer implements
|
|
}
|
|
}
|
|
return returnSet;
|
|
return returnSet;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public AuditMessage buildAuditMessageForSuccess(
|
|
|
|
+ AuditAction op, Map<String, String> auditMap) {
|
|
|
|
+ return new AuditMessage.Builder()
|
|
|
|
+ .setUser((Server.getRemoteUser() == null) ? null :
|
|
|
|
+ Server.getRemoteUser().getUserName())
|
|
|
|
+ .atIp((Server.getRemoteIp() == null) ? null :
|
|
|
|
+ Server.getRemoteIp().getHostAddress())
|
|
|
|
+ .forOperation(op.getAction())
|
|
|
|
+ .withParams(auditMap)
|
|
|
|
+ .withResult(AuditEventStatus.SUCCESS.toString())
|
|
|
|
+ .withException(null)
|
|
|
|
+ .build();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public AuditMessage buildAuditMessageForFailure(AuditAction op, Map<String,
|
|
|
|
+ String> auditMap, Throwable throwable) {
|
|
|
|
+ return new AuditMessage.Builder()
|
|
|
|
+ .setUser((Server.getRemoteUser() == null) ? null :
|
|
|
|
+ Server.getRemoteUser().getUserName())
|
|
|
|
+ .atIp((Server.getRemoteIp() == null) ? null :
|
|
|
|
+ Server.getRemoteIp().getHostAddress())
|
|
|
|
+ .forOperation(op.getAction())
|
|
|
|
+ .withParams(auditMap)
|
|
|
|
+ .withResult(AuditEventStatus.FAILURE.toString())
|
|
|
|
+ .withException(throwable)
|
|
|
|
+ .build();
|
|
|
|
+ }
|
|
}
|
|
}
|