-
Notifications
You must be signed in to change notification settings - Fork 587
HDDS-11667. Validating DatanodeID on any request to the datanode #7418
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 9 commits
42b036f
e111735
fcd8c5e
1d85c17
fae0234
989d564
844b407
aa760ce
d3094fa
2f9b611
13e000d
ae49c6c
2f8efa3
028a3ed
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -330,7 +330,8 @@ public List<DatanodeDetails> getNodesInOrder() { | |
| } | ||
|
|
||
| void reportDatanode(DatanodeDetails dn) throws IOException { | ||
| if (nodeStatus.get(dn) == null) { | ||
| if (dn == null || (nodeStatus.get(dn) == null | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Which case is this validation expected to catch?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is the case where a DN has gone down and has come back up with a different dnId. In such a case the DN could still be a part of an existing pipeline. The registeration of this dn on SCM should not fail since it is the same node. This is a very dumb way to work around https://issues.apache.org/jira/browse/HDDS-11670. Eventually these pipeline would get closed once the datanode proposes to close the existing pipeline on the next in-flight write or SCM will close it automatically marking the older datanodeId as dead and closing all the pipelines which contains the node with the old datanode id. |
||
| && nodeStatus.keySet().stream().noneMatch(node -> node.validateNodeValue(dn)))) { | ||
| throw new IOException( | ||
| String.format("Datanode=%s not part of pipeline=%s", dn, id)); | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
|
|
@@ -65,6 +65,7 @@ | |||
| import org.apache.hadoop.ozone.HddsDatanodeService; | ||||
| import org.apache.hadoop.ozone.OzoneConfigKeys; | ||||
| import org.apache.hadoop.ozone.common.utils.BufferUtils; | ||||
| import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils; | ||||
| import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; | ||||
| import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; | ||||
| import org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel; | ||||
|
|
@@ -78,6 +79,7 @@ | |||
| import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto; | ||||
| import org.apache.ratis.protocol.Message; | ||||
| import org.apache.ratis.protocol.RaftClientRequest; | ||||
| import org.apache.ratis.protocol.RaftGroup; | ||||
| import org.apache.ratis.protocol.RaftGroupId; | ||||
| import org.apache.ratis.protocol.RaftGroupMemberId; | ||||
| import org.apache.ratis.protocol.RaftPeer; | ||||
|
|
@@ -202,6 +204,7 @@ long getStartTime() { | |||
| private final boolean waitOnBothFollowers; | ||||
| private final HddsDatanodeService datanodeService; | ||||
| private static Semaphore semaphore = new Semaphore(1); | ||||
| private final AtomicBoolean peersValidated; | ||||
|
|
||||
| /** | ||||
| * CSM metrics. | ||||
|
|
@@ -252,6 +255,7 @@ public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupI | |||
| HDDS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS_DEFAULT); | ||||
| applyTransactionSemaphore = new Semaphore(maxPendingApplyTransactions); | ||||
| stateMachineHealthy = new AtomicBoolean(true); | ||||
| this.peersValidated = new AtomicBoolean(false); | ||||
|
|
||||
| ThreadFactory threadFactory = new ThreadFactoryBuilder() | ||||
| .setNameFormat( | ||||
|
|
@@ -265,6 +269,19 @@ public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupI | |||
|
|
||||
| } | ||||
|
|
||||
| private void validatePeers(RaftServer server, RaftGroupId id) throws IOException { | ||||
swamirishi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||||
| if (this.peersValidated.get()) { | ||||
| return; | ||||
| } | ||||
| final RaftGroup group = ratisServer.getServerDivision(getGroupId()).getGroup(); | ||||
| final RaftPeerId selfId = ratisServer.getServer().getId(); | ||||
| if (group.getPeer(selfId) == null) { | ||||
| throw new StorageContainerException("Current datanode " + selfId + " is not a member of " + group, | ||||
| ContainerProtos.Result.INVALID_CONFIG); | ||||
| } | ||||
| peersValidated.set(true); | ||||
| } | ||||
|
|
||||
| @Override | ||||
| public StateMachineStorage getStateMachineStorage() { | ||||
| return storage; | ||||
|
|
@@ -962,6 +979,11 @@ private CompletableFuture<ContainerCommandResponseProto> applyTransaction( | |||
| final CheckedSupplier<ContainerCommandResponseProto, Exception> task | ||||
| = () -> { | ||||
| try { | ||||
| try { | ||||
| this.validatePeers(this.ratisServer.getServer(), getGroupId()); | ||||
| } catch (StorageContainerException e) { | ||||
| return ContainerUtils.logAndReturnError(LOG, e, request); | ||||
| } | ||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What is the expectation? If the peer is not in the group, fail it? If it is the case, we should do it earlier. Why not doing it in
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The datanode proposes to close the pipeline to SCM only on the applyTransaction block. Correct me if I am wrong here.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Line 1104 in aa760ce
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just invoke
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We would want to close the entire raft group. If we do it in the start transaction, just the add log will fail on one server right? We would be just performing this check on the leader right? Correct me if I am wrong here. We need this block running on followers as well.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If you really want to check leader, check it also in the other TransactionContext startTransaction(RaftClientRequest request) throws IOException;
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we set the exception in the transactionContext?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Would this result in the stateMachine getting closed?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Discussed offline:
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We would want to some kind of callback on state machine intialization on ratis eventually which should clear up the ratis dirs and remove it from the system. I have created a follow up task item for this RATIS-2187. |
||||
| long timeNow = Time.monotonicNowNanos(); | ||||
| long queueingDelay = timeNow - context.getStartTime(); | ||||
| metrics.recordQueueingDelay(request.getCmdType(), queueingDelay); | ||||
|
|
||||
Uh oh!
There was an error while loading. Please reload this page.