You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@logging.apache.org by rp...@apache.org on 2022/04/25 03:19:15 UTC

[logging-log4j2] branch master updated: LOG4J2-3472 (master) make disruptor WaitStrategy configurable in Log4j configuration (#830)

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

rpopma pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/logging-log4j2.git


The following commit(s) were added to refs/heads/master by this push:
     new 3d1592895a LOG4J2-3472 (master) make disruptor WaitStrategy configurable in Log4j configuration (#830)
3d1592895a is described below

commit 3d1592895a27ceb7c95cd7a795c5730e3231b796
Author: Remko Popma <re...@yahoo.com>
AuthorDate: Mon Apr 25 12:19:11 2022 +0900

    LOG4J2-3472 (master) make disruptor WaitStrategy configurable in Log4j configuration (#830)
    
    LOG4J2-3472 (master) make disruptor WaitStrategy configurable in Log4j configuration (#830)
---
 ...WaitStrategyFactoryConfigGlobalLoggersTest.java |  68 ++++++++++++++
 .../async/AsyncWaitStrategyFactoryConfigTest.java  |  81 +++++++++++++++++
 ...egyFactoryIncorrectConfigGlobalLoggersTest.java |  64 +++++++++++++
 ...ncWaitStrategyFactoryConfigGlobalLoggerTest.xml |  17 ++++
 .../AsyncWaitStrategyFactoryConfigTest.xml         |  17 ++++
 ...ategyIncorrectFactoryConfigGlobalLoggerTest.xml |  16 ++++
 ...AsyncWaitStrategyIncorrectFactoryConfigTest.xml |  16 ++++
 .../logging/log4j/core/async/AsyncLogger.java      |   5 +
 .../log4j/core/async/AsyncLoggerConfig.java        |   5 +
 .../core/async/AsyncLoggerConfigDisruptor.java     |  13 ++-
 .../log4j/core/async/AsyncLoggerContext.java       |  12 +--
 .../log4j/core/async/AsyncLoggerDisruptor.java     |  15 ++-
 .../log4j/core/async/AsyncWaitStrategyFactory.java |  36 ++++++++
 .../core/async/AsyncWaitStrategyFactoryConfig.java | 101 +++++++++++++++++++++
 .../async/DefaultAsyncWaitStrategyFactory.java     |  95 +++++++++++++++++++
 .../logging/log4j/core/async/DisruptorUtil.java    |  47 ++--------
 .../log4j/core/config/AbstractConfiguration.java   |  15 ++-
 .../logging/log4j/core/config/Configuration.java   |  11 +++
 src/site/asciidoc/manual/async.adoc                |  61 ++++++++++++-
 src/site/asciidoc/manual/garbagefree.adoc          |  11 ++-
 src/site/site.xml                                  |   1 +
 21 files changed, 647 insertions(+), 60 deletions(-)

diff --git a/log4j-core-test/src/test/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryConfigGlobalLoggersTest.java b/log4j-core-test/src/test/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryConfigGlobalLoggersTest.java
new file mode 100644
index 0000000000..de4217ac51
--- /dev/null
+++ b/log4j-core-test/src/test/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryConfigGlobalLoggersTest.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache license, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the license for the specific language governing permissions and
+ * limitations under the license.
+ */
+package org.apache.logging.log4j.core.async;
+
+import com.lmax.disruptor.YieldingWaitStrategy;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.test.categories.AsyncLoggers;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.apache.logging.log4j.core.util.Constants;
+import org.apache.logging.log4j.util.Strings;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.*;
+
+@Category(AsyncLoggers.class)
+public class AsyncWaitStrategyFactoryConfigGlobalLoggersTest {
+
+    @BeforeClass
+    public static void beforeClass() {
+        System.setProperty(Constants.LOG4J_CONTEXT_SELECTOR,
+                AsyncLoggerContextSelector.class.getName());
+        System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY,
+                "AsyncWaitStrategyFactoryConfigGlobalLoggerTest.xml");
+    }
+
+    @AfterClass
+    public static void afterClass() {
+        System.clearProperty(Constants.LOG4J_CONTEXT_SELECTOR);
+        System.clearProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
+    }
+
+    @Ignore("This test succeeds when run individually but fails when run by Surefire with all other tests")
+    @Test
+    public void testConfigWaitStrategyAndFactory() throws Exception {
+        final AsyncLogger logger = (AsyncLogger) LogManager.getLogger("com.foo.Bar");
+
+        final LoggerContext context = (LoggerContext) LogManager.getContext(false);
+        assertTrue("context is AsyncLoggerContext", context instanceof AsyncLoggerContext);
+
+        AsyncWaitStrategyFactory asyncWaitStrategyFactory = context.getConfiguration().getAsyncWaitStrategyFactory();
+        assertEquals(AsyncWaitStrategyFactoryConfigTest.YieldingWaitStrategyFactory.class, asyncWaitStrategyFactory.getClass());
+        assertTrue("factory is YieldingWaitStrategyFactory", asyncWaitStrategyFactory instanceof AsyncWaitStrategyFactoryConfigTest.YieldingWaitStrategyFactory);
+
+        AsyncLoggerDisruptor delegate = logger.getAsyncLoggerDisruptor();
+
+        assertEquals(YieldingWaitStrategy.class, delegate.getWaitStrategy().getClass());
+        assertTrue("waitstrategy is YieldingWaitStrategy", delegate.getWaitStrategy() instanceof YieldingWaitStrategy);
+    }
+}
diff --git a/log4j-core-test/src/test/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryConfigTest.java b/log4j-core-test/src/test/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryConfigTest.java
new file mode 100644
index 0000000000..b27de0c15c
--- /dev/null
+++ b/log4j-core-test/src/test/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryConfigTest.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache license, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the license for the specific language governing permissions and
+ * limitations under the license.
+ */
+package org.apache.logging.log4j.core.async;
+
+import com.lmax.disruptor.TimeoutBlockingWaitStrategy;
+import com.lmax.disruptor.WaitStrategy;
+import com.lmax.disruptor.YieldingWaitStrategy;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.test.categories.AsyncLoggers;
+import org.apache.logging.log4j.core.test.junit.LoggerContextSource;
+import org.apache.logging.log4j.core.test.junit.Named;
+import org.apache.logging.log4j.core.test.appender.ListAppender;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+@Tag("async")
+public class AsyncWaitStrategyFactoryConfigTest {
+
+    @Test
+    @LoggerContextSource("AsyncWaitStrategyFactoryConfigTest.xml")
+    public void testConfigWaitStrategyFactory(final LoggerContext context) throws Exception {
+        AsyncWaitStrategyFactory asyncWaitStrategyFactory = context.getConfiguration().getAsyncWaitStrategyFactory();
+        assertThat(asyncWaitStrategyFactory.getClass()).isEqualTo(YieldingWaitStrategyFactory.class);
+        assertThat(asyncWaitStrategyFactory instanceof YieldingWaitStrategyFactory); //"factory is YieldingWaitStrategyFactory"
+    }
+
+    @Test
+    @LoggerContextSource("AsyncWaitStrategyFactoryConfigTest.xml")
+    public void testWaitStrategy(final LoggerContext context) throws Exception {
+
+        org.apache.logging.log4j.Logger logger = context.getRootLogger();
+
+        AsyncLoggerConfig loggerConfig = (AsyncLoggerConfig) ((org.apache.logging.log4j.core.Logger) logger).get();
+        AsyncLoggerConfigDisruptor delegate = (AsyncLoggerConfigDisruptor) loggerConfig.getAsyncLoggerConfigDelegate();
+        assertThat(delegate.getWaitStrategy().getClass()).isEqualTo(YieldingWaitStrategy.class);
+        assertThat(delegate.getWaitStrategy() instanceof com.lmax.disruptor.YieldingWaitStrategy);// "waitstrategy is YieldingWaitStrategy");
+    }
+
+    @Test
+    @LoggerContextSource("AsyncWaitStrategyIncorrectFactoryConfigTest.xml")
+    public void testIncorrectConfigWaitStrategyFactory(final LoggerContext context) throws Exception {
+        AsyncWaitStrategyFactory asyncWaitStrategyFactory = context.getConfiguration().getAsyncWaitStrategyFactory();
+        assertThat(asyncWaitStrategyFactory).isNull(); // because invalid configuration
+    }
+
+    @Test
+    @LoggerContextSource("AsyncWaitStrategyIncorrectFactoryConfigTest.xml")
+    public void testIncorrectWaitStrategyFallsBackToDefault(
+            @Named("WaitStrategyAppenderList") final ListAppender list1,
+            final LoggerContext context) throws Exception {
+        org.apache.logging.log4j.Logger logger = context.getRootLogger();
+
+        AsyncLoggerConfig loggerConfig = (AsyncLoggerConfig) ((org.apache.logging.log4j.core.Logger) logger).get();
+        AsyncLoggerConfigDisruptor delegate = (AsyncLoggerConfigDisruptor) loggerConfig.getAsyncLoggerConfigDelegate();
+        assertThat(delegate.getWaitStrategy().getClass()).isEqualTo(TimeoutBlockingWaitStrategy.class);
+        assertThat(delegate.getWaitStrategy() instanceof TimeoutBlockingWaitStrategy); //"waitstrategy is TimeoutBlockingWaitStrategy"
+    }
+
+    public static class YieldingWaitStrategyFactory implements AsyncWaitStrategyFactory {
+        @Override
+        public WaitStrategy createWaitStrategy() {
+            return new YieldingWaitStrategy();
+        }
+    }
+}
diff --git a/log4j-core-test/src/test/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryIncorrectConfigGlobalLoggersTest.java b/log4j-core-test/src/test/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryIncorrectConfigGlobalLoggersTest.java
new file mode 100644
index 0000000000..7adc8b85d5
--- /dev/null
+++ b/log4j-core-test/src/test/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryIncorrectConfigGlobalLoggersTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache license, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the license for the specific language governing permissions and
+ * limitations under the license.
+ */
+package org.apache.logging.log4j.core.async;
+
+import com.lmax.disruptor.TimeoutBlockingWaitStrategy;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.test.categories.AsyncLoggers;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.apache.logging.log4j.core.util.Constants;
+import org.apache.logging.log4j.util.Strings;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.*;
+
+@Category(AsyncLoggers.class)
+public class AsyncWaitStrategyFactoryIncorrectConfigGlobalLoggersTest {
+
+    @BeforeClass
+    public static void beforeClass() {
+        System.setProperty(Constants.LOG4J_CONTEXT_SELECTOR,
+                AsyncLoggerContextSelector.class.getName());
+        System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY,
+                "AsyncWaitStrategyIncorrectFactoryConfigGlobalLoggerTest.xml");
+    }
+
+    @AfterClass
+    public static void afterClass() {
+        System.clearProperty(Constants.LOG4J_CONTEXT_SELECTOR);
+        System.clearProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
+    }
+
+
+    @Test
+    public void testIncorrectConfigWaitStrategyFactory() throws Exception {
+        final LoggerContext context = (LoggerContext) LogManager.getContext(false);
+        assertTrue("context is AsyncLoggerContext", context instanceof AsyncLoggerContext);
+
+        AsyncWaitStrategyFactory asyncWaitStrategyFactory = context.getConfiguration().getAsyncWaitStrategyFactory();
+        assertNull(asyncWaitStrategyFactory);
+
+        AsyncLogger logger = (AsyncLogger) context.getRootLogger();
+        AsyncLoggerDisruptor delegate = logger.getAsyncLoggerDisruptor();
+        assertEquals(TimeoutBlockingWaitStrategy.class, delegate.getWaitStrategy().getClass());
+        assertTrue("waitstrategy is TimeoutBlockingWaitStrategy", delegate.getWaitStrategy() instanceof TimeoutBlockingWaitStrategy);
+    }
+}
diff --git a/log4j-core-test/src/test/resources/AsyncWaitStrategyFactoryConfigGlobalLoggerTest.xml b/log4j-core-test/src/test/resources/AsyncWaitStrategyFactoryConfigGlobalLoggerTest.xml
new file mode 100644
index 0000000000..5f98b2ea47
--- /dev/null
+++ b/log4j-core-test/src/test/resources/AsyncWaitStrategyFactoryConfigGlobalLoggerTest.xml
@@ -0,0 +1,17 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Configuration name="asyncwait-global1" >
+  <!--  packages="org.apache.logging.log4j.core.async" -->
+
+  <AsyncWaitStrategyFactory
+      class="org.apache.logging.log4j.core.async.AsyncWaitStrategyFactoryConfigTest$YieldingWaitStrategyFactory" />
+
+  <Appenders>
+    <List name="WaitStrategyAppenderList" />
+  </Appenders>
+  
+  <Loggers>
+    <Root level="info">
+      <AppenderRef ref="WaitStrategyAppenderList"/>
+    </Root>
+  </Loggers>
+</Configuration>
diff --git a/log4j-core-test/src/test/resources/AsyncWaitStrategyFactoryConfigTest.xml b/log4j-core-test/src/test/resources/AsyncWaitStrategyFactoryConfigTest.xml
new file mode 100644
index 0000000000..04211fd3da
--- /dev/null
+++ b/log4j-core-test/src/test/resources/AsyncWaitStrategyFactoryConfigTest.xml
@@ -0,0 +1,17 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Configuration name="async-wait-1" >
+  <!--  packages="org.apache.logging.log4j.core.async" -->
+
+  <AsyncWaitStrategyFactory
+      class="org.apache.logging.log4j.core.async.AsyncWaitStrategyFactoryConfigTest$YieldingWaitStrategyFactory" />
+
+  <Appenders>
+    <List name="WaitStrategyAppenderList" />
+  </Appenders>
+  
+  <Loggers>
+    <AsyncRoot level="info">
+      <AppenderRef ref="WaitStrategyAppenderList"/>
+    </AsyncRoot>
+  </Loggers>
+</Configuration>
diff --git a/log4j-core-test/src/test/resources/AsyncWaitStrategyIncorrectFactoryConfigGlobalLoggerTest.xml b/log4j-core-test/src/test/resources/AsyncWaitStrategyIncorrectFactoryConfigGlobalLoggerTest.xml
new file mode 100644
index 0000000000..c65a76ab34
--- /dev/null
+++ b/log4j-core-test/src/test/resources/AsyncWaitStrategyIncorrectFactoryConfigGlobalLoggerTest.xml
@@ -0,0 +1,16 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Configuration name="asyncwait-global2">
+
+  <AsyncWaitStrategyFactory
+      class="nonexisting.Factory" />
+
+  <Appenders>
+    <List name="WaitStrategyAppenderList" />
+  </Appenders>
+  
+  <Loggers>
+    <Root level="info">
+      <AppenderRef ref="WaitStrategyAppenderList"/>
+    </Root>
+  </Loggers>
+</Configuration>
diff --git a/log4j-core-test/src/test/resources/AsyncWaitStrategyIncorrectFactoryConfigTest.xml b/log4j-core-test/src/test/resources/AsyncWaitStrategyIncorrectFactoryConfigTest.xml
new file mode 100644
index 0000000000..5a3b025c3c
--- /dev/null
+++ b/log4j-core-test/src/test/resources/AsyncWaitStrategyIncorrectFactoryConfigTest.xml
@@ -0,0 +1,16 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Configuration name="async-wait-2">
+
+  <AsyncWaitStrategyFactory
+      class="nonexisting.Factory" />
+
+  <Appenders>
+    <List name="WaitStrategyAppenderList" />
+  </Appenders>
+  
+  <Loggers>
+    <AsyncRoot level="info">
+      <AppenderRef ref="WaitStrategyAppenderList"/>
+    </AsyncRoot>
+  </Loggers>
+</Configuration>
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLogger.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLogger.java
index 2d60ae70f8..1ab886fc71 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLogger.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLogger.java
@@ -516,4 +516,9 @@ public class AsyncLogger extends Logger implements EventTranslatorVararg<RingBuf
         }
         return contextData;
     }
