|
@@ -27,9 +27,21 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|
|
.ContainerDataProto;
|
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
|
.StorageContainerDatanodeProtocolProtos.ContainerAction;
|
|
|
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException;
|
|
|
-import org.apache.hadoop.hdds.scm.container.common.helpers.InvalidContainerStateException;
|
|
|
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
|
|
+import org.apache.hadoop.hdds.scm.container.common.helpers
|
|
|
+ .ContainerNotOpenException;
|
|
|
+import org.apache.hadoop.hdds.scm.container.common.helpers
|
|
|
+ .InvalidContainerStateException;
|
|
|
+import org.apache.hadoop.hdds.scm.container.common.helpers
|
|
|
+ .StorageContainerException;
|
|
|
+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.AuditMarker;
|
|
|
+import org.apache.hadoop.ozone.audit.AuditMessage;
|
|
|
+import org.apache.hadoop.ozone.audit.Auditor;
|
|
|
+import org.apache.hadoop.ozone.container.common.helpers
|
|
|
+ .ContainerCommandRequestPBHelper;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
|
|
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
|
@@ -57,10 +69,11 @@ import java.util.Optional;
|
|
|
* Ozone Container dispatcher takes a call from the netty server and routes it
|
|
|
* to the right handler function.
|
|
|
*/
|
|
|
-public class HddsDispatcher implements ContainerDispatcher {
|
|
|
+public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
|
|
|
|
|
static final Logger LOG = LoggerFactory.getLogger(HddsDispatcher.class);
|
|
|
-
|
|
|
+ private static final AuditLogger AUDIT =
|
|
|
+ new AuditLogger(AuditLoggerType.DNLOGGER);
|
|
|
private final Map<ContainerType, Handler> handlers;
|
|
|
private final Configuration conf;
|
|
|
private final ContainerSet containerSet;
|
|
@@ -123,7 +136,13 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
ContainerCommandRequestProto msg) {
|
|
|
LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
|
|
|
msg.getTraceID());
|
|
|
- Preconditions.checkNotNull(msg);
|
|
|
+ Preconditions.checkNotNull(msg.toString());
|
|
|
+
|
|
|
+ AuditAction action = ContainerCommandRequestPBHelper.getAuditAction(
|
|
|
+ msg.getCmdType());
|
|
|
+ EventType eventType = getEventType(msg);
|
|
|
+ Map<String, String> params =
|
|
|
+ ContainerCommandRequestPBHelper.getAuditParams(msg);
|
|
|
|
|
|
Container container = null;
|
|
|
ContainerType containerType = null;
|
|
@@ -149,11 +168,14 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
StorageContainerException sce = new StorageContainerException(
|
|
|
"ContainerID " + containerID + " does not exist",
|
|
|
ContainerProtos.Result.CONTAINER_NOT_FOUND);
|
|
|
+ audit(action, eventType, params, AuditEventStatus.FAILURE, sce);
|
|
|
return ContainerUtils.logAndReturnError(LOG, sce, msg);
|
|
|
}
|
|
|
containerType = getContainerType(container);
|
|
|
} else {
|
|
|
if (!msg.hasCreateContainer()) {
|
|
|
+ audit(action, eventType, params, AuditEventStatus.FAILURE,
|
|
|
+ new Exception("MALFORMED_REQUEST"));
|
|
|
return ContainerUtils.malformedRequest(msg);
|
|
|
}
|
|
|
containerType = msg.getCreateContainer().getContainerType();
|
|
@@ -168,6 +190,8 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
StorageContainerException ex = new StorageContainerException("Invalid " +
|
|
|
"ContainerType " + containerType,
|
|
|
ContainerProtos.Result.CONTAINER_INTERNAL_ERROR);
|
|
|
+ // log failure
|
|
|
+ audit(action, eventType, params, AuditEventStatus.FAILURE, ex);
|
|
|
return ContainerUtils.logAndReturnError(LOG, ex, msg);
|
|
|
}
|
|
|
responseProto = handler.handle(msg, container);
|
|
@@ -204,8 +228,19 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
.setState(ContainerDataProto.State.UNHEALTHY);
|
|
|
sendCloseContainerActionIfNeeded(container);
|
|
|
}
|
|
|
+
|
|
|
+ if(result == Result.SUCCESS) {
|
|
|
+ audit(action, eventType, params, AuditEventStatus.SUCCESS, null);
|
|
|
+ } else {
|
|
|
+ audit(action, eventType, params, AuditEventStatus.FAILURE,
|
|
|
+ new Exception(responseProto.getMessage()));
|
|
|
+ }
|
|
|
+
|
|
|
return responseProto;
|
|
|
} else {
|
|
|
+ // log failure
|
|
|
+ audit(action, eventType, params, AuditEventStatus.FAILURE,
|
|
|
+ new Exception("UNSUPPORTED_REQUEST"));
|
|
|
return ContainerUtils.unsupportedRequest(msg);
|
|
|
}
|
|
|
}
|
|
@@ -249,17 +284,24 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
public void validateContainerCommand(
|
|
|
ContainerCommandRequestProto msg) throws StorageContainerException {
|
|
|
ContainerType containerType = msg.getCreateContainer().getContainerType();
|
|
|
+ ContainerProtos.Type cmdType = msg.getCmdType();
|
|
|
+ AuditAction action =
|
|
|
+ ContainerCommandRequestPBHelper.getAuditAction(cmdType);
|
|
|
+ EventType eventType = getEventType(msg);
|
|
|
+ Map<String, String> params =
|
|
|
+ ContainerCommandRequestPBHelper.getAuditParams(msg);
|
|
|
Handler handler = getHandler(containerType);
|
|
|
if (handler == null) {
|
|
|
StorageContainerException ex = new StorageContainerException(
|
|
|
"Invalid " + "ContainerType " + containerType,
|
|
|
ContainerProtos.Result.CONTAINER_INTERNAL_ERROR);
|
|
|
+ audit(action, eventType, params, AuditEventStatus.FAILURE, ex);
|
|
|
throw ex;
|
|
|
}
|
|
|
- ContainerProtos.Type cmdType = msg.getCmdType();
|
|
|
long containerID = msg.getContainerID();
|
|
|
Container container;
|
|
|
container = getContainer(containerID);
|
|
|
+
|
|
|
if (container != null) {
|
|
|
State containerState = container.getContainerState();
|
|
|
if (!HddsUtils.isReadOnly(msg) && containerState != State.OPEN) {
|
|
@@ -274,12 +316,16 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
default:
|
|
|
// if the container is not open, no updates can happen. Just throw
|
|
|
// an exception
|
|
|
- throw new ContainerNotOpenException(
|
|
|
+ ContainerNotOpenException cex = new ContainerNotOpenException(
|
|
|
"Container " + containerID + " in " + containerState + " state");
|
|
|
+ audit(action, eventType, params, AuditEventStatus.FAILURE, cex);
|
|
|
+ throw cex;
|
|
|
}
|
|
|
} else if (HddsUtils.isReadOnly(msg) && containerState == State.INVALID) {
|
|
|
- throw new InvalidContainerStateException(
|
|
|
+ InvalidContainerStateException iex = new InvalidContainerStateException(
|
|
|
"Container " + containerID + " in " + containerState + " state");
|
|
|
+ audit(action, eventType, params, AuditEventStatus.FAILURE, iex);
|
|
|
+ throw iex;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -355,4 +401,73 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
public void setMetricsForTesting(ContainerMetrics containerMetrics) {
|
|
|
this.metrics = containerMetrics;
|
|
|
}
|
|
|
+
|
|
|
+ private EventType getEventType(ContainerCommandRequestProto msg) {
|
|
|
+ return HddsUtils.isReadOnly(msg) ? EventType.READ : EventType.WRITE;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void audit(AuditAction action, EventType eventType,
|
|
|
+ Map<String, String> params, AuditEventStatus result, Throwable exception){
|
|
|
+ AuditMessage amsg;
|
|
|
+ switch (result) {
|
|
|
+ case SUCCESS:
|
|
|
+ if(eventType == EventType.READ &&
|
|
|
+ AUDIT.getLogger().isInfoEnabled(AuditMarker.READ.getMarker())) {
|
|
|
+ amsg = buildAuditMessageForSuccess(action, params);
|
|
|
+ AUDIT.logReadSuccess(amsg);
|
|
|
+ } else if(eventType == EventType.WRITE &&
|
|
|
+ AUDIT.getLogger().isInfoEnabled(AuditMarker.WRITE.getMarker())) {
|
|
|
+ amsg = buildAuditMessageForSuccess(action, params);
|
|
|
+ AUDIT.logWriteSuccess(amsg);
|
|
|
+ }
|
|
|
+ break;
|
|
|
+
|
|
|
+ case FAILURE:
|
|
|
+ if(eventType == EventType.READ &&
|
|
|
+ AUDIT.getLogger().isErrorEnabled(AuditMarker.READ.getMarker())) {
|
|
|
+ amsg = buildAuditMessageForFailure(action, params, exception);
|
|
|
+ AUDIT.logReadFailure(amsg);
|
|
|
+ } else if(eventType == EventType.WRITE &&
|
|
|
+ AUDIT.getLogger().isErrorEnabled(AuditMarker.WRITE.getMarker())) {
|
|
|
+ amsg = buildAuditMessageForFailure(action, params, exception);
|
|
|
+ AUDIT.logWriteFailure(amsg);
|
|
|
+ }
|
|
|
+ break;
|
|
|
+
|
|
|
+ default: LOG.debug("Invalid audit event status - " + result);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ //TODO: use GRPC to fetch user and ip details
|
|
|
+ @Override
|
|
|
+ public AuditMessage buildAuditMessageForSuccess(AuditAction op,
|
|
|
+ Map<String, String> auditMap) {
|
|
|
+ return new AuditMessage.Builder()
|
|
|
+ .setUser(null)
|
|
|
+ .atIp(null)
|
|
|
+ .forOperation(op.getAction())
|
|
|
+ .withParams(auditMap)
|
|
|
+ .withResult(AuditEventStatus.SUCCESS.toString())
|
|
|
+ .withException(null)
|
|
|
+ .build();
|
|
|
+ }
|
|
|
+
|
|
|
+ //TODO: use GRPC to fetch user and ip details
|
|
|
+ @Override
|
|
|
+ public AuditMessage buildAuditMessageForFailure(AuditAction op,
|
|
|
+ Map<String, String> auditMap, Throwable throwable) {
|
|
|
+ return new AuditMessage.Builder()
|
|
|
+ .setUser(null)
|
|
|
+ .atIp(null)
|
|
|
+ .forOperation(op.getAction())
|
|
|
+ .withParams(auditMap)
|
|
|
+ .withResult(AuditEventStatus.FAILURE.toString())
|
|
|
+ .withException(throwable)
|
|
|
+ .build();
|
|
|
+ }
|
|
|
+
|
|
|
+ enum EventType {
|
|
|
+ READ,
|
|
|
+ WRITE
|
|
|
+ }
|
|
|
}
|