sumitagrawl commented on code in PR #3963: URL: https://github.com/apache/ozone/pull/3963#discussion_r1031674205
########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/move/MoveManagerImpl.java: ########## @@ -0,0 +1,777 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdds.scm.container.move; + +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; +import org.apache.hadoop.hdds.scm.ContainerPlacementStatus; +import org.apache.hadoop.hdds.scm.PlacementPolicyValidateProxy; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.ContainerManager; +import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; +import org.apache.hadoop.hdds.scm.container.ContainerReplica; +import org.apache.hadoop.hdds.scm.container.ContainerReplicaNotFoundException; +import org.apache.hadoop.hdds.scm.container.common.helpers.MoveDataNodePair; +import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaCount; +import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp; +import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType; +import org.apache.hadoop.hdds.scm.events.SCMEvents; +import org.apache.hadoop.hdds.scm.ha.SCMContext; +import org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler; +import org.apache.hadoop.hdds.scm.ha.SCMRatisServer; +import org.apache.hadoop.hdds.scm.metadata.DBTransactionBuffer; +import org.apache.hadoop.hdds.scm.metadata.Replicate; +import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.node.NodeStatus; +import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; +import org.apache.hadoop.hdds.scm.server.StorageContainerManager; +import org.apache.hadoop.hdds.server.events.EventPublisher; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; +import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand; +import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; +import org.apache.ratis.protocol.exceptions.NotLeaderException; +import org.apache.ratis.util.ExitUtils; +import org.apache.ratis.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Proxy; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +import static org.apache.hadoop.hdds.protocol.proto.SCMRatisProtocol.RequestType.MOVE; + +/** + * An HA enabled implementation of MoveManager. + */ +public final class MoveManagerImpl implements MoveManager { + public static final Logger LOG = + LoggerFactory.getLogger(MoveManagerImpl.class); + + private final SCMContext scmContext; + private final NodeManager nodeManager; + private final ContainerManager containerManager; + private final PlacementPolicyValidateProxy placementPolicyValidateProxy; + private final EventPublisher eventPublisher; + private final StorageContainerManager scm; + private final Map<ContainerID, CompletableFuture<MoveResult>> + pendingMoveFuture = new ConcurrentHashMap<>(); + private final Map<ContainerID, MoveDataNodePair> + pendingMoveOps = new ConcurrentHashMap<>(); + private final MoveDBHandler moveDBHandler; + private volatile boolean running = false; + + public MoveManagerImpl(final StorageContainerManager scm) throws IOException { + this.nodeManager = scm.getScmNodeManager(); + this.containerManager = scm.getContainerManager(); + this.scmContext = scm.getScmContext(); + this.placementPolicyValidateProxy = scm.getPlacementPolicyValidateProxy(); + this.eventPublisher = scm.getEventQueue(); + this.scm = scm; + this.moveDBHandler = new MoveDBHandlerImpl.Builder() + .setMoveTable(scm.getScmMetadataStore().getMoveTable()) + .setRatisServer(scm.getScmHAManager().getRatisServer()) + .setDBTransactionBuffer(scm.getScmHAManager().getDBTransactionBuffer()) + .build(); + } + + /** + * get all the pending move operations . + */ + public Map<ContainerID, MoveDataNodePair> getPendingMove() { + return pendingMoveOps; + } + + /** + * completeMove a move action for a given container. + * + * @param cid Container id to which the move option is finished + */ + private void completeMove(final ContainerID cid, final MoveResult mr) { + if (pendingMoveOps.containsKey(cid)) { + boolean successful = false; + try { + successful = moveDBHandler.deleteFromDB(cid); + } catch (TimeoutException t) { + //TimeoutException here is caused by ratis request timeout. + //we do not throw TimeoutException out here since completeMove is called + //in many places. Handling this exception inside itself seems + //a better way to have lower complexity. + LOG.warn("fail to delete move from DB"); + } + + // no matter deletion is successful or not, we could complete the + // corresponding pending CompletableFuture, because : + // 1 if completeMove is called , it means move is actually completed. + // 2 a CompletableFuture here means a `move` method call from + // container balancer, and balancer will block at CompletableFuture.get(), + // so we would better let it return as soon as possible. + CompletableFuture<MoveResult> future = pendingMoveFuture.get(cid); + pendingMoveFuture.remove(cid); + if (successful) { + pendingMoveOps.remove(cid); + if (future != null && mr != null) { + // when we know the future is null, and we want to complete + // the move , then we set mr to null. + // for example , after becoming a new leader, we might need + // to complete some moves and we know these futures does not + // exist. + future.complete(mr); + } + } else { + //TODO: if unsuccessful(actually rarely), cid will not be removed + // from pendingMoveOps. we need to try to delete cid from db and + // remove it from pendingMoveOps again. we could start a background + // service to handle those failed cases. + // Note that, when scm leader switches, new leader will also try + // to delete those stale completed move from db. + if (future != null) { + future.complete(MoveResult.FAIL_CAN_NOT_RECORD_TO_DB); + } + } + } + } + + /** + * start a move for a given container. + * + * @param cid Container id + * @param src source datanode + * @param tgt target datanode + */ + private void startMove( + final ContainerID cid, final DatanodeDetails src, + final DatanodeDetails tgt, final CompletableFuture<MoveResult> ret) + throws TimeoutException, ContainerReplicaNotFoundException, + ContainerNotFoundException { + if (!pendingMoveOps.containsKey(cid)) { + MoveDataNodePair mp = new MoveDataNodePair(src, tgt); + if (moveDBHandler.addToDB(cid, mp)) { + sendReplicateCommand(cid, tgt, src); + pendingMoveOps.putIfAbsent(cid, mp); + pendingMoveFuture.putIfAbsent(cid, ret); + } else { + ret.complete(MoveResult.FAIL_CAN_NOT_RECORD_TO_DB); + } + } + } + + /** + * notify MoveManager that the current scm has become leader and ready. + */ + @Override + public void onLeaderReady() { + //discard all stale records + pendingMoveOps.clear(); + pendingMoveFuture.clear(); + + //initialize with db record + try (TableIterator<ContainerID, Review Comment: When leader is ready, check if records are flushed as present in transaction buffer on switch to leader. Otherwise table will not have any records. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
