You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2018/11/16 22:43:22 UTC

[geode] 06/11: GEODE-2644: Make AlertAppender optional and support log4j2.xml

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

klund pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 5ae86bb22514e8afe567c8a20dd8a320ec34d1dd
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Fri Nov 2 16:47:42 2018 -0700

    GEODE-2644: Make AlertAppender optional and support log4j2.xml
    
    AlertAppender is now configured in log4j2.xml and it supports sessions
    that correspond with Cache lifecycle. This allows Geode to pause and
    resume AlertAppender without resorting to dynamically adding and
    removing appenders.
    
    List of changes:
    * Change AlertAppender to be pausable and session-oriented
    * Make AlertAppender support configuration from log4j2.xml
    * Log4j2 Core dependency is now optional
    * Internal Alerting interfaces allow Alerting service to be pluggable
    * Reduce coupling between Alerting and the rest of Geode
    * Greatly increase test coverage for Alerting
---
 .../org/apache/geode/annotations/TestingOnly.java  |   2 +-
 .../alerting/AlertingServiceDistributedTest.java   | 399 +++++++++++++++++
 ...rtingServiceWithoutListenerDistributedTest.java | 217 ++++++++++
 .../gms/messenger/JGroupsMessengerJUnitTest.java   |   9 +-
 .../AlertListenerRegistrationIntegrationTest.java  | 183 ++++++++
 .../AlertingServiceWithClusterIntegrationTest.java | 274 ++++++++++++
 .../AlertingServiceWithLonerIntegrationTest.java   |  89 ++++
 .../log4j/AlertAppenderIntegrationTest.java        | 261 ++++++++++++
 .../log4j/AlertAppenderIntegrationTest_log4j2.xml  |  36 ++
 .../internal/ClusterDistributionManager.java       |  21 +-
 .../distributed/internal/DistributionManager.java  |   8 +-
 .../internal/LonerDistributionManager.java         |   7 +
 .../distributed/internal/direct/DirectChannel.java |   5 +-
 .../membership/gms/messenger/JGroupsMessenger.java |   5 +-
 .../membership/gms/mgr/GMSMembershipManager.java   |   2 +-
 .../remote/AdminConsoleDisconnectMessage.java      |  31 +-
 .../internal/admin/remote/AdminConsoleMessage.java |  21 +-
 .../admin/remote/AlertLevelChangeMessage.java      |  12 +-
 .../admin/remote/AlertListenerMessage.java         |  44 ++
 .../apache/geode/internal/alerting/AlertLevel.java |  53 +++
 .../alerting/AlertListenerMessageFactory.java      |  50 +++
 .../geode/internal/alerting/AlertMessaging.java    | 100 +++++
 .../geode/internal/alerting/AlertingAction.java    |  28 +-
 .../geode/internal/alerting/AlertingProvider.java  |  21 +-
 .../alerting/AlertingProviderRegistry.java         |  80 ++++
 .../geode/internal/alerting/AlertingService.java   |  57 +++
 .../geode/internal/alerting/AlertingSession.java   | 107 +++++
 .../internal/alerting/AlertingSessionListener.java |  17 +-
 .../alerting/AlertingSessionListeners.java         |  78 ++++
 .../internal/alerting/NullAlertingProvider.java    |  40 +-
 .../internal/alerting/NullAlertingService.java     |  35 +-
 .../internal/logging/log4j/AlertAppender.java      | 474 +++++++++++----------
 .../geode/internal/logging/log4j/AlertLevel.java   |  61 ---
 .../logging/log4j/AlertLevelConverter.java         |  88 ++++
 .../internal/logging/log4j/AlertListener.java      |  66 +++
 .../org/apache/geode/internal/tcp/Connection.java  |   4 +-
 .../apache/geode/internal/tcp/ConnectionTable.java |  22 +-
 .../org/apache/geode/internal/tcp/TCPConduit.java  |   4 +-
 .../management/internal/ManagerStartupMessage.java |   4 +-
 .../geode/internal/alerting/AlertLevelTest.java    | 113 +++++
 .../alerting/AlertListenerMessageFactoryTest.java  |  68 +++
 .../internal/alerting/AlertMessagingTest.java      | 108 +++++
 .../internal/alerting/AlertingActionTest.java      |  54 +++
 .../alerting/AlertingProviderRegistryTest.java     |  94 ++++
 .../internal/alerting/AlertingServiceTest.java     |  75 ++++
 .../alerting/NullAlertingProviderTest.java         |  48 +++
 .../internal/logging/log4j/AlertAppenderTest.java  | 314 +++++---------
 .../logging/log4j/AlertLevelConverterTest.java     | 144 +++++++
 .../internal/logging/log4j/AlertLevelTest.java     | 137 ------
 49 files changed, 3412 insertions(+), 758 deletions(-)

diff --git a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java b/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
index f2a21a1..da89816 100644
--- a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
+++ b/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
@@ -20,7 +20,7 @@ import java.lang.annotation.ElementType;
 import java.lang.annotation.Target;
 
 @Documented
-@Target({ElementType.CONSTRUCTOR, ElementType.METHOD})
+@Target({ElementType.TYPE, ElementType.CONSTRUCTOR, ElementType.METHOD})
 public @interface TestingOnly {
 
   /** Optional description */
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/alerting/AlertingServiceDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/alerting/AlertingServiceDistributedTest.java
new file mode 100644
index 0000000..7fe2058
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/alerting/AlertingServiceDistributedTest.java
@@ -0,0 +1,399 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static java.lang.management.ManagementFactory.getPlatformMBeanServer;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.apache.geode.internal.admin.remote.AlertListenerMessage.addListener;
+import static org.apache.geode.internal.admin.remote.AlertListenerMessage.removeListener;
+import static org.apache.geode.internal.alerting.AlertLevel.NONE;
+import static org.apache.geode.internal.alerting.AlertLevel.SEVERE;
+import static org.apache.geode.management.JMXNotificationType.SYSTEM_ALERT;
+import static org.apache.geode.management.internal.MBeanJMXAdapter.getDistributedSystemName;
+import static org.apache.geode.management.internal.MBeanJMXAdapter.mbeanServer;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.getTimeout;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getController;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.toArray;
+import static org.apache.geode.test.dunit.standalone.DUnitLauncher.getDistributedSystemProperties;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.isA;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+import java.io.Serializable;
+import java.util.Properties;
+
+import javax.management.InstanceNotFoundException;
+import javax.management.JMX;
+import javax.management.Notification;
+import javax.management.NotificationFilter;
+import javax.management.NotificationListener;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.admin.remote.AlertListenerMessage;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.management.DistributedSystemMXBean;
+import org.apache.geode.management.internal.AlertDetails;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.categories.AlertingTest;
+import org.apache.geode.test.junit.categories.ManagementTest;
+import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
+
+/**
+ * Distributed tests for {@link AlertingService} with {@link DistributedSystemMXBean} in the
+ * JMX Manager.
+ */
+@Category({AlertingTest.class, ManagementTest.class})
+public class AlertingServiceDistributedTest implements Serializable {
+
+  private static final long TIMEOUT = getTimeout().getValueInMS();
+  private static final NotificationFilter SYSTEM_ALERT_FILTER =
+      notification -> notification.getType().equals(SYSTEM_ALERT);
+
+  private static InternalCache cache;
+  private static Logger logger;
+
+  private static AlertListenerMessage.Listener messageListener;
+  private static DistributedSystemMXBean distributedSystemMXBean;
+  private static NotificationListener notificationListener;
+  private static AlertingService alertingService;
+
+  private DistributedMember managerMember;
+
+  private String alertMessage;
+  private String exceptionMessage;
+
+  private String managerName;
+  private String memberName;
+
+  private VM managerVM;
+  private VM memberVM;
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Rule
+  public SerializableTestName testName = new SerializableTestName();
+
+  @Before
+  public void setUp() throws Exception {
+    alertMessage = "Alerting in " + testName.getMethodName();
+    exceptionMessage = "Exception in " + testName.getMethodName();
+
+    managerName = "Manager in " + testName.getMethodName();
+    memberName = "Member in " + testName.getMethodName();
+
+    managerVM = getVM(0);
+    memberVM = getController();
+
+    managerMember = managerVM.invoke(() -> createManager());
+    memberVM.invoke(() -> createMember());
+
+    addIgnoredException(alertMessage);
+    addIgnoredException(exceptionMessage);
+  }
+
+  @After
+  public void tearDown() {
+    for (VM vm : toArray(managerVM, memberVM)) {
+      vm.invoke(() -> {
+        removeListener(messageListener);
+        cache.close();
+        cache = null;
+        logger = null;
+        messageListener = null;
+        distributedSystemMXBean = null;
+        notificationListener = null;
+        alertingService = null;
+      });
+    }
+  }
+
+  @Test
+  public void distributedSystemMXBeanExists() {
+    managerVM.invoke(() -> {
+      assertThat(distributedSystemMXBean).isNotNull();
+    });
+  }
+
+  @Test
+  public void distributedSystemMXBeanIsRegistered() {
+    managerVM.invoke(() -> {
+      assertThat(mbeanServer.isRegistered(getDistributedSystemName())).isTrue();
+    });
+  }
+
+  @Test
+  public void listenerReceivesAlertFromRemoteMember() {
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureNotification()).isNotNull();
+    });
+  }
+
+  @Test
+  public void listenerReceivesAlertFromLocalMember() {
+    managerVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureNotification().getMessage()).isEqualTo(alertMessage);
+    });
+  }
+
+  @Test
+  public void notificationMessageFromRemoteMemberIsAlertMessage() {
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureNotification().getMessage()).isEqualTo(alertMessage);
+    });
+  }
+
+  @Test
+  public void notificationMessageFromLocalMemberIsAlertMessage() {
+    managerVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureNotification().getMessage()).isEqualTo(alertMessage);
+    });
+  }
+
+  @Test
+  public void alertListenerMessageIsStillReceivedAfterRemoveListener() {
+    managerVM.invoke(() -> getPlatformMBeanServer()
+        .removeNotificationListener(getDistributedSystemName(), notificationListener));
+
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      verify(messageListener, timeout(TIMEOUT)).received(isA(AlertListenerMessage.class));
+    });
+  }
+
+  @Test
+  public void alertListenerMessageIsNotReceivedForLevelNone() {
+    changeAlertLevel(NONE);
+
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> verifyNoMoreInteractions(messageListener));
+  }
+
+  @Test
+  public void alertListenerMessageIsNotReceivedForLevelsLowerThanAlertLevel() {
+    memberVM.invoke(() -> {
+      logger.warn(alertMessage);
+      logger.error(alertMessage);
+    });
+
+    managerVM.invoke(() -> verifyNoMoreInteractions(messageListener));
+  }
+
+  @Test
+  public void alertDetailsIsCreatedByAlertMessage() {
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails()).isNotNull().isInstanceOf(AlertDetails.class);
+    });
+  }
+
+  @Test
+  public void alertDetailsAlertLevelMatchesLogLevel() {
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails().getAlertLevel()).isEqualTo(SEVERE.intLevel());
+    });
+  }
+
+  @Test
+  public void alertDetailsMessageMatchesAlertMessage() {
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails().getMsg()).isEqualTo(alertMessage);
+    });
+  }
+
+  @Test
+  public void alertDetailsSenderIsNullForLocalAlert() {
+    managerVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails().getSender()).isNull();
+    });
+  }
+
+  @Test
+  public void alertDetailsSourceContainsThreadName() {
+    String threadName = memberVM.invoke(() -> {
+      logger.fatal(alertMessage);
+      return Thread.currentThread().getName();
+    });
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails().getSource()).contains(threadName);
+    });
+  }
+
+  @Test
+  public void alertDetailsConnectionNameMatchesMemberName() {
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails().getConnectionName()).isEqualTo(memberName);
+    });
+  }
+
+  @Test
+  public void alertDetailsExceptionTextIsEmptyByDefault() {
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails().getExceptionText()).isEqualTo("");
+    });
+  }
+
+  @Test
+  public void alertDetailsExceptionTextMatchesExceptionMessage() {
+    memberVM.invoke(() -> logger.fatal(alertMessage, new Exception(exceptionMessage)));
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails().getExceptionText()).contains(exceptionMessage);
+    });
+  }
+
+  @Test
+  public void alertDetailsThreadNameMatchesLoggingThreadName() {
+    String threadName = memberVM.invoke(() -> {
+      logger.fatal(alertMessage);
+      return Thread.currentThread().getName();
+    });
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails().getThreadName()).isEqualTo(threadName);
+    });
+  }
+
+  @Test
+  public void alertDetailsThreadIdMatchesLoggingThreadId() {
+    long threadId = memberVM.invoke(() -> {
+      logger.fatal(alertMessage);
+      return Thread.currentThread().getId();
+    });
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails().getTid()).isEqualTo(threadId);
+    });
+  }
+
+  @Test
+  public void alertDetailsMessageTimeIsNotNull() {
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> {
+      assertThat(captureAlertDetails().getMsgTime()).isNotNull();
+    });
+  }
+
+  private DistributedMember createManager() throws InstanceNotFoundException {
+    messageListener = spy(AlertListenerMessage.Listener.class);
+    addListener(messageListener);
+
+    Properties config = getDistributedSystemProperties();
+    config.setProperty(NAME, managerName);
+    config.setProperty(JMX_MANAGER, "true");
+    config.setProperty(JMX_MANAGER_START, "true");
+    config.setProperty(JMX_MANAGER_PORT, "0");
+    config.setProperty(HTTP_SERVICE_PORT, "0");
+
+    cache = (InternalCache) new CacheFactory(config).create();
+    alertingService = cache.getInternalDistributedSystem().getAlertingService();
+    logger = LogService.getLogger();
+
+    distributedSystemMXBean = JMX.newMXBeanProxy(getPlatformMBeanServer(),
+        getDistributedSystemName(), DistributedSystemMXBean.class);
+
+    notificationListener = spy(NotificationListener.class);
+    getPlatformMBeanServer().addNotificationListener(getDistributedSystemName(),
+        notificationListener, SYSTEM_ALERT_FILTER, null);
+
+    return cache.getDistributedSystem().getDistributedMember();
+  }
+
+  private void createMember() {
+    Properties config = getDistributedSystemProperties();
+    config.setProperty(NAME, memberName);
+    config.setProperty(JMX_MANAGER, "false");
+
+    cache = (InternalCache) new CacheFactory(config).create();
+    alertingService = cache.getInternalDistributedSystem().getAlertingService();
+    logger = LogService.getLogger();
+
+    await().until(() -> alertingService.hasAlertListener(managerMember, SEVERE));
+  }
+
+  private void changeAlertLevel(AlertLevel alertLevel) {
+    managerVM.invoke(() -> {
+      distributedSystemMXBean.changeAlertLevel(alertLevel.name());
+    });
+
+    memberVM.invoke(() -> {
+      if (alertLevel == NONE) {
+        await().until(() -> !alertingService.hasAlertListener(managerMember, alertLevel));
+      } else {
+        await().until(() -> alertingService.hasAlertListener(managerMember, alertLevel));
+      }
+    });
+  }
+
+  private Notification captureNotification() {
+    ArgumentCaptor<Notification> notificationCaptor = ArgumentCaptor.forClass(Notification.class);
+    verify(notificationListener, timeout(TIMEOUT)).handleNotification(notificationCaptor.capture(),
+        isNull());
+    return notificationCaptor.getValue();
+  }
+
+  private AlertDetails captureAlertDetails() {
+    ArgumentCaptor<AlertDetails> alertDetailsCaptor = ArgumentCaptor.forClass(AlertDetails.class);
+    verify(messageListener, timeout(TIMEOUT)).created(alertDetailsCaptor.capture());
+    return alertDetailsCaptor.getValue();
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/alerting/AlertingServiceWithoutListenerDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/alerting/AlertingServiceWithoutListenerDistributedTest.java
new file mode 100644
index 0000000..c2bf009
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/alerting/AlertingServiceWithoutListenerDistributedTest.java
@@ -0,0 +1,217 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static java.lang.management.ManagementFactory.getPlatformMBeanServer;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.apache.geode.internal.admin.remote.AlertListenerMessage.addListener;
+import static org.apache.geode.internal.admin.remote.AlertListenerMessage.removeListener;
+import static org.apache.geode.internal.alerting.AlertLevel.NONE;
+import static org.apache.geode.internal.alerting.AlertLevel.SEVERE;
+import static org.apache.geode.management.internal.MBeanJMXAdapter.getDistributedSystemName;
+import static org.apache.geode.management.internal.MBeanJMXAdapter.mbeanServer;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getController;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.toArray;
+import static org.apache.geode.test.dunit.standalone.DUnitLauncher.getDistributedSystemProperties;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+import java.io.Serializable;
+import java.util.Properties;
+
+import javax.management.JMX;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.admin.remote.AlertListenerMessage;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.management.DistributedSystemMXBean;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.categories.AlertingTest;
+import org.apache.geode.test.junit.categories.ManagementTest;
+import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
+
+/**
+ * Distributed tests for {@link AlertingService} with {@link DistributedSystemMXBean} in the
+ * JMX Manager without any {@code NotificationListener}s.
+ */
+@Category({AlertingTest.class, ManagementTest.class})
+public class AlertingServiceWithoutListenerDistributedTest implements Serializable {
+
+  private static InternalCache cache;
+  private static Logger logger;
+
+  private static AlertListenerMessage.Listener messageListener;
+  private static DistributedSystemMXBean distributedSystemMXBean;
+  private static AlertingService alertingService;
+
+  private DistributedMember managerMember;
+
+  private String alertMessage;
+
+  private String managerName;
+  private String memberName;
+
+  private VM managerVM;
+  private VM memberVM;
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Rule
+  public SerializableTestName testName = new SerializableTestName();
+
+  @Before
+  public void setUp() throws Exception {
+    alertMessage = "Alerting in " + testName.getMethodName();
+
+    managerName = "Manager in " + testName.getMethodName();
+    memberName = "Member in " + testName.getMethodName();
+
+    managerVM = getVM(0);
+    memberVM = getController();
+
+    managerMember = managerVM.invoke(() -> createManager());
+    memberVM.invoke(() -> createMember());
+
+    addIgnoredException(alertMessage);
+  }
+
+  @After
+  public void tearDown() {
+    for (VM vm : toArray(managerVM, memberVM)) {
+      vm.invoke(() -> {
+        removeListener(messageListener);
+        cache.close();
+        cache = null;
+        logger = null;
+        messageListener = null;
+        distributedSystemMXBean = null;
+        alertingService = null;
+      });
+    }
+  }
+
+  @Test
+  public void distributedSystemMXBeanExists() {
+    managerVM.invoke(() -> {
+      assertThat(distributedSystemMXBean).isNotNull();
+    });
+  }
+
+  @Test
+  public void distributedSystemMXBeanIsRegistered() {
+    managerVM.invoke(() -> {
+      assertThat(mbeanServer.isRegistered(getDistributedSystemName())).isTrue();
+    });
+  }
+
+  @Test
+  public void alertListenerIsRegisteredForManager() {
+    memberVM.invoke(() -> {
+      assertThat(alertingService.hasAlertListener(managerMember, SEVERE)).isTrue();
+    });
+  }
+
+  @Test
+  public void alertMessageIsReceivedByManager() {
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> verifyNoMoreInteractions(messageListener));
+  }
+
+  @Test
+  public void alertListenerMessageIsNotReceivedByManagerForLevelNone() {
+    changeAlertLevel(NONE);
+
+    memberVM.invoke(() -> logger.fatal(alertMessage));
+
+    managerVM.invoke(() -> verifyNoMoreInteractions(messageListener));
+  }
+
+  @Test
+  public void alertListenerMessageIsNotReceivedByManagerForLevelsLowerThanAlertLevel() {
+    memberVM.invoke(() -> {
+      logger.warn(alertMessage);
+      logger.error(alertMessage);
+    });
+
+    managerVM.invoke(() -> verifyNoMoreInteractions(messageListener));
+  }
+
+  private DistributedMember createManager() {
+    messageListener = spy(AlertListenerMessage.Listener.class);
+    addListener(messageListener);
+
+    Properties config = getDistributedSystemProperties();
+    config.setProperty(NAME, managerName);
+    config.setProperty(JMX_MANAGER, "true");
+    config.setProperty(JMX_MANAGER_START, "true");
+    config.setProperty(JMX_MANAGER_PORT, "0");
+    config.setProperty(HTTP_SERVICE_PORT, "0");
+
+    cache = (InternalCache) new CacheFactory(config).create();
+    alertingService = cache.getInternalDistributedSystem().getAlertingService();
+    logger = LogService.getLogger();
+
+    distributedSystemMXBean = JMX.newMXBeanProxy(getPlatformMBeanServer(),
+        getDistributedSystemName(), DistributedSystemMXBean.class);
+
+    return cache.getDistributedSystem().getDistributedMember();
+  }
+
+  private void createMember() {
+    Properties config = getDistributedSystemProperties();
+    config.setProperty(NAME, memberName);
+    config.setProperty(JMX_MANAGER, "false");
+
+    cache = (InternalCache) new CacheFactory(config).create();
+    alertingService = cache.getInternalDistributedSystem().getAlertingService();
+    logger = LogService.getLogger();
+
+    await().until(() -> alertingService.hasAlertListener(managerMember, SEVERE));
+  }
+
+  private void changeAlertLevel(AlertLevel alertLevel) {
+    managerVM.invoke(() -> {
+      distributedSystemMXBean.changeAlertLevel(alertLevel.name());
+    });
+
+    memberVM.invoke(() -> {
+      if (alertLevel == NONE) {
+        await().until(() -> !alertingService.hasAlertListener(managerMember, alertLevel));
+      } else {
+        await().until(() -> alertingService.hasAlertListener(managerMember, alertLevel));
+      }
+    });
+  }
+}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
index 35df7d9..cfbaacc 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
@@ -101,8 +101,8 @@ import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.HeapDataOutputStream;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
+import org.apache.geode.internal.alerting.AlertingAction;
 import org.apache.geode.internal.cache.DistributedCacheOperation;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
 import org.apache.geode.test.junit.categories.MembershipTest;
 
 @Category({MembershipTest.class})
