[ 
https://issues.apache.org/jira/browse/HDFS-16890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17679341#comment-17679341
 ] 

ASF GitHub Bot commented on HDFS-16890:
---------------------------------------

omalley commented on code in PR #5298:
URL: https://github.com/apache/hadoop/pull/5298#discussion_r1083100964


##########
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java:
##########
@@ -1730,4 +1750,39 @@ private static boolean isReadCall(Method method) {
     }
     return !method.getAnnotationsByType(ReadOnly.class)[0].activeOnly();
   }
+
+  /**
+   * Checks and sets last refresh time for a namespace's stateId.
+   * Returns true if refresh time is newer than threshold.
+   * Otherwise, return false and call should be handled by active namenode.
+   * @param nsId namespaceID
+   */
+  @VisibleForTesting
+  boolean isNamespaceStateIdFresh(String nsId) {
+    if (activeNNStateIdRefreshPeriodMs < 0) {
+      LOG.debug("Skipping freshness check and returning True since"

Review Comment:
   Move this to the initialization, so that we only get the log once rather 
than every call.



##########
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java:
##########
@@ -201,6 +201,10 @@ public class RBFConfigKeys extends 
CommonConfigurationKeysPublic {
       FEDERATION_ROUTER_PREFIX + 
"observer.federated.state.propagation.maxsize";
   public static final int 
DFS_ROUTER_OBSERVER_FEDERATED_STATE_PROPAGATION_MAXSIZE_DEFAULT = 5;
 
+  public static final String DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_KEY =
+      FEDERATION_ROUTER_PREFIX + "observer.state.id.refresh.period";
+  public static final String 
DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_DEFAULT = "5s";

Review Comment:
   Let's make this longer, like 10 or 15s?



##########
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java:
##########
@@ -211,13 +222,19 @@ public RouterRpcClient(Configuration conf, Router router,
     this.observerReadEnabledDefault = conf.getBoolean(
         RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_DEFAULT_KEY,
         RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_DEFAULT_VALUE);
-    String[] observerReadOverrides = 
conf.getStrings(RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_OVERRIDES);
+    String[] observerReadOverrides =
+        conf.getStrings(RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_OVERRIDES);
     if (observerReadOverrides != null) {
       
observerReadEnabledOverrides.addAll(Arrays.asList(observerReadOverrides));
     }
     if (this.observerReadEnabledDefault) {
       LOG.info("Observer read is enabled for router.");
     }
+    this.activeNNStateIdRefreshPeriodMs = conf.getTimeDuration(
+        RBFConfigKeys.DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_KEY,
+        RBFConfigKeys.DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_DEFAULT,
+        TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
+    this.lastActiveNNRefreshTimes = new HashMap<>();

Review Comment:
   Don't you need locking around the access to this hashmap?



##########
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java:
##########
@@ -1730,4 +1750,39 @@ private static boolean isReadCall(Method method) {
     }
     return !method.getAnnotationsByType(ReadOnly.class)[0].activeOnly();
   }
+
+  /**
+   * Checks and sets last refresh time for a namespace's stateId.
+   * Returns true if refresh time is newer than threshold.
+   * Otherwise, return false and call should be handled by active namenode.
+   * @param nsId namespaceID
+   */
+  @VisibleForTesting
+  boolean isNamespaceStateIdFresh(String nsId) {
+    if (activeNNStateIdRefreshPeriodMs < 0) {
+      LOG.debug("Skipping freshness check and returning True since"
+          + RBFConfigKeys.DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_KEY
+          + " is less than 0");
+      return true;
+    }
+
+    Call call = Server.getCurCall().get();

Review Comment:
   Let's just use the current time here.



##########
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java:
##########
@@ -1730,4 +1750,39 @@ private static boolean isReadCall(Method method) {
     }
     return !method.getAnnotationsByType(ReadOnly.class)[0].activeOnly();
   }
+
+  /**
+   * Checks and sets last refresh time for a namespace's stateId.
+   * Returns true if refresh time is newer than threshold.
+   * Otherwise, return false and call should be handled by active namenode.
+   * @param nsId namespaceID
+   */
+  @VisibleForTesting
+  boolean isNamespaceStateIdFresh(String nsId) {
+    if (activeNNStateIdRefreshPeriodMs < 0) {
+      LOG.debug("Skipping freshness check and returning True since"

Review Comment:
   Actually, you should also use expansion pattern rather than string 
concatenation.
   
   LOG.debug("Skipping ... {} is less than zero.", RBFConfigKeys....);
   





> RBF: Add period state refresh to keep router state near active namenode's
> -------------------------------------------------------------------------
>
>                 Key: HDFS-16890
>                 URL: https://issues.apache.org/jira/browse/HDFS-16890
>             Project: Hadoop HDFS
>          Issue Type: Task
>            Reporter: Simbarashe Dzinamarira
>            Assignee: Simbarashe Dzinamarira
>            Priority: Major
>              Labels: pull-request-available
>
> When using the ObserverReadProxyProvider, clients can set 
> *dfs.client.failover.observer.auto-msync-period...* to periodically get the 
> Active namenode's state. When using routers without the 
> ObserverReadProxyProvider, this periodic update is lost.
> In a busy cluster, the Router constantly gets updated with the active 
> namenode's state when
>  # There is a write operation.
>  # There is an operation (read/write) from a new clients.
> However, in the scenario when there are no new clients and no write 
> operations, the state kept in the router can lag behind the active's. The 
> router does update its state with responses from the Observer, but the 
> observer may be lagging behind too.
> We should have a periodic refresh in the router to serve a similar role as 
> *dfs.client.failover.observer.auto-msync-period*



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to