|
@@ -28,6 +28,8 @@ 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.ozone.container.common.helpers.ContainerMetrics;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
|
@@ -42,6 +44,9 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|
|
.ContainerCommandResponseProto;
|
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|
|
.ContainerType;
|
|
|
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
|
|
|
+ ContainerDataProto.State;
|
|
|
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
|
|
|
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -99,6 +104,25 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
volumeSet.shutdown();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Returns true for exceptions which can be ignored for marking the container
|
|
|
+ * unhealthy.
|
|
|
+ * @param result ContainerCommandResponse error code.
|
|
|
+ * @return true if exception can be ignored, false otherwise.
|
|
|
+ */
|
|
|
+ private boolean canIgnoreException(Result result) {
|
|
|
+ switch (result) {
|
|
|
+ case SUCCESS:
|
|
|
+ case CONTAINER_UNHEALTHY:
|
|
|
+ case CLOSED_CONTAINER_IO:
|
|
|
+ case DELETE_ON_OPEN_CONTAINER:
|
|
|
+ case ERROR_CONTAINER_NOT_EMPTY:
|
|
|
+ return true;
|
|
|
+ default:
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public ContainerCommandResponseProto dispatch(
|
|
|
ContainerCommandRequestProto msg) {
|
|
@@ -160,19 +184,30 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
// failed. All subsequent transactions on the container should fail and
|
|
|
// hence replica will be marked unhealthy here. In this case, a close
|
|
|
// container action will be sent to SCM to close the container.
|
|
|
- if (!HddsUtils.isReadOnly(msg)
|
|
|
- && responseProto.getResult() != ContainerProtos.Result.SUCCESS) {
|
|
|
- // If the container is open and the container operation has failed,
|
|
|
- // it should be first marked unhealthy and the initiate the close
|
|
|
- // container action. This also implies this is the first transaction
|
|
|
- // which has failed, so the container is marked unhealthy right here.
|
|
|
+
|
|
|
+ // ApplyTransaction called on closed Container will fail with Closed
|
|
|
+ // container exception. In such cases, ignore the exception here
|
|
|
+ // If the container is already marked unhealthy, no need to change the
|
|
|
+ // state here.
|
|
|
+
|
|
|
+ Result result = responseProto.getResult();
|
|
|
+ if (!HddsUtils.isReadOnly(msg) && !canIgnoreException(result)) {
|
|
|
+ // If the container is open/closing and the container operation
|
|
|
+ // has failed, it should be first marked unhealthy and the initiate the
|
|
|
+ // close container action. This also implies this is the first
|
|
|
+ // transaction which has failed, so the container is marked unhealthy
|
|
|
+ // right here.
|
|
|
// Once container is marked unhealthy, all the subsequent write
|
|
|
// transactions will fail with UNHEALTHY_CONTAINER exception.
|
|
|
- if (container.getContainerState() == ContainerDataProto.State.OPEN) {
|
|
|
- container.getContainerData()
|
|
|
- .setState(ContainerDataProto.State.UNHEALTHY);
|
|
|
- sendCloseContainerActionIfNeeded(container);
|
|
|
- }
|
|
|
+
|
|
|
+ // For container to be moved to unhealthy state here, the container can
|
|
|
+ // only be in open or closing state.
|
|
|
+ State containerState = container.getContainerData().getState();
|
|
|
+ Preconditions.checkState(
|
|
|
+ containerState == State.OPEN || containerState == State.CLOSING);
|
|
|
+ container.getContainerData()
|
|
|
+ .setState(ContainerDataProto.State.UNHEALTHY);
|
|
|
+ sendCloseContainerActionIfNeeded(container);
|
|
|
}
|
|
|
return responseProto;
|
|
|
} else {
|
|
@@ -206,6 +241,54 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
handler.handle(requestBuilder.build(), null);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * This will be called as a part of creating the log entry during
|
|
|
+ * startTransaction in Ratis on the leader node. In such cases, if the
|
|
|
+ * container is not in open state for writing we should just fail.
|
|
|
+ * Leader will propagate the exception to client.
|
|
|
+ * @param msg container command proto
|
|
|
+ * @throws StorageContainerException In case container state is open for write
|
|
|
+ * requests and in invalid state for read requests.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public void validateContainerCommand(
|
|
|
+ ContainerCommandRequestProto msg) throws StorageContainerException {
|
|
|
+ ContainerType containerType = msg.getCreateContainer().getContainerType();
|
|
|
+ Handler handler = getHandler(containerType);
|
|
|
+ if (handler == null) {
|
|
|
+ StorageContainerException ex = new StorageContainerException(
|
|
|
+ "Invalid " + "ContainerType " + containerType,
|
|
|
+ ContainerProtos.Result.CONTAINER_INTERNAL_ERROR);
|
|
|
+ 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) {
|
|
|
+ switch (cmdType) {
|
|
|
+ case CreateContainer:
|
|
|
+ // Create Container is idempotent. There is nothing to validate.
|
|
|
+ break;
|
|
|
+ case CloseContainer:
|
|
|
+ // If the container is unhealthy, closeContainer will be rejected
|
|
|
+ // while execution. Nothing to validate here.
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ // if the container is not open, no updates can happen. Just throw
|
|
|
+ // an exception
|
|
|
+ throw new ContainerNotOpenException(
|
|
|
+ "Container " + containerID + " in " + containerState + " state");
|
|
|
+ }
|
|
|
+ } else if (HddsUtils.isReadOnly(msg) && containerState == State.INVALID) {
|
|
|
+ throw new InvalidContainerStateException(
|
|
|
+ "Container " + containerID + " in " + containerState + " state");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* If the container usage reaches the close threshold or the container is
|
|
|
* marked unhealthy we send Close ContainerAction to SCM.
|
|
@@ -264,7 +347,6 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
public Container getContainer(long containerID) {
|
|
|
return containerSet.getContainer(containerID);
|
|
|
}
|