@@ -243,14 +243,11 @@ public class JGroupsMessengerJUnitTest {
     messenger.setMessageFlags(dmsg, jgmsg);
     assertFalse("expected no_fc to not be set in " + jgmsg.getFlags(),
         jgmsg.isFlagSet(Message.Flag.NO_FC));
-    AlertAppender.setIsAlerting(true);
-    try {
+    AlertingAction.execute(() -> {
       messenger.setMessageFlags(dmsg, jgmsg);
       assertTrue("expected no_fc to be set in " + jgmsg.getFlags(),
           jgmsg.isFlagSet(Message.Flag.NO_FC));
-    } finally {
-      AlertAppender.setIsAlerting(false);
-    }
+    });
   }
 
   @Test
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/alerting/AlertListenerRegistrationIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/alerting/AlertListenerRegistrationIntegrationTest.java
new file mode 100644
index 0000000..196d9ca
--- /dev/null
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/alerting/AlertListenerRegistrationIntegrationTest.java
@@ -0,0 +1,183 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.internal.alerting.AlertLevel.ERROR;
+import static org.apache.geode.internal.alerting.AlertLevel.NONE;
+import static org.apache.geode.internal.alerting.AlertLevel.SEVERE;
+import static org.apache.geode.internal.alerting.AlertLevel.WARNING;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.logging.log4j.AlertAppender;
+import org.apache.geode.test.junit.categories.AlertingTest;
+
+/**
+ * Integration tests for adding and removing {@code Alert} listeners.
+ */
+@Category(AlertingTest.class)
+public class AlertListenerRegistrationIntegrationTest {
+
+  private InternalDistributedSystem system;
+  private DistributedMember member;
+  private AlertingService alertingService;
+
+  @Before
+  public void setUp() {
+    Properties config = new Properties();
+    config.setProperty(LOCATORS, "");
+
+    system = (InternalDistributedSystem) DistributedSystem.connect(config);
+    member = system.getDistributedMember();
+    alertingService = system.getAlertingService();
+  }
+
+  @After
+  public void tearDown() {
+    system.disconnect();
+  }
+
+  @Test
+  public void alertingProviderIsAlertAppender() {
+    assertThat(alertingService.getAlertingProviderRegistry().getAlertingProvider())
+        .isInstanceOf(AlertAppender.class);
+  }
+
+  @Test
+  public void hasAlertListenerIsFalseByDefault() {
+    for (AlertLevel alertLevel : AlertLevel.values()) {
+      assertThat(alertingService.hasAlertListener(member, alertLevel)).isFalse();
+    }
+  }
+
+  @Test
+  public void hasAlertListenerIsTrueAfterAdding() {
+    alertingService.addAlertListener(member, WARNING);
+
+    assertThat(alertingService.hasAlertListener(member, WARNING)).isTrue();
+  }
+
+  @Test
+  public void hasAlertListenerIsTrueOnlyForLevelWarning() {
+    alertingService.addAlertListener(member, WARNING);
+
+    for (AlertLevel alertLevel : AlertLevel.values()) {
+      if (alertLevel != WARNING) {
+        assertThat(alertingService.hasAlertListener(member, alertLevel)).isFalse();
+      }
+    }
+  }
+
+  @Test
+  public void hasAlertListenerIsTrueOnlyForLevelError() {
+    alertingService.addAlertListener(member, ERROR);
+
+    for (AlertLevel alertLevel : AlertLevel.values()) {
+      if (alertLevel != ERROR) {
+        assertThat(alertingService.hasAlertListener(member, alertLevel)).isFalse();
+      }
+    }
+  }
+
+  @Test
+  public void hasAlertListenerIsTrueOnlyForLevelSevere() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    for (AlertLevel alertLevel : AlertLevel.values()) {
+      if (alertLevel != SEVERE) {
+        assertThat(alertingService.hasAlertListener(member, alertLevel)).isFalse();
+      }
+    }
+  }
+
+  @Test
+  public void addAlertListenerDoesNothingForLevelNone() {
+    alertingService.addAlertListener(member, NONE);
+
+    for (AlertLevel alertLevel : AlertLevel.values()) {
+      assertThat(alertingService.hasAlertListener(member, alertLevel)).isFalse();
+    }
+  }
+
+  @Test
+  public void removeAlertListenerReturnsFalseByDefault() {
+    assertThat(alertingService.removeAlertListener(member)).isFalse();
+  }
+
+  @Test
+  public void removeAlertListenerReturnsFalseAfterAddingForLevelNone() {
+    alertingService.addAlertListener(member, NONE);
+
+    assertThat(alertingService.removeAlertListener(member)).isFalse();
+  }
+
+  @Test
+  public void removeAlertListenerReturnsTrueAfterAdding() {
+    alertingService.addAlertListener(member, WARNING);
+
+    assertThat(alertingService.removeAlertListener(member)).isTrue();
+  }
+
+  @Test
+  public void hasAlertListenerIsFalseAfterRemoving() {
+    alertingService.addAlertListener(member, WARNING);
+    alertingService.removeAlertListener(member);
+
+    assertThat(alertingService.hasAlertListener(member, WARNING)).isFalse();
+  }
+
+  @Test
+  public void systemHasAlertListenerForMemberIsFalseByDefault() {
+    assertThat(system.hasAlertListenerFor(member)).isFalse();
+    for (AlertLevel alertLevel : AlertLevel.values()) {
+      assertThat(system.hasAlertListenerFor(member, alertLevel.intLevel())).isFalse();
+    }
+  }
+
+  @Test
+  public void systemHasAlertListenerForAlertLevelIsFalseByDefault() {
+    for (AlertLevel alertLevel : AlertLevel.values()) {
+      assertThat(system.hasAlertListenerFor(member, alertLevel.intLevel())).isFalse();
+    }
+  }
+
+  @Test
+  public void systemHasAlertListenerIsTrueAfterAdding() {
+    alertingService.addAlertListener(member, WARNING);
+
+    assertThat(system.hasAlertListenerFor(member, WARNING.intLevel())).isTrue();
+  }
+
+  @Test
+  public void systemHasAlertListenerIsFalseForOtherLevels() {
+    alertingService.addAlertListener(member, WARNING);
+
+    for (AlertLevel alertLevel : AlertLevel.values()) {
+      if (alertLevel != WARNING) {
+        assertThat(system.hasAlertListenerFor(member, alertLevel.intLevel())).isFalse();
+      }
+    }
+  }
+}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/alerting/AlertingServiceWithClusterIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/alerting/AlertingServiceWithClusterIntegrationTest.java
new file mode 100644
index 0000000..58b2b6a
--- /dev/null
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/alerting/AlertingServiceWithClusterIntegrationTest.java
@@ -0,0 +1,274 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.apache.geode.distributed.ConfigurationProperties.START_LOCATOR;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+import static org.apache.geode.internal.admin.remote.AlertListenerMessage.addListener;
+import static org.apache.geode.internal.admin.remote.AlertListenerMessage.removeListener;
+import static org.apache.geode.internal.alerting.AlertLevel.ERROR;
+import static org.apache.geode.internal.alerting.AlertLevel.NONE;
+import static org.apache.geode.internal.alerting.AlertLevel.SEVERE;
+import static org.apache.geode.internal.alerting.AlertLevel.WARNING;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.getTimeout;
+import static org.apache.geode.test.dunit.NetworkUtils.getServerHostName;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.isA;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+import java.util.Properties;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.mockito.ArgumentCaptor;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.admin.remote.AlertListenerMessage;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.management.internal.AlertDetails;
+import org.apache.geode.test.junit.categories.AlertingTest;
+
+/**
+ * Integration tests for {@link AlertingService} in a cluster member.
+ */
+@Category(AlertingTest.class)
+public class AlertingServiceWithClusterIntegrationTest {
+
+  private static final long TIMEOUT = getTimeout().getValueInMS();
+
+  private InternalDistributedSystem system;
+  private DistributedMember member;
+  private AlertListenerMessage.Listener messageListener;
+  private Logger logger;
+  private String connectionName;
+  private String alertMessage;
+  private String exceptionMessage;
+  private String threadName;
+  private long threadId;
+
+  private AlertingService alertingService;
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void setUp() {
+    alertMessage = "Alerting in " + testName.getMethodName();
+    exceptionMessage = "Exception in " + testName.getMethodName();
+    connectionName = "Member in " + testName.getMethodName();
+    threadName = Thread.currentThread().getName();
+    threadId = Thread.currentThread().getId();
+
+    messageListener = spy(AlertListenerMessage.Listener.class);
+    addListener(messageListener);
+
+    String startLocator = getServerHostName() + "[" + getRandomAvailableTCPPort() + "]";
+
+    Properties config = new Properties();
+    config.setProperty(START_LOCATOR, startLocator);
+    config.setProperty(NAME, connectionName);
+
+    system = (InternalDistributedSystem) DistributedSystem.connect(config);
+    member = system.getDistributedMember();
+    logger = LogService.getLogger();
+
+    alertingService = system.getAlertingService();
+  }
+
+  @After
+  public void tearDown() {
+    removeListener(messageListener);
+    system.disconnect();
+  }
+
+  @Test
+  public void alertMessageIsNotReceivedWithoutListener() {
+    logger.fatal(alertMessage);
+
+    verifyNoMoreInteractions(messageListener);
+  }
+
+  @Test
+  public void alertMessageIsReceivedForListenerLevelWarning() {
+    alertingService.addAlertListener(member, WARNING);
+
+    logger.warn(alertMessage);
+
+    verify(messageListener, timeout(TIMEOUT)).received(isA(AlertListenerMessage.class));
+  }
+
+  @Test
+  public void alertMessageIsReceivedForListenerLevelError() {
+    alertingService.addAlertListener(member, ERROR);
+
+    logger.error(alertMessage);
+
+    verify(messageListener, timeout(TIMEOUT)).received(isA(AlertListenerMessage.class));
+  }
+
+  @Test
+  public void alertMessageIsReceivedForListenerLevelFatal() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    verify(messageListener, timeout(TIMEOUT)).received(isA(AlertListenerMessage.class));
+  }
+
+  @Test
+  public void alertMessageIsNotReceivedForLevelNone() {
+    alertingService.addAlertListener(member, NONE);
+
+    logger.fatal(alertMessage);
+
+    verifyNoMoreInteractions(messageListener);
+  }
+
+  @Test
+  public void alertMessageIsReceivedForHigherLevels() {
+    alertingService.addAlertListener(member, WARNING);
+
+    logger.error(alertMessage);
+    logger.fatal(alertMessage);
+
+    verify(messageListener, timeout(TIMEOUT).times(2)).received(isA(AlertListenerMessage.class));
+  }
+
+  @Test
+  public void alertMessageIsNotReceivedForLowerLevels() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.warn(alertMessage);
+    logger.error(alertMessage);
+
+    verifyNoMoreInteractions(messageListener);
+  }
+
+  @Test
+  public void alertDetailsIsCreatedByAlertMessage() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    assertThat(captureAlertDetails()).isNotNull().isInstanceOf(AlertDetails.class);
+  }
+
+  @Test
+  public void alertDetailsAlertLevelMatches() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    assertThat(captureAlertDetails().getAlertLevel()).isEqualTo(SEVERE.intLevel());
+  }
+
+  @Test
+  public void alertDetailsMessageMatches() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    assertThat(captureAlertDetails().getMsg()).isEqualTo(alertMessage);
+  }
+
+  @Test
+  public void alertDetailsSenderIsNullForLocalAlert() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    assertThat(captureAlertDetails().getSender()).isNull();
+  }
+
+  @Test
+  public void alertDetailsSource() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    assertThat(captureAlertDetails().getSource()).contains(threadName);
+  }
+
+  @Test
+  public void alertDetailsConnectionName() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    assertThat(captureAlertDetails().getConnectionName()).isEqualTo(connectionName);
+  }
+
+  @Test
+  public void alertDetailsExceptionTextIsEmpty() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    assertThat(captureAlertDetails().getExceptionText()).isEqualTo("");
+  }
+
+  @Test
+  public void alertDetailsExceptionTextMatches() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage, new Exception(exceptionMessage));
+
+    assertThat(captureAlertDetails().getExceptionText()).contains(exceptionMessage);
+  }
+
+  @Test
+  public void alertDetailsThreadName() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    assertThat(captureAlertDetails().getThreadName()).isEqualTo(threadName);
+  }
+
+  @Test
+  public void alertDetailsThreadId() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    assertThat(captureAlertDetails().getTid()).isEqualTo(threadId);
+  }
+
+  @Test
+  public void alertDetailsMessageTime() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    assertThat(captureAlertDetails().getMsgTime()).isNotNull();
+  }
+
+  private AlertDetails captureAlertDetails() {
+    ArgumentCaptor<AlertDetails> alertDetailsCaptor = ArgumentCaptor.forClass(AlertDetails.class);
+    verify(messageListener, timeout(TIMEOUT)).created(alertDetailsCaptor.capture());
+    return alertDetailsCaptor.getValue();
+  }
+}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/alerting/AlertingServiceWithLonerIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/alerting/AlertingServiceWithLonerIntegrationTest.java
new file mode 100644
index 0000000..1b8e8f5
--- /dev/null
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/alerting/AlertingServiceWithLonerIntegrationTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.internal.admin.remote.AlertListenerMessage.addListener;
+import static org.apache.geode.internal.admin.remote.AlertListenerMessage.removeListener;
+import static org.apache.geode.internal.alerting.AlertLevel.SEVERE;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+import java.util.Properties;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.admin.remote.AlertListenerMessage;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.test.junit.categories.AlertingTest;
+
+/**
+ * Integration tests for {@link AlertingService} in a loner member.
+ */
+@Category(AlertingTest.class)
+public class AlertingServiceWithLonerIntegrationTest {
+
+  private InternalDistributedSystem system;
+  private DistributedMember member;
+  private AlertListenerMessage.Listener messageListener;
+  private Logger logger;
+  private String alertMessage;
+
+  private AlertingService alertingService;
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void setUp() {
+    alertMessage = "Alerting in " + testName.getMethodName();
+
+    messageListener = spy(AlertListenerMessage.Listener.class);
+    addListener(messageListener);
+
+    Properties config = new Properties();
+    config.setProperty(LOCATORS, "");
+
+    system = (InternalDistributedSystem) DistributedSystem.connect(config);
+    member = system.getDistributedMember();
+    logger = LogService.getLogger();
+
+    alertingService = system.getAlertingService();
+  }
+
+  @After
+  public void tearDown() {
+    removeListener(messageListener);
+    system.disconnect();
+  }
+
+  @Test
+  public void alertMessageIsNotReceived() {
+    alertingService.addAlertListener(member, SEVERE);
+
+    logger.fatal(alertMessage);
+
+    verifyNoMoreInteractions(messageListener);
+  }
+}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/logging/log4j/AlertAppenderIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/logging/log4j/AlertAppenderIntegrationTest.java
new file mode 100644
index 0000000..7b89360
--- /dev/null
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/logging/log4j/AlertAppenderIntegrationTest.java
@@ -0,0 +1,261 @@
+/*
+ * 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.geode.internal.logging.log4j;
+
+import static org.apache.geode.internal.alerting.AlertingProviderRegistry.getNullAlertingProvider;
+import static org.apache.geode.test.util.ResourceUtils.createFileFromResource;
+import static org.apache.geode.test.util.ResourceUtils.getResource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+
+import java.net.URL;
+import java.util.Date;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.Logger;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.junit.LoggerContextRule;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.alerting.AlertLevel;
+import org.apache.geode.internal.alerting.AlertMessaging;
+import org.apache.geode.internal.alerting.AlertingProviderRegistry;
+import org.apache.geode.internal.alerting.AlertingSession;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.test.junit.categories.AlertingTest;
+import org.apache.geode.test.junit.categories.LoggingTest;
+
+/**
+ * Integration tests for {@link AlertAppender}.
+ */
+@Category({AlertingTest.class, LoggingTest.class})
+public class AlertAppenderIntegrationTest {
+
+  private static final String CONFIG_FILE_NAME = "AlertAppenderIntegrationTest_log4j2.xml";
+  private static final String APPENDER_NAME = "ALERT";
+
+  private static String configFilePath;
+
+  private AlertAppender alertAppender;
+  private InternalDistributedMember localMember;
+  private AlertingSession alertingSession;
+  private AlertMessaging alertMessaging;
+  private AlertingProviderRegistry alertingProviderRegistry;
+  private Logger logger;
+  private String logMessage;
+
+  @ClassRule
+  public static TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public LoggerContextRule loggerContextRule = new LoggerContextRule(configFilePath);
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @BeforeClass
+  public static void setUpLogConfigFile() throws Exception {
+    URL resource = getResource(CONFIG_FILE_NAME);
+    configFilePath = createFileFromResource(resource, temporaryFolder.getRoot(), CONFIG_FILE_NAME)
+        .getAbsolutePath();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    alertAppender =
+        loggerContextRule.getAppender(APPENDER_NAME, AlertAppender.class);
+
+    InternalDistributedSystem internalDistributedSystem =
+        mock(InternalDistributedSystem.class, RETURNS_DEEP_STUBS);
+    localMember = mock(InternalDistributedMember.class);
+    ClusterDistributionManager clusterDistributionManager = mock(ClusterDistributionManager.class);
+
+    when(internalDistributedSystem.getConfig().getName()).thenReturn(testName.getMethodName());
+    when(internalDistributedSystem.getDistributedMember()).thenReturn(localMember);
+    when(internalDistributedSystem.getDistributionManager()).thenReturn(clusterDistributionManager);
+    when(clusterDistributionManager.getSystem()).thenReturn(internalDistributedSystem);
+
+    alertingSession = AlertingSession.create();
+    alertMessaging = new AlertMessaging(internalDistributedSystem);
+    alertingProviderRegistry = AlertingProviderRegistry.get();
+
+    alertingSession.createSession(alertMessaging);
+    alertingSession.startSession();
+
+    logger = LogService.getLogger();
+    logMessage = "Logging in " + testName.getMethodName();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    alertingSession.stopSession();
+    alertingProviderRegistry.clear();
+    alertAppender.clearLogEvents();
+  }
+
+  @Test
+  public void getLogEventsIsEmptyByDefault() {
+    assertThat(alertAppender.getLogEvents()).isEmpty();
+  }
+
+  @Test
+  public void getLogEventsReturnsLoggedEvents() {
+    logger.warn(logMessage);
+
+    assertThat(alertAppender.getLogEvents()).hasSize(1);
+    LogEvent event = alertAppender.getLogEvents().get(0);
+    assertThat(event.getLoggerName()).isEqualTo(getClass().getName());
+    assertThat(event.getLevel()).isEqualTo(Level.WARN);
+    assertThat(event.getMessage().getFormattedMessage()).isEqualTo(logMessage);
+  }
+
+  @Test
+  public void skipsLogEventsWithoutMatchingAlertLevel() {
+    // AlertLevelConverter does not convert trace|debug|info Level to AlertLevel
+    logger.trace("trace");
+    logger.debug("debug");
+    logger.info("info");
+
+    assertThat(alertAppender.getLogEvents()).isEmpty();
+  }
+
+  @Test
+  public void pausedDoesNotAppend() {
+    alertAppender.pause();
+
+    logger.warn(logMessage);
+
+    assertThat(alertAppender.getLogEvents()).isEmpty();
+  }
+
+  @Test
+  public void resumeAfterPausedAppends() {
+    alertAppender.pause();
+    alertAppender.resume();
+
+    logger.warn(logMessage);
+
+    assertThat(alertAppender.getLogEvents()).hasSize(1);
+  }
+
+  @Test
+  public void resumeWithoutPauseStillLogs() {
+    alertAppender.resume();
+
+    logger.warn(logMessage);
+
+    assertThat(alertAppender.getLogEvents()).hasSize(1);
+  }
+
+  @Test
+  public void isPausedReturnsTrueAfterPause() {
+    alertAppender.pause();
+
+    assertThat(alertAppender.isPaused()).isTrue();
+  }
+
+  @Test
+  public void isPausedReturnsFalseAfterResume() {
+    alertAppender.pause();
+    alertAppender.resume();
+
+    assertThat(alertAppender.isPaused()).isFalse();
+  }
+
+  @Test
+  public void resumeWithoutPauseDoesNothing() {
+    alertAppender.resume();
+
+    assertThat(alertAppender.isPaused()).isFalse();
+  }
+
+  @Test
+  public void isPausedReturnsFalseByDefault() {
+    assertThat(alertAppender.isPaused()).isFalse();
+  }
+
+  @Test
+  public void alertMessagingExistsAfterOnConnect() {
+    alertAppender.createSession(alertMessaging);
+
+    assertThat(alertAppender.getAlertMessaging()).isNotNull();
+  }
+
+  @Test
+  public void alertMessagingDoesNotExistAfterOnDisconnect() {
+    alertAppender.createSession(alertMessaging);
+    alertAppender.stopSession();
+
+    assertThat(alertAppender.getAlertMessaging()).isNull();
+  }
+
+  @Test
+  public void sendsNoAlertsIfNoListeners() {
+    alertAppender.createSession(alertMessaging);
+    alertMessaging = spy(alertAppender.getAlertMessaging());
+    alertAppender.setAlertMessaging(alertMessaging);
+
+    logger.warn(logMessage);
+
+    assertThat(alertAppender.getLogEvents()).hasSize(1);
+    verifyZeroInteractions(alertMessaging);
+  }
+
+  @Test
+  public void sendsAlertIfListenerExists() {
+    alertAppender.createSession(alertMessaging);
+    alertMessaging = spy(alertAppender.getAlertMessaging());
+    alertAppender.setAlertMessaging(alertMessaging);
+    alertAppender.addAlertListener(localMember, AlertLevel.WARNING);
+
+    logger.warn(logMessage);
+
+    assertThat(alertAppender.getLogEvents()).hasSize(1);
+    verify(alertMessaging).sendAlert(eq(localMember), eq(AlertLevel.WARNING), any(Date.class),
+        anyString(), anyString(), isNull());
+  }
+
+  @Test
+  public void isRegisteredWithAlertingProviderRegistryDuringInitialization() {
+    assertThat(alertingProviderRegistry.getAlertingProvider()).isSameAs(alertAppender);
+  }
+
+  @Test
+  public void isUnregisteredWithAlertingProviderRegistryDuringCleanUp() {
+    alertAppender.stop();
+
+    assertThat(alertingProviderRegistry.getAlertingProvider()).isSameAs(getNullAlertingProvider());
+  }
+}
diff --git a/geode-core/src/integrationTest/resources/org/apache/geode/internal/logging/log4j/AlertAppenderIntegrationTest_log4j2.xml b/geode-core/src/integrationTest/resources/org/apache/geode/internal/logging/log4j/AlertAppenderIntegrationTest_log4j2.xml
new file mode 100644
index 0000000..f750120
--- /dev/null
+++ b/geode-core/src/integrationTest/resources/org/apache/geode/internal/logging/log4j/AlertAppenderIntegrationTest_log4j2.xml
@@ -0,0 +1,36 @@
+<?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="INFO" shutdownHook="disable" packages="org.apache.geode.internal.logging.log4j">
+    <Properties>
+        <Property name="geode-pattern">[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} %memberName &lt;%thread&gt; tid=%hexTid] %message%n%throwable%n</Property>
+    </Properties>
+    <Appenders>
+        <GeodeAlert name="ALERT" debug="true"/>
+    </Appenders>
+    <Loggers>
+        <Logger name="com.gemstone" level="INFO" additivity="true"/>
+        <Logger name="org.apache.geode" level="INFO" additivity="true">
+            <filters>
+                <MarkerFilter marker="GEODE_VERBOSE" onMatch="DENY" onMismatch="NEUTRAL"/>
+            </filters>
+        </Logger>
+        <Logger name="org.jgroups" level="FATAL" additivity="true"/>
+        <Logger name="org.eclipse.jetty" level="FATAL" additivity="true"/>
+        <Root level="INFO">
+            <AppenderRef ref="ALERT"/>
+        </Root>
+    </Loggers>
+</Configuration>
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
index 2eb50e4..c523164 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
@@ -72,13 +72,13 @@ import org.apache.geode.internal.Version;
 import org.apache.geode.internal.admin.remote.AdminConsoleDisconnectMessage;
 import org.apache.geode.internal.admin.remote.RemoteGfManagerAgent;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
