Skip to content

Commit 3b77cf4

Browse files
authored
HDFS-15749. Make size of editPendingQ can be configurable (#2572)
1 parent 005b854 commit 3b77cf4

File tree

3 files changed

+21
-2
lines changed

3 files changed

+21
-2
lines changed

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -393,6 +393,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
393393
public static final String DFS_NAMENODE_EDITS_ASYNC_LOGGING =
394394
"dfs.namenode.edits.asynclogging";
395395
public static final boolean DFS_NAMENODE_EDITS_ASYNC_LOGGING_DEFAULT = true;
396+
public static final String
397+
DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE =
398+
"dfs.namenode.edits.asynclogging.pending.queue.size";
399+
public static final int
400+
DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE_DEFAULT = 4096;
396401

397402
public static final String DFS_NAMENODE_PROVIDED_ENABLED = "dfs.namenode.provided.enabled";
398403
public static final boolean DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT = false;

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import org.slf4j.Logger;
3232
import org.slf4j.LoggerFactory;
3333
import org.apache.hadoop.conf.Configuration;
34+
import org.apache.hadoop.hdfs.DFSConfigKeys;
3435
import org.apache.hadoop.ipc.Server;
3536
import org.apache.hadoop.util.ExitUtil;
3637
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -45,8 +46,7 @@ class FSEditLogAsync extends FSEditLog implements Runnable {
4546
private static final ThreadLocal<Edit> THREAD_EDIT = new ThreadLocal<Edit>();
4647

4748
// requires concurrent access from caller threads and syncing thread.
48-
private final BlockingQueue<Edit> editPendingQ =
49-
new ArrayBlockingQueue<Edit>(4096);
49+
private final BlockingQueue<Edit> editPendingQ;
5050

5151
// only accessed by syncing thread so no synchronization required.
5252
// queue is unbounded because it's effectively limited by the size
@@ -57,6 +57,12 @@ class FSEditLogAsync extends FSEditLog implements Runnable {
5757
super(conf, storage, editsDirs);
5858
// op instances cannot be shared due to queuing for background thread.
5959
cache.disableCache();
60+
int editPendingQSize = conf.getInt(
61+
DFSConfigKeys.DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE,
62+
DFSConfigKeys.
63+
DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE_DEFAULT);
64+
65+
editPendingQ = new ArrayBlockingQueue<>(editPendingQSize);
6066
}
6167

6268
private boolean isSyncThreadAlive() {

hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4951,6 +4951,14 @@
49514951
</description>
49524952
</property>
49534953

4954+
<property>
4955+
<name>dfs.namenode.edits.asynclogging.pending.queue.size</name>
4956+
<value>4096</value>
4957+
<description>
4958+
The queue size of edit pending queue for FSEditLogAsync.
4959+
</description>
4960+
</property>
4961+
49544962
<property>
49554963
<name>dfs.namenode.edits.dir.minimum</name>
49564964
<value>1</value>

0 commit comments

Comments
 (0)