You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@logging.apache.org by pk...@apache.org on 2024/04/04 21:50:52 UTC

(logging-log4j2) 14/14: Delegate `ScopedContext` functionality to interface

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

pkarwasz pushed a commit to branch ScopedContext-replace-with-interface
in repository https://gitbox.apache.org/repos/asf/logging-log4j2.git

commit c64a08a41945b31537549d104a65c82ee8374018
Author: Piotr P. Karwasz <pi...@karwasz.org>
AuthorDate: Thu Apr 4 23:46:52 2024 +0200

    Delegate `ScopedContext` functionality to interface
    
    To provide more configurability for the `ScopedContext` service, this PR
    moves its implementation details to `log4j-core` and replaces it with a
    `ScopedContextProvider` interface.
    
    In Log4j API only a NO-OP version of the provider is present, but each
    implementation of the API can provide its own.
---
 log4j-api-test/pom.xml                             |  11 +-
 .../org/apache/logging/log4j/test/TestLogger.java  |   5 +-
 .../log4j/test/spi/ScopedContextProviderSuite.java | 178 ++++++++
 .../apache/logging/log4j/ResourceLoggerTest.java   |   2 +
 .../apache/logging/log4j/ScopedContextTest.java    | 154 -------
 .../org/apache/logging/log4j/ScopedContext.java    | 487 +++------------------
 .../apache/logging/log4j/simple/SimpleLogger.java  |   7 +-
 .../org/apache/logging/log4j/spi/Provider.java     |   7 +
 .../logging/log4j/spi/ScopedContextProvider.java   |  81 ++++
 .../spi/internal/NoopScopedContextProvider.java    |  99 +++++
 .../internal/DefaultScopedContextProviderTest.java |  57 +++
 .../logging/log4j/core/impl/Log4jProvider.java     |  17 +
 .../log4j/core/impl/ScopedContextDataProvider.java |  20 +-
 .../internal/DefaultScopedContextProvider.java     | 389 ++++++++++++++++
 14 files changed, 908 insertions(+), 606 deletions(-)

diff --git a/log4j-api-test/pom.xml b/log4j-api-test/pom.xml
index 0ef2578dd8..40d4dc2525 100644
--- a/log4j-api-test/pom.xml
+++ b/log4j-api-test/pom.xml
@@ -37,6 +37,7 @@
     <bnd-module-name>org.apache.logging.log4j.test</bnd-module-name>
     <bnd-extra-package-options>
       org.apache.commons.lang3.*;resolution:=optional,
+      org.assertj.*;resolution:=optional,
       <!-- Both JUnit 4 and JUnit 5 are not required -->
       org.junit.*;resolution:=optional,
       org.hamcrest.*;resolution:=optional,
@@ -48,6 +49,7 @@
     <bnd-extra-module-options>
       <!-- Non-transitive static modules -->
       junit;transitive=false,
+      org.assertj.core;transitive=false,
       org.hamcrest;transitive=false,
       org.junit.jupiter.api;transitive=false,
       org.junitpioneer;transitive=false,
@@ -72,6 +74,10 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.assertj</groupId>
+      <artifactId>assertj-core</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
@@ -108,11 +114,6 @@
       <groupId>org.codehaus.plexus</groupId>
       <artifactId>plexus-utils</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.assertj</groupId>
-      <artifactId>assertj-core</artifactId>
-      <scope>test</scope>
-    </dependency>
     <!-- Required for JSON support -->
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
diff --git a/log4j-api-test/src/main/java/org/apache/logging/log4j/test/TestLogger.java b/log4j-api-test/src/main/java/org/apache/logging/log4j/test/TestLogger.java
index 88e02cbbed..d3f7217108 100644
--- a/log4j-api-test/src/main/java/org/apache/logging/log4j/test/TestLogger.java
+++ b/log4j-api-test/src/main/java/org/apache/logging/log4j/test/TestLogger.java
@@ -25,11 +25,11 @@ import java.util.List;
 import java.util.Map;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Marker;
-import org.apache.logging.log4j.ScopedContext;
 import org.apache.logging.log4j.ThreadContext;
 import org.apache.logging.log4j.message.Message;
 import org.apache.logging.log4j.message.MessageFactory;
 import org.apache.logging.log4j.spi.AbstractLogger;