+import org.apache.geode.internal.alerting.AlertingService;
 import org.apache.geode.internal.cache.InitialImageOperation;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingExecutors;
 import org.apache.geode.internal.logging.LoggingThread;
 import org.apache.geode.internal.logging.LoggingUncaughtExceptionHandler;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
 import org.apache.geode.internal.logging.log4j.LogMarker;
 import org.apache.geode.internal.monitoring.ThreadsMonitoring;
 import org.apache.geode.internal.monitoring.ThreadsMonitoringImpl;
@@ -490,6 +490,8 @@ public class ClusterDistributionManager implements DistributionManager {
    */
   private final Object shutdownMutex = new Object();
 
+  private final AlertingService alertingService;
+
   ////////////////////// Static Methods //////////////////////
 
   /**
@@ -531,7 +533,8 @@ public class ClusterDistributionManager implements DistributionManager {
 
       long start = System.currentTimeMillis();
 
-      distributionManager = new ClusterDistributionManager(system, transport);
+      distributionManager =
+          new ClusterDistributionManager(system, transport, system.getAlertingService());
       distributionManager.assertDistributionManagerType();
 
       beforeJoined = false; // we have now joined the system
@@ -627,11 +630,12 @@ public class ClusterDistributionManager implements DistributionManager {
    *
    */
   private ClusterDistributionManager(RemoteTransportConfig transport,
-      InternalDistributedSystem system) {
+      InternalDistributedSystem system, AlertingService alertingService) {
 
     this.dmType = transport.getVmKind();
     this.system = system;
     this.transport = transport;
+    this.alertingService = alertingService;
 
     this.membershipListeners = new ConcurrentHashMap<>();
     this.distributedSystemId = system.getConfig().getDistributedSystemId();
@@ -888,8 +892,8 @@ public class ClusterDistributionManager implements DistributionManager {
    * @param system The distributed system to which this distribution manager will send messages.
    */
   private ClusterDistributionManager(InternalDistributedSystem system,
-      RemoteTransportConfig transport) {
-    this(transport, system);
+      RemoteTransportConfig transport, AlertingService alertingService) {
+    this(transport, system, alertingService);
 
     boolean finishedConstructor = false;
     try {
@@ -1919,6 +1923,11 @@ public class ClusterDistributionManager implements DistributionManager {
     return this.system;
   }
 
+  @Override
+  public AlertingService getAlertingService() {
+    return alertingService;
+  }
+
   /**
    * Returns the transport configuration for this distribution manager
    */
@@ -2634,7 +2643,7 @@ public class ClusterDistributionManager implements DistributionManager {
   public void handleManagerDeparture(InternalDistributedMember theId, boolean p_crashed,
       String p_reason) {
 
-    AlertAppender.getInstance().removeAlertListener(theId);
+    alertingService.removeAlertListener(theId);
 
     int vmType = theId.getVmKind();
     if (vmType == ADMIN_ONLY_DM_TYPE) {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
index 9742822..09eaabc 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
@@ -32,6 +32,7 @@ import org.apache.geode.distributed.internal.locks.ElderState;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
 import org.apache.geode.internal.Version;
+import org.apache.geode.internal.alerting.AlertingService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.monitoring.ThreadsMonitoring;
 
@@ -115,7 +116,7 @@ public interface DistributionManager extends ReplySender {
   Set<InternalDistributedMember> getOtherNormalDistributionManagerIds();
 
   /**
-   * Add a membership listener and return other DistribtionManagerIds as an atomic operation
+   * Add a membership listener and return other DistributionManagerIds as an atomic operation
    */
   Set<InternalDistributedMember> addMembershipListenerAndGetDistributionManagerIds(
       MembershipListener l);
@@ -476,4 +477,9 @@ public interface DistributionManager extends ReplySender {
 
   /** returns the Threads Monitoring instance */
   public ThreadsMonitoring getThreadMonitoring();
+
+  /**
+   * Returns the {@link AlertingService}.
+   */
+  AlertingService getAlertingService();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
index 5c759dd..16c61a0 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
@@ -49,6 +49,8 @@ import org.apache.geode.distributed.internal.membership.MemberAttributes;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
 import org.apache.geode.i18n.LogWriterI18n;
 import org.apache.geode.internal.Version;
+import org.apache.geode.internal.alerting.AlertingService;
+import org.apache.geode.internal.alerting.NullAlertingService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.logging.LoggingExecutors;
@@ -1428,4 +1430,9 @@ public class LonerDistributionManager implements DistributionManager {
   public ThreadsMonitoring getThreadMonitoring() {
     return this.threadMonitor;
   }
+
+  @Override
+  public AlertingService getAlertingService() {
+    return NullAlertingService.get();
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannel.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannel.java
index 5c9dac7..dbb4068 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannel.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannel.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.distributed.internal.direct;
 
 import java.io.IOException;
@@ -45,9 +44,9 @@ import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.ReplyProcessor21;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
+import org.apache.geode.internal.alerting.AlertingAction;
 import org.apache.geode.internal.cache.DirectReplyMessage;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
 import org.apache.geode.internal.logging.log4j.LogMarker;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.tcp.BaseMsgStreamer;
@@ -226,7 +225,7 @@ public class DirectChannel {
   boolean threadOwnsResources() {
     DistributionManager d = getDM();
     if (d != null) {
-      return d.getSystem().threadOwnsResources() && !AlertAppender.isThreadAlerting();
+      return d.getSystem().threadOwnsResources() && !AlertingAction.isThreadAlerting();
     }
     return false;
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessenger.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
index 736e9e7..4be7154 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
@@ -98,13 +98,12 @@ import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.VersionedDataInputStream;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
+import org.apache.geode.internal.alerting.AlertingAction;
 import org.apache.geode.internal.cache.DirectReplyMessage;
 import org.apache.geode.internal.cache.DistributedCacheOperation;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.tcp.MemberShunnedException;
 
-
 @SuppressWarnings("StatementWithEmptyBody")
 public class JGroupsMessenger implements Messenger {
 
@@ -947,7 +946,7 @@ public class JGroupsMessenger implements Messenger {
     msg.setFlag(Flag.DONT_BUNDLE);
 
     if (gfmsg.getProcessorType() == ClusterDistributionManager.HIGH_PRIORITY_EXECUTOR
-        || gfmsg instanceof HighPriorityDistributionMessage || AlertAppender.isThreadAlerting()) {
+        || gfmsg instanceof HighPriorityDistributionMessage || AlertingAction.isThreadAlerting()) {
       msg.setFlag(Flag.OOB);
       msg.setFlag(Flag.NO_FC);
       msg.setFlag(Flag.SKIP_BARRIER);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index 63eef40..5eb36fd 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -2506,7 +2506,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     services.setShutdownCause(shutdownCause);
     services.getCancelCriterion().cancel(reason);
 
-    AlertAppender.getInstance().shuttingDown();
+    AlertAppender.getInstance().stopSession();
 
     if (!inhibitForceDisconnectLogging) {
       logger.fatal(
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleDisconnectMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleDisconnectMessage.java
index d1b0e20..e37c109 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleDisconnectMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleDisconnectMessage.java
@@ -12,8 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
-
 package org.apache.geode.internal.admin.remote;
 
 import java.io.DataInput;
@@ -27,12 +25,11 @@ import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.PooledDistributionMessage;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
 import org.apache.geode.internal.statistics.GemFireStatSampler;
 
 /**
  * A message that is sent to a particular distribution manager to let it know that the sender is an
- * administation console that just disconnected.
+ * administration console that just disconnected.
  */
 public class AdminConsoleDisconnectMessage extends PooledDistributionMessage {
   private static final Logger logger = LogService.getLogger();
@@ -62,7 +59,7 @@ public class AdminConsoleDisconnectMessage extends PooledDistributionMessage {
   }
 
   public void setIgnoreAlertListenerRemovalFailure(boolean ignore) {
-    this.ignoreAlertListenerRemovalFailure = ignore;
+    ignoreAlertListenerRemovalFailure = ignore;
   }
 
   /**
@@ -77,24 +74,22 @@ public class AdminConsoleDisconnectMessage extends PooledDistributionMessage {
   @Override
   public void process(ClusterDistributionManager dm) {
     InternalDistributedSystem sys = dm.getSystem();
-    // DistributionConfig config = sys.getConfig();
     if (alertListenerExpected) {
-      if (!AlertAppender.getInstance().removeAlertListener(this.getSender())
-          && !this.ignoreAlertListenerRemovalFailure) {
+      if (!dm.getAlertingService().removeAlertListener(getSender())
+          && !ignoreAlertListenerRemovalFailure) {
         logger.warn("Unable to remove console with id {} from alert listeners.",
-            this.getSender());
+            getSender());
       }
     }
     GemFireStatSampler sampler = sys.getStatSampler();
     if (sampler != null) {
-      sampler.removeListenersByRecipient(this.getSender());
+      sampler.removeListenersByRecipient(getSender());
     }
-    dm.handleConsoleShutdown(this.getSender(), crashed,
-        String.format("Reason for automatic admin disconnect : %s",
-            reason));
-    // AppCacheSnapshotMessage.flushSnapshots(this.getSender());
+    dm.handleConsoleShutdown(getSender(), crashed,
+        String.format("Reason for automatic admin disconnect : %s", reason));
   }
 
+  @Override
   public int getDSFID() {
     return ADMIN_CONSOLE_DISCONNECT_MESSAGE;
   }
@@ -110,13 +105,13 @@ public class AdminConsoleDisconnectMessage extends PooledDistributionMessage {
   @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
     super.fromData(in);
-    this.alertListenerExpected = in.readBoolean();
-    this.crashed = in.readBoolean();
-    this.reason = DataSerializer.readString(in);
+    alertListenerExpected = in.readBoolean();
+    crashed = in.readBoolean();
+    reason = DataSerializer.readString(in);
   }
 
   @Override
   public String toString() {
-    return "AdminConsoleDisconnectMessage from " + this.getSender();
+    return "AdminConsoleDisconnectMessage from " + getSender();
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleMessage.java
index d4e9532..6b46835 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleMessage.java
@@ -12,8 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
-
 package org.apache.geode.internal.admin.remote;
 
 import java.io.DataInput;
@@ -23,15 +21,15 @@ import java.io.IOException;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.PooledDistributionMessage;
 import org.apache.geode.internal.admin.Alert;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
+import org.apache.geode.internal.alerting.AlertLevel;
 
 /**
  * A message that is sent to a particular distribution manager to let it know that the sender is an
- * administation console that just connected.
+ * administration console that just connected.
  */
 public class AdminConsoleMessage extends PooledDistributionMessage {
   // instance variables
-  int level;
+  private int level;
 
   public static AdminConsoleMessage create(int level) {
     AdminConsoleMessage m = new AdminConsoleMessage();
@@ -45,12 +43,13 @@ public class AdminConsoleMessage extends PooledDistributionMessage {
 
   @Override
   public void process(ClusterDistributionManager dm) {
-    if (this.level != Alert.OFF) {
-      AlertAppender.getInstance().addAlertListener(this.getSender(), this.level);
+    if (level != Alert.OFF) {
+      dm.getAlertingService().addAlertListener(getSender(), AlertLevel.find(level));
     }
-    dm.addAdminConsole(this.getSender());
+    dm.addAdminConsole(getSender());
   }
 
+  @Override
   public int getDSFID() {
     return ADMIN_CONSOLE_MESSAGE;
   }
@@ -58,18 +57,18 @@ public class AdminConsoleMessage extends PooledDistributionMessage {
   @Override
   public void toData(DataOutput out) throws IOException {
     super.toData(out);
-    out.writeInt(this.level);
+    out.writeInt(level);
   }
 
   @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
     super.fromData(in);
-    this.level = in.readInt();
+    level = in.readInt();
   }
 
   @Override
   public String toString() {
-    return "AdminConsoleMessage from " + this.getSender() + " level=" + level;
+    return "AdminConsoleMessage from " + getSender() + " level=" + level;
   }
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertLevelChangeMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertLevelChangeMessage.java
index 1a950e6..5a7414d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertLevelChangeMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertLevelChangeMessage.java
@@ -20,12 +20,10 @@ import java.io.IOException;
 
 import org.apache.logging.log4j.Logger;
 
-import org.apache.geode.admin.AlertLevel;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.SerialDistributionMessage;
-import org.apache.geode.internal.admin.Alert;
+import org.apache.geode.internal.alerting.AlertLevel;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
 import org.apache.geode.internal.logging.log4j.LogMarker;
 
 /**
@@ -52,10 +50,10 @@ public class AlertLevelChangeMessage extends SerialDistributionMessage {
 
   @Override
   public void process(ClusterDistributionManager dm) {
-    AlertAppender.getInstance().removeAlertListener(getSender());
+    dm.getAlertingService().removeAlertListener(getSender());
 
-    if (newLevel != Alert.OFF) {
-      AlertAppender.getInstance().addAlertListener(getSender(), newLevel);
+    if (newLevel != AlertLevel.NONE.intLevel()) {
+      dm.getAlertingService().addAlertListener(getSender(), AlertLevel.find(newLevel));
       if (logger.isTraceEnabled(LogMarker.DM_VERBOSE)) {
         logger.trace(LogMarker.DM_VERBOSE, "Added new AlertListener");
       }
@@ -81,6 +79,6 @@ public class AlertLevelChangeMessage extends SerialDistributionMessage {
 
   @Override
   public String toString() {
-    return String.format("Changing alert level to %s", AlertLevel.forSeverity(newLevel));
+    return String.format("Changing alert level to %s", AlertLevel.find(newLevel));
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertListenerMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertListenerMessage.java
index 6c3b16b..4cc2324 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertListenerMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertListenerMessage.java
@@ -18,9 +18,11 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Date;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.geode.DataSerializer;
 import org.apache.geode.admin.AlertLevel;
+import org.apache.geode.annotations.TestingOnly;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.AdminMessageType;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
@@ -41,6 +43,8 @@ import org.apache.geode.management.internal.AlertDetails;
  */
 public class AlertListenerMessage extends PooledDistributionMessage implements AdminMessageType {
 
+  private static final AtomicReference<Listener> listenerRef = new AtomicReference<>();
+
   private int alertLevel;
   private Date date;
   private String connectionName;
@@ -78,6 +82,11 @@ public class AlertListenerMessage extends PooledDistributionMessage implements A
 
   @Override
   public void process(ClusterDistributionManager dm) {
+    Listener listener = getListener();
+    if (listener != null) {
+      listener.received(this);
+    }
+
     RemoteGfManagerAgent agent = dm.getAgent();
     if (agent != null) {
       RemoteGemFireVM manager = agent.getMemberById(getSender());
@@ -86,6 +95,11 @@ public class AlertListenerMessage extends PooledDistributionMessage implements A
       }
       Alert alert = new RemoteAlert(manager, alertLevel, date, connectionName, threadName, threadId,
           message, exceptionText, getSender());
+
+      if (listener != null) {
+        listener.created(alert);
+      }
+
       agent.callAlertListener(alert);
     } else {
       /*
@@ -94,6 +108,11 @@ public class AlertListenerMessage extends PooledDistributionMessage implements A
        */
       AlertDetails alertDetail = new AlertDetails(alertLevel, date, connectionName, threadName,
           threadId, message, exceptionText, getSender());
+
+      if (listener != null) {
+        listener.created(alertDetail);
+      }
+
       dm.getSystem().handleResourceEvent(ResourceEvent.SYSTEM_ALERT, alertDetail);
     }
   }
@@ -136,4 +155,29 @@ public class AlertListenerMessage extends PooledDistributionMessage implements A
   public String toString() {
     return "Alert \"" + message + "\" level " + AlertLevel.forSeverity(alertLevel);
   }
+
+  @TestingOnly
+  public static void addListener(Listener listener) {
+    listenerRef.compareAndSet(null, listener);
+  }
+
+  @TestingOnly
+  public static void removeListener(Listener listener) {
+    listenerRef.compareAndSet(listener, null);
+  }
+
+  @TestingOnly
+  public static Listener getListener() {
+    return listenerRef.get();
+  }
+
+  @TestingOnly
+  public interface Listener {
+
+    void received(AlertListenerMessage message);
+
+    void created(Alert alert);
+
+    void created(AlertDetails alertDetails);
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertLevel.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertLevel.java
new file mode 100644
index 0000000..0ec7984
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertLevel.java
@@ -0,0 +1,53 @@
+/*
+ * 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.geode.internal.alerting;
+
+import org.apache.geode.internal.logging.LogWriterLevel;
+
+/**
+ * Defines the {@code LogWriterLevel}s that are available for use as a threshold for generating and
+ * listening for system {@code Alert}s.
+ */
+public enum AlertLevel {
+
+  WARNING(LogWriterLevel.WARNING.intLevel()),
+  ERROR(LogWriterLevel.ERROR.intLevel()),
+  SEVERE(LogWriterLevel.SEVERE.intLevel()),
+  NONE(LogWriterLevel.NONE.intLevel());
+
+  public static AlertLevel find(int intLevel) {
+    for (AlertLevel alertLevel : values()) {
+      if (alertLevel.intLevel == intLevel) {
+        return alertLevel;
+      }
+    }
+    throw new IllegalArgumentException("No AlertLevel found for intLevel " + intLevel);
+  }
+
+  private final int intLevel;
+
+  AlertLevel(int intLevel) {
+    this.intLevel = intLevel;
+  }
+
+  public int intLevel() {
+    return intLevel;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "." + name() + "(" + intLevel + ")";
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertListenerMessageFactory.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertListenerMessageFactory.java
new file mode 100644
index 0000000..10e64b0
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertListenerMessageFactory.java
@@ -0,0 +1,50 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static org.apache.geode.internal.admin.remote.AlertListenerMessage.create;
+
+import java.util.Date;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.admin.remote.AlertListenerMessage;
+
+class AlertListenerMessageFactory {
+
+  AlertListenerMessage createAlertListenerMessage(final DistributedMember member,
+      final AlertLevel alertLevel,
+      final Date date,
+      final String connectionName,
+      final String threadName,
+      final String formattedMessage,
+      final String stackTrace) {
+    verifyDistributedMemberCanReceiveMessage(member);
+    return create(member, alertLevel.intLevel(), date, connectionName, threadName,
+        Thread.currentThread().getId(), formattedMessage, stackTrace);
+  }
+
+  /**
+   * Remove verifyDistributedMemberCanReceiveMessage when AlertListenerMessage no longer casts to
+   * InternalDistributedMember.
+   */
+  private void verifyDistributedMemberCanReceiveMessage(final DistributedMember member) {
+    if (!(member instanceof InternalDistributedMember)) {
+      throw new IllegalArgumentException(
+          "Creation of AlertListenerMessage requires InternalDistributedMember instead of "
+              + member.getClass().getSimpleName());
+    }
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertMessaging.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertMessaging.java
new file mode 100644
index 0000000..df07aad
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertMessaging.java
@@ -0,0 +1,100 @@
+/*
+ * 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.geode.internal.alerting;
+
+import java.util.Date;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
+import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.admin.remote.AlertListenerMessage;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.tcp.ReenteredConnectException;
+
+/**
+ * Provides the ability to send messages about {@code Alert}s to local or remote {@code Alert}
+ * listeners.
+ */
+public class AlertMessaging {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final InternalDistributedSystem system;
+  private final DistributionManager dm;
+  private final AlertListenerMessageFactory alertListenerMessageFactory;
+
+  public AlertMessaging(final InternalDistributedSystem system) {
+    this(system, system.getDistributionManager(), new AlertListenerMessageFactory());
+  }
+
+  AlertMessaging(final InternalDistributedSystem system,
+      final DistributionManager dm,
+      final AlertListenerMessageFactory alertListenerMessageFactory) {
+    this.system = system;
+    this.dm = dm;
+    this.alertListenerMessageFactory = alertListenerMessageFactory;
+  }
+
+  public void sendAlert(final DistributedMember member,
+      final AlertLevel alertLevel,
+      final Date date,
+      final String threadName,
+      final String formattedMessage,
+      final String stackTrace) {
+    try {
+      String connectionName = system.getConfig().getName();
+
+      AlertListenerMessage message = alertListenerMessageFactory.createAlertListenerMessage(member,
+          alertLevel, date, connectionName, threadName, formattedMessage, stackTrace);
+
+      if (member.equals(system.getDistributedMember())) {
+        // process in local member
+        logger.debug("Processing local alert message: {}, {}, {}, {}, {}, [{}], [{}].",
+            member, alertLevel, date, connectionName, threadName, formattedMessage, stackTrace);
+        processAlertListenerMessage(message);
+
+      } else {
+        // send to remote member
+        logger.debug("Sending remote alert message: {}, {}, {}, {}, {}, [{}], [{}].",
+            member, alertLevel, date, connectionName, threadName, formattedMessage, stackTrace);
+        dm.putOutgoing(message);
+      }
+    } catch (ReenteredConnectException ignore) {
+      // OK. We can't send to this recipient because we're in the middle of
+      // trying to connect to it.
+    }
+  }
+
+  void processAlertListenerMessage(final AlertListenerMessage message) {
+    verifyDistributionManagerCanProcessMessage();
+    message.process((ClusterDistributionManager) dm);
+  }
+
+  /**
+   * Remove verifyDistributionManagerCanProcessMessage when
+   * {@link AlertListenerMessage#process(ClusterDistributionManager)} no longer requires
+   * ClusterDistributionManager.
+   */
+  private void verifyDistributionManagerCanProcessMessage() {
+    if (!(dm instanceof ClusterDistributionManager)) {
+      throw new IllegalArgumentException(
+          "Processing of AlertListenerMessage requires ClusterDistributionManager instead of "
+              + dm.getClass().getSimpleName());
+    }
+  }
+}
diff --git a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingAction.java
similarity index 55%
copy from geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
copy to geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingAction.java
index f2a21a1..c94ceb5 100644
--- a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingAction.java
@@ -12,17 +12,27 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.internal.alerting;
 
-package org.apache.geode.annotations;
+/**
+ * Executes an action that is protected against generating additional {@code Alert}s. Even if the
+ * executed action generates log statements that meet the configured {@code AlertLevel}, no
+ * {@code Alert} will be raised.
+ */
+public class AlertingAction {
 
-import java.lang.annotation.Documented;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Target;
+  private static final ThreadLocal<Boolean> ALERTING = ThreadLocal.withInitial(() -> Boolean.FALSE);
 
-@Documented
-@Target({ElementType.CONSTRUCTOR, ElementType.METHOD})
-public @interface TestingOnly {
+  public static void execute(final Runnable action) {
+    ALERTING.set(true);
+    try {
+      action.run();
+    } finally {
+      ALERTING.set(false);
+    }
+  }
 
-  /** Optional description */
-  String value() default "";
+  public static boolean isThreadAlerting() {
+    return ALERTING.get();
+  }
 }
diff --git a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingProvider.java
similarity index 54%
copy from geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
copy to geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingProvider.java
index f2a21a1..9ba8061 100644
--- a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingProvider.java
@@ -12,17 +12,20 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.internal.alerting;
 
-package org.apache.geode.annotations;
+import org.apache.geode.distributed.DistributedMember;
 
-import java.lang.annotation.Documented;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Target;
+/**
+ * Handles {@code Alert} listeners for local and remote members. The implementation is responsible
+ * for maintaining registration of {@code Alert} listeners. Listeners are typically managing members
+ * that will receive a notification anytime an {@code Alert} is raised.
+ */
+public interface AlertingProvider extends AlertingSessionListener {
+
+  void addAlertListener(DistributedMember member, AlertLevel alertLevel);
 
-@Documented
-@Target({ElementType.CONSTRUCTOR, ElementType.METHOD})
-public @interface TestingOnly {
+  boolean removeAlertListener(DistributedMember member);
 
-  /** Optional description */
-  String value() default "";
+  boolean hasAlertListener(DistributedMember member, AlertLevel alertLevel);
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingProviderRegistry.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingProviderRegistry.java
new file mode 100644
index 0000000..70295e8
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingProviderRegistry.java
@@ -0,0 +1,80 @@
+/*
+ * 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.geode.internal.alerting;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.geode.annotations.TestingOnly;
+
+/**
+ * Provides publication of {@code AlertingProvider} that may be initiated by a third party logging
+ * library such as Log4J2.
+ *
+ * <p>
+ * This mechanism is used instead of a manifest-based {@code ServiceLoader} so that
+ * {@code AlertAppender} is registered only if Log4J2 creates it.
+ */
+public class AlertingProviderRegistry {
+
+  private static final AlertingProvider NULL_ALERTING_PROVIDER = new NullAlertingProvider();
+
+  private static final AlertingProviderRegistry INSTANCE = new AlertingProviderRegistry();
+
+  public static AlertingProviderRegistry get() {
+    return INSTANCE;
+  }
+
+  @TestingOnly
+  public static AlertingProvider getNullAlertingProvider() {
+    return NULL_ALERTING_PROVIDER;
+  }
+
+  private final AlertingSessionListeners alertingSessionListeners;
+  private final AtomicReference<AlertingProvider> alertingProviderRef = new AtomicReference<>();
+
+  AlertingProviderRegistry() {
+    this(AlertingSessionListeners.get(), NULL_ALERTING_PROVIDER);
+  }
+
+  private AlertingProviderRegistry(final AlertingSessionListeners alertingSessionListeners,
+      final AlertingProvider alertingProvider) {
+    if (alertingSessionListeners == null) {
+      throw new NullPointerException("alertingSessionListeners must not be null");
+    }
+    if (alertingProvider == null) {
+      throw new NullPointerException("alertingProvider must not be null");
+    }
+    this.alertingSessionListeners = alertingSessionListeners;
+    alertingProviderRef.set(alertingProvider);
+  }
+
+  public AlertingProvider getAlertingProvider() {
+    return alertingProviderRef.get();
+  }
+
+  public void registerAlertingProvider(final AlertingProvider provider) {
+    alertingSessionListeners.addAlertingSessionListener(provider);
+    alertingProviderRef.compareAndSet(NULL_ALERTING_PROVIDER, provider);
+  }
+
+  public void unregisterAlertingProvider(final AlertingProvider provider) {
+    alertingProviderRef.compareAndSet(provider, NULL_ALERTING_PROVIDER);
+    alertingSessionListeners.removeAlertingSessionListener(provider);
+  }
+
+  public void clear() {
+    unregisterAlertingProvider(alertingProviderRef.get());
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingService.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingService.java
new file mode 100644
index 0000000..fadfc6f
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingService.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.geode.internal.alerting;
+
+
+import org.apache.geode.annotations.TestingOnly;
+import org.apache.geode.distributed.DistributedMember;
+
+/**
+ * Adds and removes {@code AlertListeners} for local and remote members that need to receive
+ * notification of system {@code Alerts}.
+ *
+ * <p>
+ * The {@code AlertingService} looks up the registered {@code AlertingProvider} from
+ * {@code AlertingProviderRegistry} and delegates all calls to that provider.
+ */
+public class AlertingService {
+
+  private final AlertingProviderRegistry providerRegistry;
+
+  public AlertingService() {
+    this(AlertingProviderRegistry.get());
+  }
+
+  AlertingService(final AlertingProviderRegistry providerRegistry) {
+    this.providerRegistry = providerRegistry;
+  }
+
+  public void addAlertListener(final DistributedMember member, final AlertLevel alertLevel) {
+    providerRegistry.getAlertingProvider().addAlertListener(member, alertLevel);
+  }
+
+  public boolean removeAlertListener(final DistributedMember member) {
+    return providerRegistry.getAlertingProvider().removeAlertListener(member);
+  }
+
+  public boolean hasAlertListener(final DistributedMember member, final AlertLevel alertLevel) {
+    return providerRegistry.getAlertingProvider().hasAlertListener(member, alertLevel);
+  }
+
+  @TestingOnly
+  AlertingProviderRegistry getAlertingProviderRegistry() {
+    return providerRegistry;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingSession.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingSession.java
new file mode 100644
index 0000000..cf528df
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingSession.java
@@ -0,0 +1,107 @@
+/*
+ * 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.geode.internal.alerting;
+
+import org.apache.geode.annotations.TestingOnly;
+
+/**
+ * The {@code DistributedSystem} connection uses {@code AlertingSession} to control the lifecycle
+ * of the registered {@code AlertingProvider}.
+ *
+ * <p>
+ * During initialization of {@code DistributedSystem} a new {@code AlertingSession} is instantiated
+ * and {@code createSession} will be invoked with a new instance of {@code AlertMessaging}. After
+ * a {@code DistributionConfig} exists, it will then invoke {@code startSession}. During disconnect,
+ * it will invoke {@code stopSession}.
+ *
+ * <p>
+ * The {@code AlertAppender} will capture the {@code AlertMessaging} in order to send out
+ * {@code Alert} messages to registered {@code Alert} listeners. {@code startSession} will cause
+ * the appender to unpause and begin processing all incoming log events. Any log event that meets
+ * the {@code AlertLevel} of one or more {@code Alert} listeners will result in the generation of
+ * an {@code Alert} being sent to those listeners.
+ */
+public class AlertingSession {
+
+  private final AlertingSessionListeners alertingSessionListeners;
+  private State state = State.STOPPED;
+
+  public static AlertingSession create() {
+    return create(AlertingSessionListeners.get());
+  }
+
+  @TestingOnly
+  static AlertingSession create(final AlertingSessionListeners alertingSessionListeners) {
+    return new AlertingSession(alertingSessionListeners);
+  }
+
+  private AlertingSession(final AlertingSessionListeners alertingSessionListeners) {
+    this.alertingSessionListeners = alertingSessionListeners;
+  }
+
+  public synchronized void createSession(final AlertMessaging alertMessaging) {
+    state = state.changeTo(State.CREATED);
+    alertingSessionListeners.createSession(alertMessaging);
+  }
+
+  public synchronized void startSession() {
+    state = state.changeTo(State.STARTED);
+    alertingSessionListeners.startSession();
+  }
+
+  public synchronized void stopSession() {
+    state = state.changeTo(State.STOPPED);
+    alertingSessionListeners.stopSession();
+  }
+
+  public synchronized void shutdown() {
+    // nothing?
+  }
+
+  @TestingOnly
+  AlertingSessionListeners getAlertingSessionListeners() {
+    return alertingSessionListeners;
+  }
+
+  synchronized State getState() {
+    return state;
+  }
+
+  enum State {
+    CREATED,
+    STARTED,
+    STOPPED;
+
+    State changeTo(final State newState) {
+      switch (newState) {
+        case CREATED:
+          if (this != STOPPED) {
+            throw new IllegalStateException("Session must not exist before creating");
+          }
+          return CREATED;
+        case STARTED:
+          if (this != CREATED) {
+            throw new IllegalStateException("Session must be created before starting");
+          }
+          return STARTED;
+        case STOPPED:
+          if (this != STARTED) {
+            throw new IllegalStateException("Session must be started before stopping");
+          }
+      }
+      return STOPPED;
+    }
+  }
+}
diff --git a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingSessionListener.java
similarity index 71%
copy from geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
copy to geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingSessionListener.java
index f2a21a1..3e9a831 100644
--- a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingSessionListener.java
@@ -12,17 +12,16 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.internal.alerting;
 
-package org.apache.geode.annotations;
+/**
+ * Defines the lifecycle callbacks of an {@code AlertingSession}.
+ */
+public interface AlertingSessionListener {
 
-import java.lang.annotation.Documented;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Target;
+  void createSession(final AlertMessaging alertMessaging);
 
-@Documented
-@Target({ElementType.CONSTRUCTOR, ElementType.METHOD})
-public @interface TestingOnly {
+  void startSession();
 
-  /** Optional description */
-  String value() default "";
+  void stopSession();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingSessionListeners.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingSessionListeners.java
new file mode 100644
index 0000000..7fa3905
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/AlertingSessionListeners.java
@@ -0,0 +1,78 @@
+/*
+ * 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.geode.internal.alerting;
+
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+/**
+ * Manages a collection of registered {@code AlertingSessionListener}s and forwards
+ * {@code AlertingSession} lifecycle callbacks to each listener.
+ */
+public class AlertingSessionListeners {
+
+  private static final AlertingSessionListeners INSTANCE = new AlertingSessionListeners();
+
+  public static AlertingSessionListeners get() {
+    return INSTANCE;
+  }
+
+  private final Set<AlertingSessionListener> listeners;
+
+  AlertingSessionListeners() {
+    listeners = new LinkedHashSet<>();
+  }
+
+  /**
+   * Adds the {@code AlertingSessionListener} and returns true if it was not already
+   * registered.
+   */
+  public synchronized boolean addAlertingSessionListener(final AlertingSessionListener listener) {
+    return listeners.add(listener);
+  }
+
+  /**
+   * Removes the {@code AlertingSessionListener} and returns true if it was registered.
+   */
+  public synchronized boolean removeAlertingSessionListener(
+      final AlertingSessionListener listener) {
+    return listeners.remove(listener);
+  }
+
+  /**
+   * Removes all {@code AlertingSessionListener}s that are registered.
+   */
+  public synchronized void clear() {
+    listeners.clear();
+  }
+
+  public synchronized void createSession(final AlertMessaging alertMessaging) {
+    for (AlertingSessionListener listener : listeners) {
+      listener.createSession(alertMessaging);
+    }
+  }
+
+  public synchronized void startSession() {
+    for (AlertingSessionListener listener : listeners) {
+      listener.startSession();
+    }
+  }
+
+  public synchronized void stopSession() {
+    for (AlertingSessionListener listener : listeners) {
+      listener.stopSession();
+    }
+  }
+}
diff --git a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/NullAlertingProvider.java
similarity index 52%
copy from geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
copy to geode-core/src/main/java/org/apache/geode/internal/alerting/NullAlertingProvider.java
index f2a21a1..bb8c800 100644
--- a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/NullAlertingProvider.java
@@ -12,17 +12,39 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.internal.alerting;
 
-package org.apache.geode.annotations;
+import org.apache.geode.distributed.DistributedMember;
 
-import java.lang.annotation.Documented;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Target;
+class NullAlertingProvider implements AlertingProvider {
 
-@Documented
-@Target({ElementType.CONSTRUCTOR, ElementType.METHOD})
-public @interface TestingOnly {
+  @Override
+  public void addAlertListener(DistributedMember member, AlertLevel alertLevel) {
+    // nothing
+  }
 
-  /** Optional description */
-  String value() default "";
+  @Override
+  public boolean removeAlertListener(DistributedMember member) {
+    return false;
+  }
+
+  @Override
+  public boolean hasAlertListener(DistributedMember member, AlertLevel alertLevel) {
+    return false;
+  }
+
+  @Override
+  public void createSession(AlertMessaging alertMessaging) {
+    // nothing
+  }
+
+  @Override
+  public void startSession() {
+    // nothing
+  }
+
+  @Override
+  public void stopSession() {
+    // nothing
+  }
 }
diff --git a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java b/geode-core/src/main/java/org/apache/geode/internal/alerting/NullAlertingService.java
similarity index 50%
copy from geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
copy to geode-core/src/main/java/org/apache/geode/internal/alerting/NullAlertingService.java
index f2a21a1..ca9fa8a 100644
--- a/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/alerting/NullAlertingService.java
@@ -12,17 +12,34 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.internal.alerting;
 
-package org.apache.geode.annotations;
+import org.apache.geode.distributed.DistributedMember;
 
-import java.lang.annotation.Documented;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Target;
+/**
+ * Null implementation of {@link AlertingService} that does nothing.
+ */
+public class NullAlertingService extends AlertingService {
+
+  private static final NullAlertingService INSTANCE = new NullAlertingService();
+
+  public static NullAlertingService get() {
+    return INSTANCE;
+  }
+
+  public NullAlertingService() {
+    // nothing
+  }
+
+  public void addAlertListener(final DistributedMember member, final AlertLevel alertLevel) {
+    // nothing
+  }
 
-@Documented
-@Target({ElementType.CONSTRUCTOR, ElementType.METHOD})
-public @interface TestingOnly {
+  public boolean removeAlertListener(final DistributedMember member) {
+    return false;
+  }
 
-  /** Optional description */
-  String value() default "";
+  public boolean hasAlertListener(final DistributedMember member, final AlertLevel alertLevel) {
+    return false;
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertAppender.java b/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertAppender.java
index 7640a6d..22d1137 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertAppender.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertAppender.java
@@ -14,314 +14,328 @@
  */
 package org.apache.geode.internal.logging.log4j;
 
-import static org.apache.geode.internal.logging.log4j.AlertLevel.alertLevelToLogLevel;
-import static org.apache.geode.internal.logging.log4j.AlertLevel.logLevelToAlertLevel;
+import static org.apache.geode.internal.logging.log4j.AlertLevelConverter.hasAlertLevel;
 
-import java.beans.PropertyChangeEvent;
-import java.beans.PropertyChangeListener;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Date;
+import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.Logger;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.Layout;
 import org.apache.logging.log4j.core.LogEvent;
-import org.apache.logging.log4j.core.LoggerContext;
 import org.apache.logging.log4j.core.appender.AbstractAppender;
-import org.apache.logging.log4j.core.config.LoggerConfig;
-import org.apache.logging.log4j.core.layout.PatternLayout;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
 
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.internal.admin.Alert;
-import org.apache.geode.internal.admin.remote.AlertListenerMessage;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.tcp.ReenteredConnectException;
-
-/**
- * A Log4j Appender which will notify listeners whenever a message of the requested level is written
- * to the log file.
- *
- */
-public class AlertAppender extends AbstractAppender implements PropertyChangeListener {
-  private static final String APPENDER_NAME = AlertAppender.class.getName();
-  private static final Logger logger = LogService.getLogger();
-  private static final AlertAppender instance = createAlertAppender();
+import org.apache.geode.internal.alerting.AlertLevel;
+import org.apache.geode.internal.alerting.AlertMessaging;
+import org.apache.geode.internal.alerting.AlertingAction;
+import org.apache.geode.internal.alerting.AlertingProvider;
+import org.apache.geode.internal.alerting.AlertingProviderRegistry;
 
-  /** Is this thread in the process of alerting? */
-  private static final ThreadLocal<Boolean> alerting = new ThreadLocal<Boolean>() {
-    @Override
-    protected Boolean initialValue() {
-      return Boolean.FALSE;
-    }
-  };
+@Plugin(name = AlertAppender.PLUGIN_NAME, category = Core.CATEGORY_NAME,
+    elementType = Appender.ELEMENT_TYPE, printObject = true)
+@SuppressWarnings("unused")
+public class AlertAppender extends AbstractAppender
+    implements PausableAppender, DebuggableAppender, AlertingProvider {
 
-  // Listeners are ordered with the narrowest levels (e.g. FATAL) at the end
-  private final CopyOnWriteArrayList<Listener> listeners = new CopyOnWriteArrayList<Listener>();
+  public static final String PLUGIN_NAME = "GeodeAlert";
 
-  private final AppenderContext appenderContext = LogService.getAppenderContext();
+  private static final boolean START_PAUSED_BY_DEFAULT = true;
 
-  private final AtomicReference<InternalDistributedSystem> systemRef = new AtomicReference<>();
+  private static final AtomicReference<AlertAppender> instanceRef = new AtomicReference<>();
 
-  // This can be set by a loner distributed sytem to disable alerting
-  private volatile boolean alertingDisabled = false;
+  private final AtomicReference<AlertMessaging> alertMessagingRef = new AtomicReference<>();
 
-  private static AlertAppender createAlertAppender() {
-    AlertAppender alertAppender = new AlertAppender();
-    alertAppender.start();
-    return alertAppender;
-  }
+  // Listeners are ordered with the narrowest levels (e.g. FATAL) at the end
+  private final CopyOnWriteArrayList<AlertListener> listeners;
+
+  private final AlertingProviderRegistry alertingProviderRegistry;
 
-  private AlertAppender() {
-    super(APPENDER_NAME, null, PatternLayout.createDefaultLayout());
+  private final boolean debug;
+  private final List<LogEvent> events;
+
+  private volatile boolean paused;
+
+  protected AlertAppender(final String name,
+      final Layout<? extends Serializable> layout,
+      final Filter filter) {
+    this(name, layout, filter, AlertingProviderRegistry.get(), START_PAUSED_BY_DEFAULT, false);
   }
 
-  public void onConnect(final InternalDistributedSystem system) {
-    this.systemRef.set(system);
+  protected AlertAppender(final String name,
+      final Layout<? extends Serializable> layout,
+      final Filter filter,
+      final AlertingProviderRegistry alertingProviderRegistry,
+      final boolean startPaused,
+      final boolean debug) {
+    super(name, filter, layout);
+    listeners = new CopyOnWriteArrayList<>();
+    this.alertingProviderRegistry = alertingProviderRegistry;
+    this.debug = debug;
+    if (debug) {
+      events = Collections.synchronizedList(new ArrayList<>());
+    } else {
+      events = Collections.emptyList();
+    }
+    paused = true;
   }
 
-  public static AlertAppender getInstance() {
-    return instance;
+  @PluginBuilderFactory
+  public static <B extends AlertAppender.Builder<B>> B newBuilder() {
+    return new AlertAppender.Builder<B>().asBuilder();
   }
 
   /**
-   * Returns true if the current thread is in the process of delivering an alert message.
+   * Builds AlertAppender instances.
+   *
+   * @param <B> The type to build
    */
-  public static boolean isThreadAlerting() {
-    return alerting.get();
-  }
+  public static class Builder<B extends Builder<B>> extends AbstractAppender.Builder<B>
+      implements org.apache.logging.log4j.core.util.Builder<AlertAppender> {
 
-  public boolean isAlertingDisabled() {
-    return alertingDisabled;
-  }
+    @PluginBuilderAttribute
+    private boolean debug;
 
-  public void setAlertingDisabled(final boolean alertingDisabled) {
-    this.alertingDisabled = alertingDisabled;
-  }
+    @PluginBuilderAttribute
+    private boolean startPaused = START_PAUSED_BY_DEFAULT;
+
+    public B setStartPaused(final boolean shouldStartPaused) {
+      startPaused = shouldStartPaused;
+      return asBuilder();
+    }
+
+    public boolean isStartPaused() {
+      return debug;
+    }
+
+    public B setDebug(final boolean shouldDebug) {
+      debug = shouldDebug;
+      return asBuilder();
+    }
 
-  public static void setIsAlerting(boolean isAlerting) {
-    alerting.set(isAlerting ? Boolean.TRUE : Boolean.FALSE);
+    public boolean isDebug() {
+      return debug;
+    }
+
+    @Override
+    public AlertAppender build() {
+      Layout<? extends Serializable> layout = getOrCreateLayout();
+      instanceRef.set(new AlertAppender(getName(), layout, getFilter(),
+          AlertingProviderRegistry.get(), startPaused, debug));
+      return instanceRef.get();
+    }
   }
 
-  /**
-   * This method is optimized with the assumption that at least one listener has set a level which
-   * requires that the event be sent. This is ensured by modifying the appender's configuration
-   * whenever a listener is added or removed.
-   */
   @Override
   public void append(final LogEvent event) {
-    if (this.alertingDisabled) {
+    LOGGER.trace("Handling append of {} in {}.", event, this);
+    if (isPaused()) {
+      LOGGER.trace("Skipping append of {} because {} is paused.", event, this);
+      return;
+    }
+    if (!hasAlertLevel(event.getLevel())) {
+      LOGGER.trace("Skipping append of {} because level is {}.", event, event.getLevel());
       return;
     }
+    if (AlertingAction.isThreadAlerting()) {
+      // If already appending then don't send to avoid infinite recursion
+      LOGGER.trace("Skipping append of {} because {} is alerting.", event, Thread.currentThread());
+      return;
+    }
+    AlertingAction.execute(() -> doAppend(event));
+  }
+
+  private void doAppend(final LogEvent event) {
+    sendAlertMessage(event);
+    if (debug) {
+      events.add(event);
+    }
+  }
 
-    // If already appending then don't send to avoid infinite recursion
-    if ((alerting.get())) {
+  private void sendAlertMessage(final LogEvent event) {
+    AlertMessaging alertMessaging = alertMessagingRef.get();
+    if (alertMessaging == null || listeners.isEmpty()) {
+      LOGGER.trace("Skipping alert messaging for {} because listeners is empty.", event);
       return;
     }
-    setIsAlerting(true);
 
-    try {
+    AlertLevel alertLevel = AlertLevelConverter.fromLevel(event.getLevel());
+    Date date = new Date(event.getTimeMillis());
+    String threadName = event.getThreadName();
+    String formattedMessage = event.getMessage().getFormattedMessage();
+    String stackTrace = getStackTrace(event);
 
-      final boolean isDebugEnabled = logger.isDebugEnabled();
-      if (isDebugEnabled) {
-        logger.debug("Delivering an alert event: {}", event);
+    for (AlertListener listener : listeners) {
+      if (event.getLevel().intLevel() > listener.getLevel().intLevel()) {
+        break;
       }
 
-      InternalDistributedSystem ds = this.systemRef.get();
-      if (ds == null) {
-        // Use info level to avoid triggering another alert
-        logger.info("Did not append alert event because the distributed system is set to null.");
-        return;
-      }
-      ClusterDistributionManager distMgr = (ClusterDistributionManager) ds.getDistributionManager();
-
-      final int intLevel = logLevelToAlertLevel(event.getLevel());
-      final Date date = new Date(event.getTimeMillis());
-      final String threadName = event.getThreadName();
-      final String logMessage = event.getMessage().getFormattedMessage();
-      final String stackTrace =
-          (event.getThrown() == null) ? null : ExceptionUtils.getStackTrace(event.getThrown());
-      final String connectionName = ds.getConfig().getName();
-
-      for (Listener listener : this.listeners) {
-        if (event.getLevel().intLevel() > listener.getLevel().intLevel()) {
-          break;
-        }
-
-        try {
-          AlertListenerMessage alertMessage =
-              AlertListenerMessage.create(listener.getMember(), intLevel, date, connectionName,
-                  threadName, Thread.currentThread().getId(), logMessage, stackTrace);
-
-          if (listener.getMember().equals(distMgr.getDistributionManagerId())) {
-            if (isDebugEnabled) {
-              logger.debug("Delivering local alert message: {}, {}, {}, {}, {}, [{}], [{}].",
-                  listener.getMember(), intLevel, date, connectionName, threadName, logMessage,
-                  stackTrace);
-            }
-            alertMessage.process(distMgr);
-          } else {
-            if (isDebugEnabled) {
-              logger.debug("Delivering remote alert message: {}, {}, {}, {}, {}, [{}], [{}].",
-                  listener.getMember(), intLevel, date, connectionName, threadName, logMessage,
-                  stackTrace);
-            }
-            distMgr.putOutgoing(alertMessage);
-          }
-        } catch (ReenteredConnectException e) {
-          // OK. We can't send to this recipient because we're in the middle of
-          // trying to connect to it.
-        }
-      }
-    } finally {
-      setIsAlerting(false);
+      LOGGER.trace("Sending alert message for {} to {}.", event, listener.getMember());
+      alertMessaging.sendAlert(listener.getMember(), alertLevel, date, threadName, formattedMessage,
+          stackTrace);
     }
   }
 
-  public synchronized void addAlertListener(final DistributedMember member, final int alertLevel) {
-    final Level level = LogService.toLevel(alertLevelToLogLevel(alertLevel));
+  private String getStackTrace(final LogEvent event) {
+    return event.getThrown() == null ? null : ExceptionUtils.getStackTrace(event.getThrown());
+  }
 
-    if (this.listeners.size() == 0) {
-      this.appenderContext.getLoggerContext().addPropertyChangeListener(this);
+  @Override
+  public void start() {
+    LOGGER.info("Starting {}.", this);
+    LOGGER.debug("Registering {} with AlertingProviderRegistry.", this);
+    try {
+      alertingProviderRegistry.registerAlertingProvider(this);
+    } finally {
+      super.start();
     }
+  }
 
-    addListenerToSortedList(new Listener(level, member));
+  @Override
+  public void stop() {
+    LOGGER.info("Stopping {}.", this);
 
-    LoggerConfig loggerConfig = this.appenderContext.getLoggerConfig();
-    loggerConfig.addAppender(this, this.listeners.get(0).getLevel(), null);
-    if (logger.isDebugEnabled()) {
-      logger.debug("Added/Replaced alert listener for member {} at level {}", member, level);
-    }
-  }
+    // stop LogEvents from coming to this appender
+    super.stop();
 
-  public synchronized boolean removeAlertListener(final DistributedMember member) {
-    final boolean memberWasFound = this.listeners.remove(new Listener(null, member));
+    // unregister as provider
+    cleanUp(true);
 
-    if (memberWasFound) {
-      if (this.listeners.size() == 0) {
-        this.appenderContext.getLoggerContext().removePropertyChangeListener(this);
-        this.appenderContext.getLoggerConfig().removeAppender(APPENDER_NAME);
+    LOGGER.info("{} has stopped.", this);
+  }
 
-      } else {
-        LoggerConfig loggerConfig = this.appenderContext.getLoggerConfig();
-        loggerConfig.addAppender(this, this.listeners.get(0).getLevel(), null);
-      }
-      if (logger.isDebugEnabled()) {
-        logger.debug("Removed alert listener for member {}", member);
-      }
-    }
+  @Override
+  public void pause() {
+    LOGGER.debug("Pausing {}.", this);
+    paused = true;
+  }
 
-    return memberWasFound;
+  @Override
+  public void resume() {
+    LOGGER.debug("Resuming {}.", this);
+    paused = false;
   }
 
-  public synchronized boolean hasAlertListener(final DistributedMember member,
-      final int alertLevel) {
-    final Level level = LogService.toLevel(alertLevelToLogLevel(alertLevel));
+  @Override
+  public boolean isPaused() {
+    return paused;
+  }
 
-    for (Listener listener : this.listeners) {
-      if (listener.getMember().equals(member) && listener.getLevel().equals(level)) {
-        return true;
-      }
-    }
+  @Override
+  public void clearLogEvents() {
+    events.clear();
+  }
 
-    // Special case for alert level Alert.OFF (NONE_LEVEL), because we can never have an actual
-    // listener with
-    // this level (see AlertLevelChangeMessage.process()).
-    if (alertLevel == Alert.OFF) {
-      for (Listener listener : this.listeners) {
-        if (listener.getMember().equals(member)) {
-          return false;
-        }
-      }
-      return true;
-    }
+  @Override
+  public List<LogEvent> getLogEvents() {
+    return events;
+  }
 
-    return false;
+  @Override
+  public synchronized void createSession(final AlertMessaging alertMessaging) {
+    LOGGER.info("Creating session in {} with {}.", this, alertMessaging);
+    setAlertMessaging(alertMessaging);
   }
 
   @Override
-  public synchronized void propertyChange(final PropertyChangeEvent evt) {
-    if (logger.isDebugEnabled()) {
-      logger.debug("Responding to a property change event. Property name is {}.",
-          evt.getPropertyName());
-    }
-    if (evt.getPropertyName().equals(LoggerContext.PROPERTY_CONFIG)) {
-      LoggerConfig loggerConfig = this.appenderContext.getLoggerConfig();
-      if (!loggerConfig.getAppenders().containsKey(APPENDER_NAME)) {
-        loggerConfig.addAppender(this, this.listeners.get(0).getLevel(), null);
-      }
-    }
+  public synchronized void startSession() {
+    LOGGER.info("Starting session in {}.", this);
+    resume();
   }
 
-  /**
-   * Will add (or replace) a listener to the list of sorted listeners such that listeners with a
-   * narrower level (e.g. FATAL) will be at the end of the list.
-   *
-   * @param listener The listener to add to the list.
-   */
-  private void addListenerToSortedList(final Listener listener) {
-    if (this.listeners.contains(listener)) {
-      this.listeners.remove(listener);
+  @Override
+  public synchronized void stopSession() {
+    LOGGER.info("Stopping session in {}.", this);
+    cleanUp(false);
+  }
+
+  private synchronized void cleanUp(boolean unregister) {
+    pause();
+    if (unregister) {
+      LOGGER.debug("Unregistering {} with AlertingProviderRegistry.", this);
+      alertingProviderRegistry.unregisterAlertingProvider(this);
     }
+    listeners.clear();
+    setAlertMessaging(null);
+  }
+
+  void setAlertMessaging(final AlertMessaging alertMessaging) {
+    alertMessagingRef.set(alertMessaging);
+  }
 
-    for (int i = 0; i < this.listeners.size(); i++) {
-      if (listener.getLevel().compareTo(this.listeners.get(i).getLevel()) >= 0) {
-        this.listeners.add(i, listener);
+  AlertMessaging getAlertMessaging() {
+    return alertMessagingRef.get();
+  }
+
+  @Override
+  public synchronized void addAlertListener(final DistributedMember member,
+      final AlertLevel alertLevel) {
+    if (alertLevel == AlertLevel.NONE) {
+      return;
+    }
+    Level level = AlertLevelConverter.toLevel(alertLevel);
+    AlertListener listener = new AlertListener(level, member);
+
+    // Add (or replace) a listener to the list of sorted listeners such that listeners with a
+    // narrower level (e.g. FATAL) will be at the end of the list.
+    listeners.remove(listener);
+    for (int i = 0; i < listeners.size(); i++) {
+      if (listener.getLevel().compareTo(listeners.get(i).getLevel()) >= 0) {
+        listeners.add(i, listener);
         return;
       }
     }
+    listeners.add(listener);
 
-    this.listeners.add(listener);
+    LOGGER.debug("Added/Replaced alert listener for member {} at level {}.", member, level);
   }
 
-  public synchronized void shuttingDown() {
-    this.listeners.clear();
-    this.appenderContext.getLoggerContext().removePropertyChangeListener(this);
-    this.appenderContext.getLoggerConfig().removeAppender(APPENDER_NAME);
-    this.systemRef.set(null);
+  @Override
+  public synchronized boolean removeAlertListener(final DistributedMember member) {
+    boolean memberWasFound = listeners.remove(new AlertListener(null, member));
+    if (memberWasFound) {
+      LOGGER.debug("Removed alert listener for member {}.", member);
+    }
+    return memberWasFound;
   }
 
-  /**
-   * Simple value object which holds an InteralDistributedMember and Level pair.
-   */
-  static class Listener {
-    private Level level;
-    private DistributedMember member;
-
-    public Level getLevel() {
-      return this.level;
-    }
+  @Override
+  public synchronized boolean hasAlertListener(final DistributedMember member,
+      final AlertLevel alertLevel) {
+    Level level = AlertLevelConverter.toLevel(alertLevel);
 
-    public DistributedMember getMember() {
-      return this.member;
+    for (AlertListener listener : listeners) {
+      if (listener.getMember().equals(member) && listener.getLevel().equals(level)) {
+        return true;
+      }
     }
 
-    Listener(final Level level, final DistributedMember member) {
-      this.level = level;
-      this.member = member;
-    }
+    return false;
+  }
 
-    /**
-     * Never used, but maintain the hashCode/equals contract.
-     */
-    @Override
-    public int hashCode() {
-      return 31 + ((this.member == null) ? 0 : this.member.hashCode());
-    }
+  @Override
+  public String toString() {
+    return getClass().getName() + "@" + Integer.toHexString(hashCode()) + ":" + getName()
+        + " {alertMessaging=" + alertMessagingRef.get() + ", listeners=" + listeners + ", paused="
+        + paused + ", debug=" + debug + "}";
+  }
 
-    /**
-     * Ignore the level when determining equality.
-     */
-    @Override
-    public boolean equals(Object other) {
-      return (this.member.equals(((Listener) other).member)) ? true : false;
-    }
+  public synchronized List<AlertListener> getAlertListeners() {
+    return listeners;
+  }
 
-    @Override
-    public String toString() {
-      return "Listener [level=" + this.level + ", member=" + this.member + "]";
-    }
+  public static AlertAppender getInstance() {
+    return instanceRef.get();
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertLevel.java b/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertLevel.java
deleted file mode 100644
index 5ffe5a4..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertLevel.java
+++ /dev/null
@@ -1,61 +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.geode.internal.logging.log4j;
-
-import org.apache.logging.log4j.Level;
-
-import org.apache.geode.internal.admin.Alert;
-
-public class AlertLevel {
-
-  /**
-   * Converts an {@link Alert} level to a LOG4J2 {@code Level}.
-   *
-   * @throws IllegalArgumentException if there is no matching LOG4J2 Level
-   */
-  static int alertLevelToLogLevel(final int alertLevel) {
-    switch (alertLevel) {
-      case Alert.SEVERE:
-        return Level.FATAL.intLevel();
-      case Alert.ERROR:
-        return Level.ERROR.intLevel();
-      case Alert.WARNING:
-        return Level.WARN.intLevel();
-      case Alert.OFF:
-        return Level.OFF.intLevel();
-    }
-
-    throw new IllegalArgumentException("Unknown Alert level [" + alertLevel + "].");
-  }
-
-  /**
-   * Converts a LOG4J2 {@code Level} to an {@link Alert} level.
-   *
-   * @throws IllegalArgumentException if there is no matching Alert level
-   */
-  static int logLevelToAlertLevel(final Level logLevel) {
-    if (logLevel == Level.FATAL) {
-      return Alert.SEVERE;
-    } else if (logLevel == Level.ERROR) {
-      return Alert.ERROR;
-    } else if (logLevel == Level.WARN) {
-      return Alert.WARNING;
-    } else if (logLevel == Level.OFF) {
-      return Alert.OFF;
-    }
-
-    throw new IllegalArgumentException("Unknown LOG4J2 Level [" + logLevel + "].");
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertLevelConverter.java b/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertLevelConverter.java
new file mode 100644
index 0000000..06276fc
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertLevelConverter.java
@@ -0,0 +1,88 @@
+/*
+ * 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.geode.internal.logging.log4j;
+
+import static org.apache.geode.internal.alerting.AlertLevel.ERROR;
+import static org.apache.geode.internal.alerting.AlertLevel.NONE;
+import static org.apache.geode.internal.alerting.AlertLevel.SEVERE;
+import static org.apache.geode.internal.alerting.AlertLevel.WARNING;
+
+import org.apache.logging.log4j.Level;
+
+import org.apache.geode.internal.alerting.AlertLevel;
+
+/**
+ * Converts between {@link AlertLevel}s and Log4J2 {@code Level}s.
+ *
+ * <p>
+ * Implementation note: switch and if-else structures perform better than using a HashMap for this
+ * which matters the most if used for every log statement.
+ */
+class AlertLevelConverter {
+
+  /**
+   * True if the Log4J2 {@code Level} converts to an {@link AlertLevel}.
+   */
+  static boolean hasAlertLevel(final Level level) {
+    if (level == Level.FATAL) {
+      return true;
+    } else if (level == Level.ERROR) {
+      return true;
+    } else if (level == Level.WARN) {
+      return true;
+    } else {
+      return level == Level.OFF;
+    }
+  }
+
+  /**
+   * Converts from an {@link AlertLevel} to a Log4J2 {@code Level}.
+   *
+   * @throws IllegalArgumentException if there is no matching Log4J2 Level
+   */
+  static Level toLevel(final AlertLevel alert) {
+    switch (alert) {
+      case SEVERE:
+        return Level.FATAL;
+      case ERROR:
+        return Level.ERROR;
+      case WARNING:
+        return Level.WARN;
+      case NONE:
+        return Level.OFF;
+    }
+
+    throw new IllegalArgumentException("No matching Log4J2 Level for " + alert + ".");
+  }
+
+  /**
+   * Converts from a Log4J2 {@code Level} to an {@link AlertLevel}.
+   *
+   * @throws IllegalArgumentException if there is no matching Alert
+   */
+  static AlertLevel fromLevel(final Level level) {
+    if (level == Level.FATAL) {
+      return SEVERE;
+    } else if (level == Level.ERROR) {
+      return ERROR;
+    } else if (level == Level.WARN) {
+      return WARNING;
+    } else if (level == Level.OFF) {
+      return NONE;
+    }
+
+    throw new IllegalArgumentException("No matching AlertLevel for Log4J2 Level " + level + ".");
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertListener.java b/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertListener.java
new file mode 100644
index 0000000..b847cfe
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/log4j/AlertListener.java
@@ -0,0 +1,66 @@
+/*
+ * 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.geode.internal.logging.log4j;
+
+import org.apache.logging.log4j.Level;
+
+import org.apache.geode.distributed.DistributedMember;
+
+/**
+ * Simple value object which holds a {@link DistributedMember} and {@link Level} pair.
+ */
+class AlertListener {
+
+  private final Level level;
+  private final DistributedMember member;
+
+  AlertListener(final Level level, final DistributedMember member) {
+    this.level = level;
+    this.member = member;
+  }
+
+  Level getLevel() {
+    return level;
+  }
+
+  DistributedMember getMember() {
+    return member;
+  }
+
+  /**
+   * Never used, but maintain the hashCode/equals contract.
+   */
+  @Override
+  public int hashCode() {
+    return 31 + (member == null ? 0 : member.hashCode());
+  }
+
+  /**
+   * Ignore the level when determining equality.
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if (!(obj instanceof AlertListener)) {
+      return false;
+    }
+    AlertListener other = (AlertListener) obj;
+    return member.equals(other.member);
+  }
+
+  @Override
+  public String toString() {
+    return "Listener [level=" + level + ", member=" + member + "]";
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java b/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
index d5d20b8..f234ee7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
@@ -75,9 +75,9 @@ import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.SystemTimer;
 import org.apache.geode.internal.SystemTimer.SystemTimerTask;
 import org.apache.geode.internal.Version;
+import org.apache.geode.internal.alerting.AlertingAction;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingThread;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.tcp.MsgReader.Header;
 import org.apache.geode.internal.util.concurrent.ReentrantSemaphore;
@@ -1035,7 +1035,7 @@ public class Connection implements Runnable {
         } else {
           // if we're sending an alert and can't connect, bail out. A sick
           // alert listener should not prevent cache operations from continuing
-          if (AlertAppender.isThreadAlerting()) {
+          if (AlertingAction.isThreadAlerting()) {
             // do not change the text of this exception - it is looked for in exception handlers
             throw new IOException("Cannot form connection to alert listener " + remoteAddr);
           }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/tcp/ConnectionTable.java b/geode-core/src/main/java/org/apache/geode/internal/tcp/ConnectionTable.java
index 9bb8f21..8bf3b3a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/tcp/ConnectionTable.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/tcp/ConnectionTable.java
@@ -44,9 +44,9 @@ import org.apache.geode.distributed.internal.membership.MembershipManager;
 import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.SystemTimer;
+import org.apache.geode.internal.alerting.AlertingAction;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingExecutors;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
 import org.apache.geode.internal.net.SocketCloser;
 
 /**
@@ -172,18 +172,9 @@ public class ConnectionTable {
   boolean threadOwnsResources() {
     DistributionManager d = getDM();
     if (d != null) {
-      return d.getSystem().threadOwnsResources() && !AlertAppender.isThreadAlerting();
+      return d.getSystem().threadOwnsResources() && !AlertingAction.isThreadAlerting();
     }
     return false;
-
-    // Boolean b = getThreadOwnsResourcesRegistration();
-    // if (b == null) {
-    // // thread does not have a preference so return default
-    // return !this.owner.shareSockets;
-    // return false;
-    // } else {
-    // return b.booleanValue();
-    // }
   }
 
   public static Boolean getThreadOwnsResourcesRegistration() {
@@ -215,13 +206,6 @@ public class ConnectionTable {
     }
   }
 
-  /** conduit sends connected() after establishing the server socket */
-  // protected void connected() {
-  // /* NOMUX: if (TCPConduit.useNIO) {
-  // inputMuxManager.connected();
-  // }*/
-  // }
-
   /** conduit calls acceptConnection after an accept */
   protected void acceptConnection(Socket sock, PeerConnectionFactory peerConnectionFactory)
       throws IOException, ConnectionException, InterruptedException {
@@ -418,7 +402,7 @@ public class ConnectionTable {
     } else { // we have existing connection
       if (mEntry instanceof PendingConnection) {
 
-        if (AlertAppender.isThreadAlerting()) {
+        if (AlertingAction.isThreadAlerting()) {
           // do not change the text of this exception - it is looked for in exception handlers
           throw new IOException("Cannot form connection to alert listener " + id);
         }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/tcp/TCPConduit.java b/geode-core/src/main/java/org/apache/geode/internal/tcp/TCPConduit.java
index 18593dd..0b3b438 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/tcp/TCPConduit.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/tcp/TCPConduit.java
@@ -48,10 +48,10 @@ import org.apache.geode.distributed.internal.LonerDistributionManager;
 import org.apache.geode.distributed.internal.direct.DirectChannel;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
+import org.apache.geode.internal.alerting.AlertingAction;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingExecutors;
 import org.apache.geode.internal.logging.LoggingThread;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
 import org.apache.geode.internal.logging.log4j.LogMarker;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.net.SocketCreatorFactory;
@@ -978,7 +978,7 @@ public class TCPConduit implements Runnable {
         } catch (IOException e) {
           problem = e;
           // bug #43962 don't keep trying to connect to an alert listener
-          if (AlertAppender.isThreadAlerting()) {
+          if (AlertingAction.isThreadAlerting()) {
             if (logger.isDebugEnabled()) {
               logger.debug("Giving up connecting to alert listener {}", memberAddress);
             }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/ManagerStartupMessage.java b/geode-core/src/main/java/org/apache/geode/management/internal/ManagerStartupMessage.java
index ec4457c..10c7fce 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/ManagerStartupMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/ManagerStartupMessage.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.PooledDistributionMessage;
 import org.apache.geode.internal.admin.Alert;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
+import org.apache.geode.internal.alerting.AlertLevel;
 
 /**
  * Sent by JMX manager to all other members to notify them that it has started.
@@ -45,7 +45,7 @@ public class ManagerStartupMessage extends PooledDistributionMessage {
   @Override
   public void process(ClusterDistributionManager dm) {
     if (alertLevel != Alert.OFF) {
-      AlertAppender.getInstance().addAlertListener(getSender(), alertLevel);
+      dm.getAlertingService().addAlertListener(getSender(), AlertLevel.find(alertLevel));
     }
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertLevelTest.java b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertLevelTest.java
new file mode 100644
index 0000000..c981550
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertLevelTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static org.apache.geode.internal.alerting.AlertLevel.ERROR;
+import static org.apache.geode.internal.alerting.AlertLevel.NONE;
+import static org.apache.geode.internal.alerting.AlertLevel.SEVERE;
+import static org.apache.geode.internal.alerting.AlertLevel.WARNING;
+import static org.apache.geode.internal.alerting.AlertLevel.find;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.Serializable;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.internal.logging.LogWriterLevel;
+import org.apache.geode.test.junit.categories.AlertingTest;
+
+/**
+ * Unit tests for {@link AlertLevel}.
+ */
+@Category(AlertingTest.class)
+public class AlertLevelTest {
+
+  @Test
+  public void isSerializable() {
+    assertThat(NONE).isInstanceOf(Serializable.class);
+  }
+
+  @Test
+  public void serializes() {
+    AlertLevel logLevel = (AlertLevel) SerializationUtils.clone(NONE);
+
+    assertThat(logLevel).isEqualTo(NONE).isSameAs(NONE);
+  }
+
+  @Test
+  public void findWARNING() {
+    assertThat(find(LogWriterLevel.WARNING.intLevel())).isEqualTo(WARNING);
+  }
+
+  @Test
+  public void findERROR() {
+    assertThat(find(LogWriterLevel.ERROR.intLevel())).isEqualTo(ERROR);
+  }
+
+  @Test
+  public void findSEVERE() {
+    assertThat(find(LogWriterLevel.SEVERE.intLevel())).isEqualTo(SEVERE);
+  }
+
+  @Test
+  public void findNONE() {
+    assertThat(find(LogWriterLevel.NONE.intLevel())).isEqualTo(NONE);
+  }
+
+  @Test
+  public void findINFO() {
+    assertThatThrownBy(() -> find(LogWriterLevel.INFO.intLevel()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("No AlertLevel found for intLevel " + LogWriterLevel.INFO.intLevel());
+  }
+
+  @Test
+  public void findCONFIG() {
+    assertThatThrownBy(() -> find(LogWriterLevel.CONFIG.intLevel()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("No AlertLevel found for intLevel " + LogWriterLevel.CONFIG.intLevel());
+  }
+
+  @Test
+  public void findFINE() {
+    assertThatThrownBy(() -> find(LogWriterLevel.FINE.intLevel()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("No AlertLevel found for intLevel " + LogWriterLevel.FINE.intLevel());
+  }
+
+  @Test
+  public void findFINER() {
+    assertThatThrownBy(() -> find(LogWriterLevel.FINER.intLevel()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("No AlertLevel found for intLevel " + LogWriterLevel.FINER.intLevel());
+  }
+
+  @Test
+  public void findFINEST() {
+    assertThatThrownBy(() -> find(LogWriterLevel.FINEST.intLevel()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("No AlertLevel found for intLevel " + LogWriterLevel.FINEST.intLevel());
+  }
+
+  @Test
+  public void findALL() {
+    assertThatThrownBy(() -> find(LogWriterLevel.ALL.intLevel()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("No AlertLevel found for intLevel " + LogWriterLevel.ALL.intLevel());
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertListenerMessageFactoryTest.java b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertListenerMessageFactoryTest.java
new file mode 100644
index 0000000..92c2353
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertListenerMessageFactoryTest.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.alerting;
+
+import static org.apache.geode.internal.DataSerializableFixedID.ALERT_LISTENER_MESSAGE;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+import static org.mockito.Mockito.mock;
+
+import java.util.Date;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.admin.remote.AlertListenerMessage;
+import org.apache.geode.test.junit.categories.AlertingTest;
+
+/**
+ * Unit tests for {@link AlertListenerMessageFactory}.
+ */
+@Category(AlertingTest.class)
+public class AlertListenerMessageFactoryTest {
+
+  private DistributedMember member;
+  private AlertListenerMessageFactory alertListenerMessageFactory;
+
+  @Before
+  public void setUp() {
+    member = mock(InternalDistributedMember.class);
+    alertListenerMessageFactory = new AlertListenerMessageFactory();
+  }
+
+  @Test
+  public void createAlertListenerMessage() {
+    AlertListenerMessage message = alertListenerMessageFactory.createAlertListenerMessage(member,
+        AlertLevel.WARNING, new Date(), "connectionName", "threadName", "formattedMessage", null);
+
+    assertThat(message).isNotNull();
+    assertThat(message.getDSFID()).isEqualTo(ALERT_LISTENER_MESSAGE);
+    assertThat(message.sendViaUDP()).isTrue();
+  }
+
+  @Test
+  public void createAlertListenerMessage_requiresInternalDistributedMember() {
+    member = mock(DistributedMember.class);
+
+    Throwable thrown = catchThrowable(
+        () -> alertListenerMessageFactory.createAlertListenerMessage(member, AlertLevel.WARNING,
+            new Date(), "connectionName", "threadName", "formattedMessage", null));
+
+    assertThat(thrown).isNotNull().isInstanceOf(IllegalArgumentException.class);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertMessagingTest.java b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertMessagingTest.java
new file mode 100644
index 0000000..907fa9f
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertMessagingTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.Date;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.admin.remote.AlertListenerMessage;
+import org.apache.geode.test.junit.categories.AlertingTest;
+
+/**
+ * Unit tests for {@link AlertMessaging}.
+ */
+@Category(AlertingTest.class)
+public class AlertMessagingTest {
+
+  private InternalDistributedSystem system;
+  private InternalDistributedMember localMember;
+  private DistributionConfig config;
+  private DistributionManager dm;
+  private AlertListenerMessageFactory alertListenerMessageFactory;
+  private AlertListenerMessage alertListenerMessage;
+
+  private AlertMessaging alertMessaging;
+
+  @Before
+  public void setUp() {
+    system = mock(InternalDistributedSystem.class);
+    localMember = mock(InternalDistributedMember.class);
+    config = mock(DistributionConfig.class);
+    dm = mock(ClusterDistributionManager.class);
+    alertListenerMessageFactory = mock(AlertListenerMessageFactory.class);
+    alertListenerMessage = mock(AlertListenerMessage.class);
+
+    alertMessaging = spy(new AlertMessaging(system, dm, alertListenerMessageFactory));
+
+    when(system.getConfig()).thenReturn(config);
+    when(system.getDistributedMember()).thenReturn(localMember);
+    when(config.getName()).thenReturn("name");
+
+    when(alertListenerMessageFactory.createAlertListenerMessage(
+        any(DistributedMember.class), any(AlertLevel.class), any(Date.class),
+        anyString(), anyString(), anyString(), anyString())).thenReturn(alertListenerMessage);
+
+    doNothing().when(alertMessaging).processAlertListenerMessage(any(AlertListenerMessage.class));
+  }
+
+  @Test
+  public void sendAlertProcessesMessageIfMemberIsLocal() {
+    alertMessaging.sendAlert(localMember, AlertLevel.WARNING, new Date(), "threadName",
+        "formattedMessage", "stackTrace");
+
+    verify(alertMessaging).processAlertListenerMessage(eq(alertListenerMessage));
+  }
+
+  @Test
+  public void sendAlertSendsMessageIfMemberIsRemote() {
+    DistributedMember remoteMember = mock(DistributedMember.class);
+
+    alertMessaging.sendAlert(remoteMember, AlertLevel.WARNING, new Date(), "threadName",
+        "formattedMessage", "stackTrace");
+
+    verify(dm).putOutgoing(eq(alertListenerMessage));
+  }
+
+  @Test
+  public void processAlertListenerMessage_requires_ClusterDistributionManager() {
+    dm = mock(DistributionManager.class);
+
+    alertMessaging = new AlertMessaging(system, dm, alertListenerMessageFactory);
+
+    Throwable thrown =
+        catchThrowable(() -> alertMessaging.processAlertListenerMessage(alertListenerMessage));
+    assertThat(thrown).isInstanceOf(IllegalArgumentException.class);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertingActionTest.java b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertingActionTest.java
new file mode 100644
index 0000000..1d4a7ff
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertingActionTest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static org.apache.geode.internal.alerting.AlertingAction.execute;
+import static org.apache.geode.internal.alerting.AlertingAction.isThreadAlerting;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.hamcrest.Matchers.is;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ErrorCollector;
+
+import org.apache.geode.test.junit.categories.AlertingTest;
+
+/**
+ * Unit tests for {@link AlertingAction}.
+ */
+@Category(AlertingTest.class)
+public class AlertingActionTest {
+
+  @Rule
+  public ErrorCollector errorCollector = new ErrorCollector();
+
+  @Test
+  public void isThreadAlertingIsFalseByDefault() {
+    assertThat(isThreadAlerting()).isFalse();
+  }
+
+  @Test
+  public void isThreadAlertingIsTrueWhileExecuting() {
+    execute(() -> errorCollector.checkThat(isThreadAlerting(), is(true)));
+  }
+
+  @Test
+  public void isThreadAlertingIsFalseAfterExecuting() {
+    execute(() -> System.out.println("hi"));
+
+    assertThat(isThreadAlerting()).isFalse();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertingProviderRegistryTest.java b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertingProviderRegistryTest.java
new file mode 100644
index 0000000..a4efb1a
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertingProviderRegistryTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static org.apache.geode.internal.alerting.AlertingProviderRegistry.getNullAlertingProvider;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.AlertingTest;
+
+/**
+ * Unit tests for {@link AlertingProviderRegistry}.
+ */
+@Category(AlertingTest.class)
+public class AlertingProviderRegistryTest {
+
+  private AlertingProvider provider;
+
+  private AlertingProviderRegistry alertingProviderRegistry;
+
+  @Before
+  public void setUp() {
+    provider = mock(AlertingProvider.class);
+
+    alertingProviderRegistry = new AlertingProviderRegistry();
+  }
+
+  @Test
+  public void getAlertingProviderIsNullProviderBeforeRegister() {
+    assertThat(alertingProviderRegistry.getAlertingProvider()).isSameAs(getNullAlertingProvider());
+  }
+
+  @Test
+  public void getAlertingProviderIsSameAsRegisteredProvider() {
+    alertingProviderRegistry.registerAlertingProvider(provider);
+
+    assertThat(alertingProviderRegistry.getAlertingProvider()).isSameAs(provider);
+  }
+
+  @Test
+  public void unregisterDoesNothingIfNotRegistered() {
+    alertingProviderRegistry.unregisterAlertingProvider(provider);
+
+    assertThat(alertingProviderRegistry.getAlertingProvider()).isSameAs(getNullAlertingProvider());
+  }
+
+  @Test
+  public void unregisterWrongProviderDoesNothing() {
+    alertingProviderRegistry.registerAlertingProvider(provider);
+
+    alertingProviderRegistry.unregisterAlertingProvider(mock(AlertingProvider.class));
+
+    assertThat(alertingProviderRegistry.getAlertingProvider()).isSameAs(provider);
+  }
+
+  @Test
+  public void unregisterDoesNothingIfNullProvider() {
+    alertingProviderRegistry.unregisterAlertingProvider(getNullAlertingProvider());
+
+    assertThat(alertingProviderRegistry.getAlertingProvider()).isSameAs(getNullAlertingProvider());
+  }
+
+  @Test
+  public void unregisterRemovesRegisteredProvider() {
+    alertingProviderRegistry.registerAlertingProvider(provider);
+
+    alertingProviderRegistry.unregisterAlertingProvider(provider);
+
+    assertThat(alertingProviderRegistry.getAlertingProvider()).isSameAs(getNullAlertingProvider());
+  }
+
+  @Test
+  public void registerAddsAlertingProvider() {
+    alertingProviderRegistry.registerAlertingProvider(provider);
+
+    assertThat(alertingProviderRegistry.getAlertingProvider()).isSameAs(provider);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertingServiceTest.java b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertingServiceTest.java
new file mode 100644
index 0000000..089579d
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/alerting/AlertingServiceTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.test.junit.categories.AlertingTest;
+
+/**
+ * Unit tests for {@link AlertingService}.
+ */
+@Category(AlertingTest.class)
+public class AlertingServiceTest {
+
+  private AlertingProviderRegistry registry;
+  private AlertingProvider provider;
+  private DistributedMember member;
+
+  private AlertingService alertingService;
+
+  @Before
+  public void setUp() {
+    registry = mock(AlertingProviderRegistry.class);
+    provider = mock(AlertingProvider.class);
+    member = mock(DistributedMember.class);
+
+    when(registry.getAlertingProvider()).thenReturn(provider);
+    when(provider.hasAlertListener(eq(member), eq(AlertLevel.WARNING))).thenReturn(true);
+    when(provider.removeAlertListener(eq(member))).thenReturn(true);
+
+    alertingService = new AlertingService(registry);
+  }
+
+  @Test
+  public void hasAlertListenerDelegates() {
+    assertThat(alertingService.hasAlertListener(member, AlertLevel.WARNING)).isTrue();
+
+    verify(provider).hasAlertListener(eq(member), eq(AlertLevel.WARNING));
+  }
+
+  @Test
+  public void addAlertListenerDelegates() {
+    alertingService.addAlertListener(member, AlertLevel.WARNING);
+
+    verify(provider).addAlertListener(eq(member), eq(AlertLevel.WARNING));
+  }
+
+  @Test
+  public void removeAlertListenerDelegates() {
+    assertThat(alertingService.removeAlertListener(member)).isTrue();
+
+    verify(provider).removeAlertListener(eq(member));
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/alerting/NullAlertingProviderTest.java b/geode-core/src/test/java/org/apache/geode/internal/alerting/NullAlertingProviderTest.java
new file mode 100644
index 0000000..8d31f0d
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/alerting/NullAlertingProviderTest.java
@@ -0,0 +1,48 @@
+/*
+ * 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.geode.internal.alerting;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.test.junit.categories.AlertingTest;
+
+/**
+ * Unit test for {@link NullAlertingProvider}.
+ */
+@Category(AlertingTest.class)
+public class NullAlertingProviderTest {
+
+  private DistributedMember member;
+  private NullAlertingProvider nullAlertingProvider;
+
+  @Before
+  public void setUp() {
+    member = mock(DistributedMember.class);
+    nullAlertingProvider = new NullAlertingProvider();
+  }
+
+  @Test
+  public void doesNothingButReturnFalse() {
+    nullAlertingProvider.addAlertListener(member, AlertLevel.WARNING);
+    assertThat(nullAlertingProvider.hasAlertListener(member, AlertLevel.WARNING)).isFalse();
+    assertThat(nullAlertingProvider.removeAlertListener(member)).isFalse();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/logging/log4j/AlertAppenderTest.java b/geode-core/src/test/java/org/apache/geode/internal/logging/log4j/AlertAppenderTest.java
index e36f18e..0ba7e45 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/logging/log4j/AlertAppenderTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/logging/log4j/AlertAppenderTest.java
@@ -14,246 +14,154 @@
  */
 package org.apache.geode.internal.logging.log4j;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArrayList;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
 
 import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.core.config.LoggerConfig;
-import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.DurableClientAttributes;
-import org.apache.geode.distributed.Role;
-import org.apache.geode.internal.admin.Alert;
-import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.alerting.AlertLevel;
+import org.apache.geode.internal.alerting.AlertingProvider;
+import org.apache.geode.test.junit.categories.AlertingTest;
 import org.apache.geode.test.junit.categories.LoggingTest;
 
 /**
  * Unit tests for {@link AlertAppender}.
  */
-@Category(LoggingTest.class)
+@Category({AlertingTest.class, LoggingTest.class})
 public class AlertAppenderTest {
 
-  private List<DistributedMember> members;
-  private Level previousLogLevel;
+  private DistributedMember member;
+
+  private AlertAppender alertAppender;
+  private AlertingProvider asAlertingProvider;
+
+  @Rule
+  public TestName testName = new TestName();
 
   @Before
   public void setUp() {
-    members = new ArrayList<>();
-    previousLogLevel = LogService.getBaseLogLevel();
+    member = mock(DistributedMember.class);
+
+    alertAppender = new AlertAppender(testName.getMethodName(), null, null);
+    asAlertingProvider = alertAppender;
+  }
+
+  @Test
+  public void alertListenersIsEmptyByDefault() {
+    assertThat(alertAppender.getAlertListeners()).isEmpty();
+  }
+
+  @Test
+  public void hasAlertListenerReturnsFalseByDefault() {
+    asAlertingProvider.hasAlertListener(member, AlertLevel.WARNING);
+  }
+
+  @Test
+  public void addAlertListenerAddsListener() {
+    asAlertingProvider.addAlertListener(member, AlertLevel.WARNING);
+    assertThat(alertAppender.getAlertListeners())
+        .contains(new AlertListener(Level.WARN, member));
   }
 
-  @After
-  public void tearDown() {
-    LogService.setBaseLogLevel(previousLogLevel);
-    if (!members.isEmpty()) {
-      for (DistributedMember member : members) {
-        AlertAppender.getInstance().removeAlertListener(member);
-      }
-      members.clear();
-    }
+  @Test
+  public void hasAlertListenerReturnsTrueIfListenerExists() {
+    asAlertingProvider.addAlertListener(member, AlertLevel.WARNING);
+    assertThat(asAlertingProvider.hasAlertListener(member, AlertLevel.WARNING)).isTrue();
   }
 
-  /**
-   * Verify that adding/removing/replacing listeners works correctly.
-   */
   @Test
-  public void testListenerHandling() throws Exception {
-    DistributedMember member1 = createNamedDistributedMember("Member1");
-    DistributedMember member2 = createNamedDistributedMember("Member2");
-    DistributedMember member3 = createNamedDistributedMember("Member3");
-    DistributedMember member4 = createNamedDistributedMember("Member4");
-    DistributedMember member5 = createNamedDistributedMember("Member5");
-    DistributedMember member6 = createNamedDistributedMember("Member6");
-
-    LogService.setBaseLogLevel(Level.WARN);
-
-    AlertAppender.getInstance().addAlertListener(member1, Alert.SEVERE);
-    AlertAppender.getInstance().addAlertListener(member2, Alert.WARNING);
-    AlertAppender.getInstance().addAlertListener(member3, Alert.ERROR);
-    AlertAppender.getInstance().addAlertListener(member4, Alert.ERROR);
-    AlertAppender.getInstance().addAlertListener(member5, Alert.WARNING);
-    AlertAppender.getInstance().addAlertListener(member6, Alert.SEVERE);
-
-    Field listenersField = AlertAppender.getInstance().getClass().getDeclaredField("listeners");
-    listenersField.setAccessible(true);
-
-    CopyOnWriteArrayList<AlertAppender.Listener> listeners =
-        (CopyOnWriteArrayList<AlertAppender.Listener>) listenersField
-            .get(AlertAppender.getInstance());
-
-    // Verify add
-    assertSame(member5, listeners.get(0).getMember());
-    assertSame(member2, listeners.get(1).getMember());
-    assertSame(member4, listeners.get(2).getMember());
-    assertSame(member3, listeners.get(3).getMember());
-    assertSame(member6, listeners.get(4).getMember());
-    assertSame(member1, listeners.get(5).getMember());
-    assertSame(6, listeners.size());
-
-    // Verify replace with same level
-    AlertAppender.getInstance().addAlertListener(member5, Alert.WARNING);
-    assertSame(member5, listeners.get(0).getMember());
-    assertSame(member2, listeners.get(1).getMember());
-    assertSame(member4, listeners.get(2).getMember());
-    assertSame(member3, listeners.get(3).getMember());
-    assertSame(member6, listeners.get(4).getMember());
-    assertSame(member1, listeners.get(5).getMember());
-    assertSame(6, listeners.size());
-
-    // Verify replace with difference level
-    AlertAppender.getInstance().addAlertListener(member5, Alert.SEVERE);
-    assertSame(member2, listeners.get(0).getMember());
-    assertSame(member4, listeners.get(1).getMember());
-    assertSame(member3, listeners.get(2).getMember());
-    assertSame(member5, listeners.get(3).getMember());
-    assertSame(member6, listeners.get(4).getMember());
-    assertSame(member1, listeners.get(5).getMember());
-    assertSame(6, listeners.size());
-
-    // Verify remove
-    assertTrue(AlertAppender.getInstance().removeAlertListener(member3));
-    assertSame(member2, listeners.get(0).getMember());
-    assertSame(member4, listeners.get(1).getMember());
-    assertSame(member5, listeners.get(2).getMember());
-    assertSame(member6, listeners.get(3).getMember());
-    assertSame(member1, listeners.get(4).getMember());
-    assertSame(5, listeners.size());
-
-    assertTrue(AlertAppender.getInstance().removeAlertListener(member1));
-    assertTrue(AlertAppender.getInstance().removeAlertListener(member2));
-    assertFalse(AlertAppender.getInstance().removeAlertListener(member3));
-    assertTrue(AlertAppender.getInstance().removeAlertListener(member4));
-    assertTrue(AlertAppender.getInstance().removeAlertListener(member5));
-    assertTrue(AlertAppender.getInstance().removeAlertListener(member6));
+  public void removeAlertListenerDoesNothingByDefault() {
+    asAlertingProvider.removeAlertListener(member);
+    assertThat(alertAppender.getAlertListeners()).isEmpty();
   }
 
-  /**
-   * Verifies that the appender is correctly added and removed from the Log4j configuration and that
-   * when the configuration is changed the appender is still there.
-   */
   @Test
-  public void testAppenderToConfigHandling() {
-    LogService.setBaseLogLevel(Level.WARN);
+  public void removeAlertListenerDoesNothingIfMemberDoesNotMatch() {
+    asAlertingProvider.addAlertListener(member, AlertLevel.WARNING);
+
+    asAlertingProvider.removeAlertListener(mock(DistributedMember.class));
+
+    assertThat(asAlertingProvider.hasAlertListener(member, AlertLevel.WARNING)).isTrue();
+  }
 
-    String appenderName = AlertAppender.getInstance().getName();
-    AppenderContext appenderContext = LogService.getAppenderContext();
+  @Test
+  public void removeAlertListenerRemovesListener() {
+    asAlertingProvider.addAlertListener(member, AlertLevel.WARNING);
 
-    LoggerConfig loggerConfig = appenderContext.getLoggerConfig();
+    asAlertingProvider.removeAlertListener(member);
 
-    // Find out home many appenders exist before we get started
-    int startingSize = loggerConfig.getAppenders().size();
+    assertThat(asAlertingProvider.hasAlertListener(member, AlertLevel.WARNING)).isFalse();
+  }
 
-    // Add a listener and verify that the appender was added to log4j
-    DistributedMember member1 = createNamedDistributedMember("Member1");
-    AlertAppender.getInstance().addAlertListener(member1, Alert.SEVERE);
-    assertEquals(loggerConfig.getAppenders().values().toString(), startingSize + 1,
-        loggerConfig.getAppenders().size());
-    assertTrue(loggerConfig.getAppenders().containsKey(appenderName));
+  @Test
+  public void addAlertListenerWithAlertLevelNoneDoesNothing() {
+    asAlertingProvider.addAlertListener(member, AlertLevel.NONE);
+    assertThat(alertAppender.getAlertListeners()).isEmpty();
+  }
 
-    // Add another listener and verify that there's still only 1 alert appender
-    DistributedMember member2 = createNamedDistributedMember("Member1");
-    AlertAppender.getInstance().addAlertListener(member2, Alert.SEVERE);
-    assertEquals(startingSize + 1, loggerConfig.getAppenders().size());
+  @Test
+  public void hasAlertListenerReturnsFalseIfAlertLevelIsNone() {
+    asAlertingProvider.addAlertListener(member, AlertLevel.WARNING);
+    assertThat(asAlertingProvider.hasAlertListener(member, AlertLevel.NONE)).isFalse();
+  }
 
-    // Modify the config and verify that the appender still exists
-    assertEquals(Level.WARN, LogService.getLogger(LogService.BASE_LOGGER_NAME).getLevel());
+  @Test
+  public void addAlertListenerOrdersByAscendingAlertLevel() {
+    DistributedMember member1 = mock(DistributedMember.class);
+    DistributedMember member2 = mock(DistributedMember.class);
+    DistributedMember member3 = mock(DistributedMember.class);
 
-    LogService.setBaseLogLevel(Level.INFO);
+    asAlertingProvider.addAlertListener(member3, AlertLevel.WARNING);
+    asAlertingProvider.addAlertListener(member1, AlertLevel.SEVERE);
+    asAlertingProvider.addAlertListener(member2, AlertLevel.ERROR);
 
-    assertEquals(Level.INFO, LogService.getLogger(LogService.BASE_LOGGER_NAME).getLevel());
-    loggerConfig = appenderContext.getLoggerConfig();
-    assertEquals(startingSize + 1, loggerConfig.getAppenders().size());
-    assertTrue(loggerConfig.getAppenders().containsKey(appenderName));
+    AlertListener listener1 = new AlertListener(Level.WARN, member3);
+    AlertListener listener2 = new AlertListener(Level.ERROR, member2);
+    AlertListener listener3 = new AlertListener(Level.FATAL, member1);
 
-    // Remove the listeners and verify that the appender was removed from log4j
-    assertTrue(AlertAppender.getInstance().removeAlertListener(member2));
-    assertFalse(AlertAppender.getInstance().removeAlertListener(member1));
-    assertEquals(startingSize, loggerConfig.getAppenders().size());
-    assertFalse(loggerConfig.getAppenders().containsKey(appenderName));
+    assertThat(alertAppender.getAlertListeners()).containsExactly(listener1, listener2,
+        listener3);
   }
 
-  private DistributedMember createNamedDistributedMember(final String name) {
-    return new NamedDistributedMember(name);
+  @Test
+  public void removeAlertListenerMaintainsExistingOrder() {
+    DistributedMember member1 = mock(DistributedMember.class);
+    DistributedMember member2 = mock(DistributedMember.class);
+    DistributedMember member3 = mock(DistributedMember.class);
+
+    asAlertingProvider.addAlertListener(member3, AlertLevel.WARNING);
+    asAlertingProvider.addAlertListener(member1, AlertLevel.SEVERE);
+    asAlertingProvider.addAlertListener(member2, AlertLevel.ERROR);
+
+    AlertListener listener1 = new AlertListener(Level.WARN, member3);
+    AlertListener listener3 = new AlertListener(Level.FATAL, member1);
+
+    assertThat(alertAppender.removeAlertListener(member2)).isTrue();
+
+    assertThat(alertAppender.getAlertListeners()).containsExactly(listener1, listener3);
   }
 
-  private static class NamedDistributedMember implements DistributedMember {
-
-    private final String name;
-
-    NamedDistributedMember(final String name) {
-      this.name = name;
-    }
-
-    @Override
-    public String getName() {
-      return name;
-    }
-
-    @Override
-    public String getHost() {
-      return "";
-    }
-
-    @Override
-    public Set<Role> getRoles() {
-      return null;
-    }
-
-    @Override
-    public int getProcessId() {
-      return 0;
-    }
-
-    @Override
-    public String getId() {
-      return name;
-    }
-
-    @Override
-    public int compareTo(final DistributedMember o) {
-      return getName().compareTo(o.getName());
-    }
-
-    @Override
-    public DurableClientAttributes getDurableClientAttributes() {
-      return null;
-    }
-
-    @Override
-    public List<String> getGroups() {
-      return Collections.emptyList();
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-      if (!(obj instanceof NamedDistributedMember)) {
-        return false;
-      }
-      NamedDistributedMember other = (NamedDistributedMember) obj;
-      return getName().equals(other.getName());
-    }
-
-    @Override
-    public int hashCode() {
-      return getHost().hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return getClass().getSimpleName() + " [name=" + name + "]";
-    }
+  @Test
+  public void addAlertListenerOrdersByDescendingAddIfAlertLevelMatches() {
+    DistributedMember member1 = mock(DistributedMember.class);
+    DistributedMember member2 = mock(DistributedMember.class);
+    DistributedMember member3 = mock(DistributedMember.class);
+
+    asAlertingProvider.addAlertListener(member3, AlertLevel.WARNING);
+    asAlertingProvider.addAlertListener(member1, AlertLevel.WARNING);
+    asAlertingProvider.addAlertListener(member2, AlertLevel.WARNING);
+
+    AlertListener listener1 = new AlertListener(Level.WARN, member2);
+    AlertListener listener2 = new AlertListener(Level.WARN, member1);
+    AlertListener listener3 = new AlertListener(Level.WARN, member3);
+
+    assertThat(alertAppender.getAlertListeners()).containsExactly(listener1, listener2,
+        listener3);
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/logging/log4j/AlertLevelConverterTest.java b/geode-core/src/test/java/org/apache/geode/internal/logging/log4j/AlertLevelConverterTest.java
new file mode 100644
index 0000000..223f7f2
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/logging/log4j/AlertLevelConverterTest.java
@@ -0,0 +1,144 @@
+/*
+ * 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.geode.internal.logging.log4j;
+
+import static org.apache.geode.internal.logging.log4j.AlertLevelConverter.fromLevel;
+import static org.apache.geode.internal.logging.log4j.AlertLevelConverter.hasAlertLevel;
+import static org.apache.geode.internal.logging.log4j.AlertLevelConverter.toLevel;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import org.apache.logging.log4j.Level;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.internal.alerting.AlertLevel;
+import org.apache.geode.test.junit.categories.AlertingTest;
+import org.apache.geode.test.junit.categories.LoggingTest;
+
+/**
+ * Unit tests for {@link AlertLevelConverter}.
+ */
+@Category({AlertingTest.class, LoggingTest.class})
+public class AlertLevelConverterTest {
+
+  @Test
+  public void toLevel_AlertSevere_returnsLevelFatal() {
+    assertThat(toLevel(AlertLevel.SEVERE)).isEqualTo(Level.FATAL);
+  }
+
+  @Test
+  public void toLevel_AlertError_returnsLevelFatal() {
+    assertThat(toLevel(AlertLevel.ERROR)).isEqualTo(Level.ERROR);
+  }
+
+  @Test
+  public void toLevel_AlertWarning_returnsLevelFatal() {
+    assertThat(toLevel(AlertLevel.WARNING)).isEqualTo(Level.WARN);
+  }
+
+  @Test
+  public void toLevel_AlertOff_returnsLevelFatal() {
+    assertThat(toLevel(AlertLevel.NONE)).isEqualTo(Level.OFF);
+  }
+
+  @Test
+  public void toAlertLevel_LevelFatal_returnsAlertSevere() {
+    assertThat(fromLevel(Level.FATAL)).isEqualTo(AlertLevel.SEVERE);
+  }
+
+  @Test
+  public void toAlertLevel_LevelError_returnsAlertError() {
+    assertThat(fromLevel(Level.ERROR)).isEqualTo(AlertLevel.ERROR);
+  }
+
+  @Test
+  public void toAlertLevel_LevelWarn_returnsAlertWarning() {
+    assertThat(fromLevel(Level.WARN)).isEqualTo(AlertLevel.WARNING);
+  }
+
+  @Test
+  public void toAlertLevel_LevelOff_returnsAlertNone() {
+    assertThat(fromLevel(Level.OFF)).isEqualTo(AlertLevel.NONE);
+  }
+
+  @Test
+  public void toAlertLevel_LevelAll_throwsIllegalArgumentException() {
+    assertThatThrownBy(() -> fromLevel(Level.ALL))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("No matching AlertLevel for Log4J2 Level " + Level.ALL + ".");
+  }
+
+  @Test
+  public void toAlertLevel_LevelTrace_throwsIllegalArgumentException() {
+    assertThatThrownBy(() -> fromLevel(Level.TRACE))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("No matching AlertLevel for Log4J2 Level " + Level.TRACE + ".");
+  }
+
+  @Test
+  public void toAlertLevel_LevelDebug_throwsIllegalArgumentException() {
+    assertThatThrownBy(() -> fromLevel(Level.DEBUG))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("No matching AlertLevel for Log4J2 Level " + Level.DEBUG + ".");
+  }
+
+  @Test
+  public void toAlertLevel_LevelInfo_throwsIllegalArgumentException() {
+    assertThatThrownBy(() -> fromLevel(Level.INFO))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("No matching AlertLevel for Log4J2 Level " + Level.INFO + ".");
+  }
+
+  @Test
+  public void hasAlertLevel_LevelFatal_returnsTrue() {
+    assertThat(hasAlertLevel(Level.FATAL)).isTrue();
+  }
+
+  @Test
+  public void hasAlertLevel_LevelError_returnsTrue() {
+    assertThat(hasAlertLevel(Level.ERROR)).isTrue();
+  }
+
+  @Test
+  public void hasAlertLevel_LevelWarn_returnsTrue() {
+    assertThat(hasAlertLevel(Level.WARN)).isTrue();
+  }
+
+  @Test
+  public void hasAlertLevel_LevelOff_returnsTrue() {
+    assertThat(hasAlertLevel(Level.OFF)).isTrue();
+  }
+
+  @Test
+  public void hasAlertLevel_LevelAll_returnsFalse() {
+    assertThat(hasAlertLevel(Level.ALL)).isFalse();
+  }
+
+  @Test
+  public void hasAlertLevel_LevelTrace_returnsFalse() {
+    assertThat(hasAlertLevel(Level.TRACE)).isFalse();
+  }
+
+  @Test
+  public void hasAlertLevel_LevelDebug_returnsFalse() {
+    assertThat(hasAlertLevel(Level.DEBUG)).isFalse();
+  }
+
+  @Test
+  public void hasAlertLevel_LevelInfo_returnsFalse() {
+    assertThat(hasAlertLevel(Level.INFO)).isFalse();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/logging/log4j/AlertLevelTest.java b/geode-core/src/test/java/org/apache/geode/internal/logging/log4j/AlertLevelTest.java
deleted file mode 100644
index 06b7b9c..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/logging/log4j/AlertLevelTest.java
+++ /dev/null
@@ -1,137 +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.geode.internal.logging.log4j;
-
-import static org.apache.geode.internal.logging.log4j.AlertLevel.alertLevelToLogLevel;
-import static org.apache.geode.internal.logging.log4j.AlertLevel.logLevelToAlertLevel;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-import org.apache.logging.log4j.Level;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.internal.admin.Alert;
-import org.apache.geode.test.junit.categories.LoggingTest;
-
-/**
- * Unit tests for {@link AlertLevel}.
- */
-@Category(LoggingTest.class)
-public class AlertLevelTest {
-
-  @Test
-  public void alertLevelToLogLevel_AlertSevere_returnsLevelFatal() {
-    assertThat(alertLevelToLogLevel(Alert.SEVERE)).isEqualTo(Level.FATAL.intLevel());
-  }
-
-  @Test
-  public void alertLevelToLogLevel_AlertError_returnsLevelFatal() {
-    assertThat(alertLevelToLogLevel(Alert.ERROR)).isEqualTo(Level.ERROR.intLevel());
-  }
-
-  @Test
-  public void alertLevelToLogLevel_AlertWarning_returnsLevelFatal() {
-    assertThat(alertLevelToLogLevel(Alert.WARNING)).isEqualTo(Level.WARN.intLevel());
-  }
-
-  @Test
-  public void alertLevelToLogLevel_AlertOff_returnsLevelFatal() {
-    assertThat(alertLevelToLogLevel(Alert.OFF)).isEqualTo(Level.OFF.intLevel());
-  }
-
-  @Test
-  public void alertLevelToLogLevel_AlertAll_throwsIllegalArgumentException() {
-    assertThatThrownBy(() -> alertLevelToLogLevel(Alert.ALL))
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Unknown Alert level [" + Alert.ALL + "].");
-  }
-
-  @Test
-  public void alertLevelToLogLevel_AlertFinest_throwsIllegalArgumentException() {
-    assertThatThrownBy(() -> alertLevelToLogLevel(Alert.FINEST))
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Unknown Alert level [" + Alert.FINEST + "].");
-  }
-
-  @Test
-  public void alertLevelToLogLevel_AlertFine_throwsIllegalArgumentException() {
-    assertThatThrownBy(() -> alertLevelToLogLevel(Alert.FINE))
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Unknown Alert level [" + Alert.FINE + "].");
-  }
-
-  @Test
-  public void alertLevelToLogLevel_AlertConfig_throwsIllegalArgumentException() {
-    assertThatThrownBy(() -> alertLevelToLogLevel(Alert.CONFIG))
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Unknown Alert level [" + Alert.CONFIG + "].");
-  }
-
-  @Test
-  public void alertLevelToLogLevel_AlertInfo_throwsIllegalArgumentException() {
-    assertThatThrownBy(() -> alertLevelToLogLevel(Alert.INFO))
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Unknown Alert level [" + Alert.INFO + "].");
-  }
-
-  @Test
-  public void logLevelToAlertLevel_LevelFatal_returnsAlertSevere() {
-    assertThat(logLevelToAlertLevel(Level.FATAL)).isEqualTo(Alert.SEVERE);
-  }
-
-  @Test
-  public void logLevelToAlertLevel_LevelError_returnsAlertError() {
-    assertThat(logLevelToAlertLevel(Level.ERROR)).isEqualTo(Alert.ERROR);
-  }
-
-  @Test
-  public void logLevelToAlertLevel_LevelWarn_returnsAlertWarning() {
-    assertThat(logLevelToAlertLevel(Level.WARN)).isEqualTo(Alert.WARNING);
-  }
-
-  @Test
-  public void logLevelToAlertLevel_LevelOff_returnsAlertOff() {
-    assertThat(logLevelToAlertLevel(Level.OFF)).isEqualTo(Alert.OFF);
-  }
-
-  @Test
-  public void logLevelToAlertLevel_LevelAll_throwsIllegalArgumentException() {
-    assertThatThrownBy(() -> logLevelToAlertLevel(Level.ALL))
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Unknown LOG4J2 Level [" + Level.ALL + "].");
-  }
-
-  @Test
-  public void logLevelToAlertLevel_LevelTrace_throwsIllegalArgumentException() {
-    assertThatThrownBy(() -> logLevelToAlertLevel(Level.TRACE))
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Unknown LOG4J2 Level [" + Level.TRACE + "].");
-  }
-
-  @Test
-  public void logLevelToAlertLevel_LevelDebug_throwsIllegalArgumentException() {
-    assertThatThrownBy(() -> logLevelToAlertLevel(Level.DEBUG))
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Unknown LOG4J2 Level [" + Level.DEBUG + "].");
-  }
-
-  @Test
-  public void logLevelToAlertLevel_LevelInfo_throwsIllegalArgumentException() {
-    assertThatThrownBy(() -> logLevelToAlertLevel(Level.INFO))
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Unknown LOG4J2 Level [" + Level.INFO + "].");
-  }
-}