bharatviswa504 commented on a change in pull request #2141:
URL: https://github.com/apache/ozone/pull/2141#discussion_r613000389



##########
File path: 
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ServerNotLeaderException.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ratis;
+
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ServerNotLeaderException extends IOException {
+  private final String currentPeerId;
+  private final String leader;
+  private static final Pattern CURRENT_PEER_ID_PATTERN =
+      Pattern.compile("Server:(.*) is not the leader[.]+.*", Pattern.DOTALL);
+  private static final Pattern SUGGESTED_LEADER_PATTERN =
+      Pattern.compile(".*Suggested leader is Server:([^.]*).*", 
Pattern.DOTALL);
+
+  public ServerNotLeaderException(RaftPeerId currentPeerId) {
+    super("Server:" + currentPeerId + " is not the leader. Could not " +
+        "determine the leader node.");
+    this.currentPeerId = currentPeerId.toString();
+    this.leader = null;
+  }
+
+  public ServerNotLeaderException(RaftPeerId currentPeerId,
+      String suggestedLeader) {
+    super("Server:" + currentPeerId + " is not the leader. Suggested leader is"
+        + " Server:" + suggestedLeader + ".");
+    this.currentPeerId = currentPeerId.toString();
+    this.leader = suggestedLeader;
+  }
+
+  public ServerNotLeaderException(String message) {

Review comment:
       Minor NIT: Unused constructor.

##########
File path: 
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ServerNotLeaderException.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ratis;
+
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+

Review comment:
       Minor: Add JavaDoc.

##########
File path: 
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMContainerLocationFailoverProxyProvider.java
##########
@@ -154,13 +162,26 @@ public synchronized void performFailover(
     LOG.debug("Failing over to next proxy. {}", getCurrentProxySCMNodeId());
   }
 
-  public synchronized void performFailoverToAssignedLeader(String newLeader) {
+  public void performFailoverToAssignedLeader(String newLeader, Exception e) {
+    ServerNotLeaderException snle =
+        (ServerNotLeaderException) SCMHAUtils.getServerNotLeaderException(e);
+    if (snle != null && snle.getSuggestedLeader() != null) {
+      newLeader = scmProxyInfoMap.values().stream().filter(
+          proxyInfo -> NetUtils.getHostPortString(proxyInfo.getAddress())
+              
.equals(snle.getSuggestedLeader())).findFirst().get().getNodeId();
+      LOG.debug("Performing failover to suggested leader {}, nodeId {}",
+          snle.getSuggestedLeader(), newLeader);
+    }
     if (newLeader == null) {
-      // If newLeader is not assigned, fail over to next proxy.
-      nextProxyIndex();
-    } else if (!assignLeaderToNode(newLeader)) {
-      // If failed to fail over to newLeader, fail over to next proxy.
+      // If newLeader is not assigned, it will fail over to next proxy.
       nextProxyIndex();
+      LOG.debug("Performing failover to next proxy node {}",
+          currentProxySCMNodeId);
+    } else {
+      if (!assignLeaderToNode(newLeader)) {
+        LOG.debug("Failing over SCM proxy to nodeId: {}", newLeader);
+        nextProxyIndex();

Review comment:
       Same as above

##########
File path: 
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMBlockLocationFailoverProxyProvider.java
##########
@@ -250,7 +268,7 @@ public RetryPolicy getSCMBlockLocationRetryPolicy(String 
newLeader) {
       public RetryAction shouldRetry(Exception e, int retry,
                                      int failover, boolean b) {
         if (!SCMHAUtils.isRetriableWithNoFailoverException(e)) {
-          performFailoverToAssignedLeader(newLeader);
+          performFailoverToAssignedLeader(newLeader, e);

Review comment:
       In performFailoverToAssignedLeader when newLeader is null, we still 
failover to nextProxy. Eventough the exception type here is like 
RETRIABLE_WITH_NO_FAILOVER_EXCEPTION_LIST

##########
File path: 
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMSecurityProtocolFailoverProxyProvider.java
##########
@@ -181,6 +182,41 @@ public void performFailover(SCMSecurityProtocolPB 
currentProxy) {
     }
   }
 
+  public void performFailoverToAssignedLeader(String newLeader, Exception e) {

Review comment:
       Can we move this to a utility class, and make it a utillity method by 
passing all required parameters.
   So that we can avoid code de-duplication, and easier to fix the logic 
further.

##########
File path: 
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
##########
@@ -119,7 +119,8 @@ private SCMBlockLocationResponse submitRequest(
       if (response.getStatus() ==
           ScmBlockLocationProtocolProtos.Status.SCM_NOT_LEADER) {
         failoverProxyProvider
-            .performFailoverToAssignedLeader(response.getLeaderSCMNodeId());

Review comment:
       Do we need this at all, as server is only sending exception for NotLeader

##########
File path: 
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAUtils.java
##########
@@ -198,12 +201,27 @@ public static OzoneConfiguration removeSelfId(
     return getSCMNodeIds(configuration, scmServiceId);
   }
 
+  private static Throwable unwrapException(Exception e) {
+    IOException ioException = null;
+    Throwable cause = e.getCause();
+    if (cause instanceof RemoteException) {
+      ioException = ((RemoteException) cause).unwrapRemoteException();
+    }
+    return ioException == null ? e : ioException;
+  }
+
+  public static boolean isNonRetriableException(Exception e) {
+    Throwable t = unwrapException(e);

Review comment:
       Looks like the server returns NonRetriableException which wraps 
StateMachineException. Should we check here for NonRetriableException

##########
File path: 
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMBlockLocationFailoverProxyProvider.java
##########
@@ -145,13 +152,24 @@ public synchronized ProxyInfo getProxy() {
   @Override
   public void performFailover(ScmBlockLocationProtocolPB newLeader) {
     // Should do nothing here.
-    LOG.debug("Failing over to next proxy. {}", getCurrentProxyOMNodeId());
+    LOG.debug("Failing over to next proxy. {}", getCurrentProxySCMNodeId());
   }
 
-  public void performFailoverToAssignedLeader(String newLeader) {
+  public void performFailoverToAssignedLeader(String newLeader, Exception e) {
+    ServerNotLeaderException snle =
+        (ServerNotLeaderException) SCMHAUtils.getServerNotLeaderException(e);
+    if (snle != null && snle.getSuggestedLeader() != null) {
+      newLeader = scmProxyInfoMap.values().stream().filter(
+          proxyInfo -> NetUtils.getHostPortString(proxyInfo.getAddress())
+              
.equals(snle.getSuggestedLeader())).findFirst().get().getNodeId();
+      LOG.debug("Performing failover to suggested leader {}, nodeId {}",
+          snle.getSuggestedLeader(), newLeader);
+    }
     if (newLeader == null) {
       // If newLeader is not assigned, it will fail over to next proxy.
       nextProxyIndex();
+      LOG.debug("Performing failover to next proxy node {}",
+          currentProxySCMNodeId);
     } else {
       if (!assignLeaderToNode(newLeader)) {
         LOG.debug("Failing over SCM proxy to nodeId: {}", newLeader);

Review comment:
       Do we need to call nextProxyIndex() here?
   As we got newLeader, we should just failOver to newLeader 

##########
File path: 
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMBlockLocationFailoverProxyProvider.java
##########
@@ -250,7 +268,7 @@ public RetryPolicy getSCMBlockLocationRetryPolicy(String 
newLeader) {
       public RetryAction shouldRetry(Exception e, int retry,
                                      int failover, boolean b) {
         if (!SCMHAUtils.isRetriableWithNoFailoverException(e)) {
-          performFailoverToAssignedLeader(newLeader);
+          performFailoverToAssignedLeader(newLeader, e);

Review comment:
       And also can we remove lastAttemptedLeader, as it is not being used 
anywhere. (Though not related, it might be good as it makes code cleaner if no 
use)




-- 
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.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to