+
+    // package-protected for tests
+    AsyncLoggerDisruptor getAsyncLoggerDisruptor() {
+        return loggerDisruptor;
+    }
 }
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfig.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfig.java
index e032bd1427..210bf4812e 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfig.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfig.java
@@ -129,6 +129,11 @@ public class AsyncLoggerConfig extends LoggerConfig {
         }
     }
 
+    // package-protected for testing
+    AsyncLoggerConfigDelegate getAsyncLoggerConfigDelegate() {
+        return delegate;
+    }
+
     @Override
     protected void callAppenders(final LogEvent event) {
         super.callAppenders(event);
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDisruptor.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDisruptor.java
index 1447fdbb6b..2bce8d0344 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDisruptor.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDisruptor.java
@@ -166,10 +166,18 @@ public class AsyncLoggerConfigDisruptor extends AbstractLifeCycle implements Asy
     private EventFactory<Log4jEventWrapper> factory;
     private EventTranslatorTwoArg<Log4jEventWrapper, LogEvent, AsyncLoggerConfig> translator;
     private volatile boolean alreadyLoggedWarning;
+    private final AsyncWaitStrategyFactory asyncWaitStrategyFactory;
+    private WaitStrategy waitStrategy;
 
     private final Object queueFullEnqueueLock = new Object();
 
-    public AsyncLoggerConfigDisruptor() {
+    public AsyncLoggerConfigDisruptor(AsyncWaitStrategyFactory asyncWaitStrategyFactory) {
+        this.asyncWaitStrategyFactory = asyncWaitStrategyFactory; // may be null
+    }
+
+    // package-protected for testing
+    WaitStrategy getWaitStrategy() {
+        return waitStrategy;
     }
 
     // called from AsyncLoggerConfig constructor
@@ -195,7 +203,8 @@ public class AsyncLoggerConfigDisruptor extends AbstractLifeCycle implements Asy
         }
         LOGGER.trace("AsyncLoggerConfigDisruptor creating new disruptor for this configuration.");
         ringBufferSize = DisruptorUtil.calculateRingBufferSize("AsyncLoggerConfig.RingBufferSize");
-        final WaitStrategy waitStrategy = DisruptorUtil.createWaitStrategy("AsyncLoggerConfig.WaitStrategy");
+        waitStrategy = DisruptorUtil.createWaitStrategy(
+                "AsyncLoggerConfig.WaitStrategy", asyncWaitStrategyFactory);
 
         final ThreadFactory threadFactory = new Log4jThreadFactory("AsyncLoggerConfig", true, Thread.NORM_PRIORITY) {
             @Override
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerContext.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerContext.java
index 02dcc9f8f6..ddb1c47ba4 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerContext.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerContext.java
@@ -37,33 +37,33 @@ public class AsyncLoggerContext extends LoggerContext {
 
     public AsyncLoggerContext(final String name) {
         super(name);
-        loggerDisruptor = new AsyncLoggerDisruptor(name);
+        loggerDisruptor = new AsyncLoggerDisruptor(name, () -> getConfiguration().getAsyncWaitStrategyFactory());
     }
 
     public AsyncLoggerContext(final String name, final Object externalContext) {
         super(name, externalContext);
-        loggerDisruptor = new AsyncLoggerDisruptor(name);
+        loggerDisruptor = new AsyncLoggerDisruptor(name, () -> getConfiguration().getAsyncWaitStrategyFactory());
     }
 
     public AsyncLoggerContext(final String name, final Object externalContext, final URI configLocn) {
         super(name, externalContext, configLocn);
-        loggerDisruptor = new AsyncLoggerDisruptor(name);
+        loggerDisruptor = new AsyncLoggerDisruptor(name, () -> getConfiguration().getAsyncWaitStrategyFactory());
     }
 
     public AsyncLoggerContext(final String name, final Object externalContext, final URI configLocn, final Injector injector) {
         super(name, externalContext, configLocn, injector);
-        loggerDisruptor = new AsyncLoggerDisruptor(name);
+        loggerDisruptor = new AsyncLoggerDisruptor(name, () -> getConfiguration().getAsyncWaitStrategyFactory());
     }
 
     public AsyncLoggerContext(final String name, final Object externalContext, final String configLocn) {
         super(name, externalContext, configLocn);
-        loggerDisruptor = new AsyncLoggerDisruptor(name);
+        loggerDisruptor = new AsyncLoggerDisruptor(name, () -> getConfiguration().getAsyncWaitStrategyFactory());
     }
 
     public AsyncLoggerContext(
             final String name, final Object externalContext, final String configLocn, final Injector injector) {
         super(name, externalContext, configLocn, injector);
-        loggerDisruptor = new AsyncLoggerDisruptor(name);
+        loggerDisruptor = new AsyncLoggerDisruptor(name, () -> getConfiguration().getAsyncWaitStrategyFactory());
     }
 
     @Override
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerDisruptor.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerDisruptor.java
index 377100eab2..a25ffd5acf 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerDisruptor.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerDisruptor.java
@@ -17,8 +17,10 @@
 
 package org.apache.logging.log4j.core.async;
 
+import java.util.Objects;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
 
 import com.lmax.disruptor.EventTranslatorVararg;
 import org.apache.logging.log4j.Level;
@@ -51,14 +53,22 @@ class AsyncLoggerDisruptor extends AbstractLifeCycle {
 
     private volatile Disruptor<RingBufferLogEvent> disruptor;
     private String contextName;
+    private final Supplier<AsyncWaitStrategyFactory> waitStrategyFactorySupplier;
 
     private boolean useThreadLocalTranslator = true;
     private long backgroundThreadId;
     private AsyncQueueFullPolicy asyncQueueFullPolicy;
     private int ringBufferSize;
+    private WaitStrategy waitStrategy;
 
-    AsyncLoggerDisruptor(final String contextName) {
+    AsyncLoggerDisruptor(final String contextName, final Supplier<AsyncWaitStrategyFactory> waitStrategyFactorySupplier) {
         this.contextName = contextName;
+        this.waitStrategyFactorySupplier = Objects.requireNonNull(waitStrategyFactorySupplier, "waitStrategyFactorySupplier");
+    }
+
+    // package-protected for testing
+    WaitStrategy getWaitStrategy() {
+        return waitStrategy;
     }
 
     public String getContextName() {
@@ -93,7 +103,8 @@ class AsyncLoggerDisruptor extends AbstractLifeCycle {
         setStarting();
         LOGGER.trace("[{}] AsyncLoggerDisruptor creating new disruptor for this context.", contextName);
         ringBufferSize = DisruptorUtil.calculateRingBufferSize("AsyncLogger.RingBufferSize");
-        final WaitStrategy waitStrategy = DisruptorUtil.createWaitStrategy("AsyncLogger.WaitStrategy");
+        AsyncWaitStrategyFactory factory = waitStrategyFactorySupplier.get(); // get factory from configuration
+        waitStrategy = DisruptorUtil.createWaitStrategy("AsyncLogger.WaitStrategy", factory);
 
         final ThreadFactory threadFactory = new Log4jThreadFactory("AsyncLogger[" + contextName + "]", true, Thread.NORM_PRIORITY) {
             @Override
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactory.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactory.java
new file mode 100644
index 0000000000..51597b6003
--- /dev/null
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactory.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache license, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the license for the specific language governing permissions and
+ * limitations under the license.
+ */
+
+package org.apache.logging.log4j.core.async;
+
+import com.lmax.disruptor.WaitStrategy;
+
+/**
+ * This interface allows users to configure a custom Disruptor WaitStrategy used for
+ * Async Loggers and Async LoggerConfigs.
+ *
+ * @since 2.17.3
+ */
+public interface AsyncWaitStrategyFactory {
+    /**
+     * Creates and returns a non-null implementation of the LMAX Disruptor's WaitStrategy interface.
+     * This WaitStrategy will be used by Log4j Async Loggers and Async LoggerConfigs.
+     *
+     * @return the WaitStrategy instance to be used by Async Loggers and Async LoggerConfigs
+     */
+    WaitStrategy createWaitStrategy();
+}
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryConfig.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryConfig.java
new file mode 100644
index 0000000000..5506a47905
--- /dev/null
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncWaitStrategyFactoryConfig.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache license, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the license for the specific language governing permissions and
+ * limitations under the license.
+ */
+package org.apache.logging.log4j.core.async;
+
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.plugins.Plugin;
+import org.apache.logging.log4j.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.plugins.PluginFactory;
+import org.apache.logging.log4j.plugins.validation.constraints.Required;
+import org.apache.logging.log4j.core.util.Loader;
+import org.apache.logging.log4j.status.StatusLogger;
+
+import java.util.Objects;
+
+/**
+ * This class allows users to configure the factory used to create
+ * an instance of the LMAX disruptor WaitStrategy
+ * used by Async Loggers in the log4j configuration.
+ */
+@Plugin(name = "AsyncWaitStrategyFactory", category = Core.CATEGORY_NAME, printObject = true)
+public class AsyncWaitStrategyFactoryConfig {
+
+    /**
+     * Status logger for internal logging.
+     */
+    protected static final org.apache.logging.log4j.Logger LOGGER = StatusLogger.getLogger();
+
+    private final String factoryClassName;
+
+    public AsyncWaitStrategyFactoryConfig(final String factoryClassName) {
+        this.factoryClassName = Objects.requireNonNull(factoryClassName, "factoryClassName");
+    }
+
+    @PluginFactory
+    public static <B extends AsyncWaitStrategyFactoryConfig.Builder<B>> B newBuilder() {
+        return new AsyncWaitStrategyFactoryConfig.Builder<B>().asBuilder();
+    }
+
+    /**
+     * Builds AsyncWaitStrategyFactoryConfig instances.
+     *
+     * @param <B>
+     *            The type to build
+     */
+    public static class Builder<B extends AsyncWaitStrategyFactoryConfig.Builder<B>>
+            implements org.apache.logging.log4j.core.util.Builder<AsyncWaitStrategyFactoryConfig> {
+
+        @PluginBuilderAttribute("class")
+        @Required(message = "AsyncWaitStrategyFactory cannot be configured without a factory class name")
+        private String factoryClassName;
+
+        public String getFactoryClassName() {
+            return factoryClassName;
+        }
+
+        public B withFactoryClassName(String className) {
+            this.factoryClassName = className;
+            return asBuilder();
+        }
+
+        @Override
+        public AsyncWaitStrategyFactoryConfig build() {
+            return new AsyncWaitStrategyFactoryConfig(factoryClassName);
+        }
+
+        @SuppressWarnings("unchecked")
+        public B asBuilder() {
+            return (B) this;
+        }
+    }
+
+    public AsyncWaitStrategyFactory createWaitStrategyFactory() {
+        try {
+            @SuppressWarnings("unchecked")
+            final Class<? extends AsyncWaitStrategyFactory> klass = (Class<? extends AsyncWaitStrategyFactory>) Loader.loadClass(factoryClassName);
+            if (AsyncWaitStrategyFactory.class.isAssignableFrom(klass)) {
+                return klass.newInstance();
+            }
+            LOGGER.error("Ignoring factory '{}': it is not assignable to AsyncWaitStrategyFactory", factoryClassName);
+            return null;
+        } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) {
+            LOGGER.info("Invalid implementation class name value: error creating AsyncWaitStrategyFactory {}: {}", factoryClassName, e);
+            return null;
+        }
+
+    }
+}
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/DefaultAsyncWaitStrategyFactory.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/DefaultAsyncWaitStrategyFactory.java
new file mode 100644
index 0000000000..bbe3af696a
--- /dev/null
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/DefaultAsyncWaitStrategyFactory.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache license, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the license for the specific language governing permissions and
+ * limitations under the license.
+ */
+package org.apache.logging.log4j.core.async;
+
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.BusySpinWaitStrategy;
+import com.lmax.disruptor.SleepingWaitStrategy;
+import com.lmax.disruptor.TimeoutBlockingWaitStrategy;
+import com.lmax.disruptor.WaitStrategy;
+import com.lmax.disruptor.YieldingWaitStrategy;
+import org.apache.logging.log4j.Logger;
+import org.apache.logging.log4j.status.StatusLogger;
+import org.apache.logging.log4j.util.PropertiesUtil;
+import org.apache.logging.log4j.util.Strings;
+
+import java.util.concurrent.TimeUnit;
+
+class DefaultAsyncWaitStrategyFactory implements AsyncWaitStrategyFactory {
+    static final String DEFAULT_WAIT_STRATEGY_CLASSNAME = TimeoutBlockingWaitStrategy.class.getName();
+    private static final Logger LOGGER = StatusLogger.getLogger();
+    private final String propertyName;
+
+    public DefaultAsyncWaitStrategyFactory(String propertyName) {
+        this.propertyName = propertyName;
+    }
+
+    @Override
+    public WaitStrategy createWaitStrategy() {
+        final String strategy = PropertiesUtil.getProperties().getStringProperty(propertyName, "TIMEOUT");
+        LOGGER.trace("DefaultAsyncWaitStrategyFactory property {}={}", propertyName, strategy);
+        final String strategyUp = Strings.toRootUpperCase(strategy);
+        // String (not enum) is deliberately used here to avoid IllegalArgumentException being thrown. In case of
+        // incorrect property value, default WaitStrategy is created.
+        switch (strategyUp) {
+            case "SLEEP":
+                final long sleepTimeNs =
+                        parseAdditionalLongProperty(propertyName, "SleepTimeNs", 100L);
+                final String key = getFullPropertyKey(propertyName, "Retries");
+                final int retries =
+                        PropertiesUtil.getProperties().getIntegerProperty(key, 200);
+                LOGGER.trace("DefaultAsyncWaitStrategyFactory creating SleepingWaitStrategy(retries={}, sleepTimeNs={})", retries, sleepTimeNs);
+                return new SleepingWaitStrategy(retries, sleepTimeNs);
+            case "YIELD":
+                LOGGER.trace("DefaultAsyncWaitStrategyFactory creating YieldingWaitStrategy");
+                return new YieldingWaitStrategy();
+            case "BLOCK":
+                LOGGER.trace("DefaultAsyncWaitStrategyFactory creating BlockingWaitStrategy");
+                return new BlockingWaitStrategy();
+            case "BUSYSPIN":
+                LOGGER.trace("DefaultAsyncWaitStrategyFactory creating BusySpinWaitStrategy");
+                return new BusySpinWaitStrategy();
+            case "TIMEOUT":
+                return createDefaultWaitStrategy(propertyName);
+            default:
+                return createDefaultWaitStrategy(propertyName);
+        }
+    }
+
+    static WaitStrategy createDefaultWaitStrategy(final String propertyName) {
+        final long timeoutMillis = parseAdditionalLongProperty(propertyName, "Timeout", 10L);
+        LOGGER.trace("DefaultAsyncWaitStrategyFactory creating TimeoutBlockingWaitStrategy(timeout={}, unit=MILLIS)", timeoutMillis);
+        return new TimeoutBlockingWaitStrategy(timeoutMillis, TimeUnit.MILLISECONDS);
+    }
+
+    private static String getFullPropertyKey(final String strategyKey, final String additionalKey) {
+        if (strategyKey.startsWith("AsyncLogger.")) {
+            return "AsyncLogger." + additionalKey;
+        } else if (strategyKey.startsWith("AsyncLoggerConfig.")) {
+            return "AsyncLoggerConfig." + additionalKey;
+        }
+        return strategyKey + additionalKey;
+    }
+
+    private static long parseAdditionalLongProperty(
+            final String propertyName,
+            final String additionalKey,
+            long defaultValue) {
+        final String key = getFullPropertyKey(propertyName, additionalKey);
+        return PropertiesUtil.getProperties().getLongProperty(key, defaultValue);
+    }
+}
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/DisruptorUtil.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/DisruptorUtil.java
index a221afd482..e37f45473a 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/DisruptorUtil.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/DisruptorUtil.java
@@ -60,46 +60,15 @@ final class DisruptorUtil {
     private DisruptorUtil() {
     }
 
-    static WaitStrategy createWaitStrategy(final String propertyName) {
-        final String strategy = PropertiesUtil.getProperties().getStringProperty(propertyName, "Timeout");
-        LOGGER.trace("property {}={}", propertyName, strategy);
-        final String strategyUp = Strings.toRootUpperCase(strategy);
-        final long timeoutMillis = parseAdditionalLongProperty(propertyName, "Timeout", 10L);
-        // String (not enum) is deliberately used here to avoid IllegalArgumentException being thrown. In case of
-        // incorrect property value, default WaitStrategy is created.
-        switch (strategyUp) {
-            case "SLEEP":
-                final long sleepTimeNs =
-                        parseAdditionalLongProperty(propertyName, "SleepTimeNs", 100L);
-                final String key = getFullPropertyKey(propertyName, "Retries");
-                final int retries =
-                        PropertiesUtil.getProperties().getIntegerProperty(key, 200);
-                return new SleepingWaitStrategy(retries, sleepTimeNs);
-            case "YIELD":
-                return new YieldingWaitStrategy();
-            case "BLOCK":
-                return new BlockingWaitStrategy();
-            case "BUSYSPIN":
-                return new BusySpinWaitStrategy();
-            case "TIMEOUT":
-                return new TimeoutBlockingWaitStrategy(timeoutMillis, TimeUnit.MILLISECONDS);
-            default:
-                return new TimeoutBlockingWaitStrategy(timeoutMillis, TimeUnit.MILLISECONDS);
-        }
-    }
+    static WaitStrategy createWaitStrategy(final String propertyName,
+                                           final AsyncWaitStrategyFactory asyncWaitStrategyFactory) {
 
-    private static String getFullPropertyKey(final String strategyKey, final String additionalKey) {
-        return strategyKey.startsWith("AsyncLogger.")
-                ? "AsyncLogger." + additionalKey
-                : "AsyncLoggerConfig." + additionalKey;
-    }
-
-    private static long parseAdditionalLongProperty(
-            final String propertyName,
-            final String additionalKey,
-            final long defaultValue) {
-        final String key = getFullPropertyKey(propertyName, additionalKey);
-        return PropertiesUtil.getProperties().getLongProperty(key, defaultValue);
+        if (asyncWaitStrategyFactory == null) {
+            LOGGER.debug("No AsyncWaitStrategyFactory was configured in the configuration, using default factory...");
+            return new DefaultAsyncWaitStrategyFactory(propertyName).createWaitStrategy();
+        }
+        LOGGER.debug("Using configured AsyncWaitStrategyFactory {}", asyncWaitStrategyFactory.getClass().getName());
+        return asyncWaitStrategyFactory.createWaitStrategy();
     }
 
     static int calculateRingBufferSize(final String propertyName) {
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AbstractConfiguration.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AbstractConfiguration.java
index 849b27b30e..c575b45bd3 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AbstractConfiguration.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AbstractConfiguration.java
@@ -29,6 +29,8 @@ import org.apache.logging.log4j.core.appender.ConsoleAppender;
 import org.apache.logging.log4j.core.async.AsyncLoggerConfig;
 import org.apache.logging.log4j.core.async.AsyncLoggerConfigDelegate;
 import org.apache.logging.log4j.core.async.AsyncLoggerConfigDisruptor;
+import org.apache.logging.log4j.core.async.AsyncWaitStrategyFactory;
+import org.apache.logging.log4j.core.async.AsyncWaitStrategyFactoryConfig;
 import org.apache.logging.log4j.core.config.arbiters.Arbiter;
 import org.apache.logging.log4j.core.config.arbiters.SelectArbiter;
 import org.apache.logging.log4j.core.filter.AbstractFilterable;
@@ -145,6 +147,7 @@ public abstract class AbstractConfiguration extends AbstractFilterable implement
     private final ConfigurationScheduler configurationScheduler;
     private final WatchManager watchManager;
     private AsyncLoggerConfigDisruptor asyncLoggerConfigDisruptor;
+    private AsyncWaitStrategyFactory asyncWaitStrategyFactory;
     private final WeakReference<LoggerContext> loggerContext;
 
     /**
@@ -225,11 +228,16 @@ public abstract class AbstractConfiguration extends AbstractFilterable implement
         // lazily instantiate only when requested by AsyncLoggers:
         // loading AsyncLoggerConfigDisruptor requires LMAX Disruptor jar on classpath
         if (asyncLoggerConfigDisruptor == null) {
-            asyncLoggerConfigDisruptor = new AsyncLoggerConfigDisruptor();
+            asyncLoggerConfigDisruptor = new AsyncLoggerConfigDisruptor(asyncWaitStrategyFactory);
         }
         return asyncLoggerConfigDisruptor;
     }
 
+    @Override
+    public AsyncWaitStrategyFactory getAsyncWaitStrategyFactory() {
+        return asyncWaitStrategyFactory;
+    }
+
     /**
      * Initialize the configuration.
      */
@@ -315,7 +323,7 @@ public abstract class AbstractConfiguration extends AbstractFilterable implement
         }
     }
 
-	/**
+    /**
      * Start the configuration.
      */
     @Override
@@ -691,6 +699,9 @@ public abstract class AbstractConfiguration extends AbstractFilterable implement
                 final List<CustomLevelConfig> copy = new ArrayList<>(customLevels);
                 copy.add(child.getObject(CustomLevelConfig.class));
                 customLevels = copy;
+            } else if (child.isInstanceOf(AsyncWaitStrategyFactoryConfig.class)) {
+                AsyncWaitStrategyFactoryConfig awsfc = child.getObject(AsyncWaitStrategyFactoryConfig.class);
+                asyncWaitStrategyFactory = awsfc.createWaitStrategyFactory();
             } else {
                 final List<String> expected = Arrays.asList("\"Appenders\"", "\"Loggers\"", "\"Properties\"",
                         "\"Scripts\"", "\"CustomLevels\"");
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/Configuration.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/Configuration.java
index eaa2161812..53fa9a4f94 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/Configuration.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/Configuration.java
@@ -23,6 +23,7 @@ import org.apache.logging.log4j.core.LogEvent;
 import org.apache.logging.log4j.core.Logger;
 import org.apache.logging.log4j.core.LoggerContext;
 import org.apache.logging.log4j.core.async.AsyncLoggerConfigDelegate;
+import org.apache.logging.log4j.core.async.AsyncWaitStrategyFactory;
 import org.apache.logging.log4j.core.filter.Filterable;
 import org.apache.logging.log4j.core.lookup.ConfigurationStrSubstitutor;
 import org.apache.logging.log4j.core.lookup.StrSubstitutor;
@@ -190,6 +191,16 @@ public interface Configuration extends Filterable {
      */
     AsyncLoggerConfigDelegate getAsyncLoggerConfigDelegate();
 
+    /**
+     * Returns the {@code AsyncWaitStrategyFactory} defined in this Configuration;
+     * this factory is used to create the LMAX disruptor {@code WaitStrategy} used
+     * by the disruptor ringbuffer for Async Loggers.
+     *
+     * @return the {@code AsyncWaitStrategyFactory}
+     * @since 2.17.3
+     */
+    AsyncWaitStrategyFactory getAsyncWaitStrategyFactory();
+
     /**
      * Return the WatchManager.
      *
diff --git a/src/site/asciidoc/manual/async.adoc b/src/site/asciidoc/manual/async.adoc
index a9b70d50df..dca45f00f2 100644
--- a/src/site/asciidoc/manual/async.adoc
+++ b/src/site/asciidoc/manual/async.adoc
@@ -217,20 +217,21 @@ link:../log4j-core/apidocs/org/apache/logging/log4j/core/async/AsyncQueueFullPol
 |`Timeout`
 |Valid values: Block,
 Timeout, Sleep, Yield.
+(See also the <<Custom WaitStrategy>> section below.) +
 `Block` is a strategy that uses a lock and condition variable for the
 I/O thread waiting for log events. Block can be used when throughput and
 low-latency are not as important as CPU resource. Recommended for
-resource constrained/virtualised environments.
+resource constrained/virtualised environments. +
 `Timeout` is a variation of the `Block` strategy that will periodically
 wake up from the lock condition await() call. This ensures that if a
 notification is missed somehow the consumer thread is not stuck but will
-recover with a small latency delay (default 10ms).
+recover with a small latency delay (default 10ms). +
 `Sleep` is a strategy that initially spins, then uses a Thread.yield(),
 and eventually parks for the minimum number of nanos the OS and JVM will
 allow while the I/O thread is waiting for log events. Sleep is a good
 compromise between performance and CPU resource. This strategy has very
 low impact on the application thread, in exchange for some additional
-latency for actually getting the message logged.
+latency for actually getting the message logged. +
 `Yield` is a strategy that uses a Thread.yield() for waiting for log
 events after an initially spinning. Yield is a good compromise between
 performance and CPU resource, but may use more CPU than Sleep in order
@@ -399,7 +400,8 @@ link:../log4j-core/apidocs/org/apache/logging/log4j/core/async/AsyncQueueFullPol
 |[[asyncLoggerConfigWaitStrategy]]log4j2.asyncLoggerConfigWaitStrategy
 |`Timeout`
 |Valid values: Block,
-Timeout, Sleep, Yield. +
+Timeout, Sleep, Yield.
+(See also the <<Custom WaitStrategy>> section below.) +
 `Block` is a strategy that uses a lock and condition variable for the
 I/O thread waiting for log events. Block can be used when throughput and
 low-latency are not as important as CPU resource. Recommended for
@@ -452,6 +454,57 @@ system properties
 link:configuration.html#asyncQueueFullPolicy[`log4j2.asyncQueueFullPolicy`
 and `log4j2.discardThreshold`].
 
+[#WaitStrategy]
+== Custom WaitStrategy
+The system properties mentioned above allow only choice from among a fixed set of predefined WaitStrategies.
+There may be cases where you want to configure a custom WaitStrategy that is not in this list.
+This is possible by using a `AsyncWaitStrategyFactory` element in the Log4j configuration.
+
+A configuration that configures a custom WaitStrategy can look as follows:
+
+[source,xml]
+----
+<?xml version="1.0" encoding="UTF-8"?>
+<Configuration status="WARN">
+
+  <AsyncWaitStrategyFactory
+      class="my.custom.AsyncWaitStrategyFactory" />
+
+  <Appenders>
+    <File name="MyFile" fileName="logs/app.log">
+      <PatternLayout pattern="%d %p %c{1.} [%t] %m%n" />
+    </File>
+  </Appenders>
+  <Loggers>
+    <AsyncRoot level="info">
+      <AppenderRef ref="MyFile"/>
+    </AsyncRoot>
+  </Loggers>
+</Configuration>
+----
+
+The specified class must implement the
+`org.apache.logging.log4j.core.async.AsyncWaitStrategyFactory` interface, which is defined as follows:
+
+[source,java]
+----
+public interface AsyncWaitStrategyFactory {
+  /**
+  * Returns a non-null implementation of the LMAX Disruptor's WaitStrategy interface.
+  * This WaitStrategy will be used by Log4j Async Loggers and Async LoggerConfigs.
+  *
+  * @return the WaitStrategy instance to be used by Async Loggers and Async LoggerConfigs
+  */
+  WaitStrategy createWaitStrategy();
+}
+----
+
+The specified class must also have a public no-argument constructor;
+Log4j will instantiate an instance of the specified factory class and use this factory to create the WaitStrategy used by all Async Loggers.
+
+WaitStrategy-related system properties are ignored if a `AsyncWaitStrategyFactory` is configured.
+
+
 [#Location]
 == Location, location, location...
 
diff --git a/src/site/asciidoc/manual/garbagefree.adoc b/src/site/asciidoc/manual/garbagefree.adoc
index edf4bcde12..2dc2b74d81 100644
--- a/src/site/asciidoc/manual/garbagefree.adoc
+++ b/src/site/asciidoc/manual/garbagefree.adoc
@@ -100,12 +100,13 @@ NOTE: As of version 2.6, a Log4j configuration containing a
 `<Properties>` section will result in temporary objects being created
 during steady-state logging.
 
-NOTE: The Async Logger Timeout wait strategy (the default) and the
-Block wait strategy cause
-`java.util.concurrent.locks.AbstractQueuedSynchronizer$Node` objects to
-be created. The Yield and Sleep wait strategies are garbage-free. (See
+NOTE: The Async Logger Timeout wait strategy (the default) and the Block wait strategy
+are not garbage-free since they
+cause `java.util.concurrent.locks.AbstractQueuedSynchronizer$Node` objects to be created.
+The Yield and Sleep wait strategies are garbage-free. (For configuring predefined wait strategies, see
 link:async.html#SysPropsAllAsync[here] and
-link:async.html#SysPropsMixedSync-Async[here].)
+link:async.html#SysPropsMixedSync-Async[here],
+you may also configure a link:async.html#WaitStrategy[custom wait strategy].)
 
 === Disabling Garbage-free Logging
 
diff --git a/src/site/site.xml b/src/site/site.xml
index b161a60d55..26e152f913 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -229,6 +229,7 @@
         <item name="Trade-offs" href="/manual/async.html#Trade-offs"/>
         <item name="All Loggers Async" href="/manual/async.html#AllAsync"/>
         <item name="Mixed Sync &amp; Async" href="/manual/async.html#MixedSync-Async"/>
+        <item name="WaitStrategy" href="/manual/async.html#WaitStrategy"/>
         <item name="Location" href="/manual/async.html#Location"/>
         <item name="Performance" href="/manual/async.html#Performance"/>
         <item name="Under The Hood" href="/manual/async.html#UnderTheHood"/>