You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@logging.apache.org by GitBox <gi...@apache.org> on 2023/01/20 13:34:15 UTC

[GitHub] [logging-log4j2] vy commented on a diff in pull request #1194: Migrate Recycler API to log4j-api

vy commented on code in PR #1194:
URL: https://github.com/apache/logging-log4j2/pull/1194#discussion_r1082265434


##########
log4j-api/src/main/java/org/apache/logging/log4j/message/ReusableMessageFactory.java:
##########
@@ -89,9 +83,21 @@ public static void release(final Message message) { // LOG4J2-1583
         }
     }
 
+    @Override
+    public void recycle(final Message message) {

Review Comment:
   How does this new method compare to `release(Message)`?



##########
log4j-api/src/main/java/org/apache/logging/log4j/spi/RecyclerFactory.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.function.Consumer;
+import java.util.function.Supplier;
+
+/**
+ * Factory for {@link Recycler} strategies. Depending on workloads, different instance recycling strategies may be
+ * most performant. For example, traditional multithreaded workloads may benefit from using thread-local instance
+ * recycling while different models of concurrency or versions of the JVM may benefit from using an object pooling
+ * strategy instead.
+ *
+ * @since 3.0.0
+ */
+@FunctionalInterface
+public interface RecyclerFactory {
+
+    /**
+     * Creates a new recycler using the given supplier function for initial instances. These instances have
+     * no cleaner function and are assumed to always be reusable.
+     *
+     * @param supplier function to provide new instances of a recyclable object
+     * @param <V> the recyclable type
+     * @return a new recycler for V-type instances
+     */
+    default <V> Recycler<V> create(final Supplier<V> supplier) {
+        return create(supplier, defaultCleaner());
+    }
+
+    /**
+     * Creates a new recycler using the given functions for providing fresh instances and for cleaning up
+     * existing instances for reuse. For example, a StringBuilder recycler would provide two functions:
+     * a supplier function for constructing a new StringBuilder with a preselected initial capacity and
+     * another function for trimming the StringBuilder to some preselected maximum capacity and setting
+     * its length back to 0 as if it were a fresh StringBuilder.
+     *
+     * @param supplier function to provide new instances of a recyclable object
+     * @param cleaner function to reset a recyclable object to a fresh state
+     * @param <V> the recyclable type
+     * @return a new recycler for V-type instances
+     */
+    default <V> Recycler<V> create(Supplier<V> supplier, Consumer<V> cleaner) {
+        return create(supplier, cleaner, defaultCleaner());
+    }
+
+    /**
+     * Creates a new recycler using the given functions for providing fresh instances and for cleaning recycled
+     * instances lazily or eagerly. The lazy cleaner function is invoked on recycled instances before being
+     * returned by {@link Recycler#acquire()}. The eager cleaner function is invoked on recycled instances
+     * during {@link Recycler#release(Object)}.
+     *
+     * @param supplier function to provide new instances of a recyclable object
+     * @param lazyCleaner function to invoke to clean a recycled object before being acquired
+     * @param eagerCleaner function to invoke to clean a recycled object after being released
+     * @param <V> the recyclable type
+     * @return a new recycler for V-type instances
+     */
+    <V> Recycler<V> create(Supplier<V> supplier, Consumer<V> lazyCleaner, Consumer<V> eagerCleaner);

Review Comment:
   What is the reason we need to introduce a lazy-cleaner? Why doesn't simply an eager one suffice?



##########
log4j-api-test/src/test/java/org/apache/logging/log4j/spi/ThreadLocalRecyclerFactoryTest.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.concurrent.atomic.AtomicInteger;
+
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+class ThreadLocalRecyclerFactoryTest {
+    @Test
+    void nestedAcquiresDoNotInterfere() {
+        final Recycler<AtomicInteger> r = ThreadLocalRecyclerFactory.getInstance()
+                .create(AtomicInteger::new, i -> i.set(0));
+        final var recycler = (ThreadLocalRecyclerFactory.ThreadLocalRecycler<AtomicInteger>) r;
+
+        assertThat(recycler.getQueue()).isEmpty();
+        final AtomicInteger first = recycler.acquire();
+        assertThat(recycler.getQueue()).isEmpty();
+        final AtomicInteger second = recycler.acquire();
+        assertThat(recycler.getQueue()).isEmpty();
+        first.set(1);

Review Comment:
   It would be nice to check that `first` and `second` are not ~equal~ _the same_ in terms of their object identity.



##########
log4j-core/src/main/java/org/apache/logging/log4j/core/layout/AbstractStringLayout.java:
##########
@@ -235,9 +228,8 @@ public Serializer getHeaderSerializer() {
         return headerSerializer;
     }
 
-    private DefaultLogEventFactory getLogEventFactory() {
-        // TODO: inject this
-        return DefaultLogEventFactory.newInstance();
+    private LogEventFactory getLogEventFactory() {
+        return configuration != null ? configuration.getLogEventFactory() : DefaultLogEventFactory.newInstance();

Review Comment:
   Shouldn't this rather be an instance field that returned here? Otherwise, if `configuration` is null, we will return a _new_ instance in a _get_ method.



##########
log4j-api/src/main/java/org/apache/logging/log4j/message/ReusableParameterizedMessage.java:
##########
@@ -43,12 +45,25 @@ public class ReusableParameterizedMessage implements ReusableMessage, ParameterV
     private Object[] varargs;
     private Object[] params = new Object[MAX_PARMS];
     private Throwable throwable;
-    boolean reserved = false; // LOG4J2-1583 prevent scrambled logs with nested logging calls
+
+    private final Recycler<StringBuilder> bufferRecycler; // non-static: LOG4J2-1583
 
     /**
      * Creates a reusable message.
      */
     public ReusableParameterizedMessage() {
+        this(RecyclerFactories.getDefault());
+    }
+
+    public ReusableParameterizedMessage(final RecyclerFactory recyclerFactory) {
+        bufferRecycler = recyclerFactory.create(
+                () -> {
+                    final int currentPatternLength = messagePattern == null ? 0 : messagePattern.length();
+                    return new StringBuilder(Math.max(MIN_BUILDER_SIZE, currentPatternLength * 2));
+                },
+                buffer -> buffer.setLength(0),
+                buffer -> StringBuilders.trimToMaxSize(buffer, Constants.MAX_REUSABLE_MESSAGE_SIZE)

Review Comment:
   Shouldn't we extract the `Math.max(MIN_BUILDER_SIZE, currentPatternLength * 2)` value above to a `maxSize` variable and use it in both `trimToMaxSize(maxSize)` and `new StringBuilder(maxSize)`?



##########
log4j-api/src/main/java/org/apache/logging/log4j/spi/RecyclerFactories.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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 java.util.Objects;
+import java.util.Set;
+
+import org.apache.logging.log4j.util.QueueFactory;
+import org.apache.logging.log4j.util.Queues;
+import org.apache.logging.log4j.util.StringParameterParser;
+
+import static org.apache.logging.log4j.util.Constants.isThreadLocalsEnabled;
+
+public final class RecyclerFactories {
+
+    private RecyclerFactories() {}
+
+    private static int getDefaultCapacity() {
+        return Math.max(
+                2 * Runtime.getRuntime().availableProcessors() + 1,
+                8);
+    }
+
+    public static RecyclerFactory getDefault() {
+        return isThreadLocalsEnabled()
+                ? ThreadLocalRecyclerFactory.getInstance()
+                : new QueueingRecyclerFactory(Queues.MPMC.factory(getDefaultCapacity()));
+    }
+
+    public static RecyclerFactory ofSpec(final String recyclerFactorySpec) {
+
+        // TLA-, MPMC-, or ABQ-based queueing factory -- if nothing is specified.
+        if (recyclerFactorySpec == null) {
+            return getDefault();
+        }
+
+        // Is a dummy factory requested?
+        else if (recyclerFactorySpec.equals("dummy")) {
+            return DummyRecyclerFactory.getInstance();
+        }
+
+        // Is a TLA factory requested?
+        else if (recyclerFactorySpec.equals("threadLocal")) {
+            return ThreadLocalRecyclerFactory.getInstance();
+        }
+
+        // Is a queueing factory requested?
+        else if (recyclerFactorySpec.startsWith("queue")) {
+
+            // Determine the default capacity.
+            final int defaultCapacity = getDefaultCapacity();
+
+            return readQueueingRecyclerFactory(recyclerFactorySpec, defaultCapacity);
+        }
+
+        // Bogus input, bail out.
+        else {
+            throw new IllegalArgumentException(
+                    "invalid recycler factory: " + recyclerFactorySpec);
+        }
+
+    }
+
+    private static RecyclerFactory readQueueingRecyclerFactory(
+            final String recyclerFactorySpec,
+            final int defaultCapacity) {
+
+        // Parse the spec.
+        final String queueFactorySpec = recyclerFactorySpec.substring(
+                "queue".length() +
+                        (recyclerFactorySpec.startsWith("queue:")
+                                ? 1
+                                : 0));
+        final Map<String, StringParameterParser.Value> parsedValues =
+                StringParameterParser.parse(
+                        queueFactorySpec, Set.of("supplier", "capacity"));
+
+        // Read the supplier path.
+        final StringParameterParser.Value supplierValue = parsedValues.get("supplier");
+        final String supplierPath;
+        if (supplierValue == null || supplierValue instanceof StringParameterParser.NullValue) {
+            supplierPath = null;
+        } else {
+            supplierPath = supplierValue.toString();
+        }
+
+        // Read the capacity.
+        final StringParameterParser.Value capacityValue = parsedValues.get("capacity");
+        final int capacity;
+        if (capacityValue == null || capacityValue instanceof StringParameterParser.NullValue) {
+            capacity = defaultCapacity;
+        } else {
+            try {
+                capacity = Integer.parseInt(capacityValue.toString());
+            } catch (final NumberFormatException error) {
+                throw new IllegalArgumentException(
+                        "failed reading capacity in queueing recycler " +
+                                "factory: " + queueFactorySpec, error);
+            }
+        }
+
+        // Execute the read spec.
+        final QueueFactory queueFactory = Objects.isNull(supplierPath)
+                ? Queues.MPMC.factory(capacity)
+                : Queues.createQueueFactory(queueFactorySpec, supplierPath, capacity);

Review Comment:
   You can do something as follows:
   
   ```suggestion
           // Read the capacity.
           final StringParameterParser.Value capacityValue = parsedValues.get("capacity");
           final int capacity;
           if (capacityValue == null || capacityValue instanceof StringParameterParser.NullValue) {
               capacity = defaultCapacity;
           } else {
               try {
                   capacity = Integer.parseInt(capacityValue.toString());
               } catch (final NumberFormatException error) {
                   throw new IllegalArgumentException(
                           "failed reading capacity in queueing recycler " +
                                   "factory: " + queueFactorySpec, error);
               }
           }
   
           // Read the supplier path.
           final StringParameterParser.Value supplierValue = parsedValues.get("supplier");
           final String supplierPath;
           if (supplierValue == null || supplierValue instanceof StringParameterParser.NullValue) {
               supplierPath = Queues.MPMC.factory(capacity);
           } else {
               supplierPath = supplierValue.toString();
           }
   
           // Execute the read spec.
           final QueueFactory queueFactory = Queues.createQueueFactory(queueFactorySpec, supplierPath, capacity);
   ```
   
   Note that
   1. `supplierPath` is not nullable anymore
   2. `Objects.isNull()` is not meant to be called like that, but it rather aids as a lambda whenever that check is needed



##########
log4j-api/src/main/java/org/apache/logging/log4j/spi/RecyclerFactory.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.function.Consumer;
+import java.util.function.Supplier;
+
+/**
+ * Factory for {@link Recycler} strategies. Depending on workloads, different instance recycling strategies may be
+ * most performant. For example, traditional multithreaded workloads may benefit from using thread-local instance
+ * recycling while different models of concurrency or versions of the JVM may benefit from using an object pooling
+ * strategy instead.
+ *
+ * @since 3.0.0
+ */
+@FunctionalInterface
+public interface RecyclerFactory {
+
+    /**
+     * Creates a new recycler using the given supplier function for initial instances. These instances have
+     * no cleaner function and are assumed to always be reusable.
+     *
+     * @param supplier function to provide new instances of a recyclable object
+     * @param <V> the recyclable type
+     * @return a new recycler for V-type instances
+     */
+    default <V> Recycler<V> create(final Supplier<V> supplier) {
+        return create(supplier, defaultCleaner());
+    }
+
+    /**
+     * Creates a new recycler using the given functions for providing fresh instances and for cleaning up
+     * existing instances for reuse. For example, a StringBuilder recycler would provide two functions:
+     * a supplier function for constructing a new StringBuilder with a preselected initial capacity and
+     * another function for trimming the StringBuilder to some preselected maximum capacity and setting
+     * its length back to 0 as if it were a fresh StringBuilder.
+     *
+     * @param supplier function to provide new instances of a recyclable object
+     * @param cleaner function to reset a recyclable object to a fresh state
+     * @param <V> the recyclable type
+     * @return a new recycler for V-type instances
+     */
+    default <V> Recycler<V> create(Supplier<V> supplier, Consumer<V> cleaner) {
+        return create(supplier, cleaner, defaultCleaner());
+    }
+
+    /**
+     * Creates a new recycler using the given functions for providing fresh instances and for cleaning recycled
+     * instances lazily or eagerly. The lazy cleaner function is invoked on recycled instances before being
+     * returned by {@link Recycler#acquire()}. The eager cleaner function is invoked on recycled instances
+     * during {@link Recycler#release(Object)}.
+     *
+     * @param supplier function to provide new instances of a recyclable object
+     * @param lazyCleaner function to invoke to clean a recycled object before being acquired
+     * @param eagerCleaner function to invoke to clean a recycled object after being released
+     * @param <V> the recyclable type
+     * @return a new recycler for V-type instances
+     */
+    <V> Recycler<V> create(Supplier<V> supplier, Consumer<V> lazyCleaner, Consumer<V> eagerCleaner);
+
+    /**
+     * Creates a default cleaner function that does nothing.
+     */
+    static <V> Consumer<V> defaultCleaner() {

Review Comment:
   `defaultCleaner()` -> `getDefaultCleaner()`



##########
log4j-api/src/main/java/org/apache/logging/log4j/spi/ThreadLocalRecyclerFactory.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.spi;
+
+import java.util.Queue;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import org.apache.logging.log4j.util.Queues;
+
+/**
+ * Recycling strategy that caches instances in a ThreadLocal value to allow threads to reuse objects. This strategy
+ * may not be appropriate in workloads where units of work are independent of operating system threads such as
+ * reactive streams, coroutines, or virtual threads.
+ */
+public class ThreadLocalRecyclerFactory implements RecyclerFactory {
+
+    private static final ThreadLocalRecyclerFactory INSTANCE =
+            new ThreadLocalRecyclerFactory();
+
+    private ThreadLocalRecyclerFactory() {}
+
+    public static ThreadLocalRecyclerFactory getInstance() {
+        return INSTANCE;
+    }
+
+    @Override
+    public <V> Recycler<V> create(
+            final Supplier<V> supplier,
+            final Consumer<V> lazyCleaner,
+            final Consumer<V> eagerCleaner) {
+        return new ThreadLocalRecycler<>(supplier, lazyCleaner, eagerCleaner);
+    }
+
+    // Visible for testing
+    static class ThreadLocalRecycler<V> implements Recycler<V> {
+
+        private final Supplier<V> supplier;
+
+        private final Consumer<V> lazyCleaner;
+
+        private final Consumer<V> eagerCleaner;
+
+        private final ThreadLocal<Queue<V>> holder;
+
+        private ThreadLocalRecycler(
+                final Supplier<V> supplier,
+                final Consumer<V> lazyCleaner,
+                final Consumer<V> eagerCleaner) {
+            this.supplier = supplier;
+            this.lazyCleaner = lazyCleaner;
+            this.eagerCleaner = eagerCleaner;
+            // allow for some reasonable level of recursive calls before we stop caring to reuse things
+            this.holder = ThreadLocal.withInitial(() -> Queues.SPSC.create(8));

Review Comment:
   What does happen when the recursion limit (i.e., 8) is exceeded?



##########
log4j-api/src/main/java/org/apache/logging/log4j/util/Queues.java:
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.util;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Method;
+import java.util.Queue;
+import java.util.concurrent.ArrayBlockingQueue;
+
+import org.jctools.queues.MpmcArrayQueue;
+import org.jctools.queues.MpscArrayQueue;
+import org.jctools.queues.SpmcArrayQueue;
+import org.jctools.queues.SpscArrayQueue;
+
+/**
+ * Provides {@link QueueFactory} and {@link Queue} instances for different use cases. When the
+ * <a href="https://jctools.github.io/JCTools/">JCTools</a> library is included at runtime, then
+ * the specialized lock free or wait free queues are used from there. Otherwise, {@link ArrayBlockingQueue}
+ * is provided as a fallback for thread-safety. Custom implementations of {@link QueueFactory} may also be
+ * created via {@link #createQueueFactory(String, String, int)}.
+ */
+@InternalApi
+public enum Queues {
+    /**
+     * Provides a bounded queue for single-producer/single-consumer usage. Only one thread may offer objects
+     * while only one thread may poll for them.
+     */
+    SPSC(Lazy.lazy(JCToolsQueueFactory.SPSC::load)),
+    /**
+     * Provides a bounded queue for multi-producer/single-consumer usage. Any thread may offer objects while only
+     * one thread may poll for them.
+     */
+    MPSC(Lazy.lazy(JCToolsQueueFactory.MPSC::load)),
+    /**
+     * Provides a bounded queue for single-producer/multi-consumer usage. Only one thread may offer objects but
+     * any thread may poll for them.
+     */
+    SPMC(Lazy.lazy(JCToolsQueueFactory.SPMC::load)),
+    /**
+     * Provides a bounded queue for multi-producer/multi-consumer usage. Any thread may offer objects and any thread
+     * may poll for them.
+     */
+    MPMC(Lazy.lazy(JCToolsQueueFactory.MPMC::load));
+
+    private final Lazy<BoundedQueueFactory> queueFactory;
+
+    Queues(final Lazy<BoundedQueueFactory> queueFactory) {
+        this.queueFactory = queueFactory;
+    }
+
+    public QueueFactory factory(final int capacity) {
+        return new ProxyQueueFactory(queueFactory.get(), capacity);
+    }
+
+    public <E> Queue<E> create(final int capacity) {
+        return queueFactory.get().create(capacity);
+    }
+
+    public static QueueFactory createQueueFactory(final String queueFactorySpec,

Review Comment:
   `queueFactorySpec` is not needed, you can remove that argument. It needs to be cleaned up from this class and its other usages.



##########
log4j-api/src/main/java/org/apache/logging/log4j/util/Queues.java:
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.util;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Method;
+import java.util.Queue;
+import java.util.concurrent.ArrayBlockingQueue;
+
+import org.jctools.queues.MpmcArrayQueue;
+import org.jctools.queues.MpscArrayQueue;
+import org.jctools.queues.SpmcArrayQueue;
+import org.jctools.queues.SpscArrayQueue;
+
+/**
+ * Provides {@link QueueFactory} and {@link Queue} instances for different use cases. When the
+ * <a href="https://jctools.github.io/JCTools/">JCTools</a> library is included at runtime, then
+ * the specialized lock free or wait free queues are used from there. Otherwise, {@link ArrayBlockingQueue}
+ * is provided as a fallback for thread-safety. Custom implementations of {@link QueueFactory} may also be
+ * created via {@link #createQueueFactory(String, String, int)}.
+ */
+@InternalApi
+public enum Queues {
+    /**
+     * Provides a bounded queue for single-producer/single-consumer usage. Only one thread may offer objects
+     * while only one thread may poll for them.
+     */
+    SPSC(Lazy.lazy(JCToolsQueueFactory.SPSC::load)),
+    /**
+     * Provides a bounded queue for multi-producer/single-consumer usage. Any thread may offer objects while only
+     * one thread may poll for them.
+     */
+    MPSC(Lazy.lazy(JCToolsQueueFactory.MPSC::load)),
+    /**
+     * Provides a bounded queue for single-producer/multi-consumer usage. Only one thread may offer objects but
+     * any thread may poll for them.
+     */
+    SPMC(Lazy.lazy(JCToolsQueueFactory.SPMC::load)),
+    /**
+     * Provides a bounded queue for multi-producer/multi-consumer usage. Any thread may offer objects and any thread
+     * may poll for them.
+     */
+    MPMC(Lazy.lazy(JCToolsQueueFactory.MPMC::load));
+
+    private final Lazy<BoundedQueueFactory> queueFactory;
+
+    Queues(final Lazy<BoundedQueueFactory> queueFactory) {
+        this.queueFactory = queueFactory;
+    }
+
+    public QueueFactory factory(final int capacity) {
+        return new ProxyQueueFactory(queueFactory.get(), capacity);
+    }
+
+    public <E> Queue<E> create(final int capacity) {
+        return queueFactory.get().create(capacity);
+    }
+
+    public static QueueFactory createQueueFactory(final String queueFactorySpec,
+                                                  final String supplierPath,
+                                                  final int capacity) {
+        final int supplierPathSplitterIndex = supplierPath.lastIndexOf('.');
+        if (supplierPathSplitterIndex < 0) {
+            throw new IllegalArgumentException(
+                    "invalid supplier in queue factory: " +
+                            queueFactorySpec);
+        }
+        final String supplierClassName = supplierPath.substring(0, supplierPathSplitterIndex);
+        final String supplierMethodName = supplierPath.substring(supplierPathSplitterIndex + 1);
+        try {
+            final Class<?> supplierClass = LoaderUtil.loadClass(supplierClassName);
+            final BoundedQueueFactory queueFactory;
+            if ("new".equals(supplierMethodName)) {
+                final Constructor<?> supplierCtor =
+                        supplierClass.getDeclaredConstructor(int.class);
+                queueFactory = new ConstructorProvidedQueueFactory(
+                        queueFactorySpec, supplierCtor);
+            } else {
+                final Method supplierMethod =
+                        supplierClass.getMethod(supplierMethodName, int.class);
+                queueFactory = new StaticMethodProvidedQueueFactory(
+                        queueFactorySpec, supplierMethod);
+            }
+            return new ProxyQueueFactory(queueFactory, capacity);
+        } catch (final ReflectiveOperationException | LinkageError | SecurityException error) {
+            throw new RuntimeException(
+                    "failed executing queue factory: " +
+                            queueFactorySpec, error);
+        }
+    }
+
+    static class ProxyQueueFactory implements QueueFactory {

Review Comment:
   I see many package-local classes (for instance, this one) and ctors which could have been private. Any particular reasons for this visibility preference?



##########
log4j-core/src/main/java/org/apache/logging/log4j/core/layout/GelfLayout.java:
##########
@@ -468,6 +486,11 @@ private GelfLayout(final Configuration config, final String host, final KeyValue
         this.mdcWriter = new FieldWriter(mdcChecker, mdcPrefix);
         this.mapWriter = new FieldWriter(mapChecker, mapPrefix);
         this.layout = patternLayout;
+        stacktraceRecycler = recyclerFactory.create(
+                () -> new StringBuilderWriter(2048),
+                writer -> writer.getBuilder().setLength(0),
+                writer -> StringBuilders.trimToMaxSize(writer.getBuilder(), 2048)
+        );

Review Comment:
   I would rather replace this ad-hoc 2048 value with `AbstractStringLayout.MAX_STRING_BUILDER_SIZE`. (There is one more usage below as well.)



##########
log4j-api-test/src/test/java/org/apache/logging/log4j/spi/ThreadLocalRecyclerFactoryTest.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.concurrent.atomic.AtomicInteger;
+
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+class ThreadLocalRecyclerFactoryTest {

Review Comment:
   Would you mind adding tests stressing the capacity (i.e., 8) too, please?
   1. A test stressing the happy path (I'd personally make `nestedAcquiresDoNotInterfere()` a parametrized test accepting `int acquisitionCount` from 1 to 8)
   2. A test stressing the excessive acquisition (i.e., trying to acquire more than 8)



##########
log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLogger.java:
##########
@@ -492,24 +493,24 @@ public void actualAsyncLog(final RingBufferLogEvent event) {
         privateConfigLoggerConfig.getReliabilityStrategy().log(this, event);
     }
 
-    @SuppressWarnings("ForLoopReplaceableByForEach") // Avoid iterator allocation
+    @PerformanceSensitive("allocation")
     private void onPropertiesPresent(final RingBufferLogEvent event, final List<Property> properties) {
         final StringMap contextData = getContextData(event);
-        for (int i = 0, size = properties.size(); i < size; i++) {
-            final Property prop = properties.get(i);
+        // List::forEach is garbage-free when using an ArrayList or Arrays.asList

Review Comment:
   Is it? You need to pass a lambda, which needs to be allocated. I think this needs to be reverted along with the switch from `List.of()` to `Arrays.asList()` change in `LoggerConfig`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@logging.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org