You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@logging.apache.org by vy...@apache.org on 2020/11/06 12:44:54 UTC

[logging-log4j2] 06/07: Catch all type of exceptions in AppenderControl.

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

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

commit 17e074c69689c407b27f7418817dd6ca609dfa0b
Author: Volkan Yazici <vo...@gmail.com>
AuthorDate: Thu Nov 5 15:51:16 2020 +0100

    Catch all type of exceptions in AppenderControl.
---
 .../org/apache/logging/log4j/util/SneakyThrow.java | 13 +++
 .../logging/log4j/core/config/AppenderControl.java |  8 +-
 .../log4j/core/appender/FailoverAppenderTest.java  |  2 +-
 .../async/AsyncAppenderExceptionHandlingTest.java  | 97 ++++++++++++++++++++++
 .../core/config/AppenderControlArraySetTest.java   |  2 +-
 .../log4j/test/appender/FailOnceAppender.java      | 91 +++++++++++++++-----
 .../AsyncAppenderExceptionHandlingTest.xml         | 32 +++++++
 7 files changed, 219 insertions(+), 26 deletions(-)

diff --git a/log4j-api/src/test/java/org/apache/logging/log4j/util/SneakyThrow.java b/log4j-api/src/test/java/org/apache/logging/log4j/util/SneakyThrow.java
new file mode 100644
index 0000000..af26e6e
--- /dev/null
+++ b/log4j-api/src/test/java/org/apache/logging/log4j/util/SneakyThrow.java
@@ -0,0 +1,13 @@
+package org.apache.logging.log4j.util;
+
+public enum SneakyThrow {;
+
+    /**
+     * Throws any exception (including checked ones!) without defining it in the method signature.
+     */
+    @SuppressWarnings("unchecked")
+    public static <E extends Throwable> void sneakyThrow(final Throwable throwable) throws E {
+        throw (E) throwable;
+    }
+
+}
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AppenderControl.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AppenderControl.java
index ee2f28f..1e71afa 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AppenderControl.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AppenderControl.java
@@ -154,10 +154,10 @@ public class AppenderControl extends AbstractFilterable {
     private void tryCallAppender(final LogEvent event) {
         try {
             appender.append(event);
-        } catch (final RuntimeException ex) {
-            handleAppenderError(event, ex);
-        } catch (final Exception ex) {
-            handleAppenderError(event, new AppenderLoggingException(ex));
+        } catch (final RuntimeException error) {
+            handleAppenderError(event, error);
+        } catch (final Throwable error) {
+            handleAppenderError(event, new AppenderLoggingException(error));
         }
     }
 
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/FailoverAppenderTest.java b/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/FailoverAppenderTest.java
index 54f8dbf..56c886d 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/FailoverAppenderTest.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/FailoverAppenderTest.java
@@ -77,7 +77,7 @@ public class FailoverAppenderTest {
         onceLogger.error("Second log message");
         events = app.getEvents();
         assertEquals(events.size(), 0, "Did not recover");
-        events = foApp.getEvents();
+        events = foApp.drainEvents();
         assertEquals(events.size(), 2, "Incorrect number of events in primary appender");
     }
 }
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncAppenderExceptionHandlingTest.java b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncAppenderExceptionHandlingTest.java
new file mode 100644
index 0000000..8318c81
--- /dev/null
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncAppenderExceptionHandlingTest.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache license, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the license for the specific language governing permissions and
+ * limitations under the license.
+ */
+package org.apache.logging.log4j.core.async;
+
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.Logger;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.appender.AsyncAppender;
+import org.apache.logging.log4j.core.config.AppenderControl;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.config.Configurator;
+import org.apache.logging.log4j.message.Message;
+import org.apache.logging.log4j.test.appender.FailOnceAppender;
+import org.apache.logging.log4j.util.Strings;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Verifies {@link AsyncAppender} works after certain type of {@link Appender}
+ * failures.
+ * <p>
+ * {@link AsyncAppender} thread is known to get killed due to
+ * {@link AppenderControl} leaking exceptions in the past. This class is more
+ * of an end-to-end test to verify that {@link AppenderControl} catches all kind
+ * of {@link Throwable}s.
+ */
+class AsyncAppenderExceptionHandlingTest {
+
+    @ParameterizedTest
+    @ValueSource(strings = {
+            FailOnceAppender.ThrowableClassName.RUNTIME_EXCEPTION,
+            FailOnceAppender.ThrowableClassName.LOGGING_EXCEPTION,
+            FailOnceAppender.ThrowableClassName.EXCEPTION,
+            FailOnceAppender.ThrowableClassName.ERROR,
+            FailOnceAppender.ThrowableClassName.THROWABLE
+    })
+    void AsyncAppender_should_not_stop_on_appender_failures(String throwableClassName) {
+
+        // Create the logger.
+        final String throwableClassNamePropertyName = "throwableClassName";
+        System.setProperty(throwableClassNamePropertyName, throwableClassName);
+        try (final LoggerContext loggerContext =
+                     Configurator.initialize("Test", "AsyncAppenderExceptionHandlingTest.xml")) {
+            final Logger logger = loggerContext.getRootLogger();
+
+            // Log the 1st message, which should fail due to the FailOnceAppender.
+            logger.info("message #1");
+
+            // Log the 2nd message, which should succeed.
+            final String lastLogMessage = "message #2";
+            logger.info(lastLogMessage);
+
+            // Stop the AsyncAppender to drain the queued events.
+            Configuration configuration = loggerContext.getConfiguration();
+            AsyncAppender asyncAppender = configuration.getAppender("Async");
+            Assertions.assertNotNull(asyncAppender, "couldn't obtain the FailOnceAppender");
+            asyncAppender.stop();
+
+            // Verify the logged message.
+            final FailOnceAppender failOnceAppender = configuration.getAppender("FailOnce");
+            Assertions.assertNotNull(failOnceAppender, "couldn't obtain the FailOnceAppender");
+            Assertions.assertTrue(failOnceAppender.isFailed(), "FailOnceAppender hasn't failed yet");
+            final List<String> accumulatedMessages = failOnceAppender
+                    .drainEvents()
+                    .stream()
+                    .map(LogEvent::getMessage)
+                    .map(Message::getFormattedMessage)
+                    .collect(Collectors.toList());
+            Assertions.assertEquals(Collections.singletonList(lastLogMessage), accumulatedMessages);
+
+        } finally {
+            System.setProperty(throwableClassNamePropertyName, Strings.EMPTY);
+        }
+
+    }
+
+}
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/core/config/AppenderControlArraySetTest.java b/log4j-core/src/test/java/org/apache/logging/log4j/core/config/AppenderControlArraySetTest.java
index e8469af..d102cfa 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/core/config/AppenderControlArraySetTest.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/core/config/AppenderControlArraySetTest.java
@@ -38,7 +38,7 @@ public class AppenderControlArraySetTest {
     }
 
     private AppenderControl createControl(final String name) {
-        final Appender appender = FailOnceAppender.createAppender(name);
+        final Appender appender = FailOnceAppender.createAppender(name, null);
         return new AppenderControl(appender, Level.INFO, null);
     }
 
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/test/appender/FailOnceAppender.java b/log4j-core/src/test/java/org/apache/logging/log4j/test/appender/FailOnceAppender.java
index 2a18bc3..7fe0296 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/test/appender/FailOnceAppender.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/test/appender/FailOnceAppender.java
@@ -16,51 +16,102 @@
  */
 package org.apache.logging.log4j.test.appender;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.logging.log4j.LoggingException;
 import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.LogEvent;
 import org.apache.logging.log4j.core.appender.AbstractAppender;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.config.plugins.PluginAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginFactory;
 import org.apache.logging.log4j.plugins.Plugin;
