From 3b77cf444622a9ccc56e30a1dff6dfdb83676aca Mon Sep 17 00:00:00 2001 From: maobaolong Date: Mon, 28 Dec 2020 14:18:18 +0800 Subject: [PATCH] HDFS-15749. Make size of editPendingQ can be configurable (#2572) --- .../java/org/apache/hadoop/hdfs/DFSConfigKeys.java | 5 +++++ .../hadoop/hdfs/server/namenode/FSEditLogAsync.java | 10 ++++++++-- .../hadoop-hdfs/src/main/resources/hdfs-default.xml | 8 ++++++++ 3 files changed, 21 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 0a5caed0dec..9260916d7e2 100755 --- 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 @@ -393,6 +393,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final String DFS_NAMENODE_EDITS_ASYNC_LOGGING = "dfs.namenode.edits.asynclogging"; public static final boolean DFS_NAMENODE_EDITS_ASYNC_LOGGING_DEFAULT = true; + public static final String + DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE = + "dfs.namenode.edits.asynclogging.pending.queue.size"; + public static final int + DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE_DEFAULT = 4096; public static final String DFS_NAMENODE_PROVIDED_ENABLED = "dfs.namenode.provided.enabled"; public static final boolean DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT = false; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java index e73dfa7797d..68af0c163dd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java @@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -45,8 +46,7 @@ class FSEditLogAsync extends FSEditLog implements Runnable { private static final ThreadLocal THREAD_EDIT = new ThreadLocal(); // requires concurrent access from caller threads and syncing thread. - private final BlockingQueue editPendingQ = - new ArrayBlockingQueue(4096); + private final BlockingQueue editPendingQ; // only accessed by syncing thread so no synchronization required. // queue is unbounded because it's effectively limited by the size @@ -57,6 +57,12 @@ class FSEditLogAsync extends FSEditLog implements Runnable { super(conf, storage, editsDirs); // op instances cannot be shared due to queuing for background thread. cache.disableCache(); + int editPendingQSize = conf.getInt( + DFSConfigKeys.DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE, + DFSConfigKeys. + DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE_DEFAULT); + + editPendingQ = new ArrayBlockingQueue<>(editPendingQSize); } private boolean isSyncThreadAlive() { 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 b1a0b1feff1..e5d626d28ef 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -4951,6 +4951,14 @@ + + dfs.namenode.edits.asynclogging.pending.queue.size + 4096 + + The queue size of edit pending queue for FSEditLogAsync. + + + dfs.namenode.edits.dir.minimum 1