HDFS-15749. Make size of editPendingQ can be configurable (#2572)

This commit is contained in:
maobaolong 2020-12-28 14:18:18 +08:00 committed by GitHub
parent 005b854f6b
commit 3b77cf4446
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 21 additions and 2 deletions

View File

@ -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;

View File

@ -31,6 +31,7 @@
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<Edit> THREAD_EDIT = new ThreadLocal<Edit>();
// requires concurrent access from caller threads and syncing thread.
private final BlockingQueue<Edit> editPendingQ =
new ArrayBlockingQueue<Edit>(4096);
private final BlockingQueue<Edit> 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() {

View File

@ -4951,6 +4951,14 @@
</description>
</property>
<property>
<name>dfs.namenode.edits.asynclogging.pending.queue.size</name>
<value>4096</value>
<description>
The queue size of edit pending queue for FSEditLogAsync.
</description>
</property>
<property>
<name>dfs.namenode.edits.dir.minimum</name>
<value>1</value>