-import org.apache.logging.log4j.plugins.PluginAttribute;
-import org.apache.logging.log4j.plugins.PluginFactory;
 import org.apache.logging.log4j.plugins.validation.constraints.Required;
+import org.apache.logging.log4j.util.SneakyThrow;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Supplier;
 
 /**
- *
+ * An {@link Appender} that fails on the first use and works for the rest.
  */
 @Plugin(name="FailOnce", category ="Core", elementType=Appender.ELEMENT_TYPE, printObject=true)
 public class FailOnceAppender extends AbstractAppender {
 
-    boolean fail = true;
+    private final Supplier<Throwable> throwableSupplier;
 
-    private final List<LogEvent> events = new ArrayList<>();
+    private boolean failed = false;
 
-    private FailOnceAppender(final String name) {
-        super(name, null, null, false, null);
+    private List<LogEvent> events = new ArrayList<>();
+
+    private FailOnceAppender(final String name, final Supplier<Throwable> throwableSupplier) {
+        super(name, null, null, false, Property.EMPTY_ARRAY);
+        this.throwableSupplier = throwableSupplier;
     }
 
     @Override
-    public void append(final LogEvent event) {
-        if (fail) {
-            fail = false;
-            throw new LoggingException("Always fail");
+    public synchronized void append(final LogEvent event) {
+        if (!failed) {
+            failed = true;
+            Throwable throwable = throwableSupplier.get();
+            SneakyThrow.sneakyThrow(throwable);
         }
         events.add(event);
     }
 
-    public List<LogEvent> getEvents() {
-        final List<LogEvent> list = new ArrayList<>(events);
-        events.clear();
-        return list;
+    public synchronized boolean isFailed() {
+        return failed;
+    }
+
+    /**
+     * Returns the list of accumulated events and resets the internal buffer.
+     */
+    public synchronized List<LogEvent> drainEvents() {
+        final List<LogEvent> oldEvents = events;
+        this.events = new ArrayList<>();
+        return oldEvents;
     }
 
     @PluginFactory
     public static FailOnceAppender createAppender(
-        @PluginAttribute @Required(message = "A name for the Appender must be specified") final String name) {
-        return new FailOnceAppender(name);
+        @PluginAttribute("name") @Required(message = "A name for the Appender must be specified") final String name,
+        @PluginAttribute("throwableClassName") final String throwableClassName) {
+        final Supplier<Throwable> throwableSupplier = createThrowableSupplier(name, throwableClassName);
+        return new FailOnceAppender(name, throwableSupplier);
+    }
+
+    private static Supplier<Throwable> createThrowableSupplier(
+            final String name,
+            final String throwableClassName) {
+
+        // Fallback to LoggingException if none is given.
+        final String message = String.format("failing on purpose for appender '%s'", name);
+        if (throwableClassName == null || ThrowableClassName.LOGGING_EXCEPTION.equals(throwableClassName)) {
+            return () -> new LoggingException(message);
+        }
+
+        // Check against the expected exception classes.
+        switch (throwableClassName) {
+            case ThrowableClassName.RUNTIME_EXCEPTION: return () -> new RuntimeException(message);
+            case ThrowableClassName.EXCEPTION: return () -> new Exception(message);
+            case ThrowableClassName.ERROR: return () -> new Error(message);
+            case ThrowableClassName.THROWABLE: return () -> new Throwable(message);
+            default: throw new IllegalArgumentException("unknown throwable class name: " + throwableClassName);
+        }
+
+    }
+
+    public enum ThrowableClassName {;
+
+        public static final String RUNTIME_EXCEPTION = "RuntimeException";
+
+        public static final String LOGGING_EXCEPTION = "LoggingException";
+
+        public static final String EXCEPTION = "Exception";
+
+        public static final String ERROR = "Error";
+
+        public static final String THROWABLE = "Throwable";
+
     }
 
 }
diff --git a/log4j-core/src/test/resources/AsyncAppenderExceptionHandlingTest.xml b/log4j-core/src/test/resources/AsyncAppenderExceptionHandlingTest.xml
new file mode 100644
index 0000000..84764fe
--- /dev/null
+++ b/log4j-core/src/test/resources/AsyncAppenderExceptionHandlingTest.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+ 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.
+
+-->
+<Configuration status="OFF" name="AsyncAppenderExceptionHandlingTest">
+  <Appenders>
+    <FailOnce name="FailOnce" throwableClassName="${sys:throwableClassName}"/>
+    <Async name="Async">
+      <AppenderRef ref="FailOnce" />
+    </Async>
+  </Appenders>
+  <Loggers>
+    <Root level="info">
+      <AppenderRef ref="Async"/>
+    </Root>
+  </Loggers>
+</Configuration>