+import org.apache.logging.log4j.util.ProviderUtil;
 
 /**
  *
@@ -81,7 +81,8 @@ public class TestLogger extends AbstractLogger {
             sb.append(' ');
         }
         sb.append(message.getFormattedMessage());
-        Map<String, Object> contextMap = ScopedContext.getContextMap();
+        final Map<String, ?> contextMap =
+                ProviderUtil.getProvider().getScopedContextProvider().getContextMap();
         final Map<String, String> mdc = new HashMap<>(ThreadContext.getImmutableContext());
         if (contextMap != null && !contextMap.isEmpty()) {
             contextMap.forEach((key, value) -> mdc.put(key, value.toString()));
diff --git a/log4j-api-test/src/main/java/org/apache/logging/log4j/test/spi/ScopedContextProviderSuite.java b/log4j-api-test/src/main/java/org/apache/logging/log4j/test/spi/ScopedContextProviderSuite.java
new file mode 100644
index 0000000000..dc0e747776
--- /dev/null
+++ b/log4j-api-test/src/main/java/org/apache/logging/log4j/test/spi/ScopedContextProviderSuite.java
@@ -0,0 +1,178 @@
+/*
+ * 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.test.spi;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.logging.log4j.ScopedContext;
+import org.apache.logging.log4j.spi.ScopedContextProvider;
+
+/**
+ * Provides test that should be passed by all implementations of {@link ScopedContextProviderSuite}.
+ * @since 2.24.0
+ */
+public abstract class ScopedContextProviderSuite {
+
+    private static ScopedContext.Instance where(
+            final ScopedContextProvider provider, final String key, final Object value) {
+        return provider.newScopedContext(key, value);
+    }
+
+    protected static void testScope(final ScopedContextProvider scopedContext) {
+        where(scopedContext, "key1", "Log4j2")
+                .run(() -> assertThat(scopedContext.getValue("key1")).isEqualTo("Log4j2"));
+        where(scopedContext, "key1", "value1").run(() -> {
+            assertThat(scopedContext.getValue("key1")).isEqualTo("value1");
+            where(scopedContext, "key2", "value2").run(() -> {
+                assertThat(scopedContext.getValue("key1")).isEqualTo("value1");
+                assertThat(scopedContext.getValue("key2")).isEqualTo("value2");
+            });
+        });
+    }
+
+    private static void runWhere(
+            final ScopedContextProvider provider, final String key, final Object value, final Runnable task) {
+        provider.newScopedContext(key, value).run(task);
+    }
+
+    private static Future<Void> runWhere(
+            final ScopedContextProvider provider,
+            final String key,
+            final Object value,
+            final ExecutorService executorService,
+            final Runnable task) {
+        return provider.newScopedContext(key, value).run(executorService, task);
+    }
+
+    protected static void testRunWhere(final ScopedContextProvider scopedContext) {
+        runWhere(scopedContext, "key1", "Log4j2", () -> assertThat(scopedContext.getValue("key1"))
+                .isEqualTo("Log4j2"));
+        runWhere(scopedContext, "key1", "value1", () -> {
+            assertThat(scopedContext.getValue("key1")).isEqualTo("value1");
+            runWhere(scopedContext, "key2", "value2", () -> {
+                assertThat(scopedContext.getValue("key1")).isEqualTo("value1");
+                assertThat(scopedContext.getValue("key2")).isEqualTo("value2");
+            });
+        });
+    }
+
+    protected static void testRunThreads(final ScopedContextProvider scopedContext) {
+        BlockingQueue<Runnable> workQueue = new ArrayBlockingQueue<>(5);
+        ExecutorService executorService = new ThreadPoolExecutor(1, 2, 30, TimeUnit.SECONDS, workQueue);
+        final long id = Thread.currentThread().getId();
+        final AtomicLong counter = new AtomicLong(0);
+        runWhere(scopedContext, "key1", "Log4j2", () -> {
+            assertThat(scopedContext.getValue("key1")).isEqualTo("Log4j2");
+            Future<?> future = runWhere(scopedContext, "key2", "value2", executorService, () -> {
+                assertNotEquals(Thread.currentThread().getId(), id);
+                assertThat(scopedContext.getValue("key1")).isEqualTo("Log4j2");
+                counter.incrementAndGet();
+            });
+            assertDoesNotThrow(() -> {
+                future.get();
+                assertTrue(future.isDone());
+                assertThat(counter.get()).isEqualTo(1);
+            });
+        });
+    }
+
+    protected static void testThreads(final ScopedContextProvider scopedContext) throws Exception {
+        BlockingQueue<Runnable> workQueue = new ArrayBlockingQueue<>(5);
+        ExecutorService executorService = new ThreadPoolExecutor(1, 2, 30, TimeUnit.SECONDS, workQueue);
+        final long id = Thread.currentThread().getId();
+        final AtomicLong counter = new AtomicLong(0);
+        where(scopedContext, "key1", "Log4j2").run(() -> {
+            assertThat(scopedContext.getValue("key1")).isEqualTo("Log4j2");
+            Future<?> future = where(scopedContext, "key2", "value2").run(executorService, () -> {
+                assertNotEquals(Thread.currentThread().getId(), id);
+                assertThat(scopedContext.getValue("key1")).isEqualTo("Log4j2");
+                counter.incrementAndGet();
+            });
+            assertDoesNotThrow(() -> {
+                future.get();
+                assertTrue(future.isDone());
+                assertThat(counter.get()).isEqualTo(1);
+            });
+        });
+    }
+
+    protected static void testThreadException(final ScopedContextProvider scopedContext) throws Exception {
+        BlockingQueue<Runnable> workQueue = new ArrayBlockingQueue<>(5);
+        final AtomicBoolean exceptionCaught = new AtomicBoolean(false);
+        ExecutorService executorService = new ThreadPoolExecutor(1, 2, 30, TimeUnit.SECONDS, workQueue);
+        long id = Thread.currentThread().getId();
+        runWhere(scopedContext, "key1", "Log4j2", () -> {
+            assertThat(scopedContext.getValue("key1")).isEqualTo("Log4j2");
+            Future<?> future = where(scopedContext, "key2", "value2").run(executorService, () -> {
+                assertNotEquals(Thread.currentThread().getId(), id);
+                throw new NullPointerException("On purpose NPE");
+            });
+            assertThatThrownBy(future::get)
+                    .hasRootCauseInstanceOf(NullPointerException.class)
+                    .hasRootCauseMessage("On purpose NPE");
+        });
+    }
+
+    private static <R> R callWhere(
+            final ScopedContextProvider provider, final String key, final Object value, final Callable<R> task)
+            throws Exception {
+        return provider.newScopedContext(key, value).call(task);
+    }
+
+    private static <R> Future<R> callWhere(
+            final ScopedContextProvider provider,
+            final String key,
+            final Object value,
+            final ExecutorService executorService,
+            final Callable<R> task) {
+        return provider.newScopedContext(key, value).call(executorService, task);
+    }
+
+    protected static void testThreadCall(final ScopedContextProvider scopedContext) throws Exception {
+        BlockingQueue<Runnable> workQueue = new ArrayBlockingQueue<>(5);
+        ExecutorService executorService = new ThreadPoolExecutor(1, 2, 30, TimeUnit.SECONDS, workQueue);
+        final long id = Thread.currentThread().getId();
+        final AtomicInteger counter = new AtomicInteger(0);
+        int returnVal = callWhere(scopedContext, "key1", "Log4j2", () -> {
+            assertThat(scopedContext.getValue("key1")).isEqualTo("Log4j2");
+            Future<Integer> future = callWhere(scopedContext, "key2", "value2", executorService, () -> {
+                assertNotEquals(Thread.currentThread().getId(), id);
+                assertThat(scopedContext.getValue("key1")).isEqualTo("Log4j2");
+                return counter.incrementAndGet();
+            });
+            Integer val = future.get();
+            assertTrue(future.isDone());
+            assertThat(counter.get()).isEqualTo(1);
+            return val;
+        });
+        assertThat(returnVal).isEqualTo(1);
+    }
+}
diff --git a/log4j-api-test/src/test/java/org/apache/logging/log4j/ResourceLoggerTest.java b/log4j-api-test/src/test/java/org/apache/logging/log4j/ResourceLoggerTest.java
index 4822a52e12..c2273e2790 100644
--- a/log4j-api-test/src/test/java/org/apache/logging/log4j/ResourceLoggerTest.java
+++ b/log4j-api-test/src/test/java/org/apache/logging/log4j/ResourceLoggerTest.java
@@ -29,11 +29,13 @@ import java.util.function.Supplier;
 import org.apache.logging.log4j.test.TestLogger;
 import org.apache.logging.log4j.test.TestLoggerContextFactory;
 import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 /**
  * Class Description goes here.
  */
