You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cl...@apache.org on 2019/10/02 23:39:44 UTC

[hadoop] branch branch-3.2 updated: HDFS-14858. [SBN read] Allow configurably enable/disable AlignmentContext on NameNode. Contributed by Chen Liang.

This is an automated email from the ASF dual-hosted git repository.

cliang pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
     new 8a866f0  HDFS-14858. [SBN read] Allow configurably enable/disable AlignmentContext on NameNode. Contributed by Chen Liang.
8a866f0 is described below

commit 8a866f01c98fa2d74ead9e72f7ea0cf554ae0c82
Author: Chen Liang <cl...@apache.org>
AuthorDate: Wed Oct 2 16:38:20 2019 -0700

    HDFS-14858. [SBN read] Allow configurably enable/disable AlignmentContext on NameNode. Contributed by Chen Liang.
---
 .../main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java    |  6 +++++-
 .../hadoop/hdfs/server/namenode/NameNodeRpcServer.java     | 14 +++++++++++++-
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml        | 11 +++++++++++
 .../hadoop-hdfs/src/site/markdown/ObserverNameNode.md      | 14 ++++++++++++++
 .../hadoop/hdfs/TestStateAlignmentContextWithHA.java       |  1 +
 .../server/namenode/ha/TestConsistentReadsObserver.java    |  2 ++
 .../hdfs/server/namenode/ha/TestMultiObserverNode.java     |  2 ++
 .../hadoop/hdfs/server/namenode/ha/TestObserverNode.java   |  2 ++
 8 files changed, 50 insertions(+), 2 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 428c537..e380303 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -1309,7 +1309,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.send.qop.enabled";
   public static final boolean DFS_NAMENODE_SEND_QOP_ENABLED_DEFAULT = false;
 
-  // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
+  public static final String DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY =
+      "dfs.namenode.state.context.enabled";
+  public static final boolean DFS_NAMENODE_STATE_CONTEXT_ENABLED_DEFAULT = false;
+
+  // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY
       = HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index a784107..f34a298 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -27,6 +27,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIFELINE_HANDLER
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_AUXILIARY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_PATH_DEPTH;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_PATH_LENGTH;
 import static org.apache.hadoop.util.Time.now;
@@ -447,6 +449,16 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     }
     LOG.info("RPC server is binding to " + bindHost + ":" + rpcAddr.getPort());
 
+    boolean enableStateContext = conf.getBoolean(
+        DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY,
+        DFS_NAMENODE_STATE_CONTEXT_ENABLED_DEFAULT);
+    LOG.info("Enable NameNode state context:" + enableStateContext);
+
+    GlobalStateIdContext stateIdContext = null;
+    if (enableStateContext) {
+      stateIdContext = new GlobalStateIdContext((namesystem));
+    }
+
     clientRpcServer = new RPC.Builder(conf)
         .setProtocol(
             org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
@@ -456,7 +468,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
         .setNumHandlers(handlerCount)
         .setVerbose(false)
         .setSecretManager(namesystem.getDelegationTokenSecretManager())
-        .setAlignmentContext(new GlobalStateIdContext(namesystem))
+        .setAlignmentContext(stateIdContext)
         .build();
 
     // Add all the RPC protocols that the namenode implements
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 66d504b..261e951 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -3186,6 +3186,17 @@
 </property>
 
 <property>
+  <name>dfs.namenode.state.context.enabled</name>
+  <value>false</value>
+  <description>
+    Whether enable namenode sending back its current txnid back to client.
+    Setting this to true is required by Consistent Read from Standby feature.
+    But for regular cases, this should be set to false to avoid the overhead
+    of updating and maintaining this state.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.ec.system.default.policy</name>
   <value>RS-6-3-1024k</value>
   <description>The default erasure coding policy name will be used
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md
index 07c384c..af1569c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md
@@ -120,6 +120,20 @@ Deployment
 To enable consistent reads from Observer NameNode, you'll need to add a
 few configurations to your **hdfs-site.xml**:
 
+*  **dfs.namenode.state.context.enabled** - to enable NameNode to maintain
+   and update server state and id.
+
+   This will lead to NameNode creating alignment context instance, which
+   keeps track of current server state id. Server state id will be carried
+   back to client. It is disabled by default to optimize performance of
+   Observer read cases. But this is **required to be turned on**
+   for the Observer NameNode feature.
+
+        <property>
+           <name>dfs.namenode.state.context.enabled</name>
+           <value>true</value>
+        </property>
+
 *  **dfs.ha.tail-edits.in-progress** - to enable fast tailing on
    in-progress edit logs.
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStateAlignmentContextWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStateAlignmentContextWithHA.java
index 3dbeea7..3056b43 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStateAlignmentContextWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStateAlignmentContextWithHA.java
@@ -94,6 +94,7 @@ public class TestStateAlignmentContextWithHA {
     CONF.setBoolean(String.format(
         "fs.%s.impl.disable.cache", HdfsConstants.HDFS_URI_SCHEME), true);
     CONF.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, NUMDATANODES);
+    CONF.setBoolean(DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY, true);
 
     qjmhaCluster = HATestUtil.setUpObserverCluster(CONF, 1, NUMDATANODES, true);
     cluster = qjmhaCluster.getDfsCluster();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java
index 5cd0fa4..2f8f115 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -67,6 +68,7 @@ public class TestConsistentReadsObserver {
   @BeforeClass
   public static void startUpCluster() throws Exception {
     conf = new Configuration();
+    conf.setBoolean(DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY, true);
     // disable fast tailing here because this test's assertions are based on the
     // timing of explicitly called rollEditLogAndTail. Although this means this
     // test takes some time to run
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestMultiObserverNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestMultiObserverNode.java
index a8e1245..a0913e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestMultiObserverNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestMultiObserverNode.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -46,6 +47,7 @@ public class TestMultiObserverNode {
   @BeforeClass
   public static void startUpCluster() throws Exception {
     conf = new Configuration();
+    conf.setBoolean(DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY, true);
     qjmhaCluster = HATestUtil.setUpObserverCluster(conf, 2, 0, true);
     dfsCluster = qjmhaCluster.getDfsCluster();
     dfs = HATestUtil.configureObserverReadFs(
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestObserverNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestObserverNode.java
index 061f6ed..b923473 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestObserverNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestObserverNode.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getServiceState;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -76,6 +77,7 @@ public class TestObserverNode {
   @BeforeClass
   public static void startUpCluster() throws Exception {
     conf = new Configuration();
+    conf.setBoolean(DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY, true);
     qjmhaCluster = HATestUtil.setUpObserverCluster(conf, 1, 0, true);
     dfsCluster = qjmhaCluster.getDfsCluster();
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org