+@Disabled("Does not work with the NO-OP implementation of ScopedContextProvider in the API.")
 public class ResourceLoggerTest {
     @BeforeAll
     public static void beforeAll() {
diff --git a/log4j-api-test/src/test/java/org/apache/logging/log4j/ScopedContextTest.java b/log4j-api-test/src/test/java/org/apache/logging/log4j/ScopedContextTest.java
deleted file mode 100644
index d9ba5872e6..0000000000
--- a/log4j-api-test/src/test/java/org/apache/logging/log4j/ScopedContextTest.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * 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;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import org.junit.jupiter.api.Test;
-
-public class ScopedContextTest {
-
-    @Test
-    public void testScope() {
-        ScopedContext.where("key1", "Log4j2").run(() -> assertThat(ScopedContext.get("key1"), equalTo("Log4j2")));
-        ScopedContext.where("key1", "value1").run(() -> {
-            assertThat(ScopedContext.get("key1"), equalTo("value1"));
-            ScopedContext.where("key2", "value2").run(() -> {
-                assertThat(ScopedContext.get("key1"), equalTo("value1"));
-                assertThat(ScopedContext.get("key2"), equalTo("value2"));
-            });
-        });
-    }
-
-    @Test
-    public void testRunWhere() {
-        ScopedContext.runWhere("key1", "Log4j2", () -> assertThat(ScopedContext.get("key1"), equalTo("Log4j2")));
-        ScopedContext.runWhere("key1", "value1", () -> {
-            assertThat(ScopedContext.get("key1"), equalTo("value1"));
-            ScopedContext.runWhere("key2", "value2", () -> {
-                assertThat(ScopedContext.get("key1"), equalTo("value1"));
-                assertThat(ScopedContext.get("key2"), equalTo("value2"));
-            });
-        });
-    }
-
-    @Test
-    public void testRunThreads() throws Exception {
-        BlockingQueue<Runnable> workQueue = new ArrayBlockingQueue<>(5);
-        ExecutorService executorService = new ThreadPoolExecutor(1, 2, 30, TimeUnit.SECONDS, workQueue);
-        final long id = Thread.currentThread().getId();
-        final AtomicLong counter = new AtomicLong(0);
-        ScopedContext.runWhere("key1", "Log4j2", () -> {
-            assertThat(ScopedContext.get("key1"), equalTo("Log4j2"));
-            Future<?> future = ScopedContext.runWhere("key2", "value2", executorService, () -> {
-                assertNotEquals(Thread.currentThread().getId(), id);
-                assertThat(ScopedContext.get("key1"), equalTo("Log4j2"));
-                counter.incrementAndGet();
-            });
-            try {
-                future.get();
-                assertTrue(future.isDone());
-                assertEquals(1, counter.get());
-            } catch (Exception ex) {
-                fail("Failed with " + ex.getMessage());
-            }
-        });
-    }
-
-    @Test
-    public void testThreads() throws Exception {
-        BlockingQueue<Runnable> workQueue = new ArrayBlockingQueue<>(5);
-        ExecutorService executorService = new ThreadPoolExecutor(1, 2, 30, TimeUnit.SECONDS, workQueue);
-        final long id = Thread.currentThread().getId();
-        final AtomicLong counter = new AtomicLong(0);
-        ScopedContext.where("key1", "Log4j2").run(() -> {
-            assertThat(ScopedContext.get("key1"), equalTo("Log4j2"));
-            Future<?> future = ScopedContext.where("key2", "value2").run(executorService, () -> {
-                assertNotEquals(Thread.currentThread().getId(), id);
-                assertThat(ScopedContext.get("key1"), equalTo("Log4j2"));
-                counter.incrementAndGet();
-            });
-            try {
-                future.get();
-                assertTrue(future.isDone());
-                assertEquals(1, counter.get());
-            } catch (Exception ex) {
-                fail("Failed with " + ex.getMessage());
-            }
-        });
-    }
-
-    @Test
-    public void testThreadException() throws Exception {
-        BlockingQueue<Runnable> workQueue = new ArrayBlockingQueue<>(5);
-        final AtomicBoolean exceptionCaught = new AtomicBoolean(false);
-        ExecutorService executorService = new ThreadPoolExecutor(1, 2, 30, TimeUnit.SECONDS, workQueue);
-        long id = Thread.currentThread().getId();
-        ScopedContext.runWhere("key1", "Log4j2", () -> {
-            assertThat(ScopedContext.get("key1"), equalTo("Log4j2"));
-            Future<?> future = ScopedContext.where("key2", "value2").run(executorService, () -> {
-                assertNotEquals(Thread.currentThread().getId(), id);
-                throw new NullPointerException("On purpose NPE");
-            });
-            try {
-                future.get();
-            } catch (ExecutionException ex) {
-                assertThat(ex.getMessage(), equalTo("java.lang.NullPointerException: On purpose NPE"));
-                return;
-            } catch (Exception ex) {
-                fail("Failed with " + ex.getMessage());
-            }
-            fail("No exception caught");
-        });
-    }
-
-    @Test
-    public void testThreadCall() throws Exception {
-        BlockingQueue<Runnable> workQueue = new ArrayBlockingQueue<>(5);
-        ExecutorService executorService = new ThreadPoolExecutor(1, 2, 30, TimeUnit.SECONDS, workQueue);
-        final long id = Thread.currentThread().getId();
-        final AtomicInteger counter = new AtomicInteger(0);
-        int returnVal = ScopedContext.callWhere("key1", "Log4j2", () -> {
-            assertThat(ScopedContext.get("key1"), equalTo("Log4j2"));
-            Future<Integer> future = ScopedContext.callWhere("key2", "value2", executorService, () -> {
-                assertNotEquals(Thread.currentThread().getId(), id);
-                assertThat(ScopedContext.get("key1"), equalTo("Log4j2"));
-                return counter.incrementAndGet();
-            });
-            Integer val = future.get();
-            assertTrue(future.isDone());
-            assertEquals(1, counter.get());
-            return val;
-        });
-        assertThat(returnVal, equalTo(1));
-    }
-}
diff --git a/log4j-api/src/main/java/org/apache/logging/log4j/ScopedContext.java b/log4j-api/src/main/java/org/apache/logging/log4j/ScopedContext.java
index 8a57ac4361..3962190572 100644
--- a/log4j-api/src/main/java/org/apache/logging/log4j/ScopedContext.java
+++ b/log4j-api/src/main/java/org/apache/logging/log4j/ScopedContext.java
@@ -16,151 +16,42 @@
  */
 package org.apache.logging.log4j;
 
-import java.util.ArrayDeque;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.HashMap;
 import java.util.Map;
-import java.util.Optional;
 import java.util.concurrent.Callable;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.function.Supplier;
-import org.apache.logging.log4j.status.StatusLogger;
+import org.apache.logging.log4j.spi.ScopedContextProvider;
+import org.apache.logging.log4j.util.ProviderUtil;
 
 /**
  * Context that can be used for data to be logged in a block of code.
- *
+ * <p>
  * While this is influenced by ScopedValues from Java 21 it does not share the same API. While it can perform a
  * similar function as a set of ScopedValues it is really meant to allow a block of code to include a set of keys and
  * values in all the log events within that block. The underlying implementation must provide support for
  * logging the ScopedContext for that to happen.
- *
+ * </p>
+ * <p>
  * The ScopedContext will not be bound to the current thread until either a run or call method is invoked. The
  * contexts are nested so creating and running or calling via a second ScopedContext will result in the first
  * ScopedContext being hidden until the call is returned. Thus the values from the first ScopedContext need to
  * be added to the second to be included.
- *
+ * </p>
+ * <p>
  * The ScopedContext can be passed to child threads by including the ExecutorService to be used to manage the
  * run or call methods. The caller should interact with the ExecutorService as if they were submitting their
  * run or call methods directly to it. The ScopedContext performs no error handling other than to ensure the
  * ThreadContext and ScopedContext are cleaned up from the executed Thread.
- *
+ * </p>
  * @since 2.24.0
  */
-public class ScopedContext {
-
-    public static final Logger LOGGER = StatusLogger.getLogger();
-
-    private static final ThreadLocal<Deque<Instance>> scopedContext = new ThreadLocal<>();
-
-    /**
-     * Returns an immutable Map containing all the key/value pairs as Object objects.
-     * @return An immutable copy of the Map at the current scope.
-     */
-    private static Optional<Instance> getContext() {
-        Deque<Instance> stack = scopedContext.get();
-        if (stack != null) {
-            return Optional.of(stack.getFirst());
-        }
-        return Optional.empty();
-    }
-
-    /**
-     * Add the ScopeContext.
-     * @param context The ScopeContext.
-     */
-    private static void addScopedContext(Instance context) {
-        Deque<Instance> stack = scopedContext.get();
-        if (stack == null) {
-            stack = new ArrayDeque<>();
-            scopedContext.set(stack);
-        }
-        stack.addFirst(context);
-    }
+public final class ScopedContext {
 
-    /**
-     * Remove the top ScopeContext.
-     */
-    private static void removeScopedContext() {
-        Deque<Instance> stack = scopedContext.get();
-        if (stack != null) {
-            if (!stack.isEmpty()) {
-                stack.removeFirst();
-            }
-            if (stack.isEmpty()) {
-                scopedContext.remove();
-            }
-        }
-    }
-
-    /**
-     * @hidden
-     * Returns an unmodifiable copy of the current ScopedContext Map. This method should
-     * only be used by implementations of Log4j API.
-     * @return the Map of Object objects.
-     */
-    public static Map<String, Object> getContextMap() {
-        Optional<Instance> context = getContext();
-        if (context.isPresent()
-                && context.get().contextMap != null
-                && !context.get().contextMap.isEmpty()) {
-            return Collections.unmodifiableMap(context.get().contextMap);
-        }
-        return Collections.emptyMap();
-    }
+    private static final ScopedContextProvider provider =
+            ProviderUtil.getProvider().getScopedContextProvider();
 
-    /**
-     * @hidden
-     * Returns the number of entries in the context map.
-     * @return the number of items in the context map.
-     */
-    public static int size() {
-        Optional<Instance> context = getContext();
-        return context.map(instance -> instance.contextMap.size()).orElse(0);
-    }
-
-    /**
-     * Return the value of the key from the current ScopedContext, if there is one and the key exists.
-     * @param key The key.
-     * @return The value of the key in the current ScopedContext.
-     */
-    @SuppressWarnings("unchecked")
-    public static <T> T get(String key) {
-        Optional<Instance> context = getContext();
-        return context.map(instance -> (T) instance.contextMap.get(key)).orElse(null);
-    }
-
-    /**
-     * Return String value of the key from the current ScopedContext, if there is one and the key exists.
-     * @param key The key.
-     * @return The value of the key in the current ScopedContext.
-     */
-    public static String getString(String key) {
-        Optional<Instance> context = getContext();
-        if (context.isPresent()) {
-            Object obj = context.get().contextMap.get(key);
-            if (obj != null) {
-                return obj.toString();
-            }
-        }
-        return null;
-    }
-
-    /**
-     * Adds all the String rendered objects in the context map to the provided Map.
-     * @param map The Map to add entries to.
-     */
-    public static void addAll(Map<String, String> map) {
-        Optional<Instance> context = getContext();
-        if (context.isPresent()) {
-            Map<String, Object> contextMap = context.get().contextMap;
-            if (contextMap != null && !contextMap.isEmpty()) {
-                contextMap.forEach((key, value) -> map.put(key, value.toString()));
-            }
-        }
-    }
+    private ScopedContext() {}
 
     /**
      * Creates a ScopedContext Instance with a key/value pair.
@@ -170,18 +61,8 @@ public class ScopedContext {
      * @return the Instance constructed if a valid key and value were provided. Otherwise, either the
      * current Instance is returned or a new Instance is created if there is no current Instance.
      */
-    public static Instance where(String key, Object value) {
-        if (value != null) {
-            Instance parent = getContext().isPresent() ? getContext().get() : null;
-            return new Instance(parent, key, value);
-        } else {
-            if (getContext().isPresent()) {
-                Map<String, Object> map = getContextMap();
-                map.remove(key);
-                return new Instance(map);
-            }
-        }
-        return getContext().isPresent() ? getContext().get() : new Instance();
+    public static Instance where(final String key, final Object value) {
+        return provider.newScopedContext(key, value);
     }
 
     /**
@@ -191,7 +72,7 @@ public class ScopedContext {
      * @param supplier the function to generate the value.
      * @return the ScopedContext being constructed.
      */
-    public static Instance where(String key, Supplier<Object> supplier) {
+    public static Instance where(final String key, final Supplier<Object> supplier) {
         return where(key, supplier.get());
     }
 
@@ -200,211 +81,76 @@ public class ScopedContext {
      * @param map the Map.
      * @return the ScopedContext Instance constructed.
      */
-    public static Instance where(Map<String, ?> map) {
-        if (map != null && !map.isEmpty()) {
-            Map<String, Object> objectMap = new HashMap<>();
-            if (getContext().isPresent()) {
-                objectMap.putAll(getContext().get().contextMap);
-            }
-            map.forEach((key, value) -> {
-                if (value == null || (value instanceof String && ((String) value).isEmpty())) {
-                    objectMap.remove(key);
-                } else {
-                    objectMap.put(key, value);
-                }
-            });
-            return new Instance(objectMap);
-        } else {
-            return getContext().isPresent() ? getContext().get() : new Instance();
-        }
+    public static Instance where(final Map<String, ?> map) {
+        return provider.newScopedContext(map);
     }
 
     /**
      * Creates a ScopedContext with a single key/value pair and calls a method.
      * @param key the key.
-     * @param obj the value associated with the key.
-     * @param op the Runnable to call.
+     * @param value the value associated with the key.
+     * @param task the Runnable to call.
      */
-    public static void runWhere(String key, Object obj, Runnable op) {
-        if (obj != null) {
-            Map<String, Object> map = new HashMap<>();
-            if (getContext().isPresent()) {
-                map.putAll(getContext().get().contextMap);
-            }
-            map.put(key, obj);
-            new Instance(map).run(op);
-        } else {
-            Map<String, Object> map = new HashMap<>();
-            if (getContext().isPresent()) {
-                map.putAll(getContext().get().contextMap);
-            }
-            map.remove(key);
-            new Instance(map).run(op);
-        }
+    public static void runWhere(final String key, final Object value, final Runnable task) {
+        provider.newScopedContext(key, value).run(task);
     }
 
     /**
      * Creates a ScopedContext with a single key/value pair and calls a method on a separate Thread.
      * @param key the key.
-     * @param obj the value associated with the key.
+     * @param value the value associated with the key.
      * @param executorService the ExecutorService to dispatch the work.
-     * @param op the Runnable to call.
+     * @param task the Runnable to call.
      */
-    public static Future<?> runWhere(String key, Object obj, ExecutorService executorService, Runnable op) {
-        if (obj != null) {
-            Map<String, Object> map = new HashMap<>();
-            if (getContext().isPresent()) {
-                map.putAll(getContext().get().contextMap);
-            }
-            map.put(key, obj);
-            if (executorService != null) {
-                return executorService.submit(new Runner(
-                        new Instance(map), ThreadContext.getContext(), ThreadContext.getImmutableStack(), op));
-            } else {
-                new Instance(map).run(op);
-                return CompletableFuture.completedFuture(0);
-            }
-        } else {
-            Map<String, Object> map = new HashMap<>();
-            if (getContext().isPresent()) {
-                map.putAll(getContext().get().contextMap);
-            }
-            map.remove(key);
-            if (executorService != null) {
-                return executorService.submit(new Runner(
-                        new Instance(map), ThreadContext.getContext(), ThreadContext.getImmutableStack(), op));
-            } else {
-                new Instance(map).run(op);
-                return CompletableFuture.completedFuture(0);
-            }
-        }
+    public static Future<Void> runWhere(
+            final String key, final Object value, final ExecutorService executorService, final Runnable task) {
+        return provider.newScopedContext(key, value).run(executorService, task);
     }
 
     /**
      * Creates a ScopedContext with a Map of keys and values and calls a method.
      * @param map the Map.
-     * @param op the Runnable to call.
+     * @param task the Runnable to call.
      */
-    public static void runWhere(Map<String, ?> map, Runnable op) {
-        if (map != null && !map.isEmpty()) {
-            Map<String, Object> objectMap = new HashMap<>();
-            if (getContext().isPresent()) {
-                objectMap.putAll(getContext().get().contextMap);
-            }
-            objectMap.putAll(map);
-            new Instance(objectMap).run(op);
-        } else {
-            op.run();
-        }
+    public static void runWhere(final Map<String, ?> map, final Runnable task) {
+        provider.newScopedContext(map).run(task);
     }
 
     /**
      * Creates a ScopedContext with a single key/value pair and calls a method.
      * @param key the key.
-     * @param obj the value associated with the key.
-     * @param op the Runnable to call.
+     * @param value the value associated with the key.
+     * @param task the Runnable to call.
      */
-    public static <R> R callWhere(String key, Object obj, Callable<R> op) throws Exception {
-        if (obj != null) {
-            Map<String, Object> map = new HashMap<>();
-            if (getContext().isPresent()) {
-                map.putAll(getContext().get().contextMap);
-            }
-            map.put(key, obj);
-            return new Instance(map).call(op);
-        } else {
-            Map<String, Object> map = new HashMap<>();
-            if (getContext().isPresent()) {
-                map.putAll(getContext().get().contextMap);
-            }
-            map.remove(key);
-            return new Instance(map).call(op);
-        }
+    public static <R> R callWhere(final String key, final Object value, final Callable<R> task) throws Exception {
+        return provider.newScopedContext(key, value).call(task);
     }
 
     /**
      * Creates a ScopedContext with a single key/value pair and calls a method on a separate Thread.
      * @param key the key.
-     * @param obj the value associated with the key.
+     * @param value the value associated with the key.
      * @param executorService the ExecutorService to dispatch the work.
-     * @param op the Callable to call.
+     * @param task the Callable to call.
      */
-    public static <R> Future<R> callWhere(String key, Object obj, ExecutorService executorService, Callable<R> op)
-            throws Exception {
-        if (obj != null) {
-            Map<String, Object> map = new HashMap<>();
-            if (getContext().isPresent()) {
-                map.putAll(getContext().get().contextMap);
-            }
-            map.put(key, obj);
-            if (executorService != null) {
-                return executorService.submit(new Caller<R>(
-                        new Instance(map), ThreadContext.getContext(), ThreadContext.getImmutableStack(), op));
-            } else {
-                R ret = new Instance(map).call(op);
-                return CompletableFuture.completedFuture(ret);
-            }
-        } else {
-            if (executorService != null) {
-                Map<String, Object> map = new HashMap<>();
-                if (getContext().isPresent()) {
-                    map.putAll(getContext().get().contextMap);
-                }
-                map.remove(key);
-                return executorService.submit(new Caller<R>(
-                        new Instance(map), ThreadContext.getContext(), ThreadContext.getImmutableStack(), op));
-            } else {
-                R ret = op.call();
-                return CompletableFuture.completedFuture(ret);
-            }
-        }
+    public static <R> Future<R> callWhere(
+            final String key, final Object value, final ExecutorService executorService, final Callable<R> task) {
+        return provider.newScopedContext(key, value).call(executorService, task);
     }
 
     /**
      * Creates a ScopedContext with a Map of keys and values and calls a method.
      * @param map the Map.
-     * @param op the Runnable to call.
+     * @param task the Runnable to call.
      */
-    public static <R> R callWhere(Map<String, ?> map, Callable<R> op) throws Exception {
-        if (map != null && !map.isEmpty()) {
-            Map<String, Object> objectMap = new HashMap<>();
-            if (getContext().isPresent()) {
-                objectMap.putAll(getContext().get().contextMap);
-            }
-            objectMap.putAll(map);
-            return new Instance(objectMap).call(op);
-        } else {
-            return op.call();
-        }
+    public static <R> R callWhere(final Map<String, ?> map, final Callable<R> task) throws Exception {
+        return provider.newScopedContext(map).call(task);
     }
 
-    public static class Instance {
-
-        private final Instance parent;
-        private final String key;
-        private final Object value;
-        private final Map<String, Object> contextMap;
-
-        private Instance() {
-            this.parent = null;
-            this.key = null;
-            this.value = null;
-            this.contextMap = null;
-        }
-
-        private Instance(Map<String, Object> map) {
-            this.parent = null;
-            this.key = null;
-            this.value = null;
-            this.contextMap = map;
-        }
-
-        private Instance(Instance parent, String key, Object value) {
-            this.parent = parent;
-            this.key = key;
-            this.value = value;
-            this.contextMap = null;
-        }
+    /**
+     * A holder of scoped context data.
+     */
+    public interface Instance {
 
         /**
          * Adds a key/value pair to the ScopedContext being constructed.
@@ -413,9 +159,7 @@ public class ScopedContext {
          * @param value the value associated with the key.
          * @return the ScopedContext being constructed.
          */
-        public Instance where(String key, Object value) {
-            return addObject(key, value);
-        }
+        Instance where(String key, Object value);
 
         /**
          * Adds a key/value pair to the ScopedContext being constructed.
@@ -424,158 +168,37 @@ public class ScopedContext {
          * @param supplier the function to generate the value.
          * @return the ScopedContext being constructed.
          */
-        public Instance where(String key, Supplier<Object> supplier) {
-            return addObject(key, supplier.get());
-        }
-
-        private Instance addObject(String key, Object obj) {
-            if (obj != null) {
-                return new Instance(this, key, obj);
-            }
-            return this;
-        }
+        Instance where(String key, Supplier<Object> supplier);
 
         /**
          * Executes a code block that includes all the key/value pairs added to the ScopedContext.
          *
-         * @param op the code block to execute.
+         * @param task the code block to execute.
          */
-        public void run(Runnable op) {
-            new Runner(this, null, null, op).run();
-        }
+        void run(Runnable task);
 
         /**
          * Executes a code block that includes all the key/value pairs added to the ScopedContext on a different Thread.
          *
-         * @param op the code block to execute.
+         * @param task the code block to execute.
          * @return a Future representing pending completion of the task
          */
-        public Future<?> run(ExecutorService executorService, Runnable op) {
-            return executorService.submit(
-                    new Runner(this, ThreadContext.getContext(), ThreadContext.getImmutableStack(), op));
-        }
+        Future<Void> run(ExecutorService executorService, Runnable task);
 
         /**
          * Executes a code block that includes all the key/value pairs added to the ScopedContext.
          *
-         * @param op the code block to execute.
+         * @param task the code block to execute.
          * @return the return value from the code block.
          */
-        public <R> R call(Callable<R> op) throws Exception {
-            return new Caller<R>(this, null, null, op).call();
-        }
+        <R> R call(Callable<R> task) throws Exception;
 
         /**
          * Executes a code block that includes all the key/value pairs added to the ScopedContext on a different Thread.
          *
-         * @param op the code block to execute.
+         * @param task the code block to execute.
          * @return a Future representing pending completion of the task
          */
-        public <R> Future<R> call(ExecutorService executorService, Callable<R> op) {
-            return executorService.submit(
-                    new Caller<R>(this, ThreadContext.getContext(), ThreadContext.getImmutableStack(), op));
-        }
-    }
-
-    private static class Runner implements Runnable {
-        private final Map<String, Object> contextMap = new HashMap<>();
-        private final Map<String, String> threadContextMap;
-        private final ThreadContext.ContextStack contextStack;
-        private final Instance context;
-        private final Runnable op;
-
-        public Runner(
-                Instance context,
-                Map<String, String> threadContextMap,
-                ThreadContext.ContextStack contextStack,
-                Runnable op) {
-            this.context = context;
-            this.threadContextMap = threadContextMap;
-            this.contextStack = contextStack;
-            this.op = op;
-        }
-
-        @Override
-        public void run() {
-            Instance scopedContext = context;
-            // If the current context has a Map then we can just use it.
-            if (context.contextMap == null) {
-                do {
-                    if (scopedContext.contextMap != null) {
-                        // Once we hit a scope with an already populated Map we won't need to go any further.
-                        contextMap.putAll(scopedContext.contextMap);
-                        break;
-                    } else if (scopedContext.key != null) {
-                        contextMap.putIfAbsent(scopedContext.key, scopedContext.value);
-                    }
-                    scopedContext = scopedContext.parent;
-                } while (scopedContext != null);
-                scopedContext = new Instance(contextMap);
-            }
-            if (threadContextMap != null && !threadContextMap.isEmpty()) {
-                ThreadContext.putAll(threadContextMap);
-            }
-            if (contextStack != null) {
-                ThreadContext.setStack(contextStack);
-            }
-            addScopedContext(scopedContext);
-            try {
-                op.run();
-            } finally {
-                removeScopedContext();
-                ThreadContext.clearAll();
-            }
-        }
-    }
-
-    private static class Caller<R> implements Callable<R> {
-        private final Map<String, Object> contextMap = new HashMap<>();
-        private final Instance context;
-        private final Map<String, String> threadContextMap;
-        private final ThreadContext.ContextStack contextStack;
-        private final Callable<R> op;
-
-        public Caller(
-                Instance context,
-                Map<String, String> threadContextMap,
-                ThreadContext.ContextStack contextStack,
-                Callable<R> op) {
-            this.context = context;
-            this.threadContextMap = threadContextMap;
-            this.contextStack = contextStack;
-            this.op = op;
-        }
-
-        @Override
-        public R call() throws Exception {
-            Instance scopedContext = context;
-            // If the current context has a Map then we can just use it.
-            if (context.contextMap == null) {
-                do {
-                    if (scopedContext.contextMap != null) {
-                        // Once we hit a scope with an already populated Map we won't need to go any further.
-                        contextMap.putAll(scopedContext.contextMap);
-                        break;
-                    } else if (scopedContext.key != null) {
-                        contextMap.putIfAbsent(scopedContext.key, scopedContext.value);
-                    }
-                    scopedContext = scopedContext.parent;
-                } while (scopedContext != null);
-                scopedContext = new Instance(contextMap);
-            }
-            if (threadContextMap != null && !threadContextMap.isEmpty()) {
-                ThreadContext.putAll(threadContextMap);
-            }
-            if (contextStack != null) {
-                ThreadContext.setStack(contextStack);
-            }
-            addScopedContext(scopedContext);
-            try {
-                return op.call();
-            } finally {
-                removeScopedContext();
-                ThreadContext.clearAll();
-            }
-        }
+        <R> Future<R> call(ExecutorService executorService, Callable<R> task);
     }
 }
diff --git a/log4j-api/src/main/java/org/apache/logging/log4j/simple/SimpleLogger.java b/log4j-api/src/main/java/org/apache/logging/log4j/simple/SimpleLogger.java
index 053ac45dcb..1690893187 100644
--- a/log4j-api/src/main/java/org/apache/logging/log4j/simple/SimpleLogger.java
+++ b/log4j-api/src/main/java/org/apache/logging/log4j/simple/SimpleLogger.java
@@ -21,11 +21,9 @@ import java.io.PrintStream;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.Date;
-import java.util.HashMap;
 import java.util.Map;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Marker;
-import org.apache.logging.log4j.ScopedContext;
 import org.apache.logging.log4j.ThreadContext;
 import org.apache.logging.log4j.message.Message;
 import org.apache.logging.log4j.message.MessageFactory;
@@ -296,9 +294,8 @@ public class SimpleLogger extends AbstractLogger {
         }
         sb.append(msg.getFormattedMessage());
         if (showContextMap) {
-            final Map<String, String> mdc = new HashMap<>(ThreadContext.getImmutableContext());
-            ScopedContext.getContextMap().forEach((key, value) -> mdc.put(key, value.toString()));
-            if (!mdc.isEmpty()) {
+            final Map<String, String> mdc = ThreadContext.getImmutableContext();
+            if (mdc.size() > 0) {
                 sb.append(SPACE);
                 sb.append(mdc.toString());
                 sb.append(SPACE);
diff --git a/log4j-api/src/main/java/org/apache/logging/log4j/spi/Provider.java b/log4j-api/src/main/java/org/apache/logging/log4j/spi/Provider.java
index bca1a9ecae..3eff6e8b3d 100644
--- a/log4j-api/src/main/java/org/apache/logging/log4j/spi/Provider.java
+++ b/log4j-api/src/main/java/org/apache/logging/log4j/spi/Provider.java
@@ -434,6 +434,13 @@ public class Provider {
         return threadContextMapLazy.get();
     }
 
+    /**
+     * @return An implementation of the {@link ScopedContextProvider} service to use.
+     */
+    public ScopedContextProvider getScopedContextProvider() {
+        return ScopedContextProvider.noop();
+    }
+
     /**
      * Gets the URL containing this Provider's Log4j details.
      *
diff --git a/log4j-api/src/main/java/org/apache/logging/log4j/spi/ScopedContextProvider.java b/log4j-api/src/main/java/org/apache/logging/log4j/spi/ScopedContextProvider.java
new file mode 100644
index 0000000000..e2281e3c34
--- /dev/null
+++ b/log4j-api/src/main/java/org/apache/logging/log4j/spi/ScopedContextProvider.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.spi;
+
+import java.util.Map;
+import org.apache.logging.log4j.ScopedContext;
+import org.apache.logging.log4j.spi.internal.NoopScopedContextProvider;
+import org.apache.logging.log4j.util.StringMap;
+
+/**
+ * The service underlying {@link ScopedContext}.
+ * @since 2.24.0
+ */
+public interface ScopedContextProvider {
+
+    static ScopedContextProvider noop() {
+        return NoopScopedContextProvider.SCOPED_CONTEXT_PROVIDER_INSTANCE;
+    }
+
+    /**
+     * @return An immutable map with the current context data.
+     */
+    Map<String, ?> getContextMap();
+
+    /**
+     * Adds the current data to the provided {@link StringMap}.
+     * @param map The {@link StringMap} to add data to.
+     */
+    default void addContextMapTo(final StringMap map) {
+        getContextMap().forEach(map::putValue);
+    }
+
+    /**
+     * Return the value of the key from the current ScopedContext, if there is one and the key exists.
+     * @param key The key.
+     * @return The value of the key in the current ScopedContext.
+     */
+    Object getValue(String key);
+
+    /**
+     * Return the value of the key from the current ScopedContext, if there is one and the key exists.
+     * @param key The key.
+     * @return The value of the key in the current ScopedContext converted to {@link String}.
+     */
+    String getString(String key);
+
+    /**
+     * Creates a new context containing the current context data from {@link org.apache.logging.log4j.ThreadContext}.
+     * @return A new instance of a scoped context.
+     */
+    ScopedContext.Instance newScopedContext();
+
+    /**
+     * Creates a new context containing the current context data from {@link org.apache.logging.log4j.ThreadContext}.
+     * @param key An additional key for the context.
+     * @param value An additional value for the context.
+     * @return A new instance of a scoped context.
+     */
+    ScopedContext.Instance newScopedContext(String key, Object value);
+
+    /**
+     * Creates a new context containing the current context data from {@link org.apache.logging.log4j.ThreadContext}.
+     * @param map Additional data to include in the context.
+     * @return A new instance of a scoped context.
+     */
+    ScopedContext.Instance newScopedContext(Map<String, ?> map);
+}
diff --git a/log4j-api/src/main/java/org/apache/logging/log4j/spi/internal/NoopScopedContextProvider.java b/log4j-api/src/main/java/org/apache/logging/log4j/spi/internal/NoopScopedContextProvider.java
new file mode 100644
index 0000000000..65c5376102
--- /dev/null
+++ b/log4j-api/src/main/java/org/apache/logging/log4j/spi/internal/NoopScopedContextProvider.java
@@ -0,0 +1,99 @@
+/*
+ * 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.spi.internal;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.function.Supplier;
+import org.apache.logging.log4j.ScopedContext;
+import org.apache.logging.log4j.spi.ScopedContextProvider;
+
+/**
+ * An implementation of {@link ScopedContextProvider} that does not propagate any data.
+ * @since 2.24.0
+ */
+public class NoopScopedContextProvider implements ScopedContextProvider {
+
+    private static final ScopedContext.Instance SCOPED_CONTEXT_INSTANCE = new NoopInstance();
+    public static final ScopedContextProvider SCOPED_CONTEXT_PROVIDER_INSTANCE = new NoopScopedContextProvider();
+
+    @Override
+    public Map<String, ?> getContextMap() {
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public Object getValue(final String key) {
+        return null;
+    }
+
+    @Override
+    public String getString(final String key) {
+        return null;
+    }
+
+    @Override
+    public ScopedContext.Instance newScopedContext() {
+        return SCOPED_CONTEXT_INSTANCE;
+    }
+
+    @Override
+    public ScopedContext.Instance newScopedContext(final String key, final Object value) {
+        return SCOPED_CONTEXT_INSTANCE;
+    }
+
+    @Override
+    public ScopedContext.Instance newScopedContext(final Map<String, ?> map) {
+        return SCOPED_CONTEXT_INSTANCE;
+    }
+
+    private static class NoopInstance implements ScopedContext.Instance {
+
+        @Override
+        public ScopedContext.Instance where(final String key, final Object value) {
+            return this;
+        }
+
+        @Override
+        public ScopedContext.Instance where(final String key, final Supplier<Object> supplier) {
+            return this;
+        }
+
+        @Override
+        public void run(final Runnable task) {
+            task.run();
+        }
+
+        @Override
+        public Future<Void> run(final ExecutorService executorService, final Runnable task) {
+            return executorService.submit(task, null);
+        }
+
+        @Override
+        public <R> R call(final Callable<R> task) throws Exception {
+            return task.call();
+        }
+
+        @Override
+        public <R> Future<R> call(final ExecutorService executorService, final Callable<R> task) {
+            return executorService.submit(task);
+        }
+    }
+}
diff --git a/log4j-core-test/src/test/java/org/apache/logging/log4j/core/impl/internal/DefaultScopedContextProviderTest.java b/log4j-core-test/src/test/java/org/apache/logging/log4j/core/impl/internal/DefaultScopedContextProviderTest.java
new file mode 100644
index 0000000000..2ca67c1e23
--- /dev/null
+++ b/log4j-core-test/src/test/java/org/apache/logging/log4j/core/impl/internal/DefaultScopedContextProviderTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.impl.internal;
+
+import org.apache.logging.log4j.test.spi.ScopedContextProviderSuite;
+import org.junit.jupiter.api.Test;
+
+class DefaultScopedContextProviderTest extends ScopedContextProviderSuite {
+
+    private static DefaultScopedContextProvider createProvider() {
+        return new DefaultScopedContextProvider();
+    }
+
+    @Test
+    void testScope() {
+        testScope(createProvider());
+    }
+
+    @Test
+    void testRunWhere() {
+        testRunWhere(createProvider());
+    }
+
+    @Test
+    void testRunThreads() {
+        testRunThreads(createProvider());
+    }
+
+    @Test
+    void testThreads() throws Exception {
+        testThreads(createProvider());
+    }
+
+    @Test
+    void testThreadException() throws Exception {
+        testThreadException(createProvider());
+    }
+
+    @Test
+    void testThreadCall() throws Exception {
+        testThreadCall(createProvider());
+    }
+}
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/Log4jProvider.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/Log4jProvider.java
index bd0b62337c..cd4ddfbe32 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/Log4jProvider.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/Log4jProvider.java
@@ -17,15 +17,32 @@
 package org.apache.logging.log4j.core.impl;
 
 import aQute.bnd.annotation.Resolution;
+import aQute.bnd.annotation.spi.ServiceConsumer;
 import aQute.bnd.annotation.spi.ServiceProvider;
+import java.util.ServiceLoader;
+import org.apache.logging.log4j.core.impl.internal.DefaultScopedContextProvider;
 import org.apache.logging.log4j.spi.Provider;
+import org.apache.logging.log4j.spi.ScopedContextProvider;
+import org.apache.logging.log4j.status.StatusLogger;
+import org.apache.logging.log4j.util.ServiceLoaderUtil;
 
 /**
  * Binding for the Log4j API.
  */
 @ServiceProvider(value = Provider.class, resolution = Resolution.OPTIONAL)
+@ServiceConsumer(value = ScopedContextProvider.class, resolution = Resolution.OPTIONAL)
 public class Log4jProvider extends Provider {
     public Log4jProvider() {
         super(10, CURRENT_VERSION, Log4jContextFactory.class);
     }
+
+    @Override
+    public ScopedContextProvider getScopedContextProvider() {
+        return ServiceLoaderUtil.safeStream(
+                        ScopedContextProvider.class,
+                        ServiceLoader.load(ScopedContextProvider.class),
+                        StatusLogger.getLogger())
+                .findFirst()
+                .orElse(DefaultScopedContextProvider.INSTANCE);
+    }
 }
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/ScopedContextDataProvider.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/ScopedContextDataProvider.java
index 653e17b7cd..948d9c04b6 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/ScopedContextDataProvider.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/ScopedContextDataProvider.java
@@ -21,8 +21,9 @@ import aQute.bnd.annotation.spi.ServiceProvider;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
-import org.apache.logging.log4j.ScopedContext;
 import org.apache.logging.log4j.core.util.ContextDataProvider;
+import org.apache.logging.log4j.spi.ScopedContextProvider;
+import org.apache.logging.log4j.util.ProviderUtil;
 
 /**
  * ContextDataProvider for {@code Map<String, String>} data.
@@ -31,16 +32,19 @@ import org.apache.logging.log4j.core.util.ContextDataProvider;
 @ServiceProvider(value = ContextDataProvider.class, resolution = Resolution.OPTIONAL)
 public class ScopedContextDataProvider implements ContextDataProvider {
 
+    private final ScopedContextProvider scopedContext =
+            ProviderUtil.getProvider().getScopedContextProvider();
+
     @Override
-    public String get(String key) {
-        return ScopedContext.getString(key);
+    public String get(final String key) {
+        return scopedContext.getString(key);
     }
 
     @Override
     public Map<String, String> supplyContextData() {
-        Map<String, Object> contextMap = ScopedContext.getContextMap();
+        final Map<String, ?> contextMap = scopedContext.getContextMap();
         if (!contextMap.isEmpty()) {
-            Map<String, String> map = new HashMap<>();
+            final Map<String, String> map = new HashMap<>();
             contextMap.forEach((key, value) -> map.put(key, value.toString()));
             return map;
         } else {
@@ -50,11 +54,11 @@ public class ScopedContextDataProvider implements ContextDataProvider {
 
     @Override
     public int size() {
-        return ScopedContext.size();
+        return scopedContext.getContextMap().size();
     }
 
     @Override
-    public void addAll(Map<String, String> map) {
-        ScopedContext.addAll(map);
+    public void addAll(final Map<String, String> map) {
+        scopedContext.getContextMap().forEach((key, value) -> map.put(key, String.valueOf(value)));
     }
 }
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/internal/DefaultScopedContextProvider.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/internal/DefaultScopedContextProvider.java
new file mode 100644
index 0000000000..ea9465337d
--- /dev/null
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/impl/internal/DefaultScopedContextProvider.java
@@ -0,0 +1,389 @@
+/*
+ * 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.impl.internal;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.function.Supplier;
+import org.apache.logging.log4j.Logger;
+import org.apache.logging.log4j.ScopedContext;
+import org.apache.logging.log4j.ThreadContext;
+import org.apache.logging.log4j.spi.ScopedContextProvider;
+import org.apache.logging.log4j.status.StatusLogger;
+import org.apache.logging.log4j.util.StringMap;
+
+public class DefaultScopedContextProvider implements ScopedContextProvider {
+
+    public static final Logger LOGGER = StatusLogger.getLogger();
+    public static final ScopedContextProvider INSTANCE = new DefaultScopedContextProvider();
+
+    private final ThreadLocal<Deque<Instance>> scopedContext = new ThreadLocal<>();
+
+    private final Instance EMPTY_INSTANCE = new Instance(this);
+
+    /**
+     * Returns an immutable Map containing all the key/value pairs as Object objects.
+     * @return An immutable copy of the Map at the current scope.
+     */
+    private Optional<Instance> getContext() {
+        final Deque<Instance> stack = scopedContext.get();
+        return stack != null ? Optional.of(stack.getFirst()) : Optional.empty();
+    }
+
+    /**
+     * Add the ScopeContext.
+     * @param context The ScopeContext.
+     */
+    private void addScopedContext(final Instance context) {
+        Deque<Instance> stack = scopedContext.get();
+        if (stack == null) {
+            stack = new ArrayDeque<>();
+            scopedContext.set(stack);
+        }
+        stack.addFirst(context);
+    }
+
+    /**
+     * Remove the top ScopeContext.
+     */
+    private void removeScopedContext() {
+        final Deque<Instance> stack = scopedContext.get();
+        if (stack != null) {
+            if (!stack.isEmpty()) {
+                stack.removeFirst();
+            }
+            if (stack.isEmpty()) {
+                scopedContext.remove();
+            }
+        }
+    }
+
+    @Override
+    public Map<String, ?> getContextMap() {
+        final Optional<Instance> context = getContext();
+        return context.isPresent()
+                        && context.get().contextMap != null
+                        && !context.get().contextMap.isEmpty()
+                ? Collections.unmodifiableMap(context.get().contextMap)
+                : Collections.emptyMap();
+    }
+
+    /**
+     * Return the value of the key from the current ScopedContext, if there is one and the key exists.
+     * @param key The key.
+     * @return The value of the key in the current ScopedContext.
+     */
+    @Override
+    public Object getValue(final String key) {
+        final Optional<Instance> context = getContext();
+        return context.map(instance -> instance.contextMap)
+                .map(map -> map.get(key))
+                .orElse(null);
+    }
+
+    /**
+     * Return String value of the key from the current ScopedContext, if there is one and the key exists.
+     * @param key The key.
+     * @return The value of the key in the current ScopedContext.
+     */
+    @Override
+    public String getString(final String key) {
+        final Optional<Instance> context = getContext();
+        if (context.isPresent()) {
+            final Object obj = context.get().contextMap.get(key);
+            if (obj != null) {
+                return obj.toString();
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Adds all the String rendered objects in the context map to the provided Map.
+     * @param map The Map to add entries to.
+     */
+    @Override
+    public void addContextMapTo(final StringMap map) {
+        final Optional<Instance> context = getContext();
+        if (context.isPresent()) {
+            final Map<String, ?> contextMap = context.get().contextMap;
+            if (contextMap != null && !contextMap.isEmpty()) {
+                contextMap.forEach((key, value) -> map.putValue(key, value.toString()));
+            }
+        }
+    }
+
+    @Override
+    public ScopedContext.Instance newScopedContext() {
+        return getContext().isPresent() ? getContext().get() : EMPTY_INSTANCE;
+    }
+
+    /**
+     * Creates a ScopedContext Instance with a key/value pair.
+     *
+     * @param key   the key to add.
+     * @param value the value associated with the key.
+     * @return the Instance constructed if a valid key and value were provided. Otherwise, either the
+     * current Instance is returned or a new Instance is created if there is no current Instance.
+     */
+    @Override
+    public ScopedContext.Instance newScopedContext(final String key, final Object value) {
+        if (value != null) {
+            final Instance parent = getContext().isPresent() ? getContext().get() : EMPTY_INSTANCE;
+            return new Instance(parent, key, value);
+        } else {
+            if (getContext().isPresent()) {
+                final Map<String, ?> map = getContextMap();
+                map.remove(key);
+                return new Instance(this, map);
+            }
+        }
+        return newScopedContext();
+    }
+
+    /**
+     * Creates a ScopedContext Instance with a Map of keys and values.
+     * @param map the Map.
+     * @return the ScopedContext Instance constructed.
+     */
+    @Override
+    public ScopedContext.Instance newScopedContext(final Map<String, ?> map) {
+        if (map != null && !map.isEmpty()) {
+            final Map<String, Object> objectMap = new HashMap<>();
+            if (getContext().isPresent()) {
+                objectMap.putAll(getContext().get().contextMap);
+            }
+            map.forEach((key, value) -> {
+                if (value == null || (value instanceof String && ((String) value).isEmpty())) {
+                    objectMap.remove(key);
+                } else {
+                    objectMap.put(key, value);
+                }
+            });
+            return new Instance(this, objectMap);
+        } else {
+            return getContext().isPresent() ? getContext().get() : EMPTY_INSTANCE;
+        }
+    }
+
+    private static void setupContext(
+            final Map<String, Object> contextMap,
+            final Map<String, String> threadContextMap,
+            final Collection<String> contextStack,
+            final Instance context) {
+        Instance scopedContext = context;
+        // If the current context has a Map then we can just use it.
+        if (context.contextMap == null) {
+            do {
+                if (scopedContext.contextMap != null) {
+                    // Once we hit a scope with an already populated Map we won't need to go any further.
+                    contextMap.putAll(scopedContext.contextMap);
+                    break;
+                } else if (scopedContext.key != null) {
+                    contextMap.putIfAbsent(scopedContext.key, scopedContext.value);
+                }
+                scopedContext = scopedContext.parent;
+            } while (scopedContext != null);
+            scopedContext = new Instance(context.getProvider(), contextMap);
+        }
+        if (threadContextMap != null && !threadContextMap.isEmpty()) {
+            ThreadContext.putAll(threadContextMap);
+        }
+        if (contextStack != null) {
+            ThreadContext.setStack(contextStack);
+        }
+        context.getProvider().addScopedContext(scopedContext);
+    }
+
+    private static final class Instance implements ScopedContext.Instance {
+
+        private final DefaultScopedContextProvider provider;
+        private final Instance parent;
+        private final String key;
+        private final Object value;
+        private final Map<String, ?> contextMap;
+
+        private Instance(final DefaultScopedContextProvider provider) {
+            this.provider = provider;
+            parent = null;
+            key = null;
+            value = null;
+            contextMap = null;
+        }
+
+        private Instance(final DefaultScopedContextProvider provider, final Map<String, ?> map) {
+            this.provider = provider;
+            parent = null;
+            key = null;
+            value = null;
+            contextMap = map;
+        }
+
+        private Instance(final Instance parent, final String key, final Object value) {
+            provider = parent.getProvider();
+            this.parent = parent;
+            this.key = key;
+            this.value = value;
+            contextMap = null;
+        }
+
+        /**
+         * Adds a key/value pair to the ScopedContext being constructed.
+         *
+         * @param key   the key to add.
+         * @param value the value associated with the key.
+         * @return the ScopedContext being constructed.
+         */
+        @Override
+        public Instance where(final String key, final Object value) {
+            return addObject(key, value);
+        }
+
+        /**
+         * Adds a key/value pair to the ScopedContext being constructed.
+         *
+         * @param key      the key to add.
+         * @param supplier the function to generate the value.
+         * @return the ScopedContext being constructed.
+         */
+        @Override
+        public Instance where(final String key, final Supplier<Object> supplier) {
+            return addObject(key, supplier.get());
+        }
+
+        private Instance addObject(final String key, final Object obj) {
+            return obj != null ? new Instance(this, key, obj) : this;
+        }
+
+        /**
+         * Executes a code block that includes all the key/value pairs added to the ScopedContext.
+         *
+         * @param task the code block to execute.
+         */
+        @Override
+        public void run(final Runnable task) {
+            new Runner(this, null, null, task).run();
+        }
+
+        /**
+         * Executes a code block that includes all the key/value pairs added to the ScopedContext on a different Thread.
+         *
+         * @param task the code block to execute.
+         * @return a Future representing pending completion of the task
+         */
+        @Override
+        public Future<Void> run(final ExecutorService executorService, final Runnable task) {
+            return executorService.submit(
+                    new Runner(this, ThreadContext.getContext(), ThreadContext.getImmutableStack(), task), null);
+        }
+
+        /**
+         * Executes a code block that includes all the key/value pairs added to the ScopedContext.
+         *
+         * @param task the code block to execute.
+         * @return the return value from the code block.
+         */
+        @Override
+        public <R> R call(final Callable<R> task) throws Exception {
+            return new Caller<>(this, null, null, task).call();
+        }
+
+        /**
+         * Executes a code block that includes all the key/value pairs added to the ScopedContext on a different Thread.
+         *
+         * @param task the code block to execute.
+         * @return a Future representing pending completion of the task
+         */
+        @Override
+        public <R> Future<R> call(final ExecutorService executorService, final Callable<R> task) {
+            return executorService.submit(
+                    new Caller<>(this, ThreadContext.getContext(), ThreadContext.getImmutableStack(), task));
+        }
+
+        private DefaultScopedContextProvider getProvider() {
+            return provider;
+        }
+    }
+
+    private static class Runner implements Runnable {
+        private final Map<String, Object> contextMap = new HashMap<>();
+        private final Map<String, String> threadContextMap;
+        private final ThreadContext.ContextStack contextStack;
+        private final Instance context;
+        private final Runnable op;
+
+        public Runner(
+                final Instance context,
+                final Map<String, String> threadContextMap,
+                final ThreadContext.ContextStack contextStack,
+                final Runnable op) {
+            this.context = context;
+            this.threadContextMap = threadContextMap;
+            this.contextStack = contextStack;
+            this.op = op;
+        }
+
+        @Override
+        public void run() {
+            setupContext(contextMap, threadContextMap, contextStack, context);
+            try {
+                op.run();
+            } finally {
+                context.getProvider().removeScopedContext();
+                ThreadContext.clearAll();
+            }
+        }
+    }
+
+    private static class Caller<R> implements Callable<R> {
+        private final Map<String, Object> contextMap = new HashMap<>();
+        private final Instance context;
+        private final Map<String, String> threadContextMap;
+        private final ThreadContext.ContextStack contextStack;
+        private final Callable<R> op;
+
+        public Caller(
+                final Instance context,
+                final Map<String, String> threadContextMap,
+                final ThreadContext.ContextStack contextStack,
+                final Callable<R> op) {
+            this.context = context;
+            this.threadContextMap = threadContextMap;
+            this.contextStack = contextStack;
+            this.op = op;
+        }
+
+        @Override
+        public R call() throws Exception {
+            setupContext(contextMap, threadContextMap, contextStack, context);
+            try {
+                return op.call();
+            } finally {
+                context.getProvider().removeScopedContext();
+                ThreadContext.clearAll();
+            }
+        }
+    }
+}