You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ap...@apache.org on 2023/12/25 14:42:48 UTC

(ignite-3) branch main updated: IGNITE-21065 Enhance granularity of authentication events (#2962)

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

apkhmv pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new bf7dca5708 IGNITE-21065 Enhance granularity of authentication events (#2962)
bf7dca5708 is described below

commit bf7dca5708be79659f8fa176a235f240a6847a8e
Author: Ivan Gagarkin <ga...@gmail.com>
AuthorDate: Mon Dec 25 21:42:42 2023 +0700

    IGNITE-21065 Enhance granularity of authentication events (#2962)
    
    * Add support for new event types: USER_UPDATED, USER_REMOVED.
    * Replace configuration parsing with listening for configuration changes to trigger events.
---
 .../java/org/apache/ignite/lang/ErrorGroups.java   |   3 +
 modules/client-handler/build.gradle                |   2 +
 .../ignite/client/handler/ItClientHandlerTest.java |   3 +-
 .../apache/ignite/client/handler/TestServer.java   |  17 +-
 .../ignite/client/handler/ClientHandlerModule.java |   3 -
 .../handler/ClientInboundMessageHandler.java       | 126 ++++++--
 .../handler/ClientInboundMessageHandlerTest.java   | 275 -----------------
 .../client/handler/DummyAuthenticationManager.java |  60 ++++
 .../ignite/client/ClientAuthenticationTest.java    |  12 +-
 .../ignite/client/TestClientHandlerModule.java     |  19 +-
 .../java/org/apache/ignite/client/TestServer.java  |  47 +--
 .../notifications/ConfigurationNotifier.java       |   5 +-
 .../notifications/ConfigurationListenerTest.java   | 105 +++++++
 .../testframework/ConfigurationExtension.java      |  24 +-
 .../testframework/InjectConfiguration.java         |   8 +
 .../cluster/ItClusterManagementControllerTest.java |   4 +-
 .../ClusterConfigurationControllerTest.java        |   9 +-
 .../NodeConfigurationControllerTest.java           |   9 +-
 .../app/client/ItThinClientAuthenticationTest.java | 206 +++++++++++++
 .../org/apache/ignite/internal/app/IgniteImpl.java |  12 +-
 .../authentication/AuthenticationManager.java      |  24 +-
 .../security/authentication/UserDetails.java       |   2 +
 .../authentication/event/AuthenticationEvent.java  |  18 +-
 ...ype.java => AuthenticationEventParameters.java} |  11 +-
 ... => AuthenticationProviderEventParameters.java} |  26 +-
 .../event/AuthenticationSwitchedParameters.java    |  48 +++
 .../UserEventParameters.java}                      |  32 +-
 .../authentication/AuthenticationManagerImpl.java  | 183 ++++++------
 .../AuthenticationProviderEqualityVerifier.java    |  86 ------
 .../authentication/AuthenticationUtils.java        |  56 ++++
 .../authentication/AuthenticatorFactory.java       |  12 +-
 .../SecurityConfigurationModule.java               |   6 +-
 .../basic/BasicProviderNotFoundException.java}     |  18 +-
 .../event/AuthenticationProviderEventFactory.java  |  97 ++++++
 .../event/SecurityEnabledDisabledEventFactory.java |  48 +++
 .../authentication/event/UserEventFactory.java     |  58 ++++
 .../AuthenticationManagerImplTest.java             | 326 +++++++++++++--------
 ...mAuthenticationProviderConfigurationSchema.java |   4 +
 38 files changed, 1256 insertions(+), 748 deletions(-)

diff --git a/modules/api/src/main/java/org/apache/ignite/lang/ErrorGroups.java b/modules/api/src/main/java/org/apache/ignite/lang/ErrorGroups.java
index c21f07d464..3d7e3d1211 100755
--- a/modules/api/src/main/java/org/apache/ignite/lang/ErrorGroups.java
+++ b/modules/api/src/main/java/org/apache/ignite/lang/ErrorGroups.java
@@ -495,6 +495,9 @@ public class ErrorGroups {
 
         /** Authentication error caused by invalid credentials. */
         public static final int INVALID_CREDENTIALS_ERR = AUTHENTICATION_ERR_GROUP.registerErrorCode((short) 2);
+
+        /** Basic authentication provider is not found. */
+        public static final int BASIC_PROVIDER_ERR = AUTHENTICATION_ERR_GROUP.registerErrorCode((short) 3);
     }
 
     /**
diff --git a/modules/client-handler/build.gradle b/modules/client-handler/build.gradle
index 381aa8a5a5..18e350e975 100644
--- a/modules/client-handler/build.gradle
+++ b/modules/client-handler/build.gradle
@@ -73,6 +73,7 @@ dependencies {
     integrationTestImplementation(testFixtures(project(':ignite-configuration')))
     integrationTestImplementation(testFixtures(project(':ignite-core')))
     integrationTestImplementation(testFixtures(project(':ignite-table')))
+    integrationTestImplementation(testFixtures(project(':ignite-client-handler')))
     integrationTestImplementation libs.msgpack.core
     integrationTestImplementation libs.netty.handler
     integrationTestImplementation libs.jetbrains.annotations
@@ -80,6 +81,7 @@ dependencies {
     testFixturesImplementation project(':ignite-core')
     testFixturesImplementation project(':ignite-placement-driver-api')
     testFixturesImplementation project(':ignite-catalog')
+    testFixturesImplementation project(':ignite-security-api')
     testFixturesImplementation libs.mockito.junit
 }
 
diff --git a/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/ItClientHandlerTest.java b/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/ItClientHandlerTest.java
index 288552f77b..d3c2ed60c3 100644
--- a/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/ItClientHandlerTest.java
+++ b/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/ItClientHandlerTest.java
@@ -56,8 +56,7 @@ public class ItClientHandlerTest extends BaseIgniteAbstractTest {
 
     private int serverPort;
 
-    @SuppressWarnings("unused")
-    @InjectConfiguration
+    @InjectConfiguration(rootName = "security")
     private SecurityConfiguration securityConfiguration;
 
     @BeforeEach
diff --git a/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/TestServer.java b/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/TestServer.java
index bb376a1a79..07dfcbea2e 100644
--- a/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/TestServer.java
+++ b/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/TestServer.java
@@ -59,7 +59,7 @@ public class TestServer {
 
     private final TestSslConfig testSslConfig;
 
-    private final SecurityConfiguration securityConfiguration;
+    private final AuthenticationManager authenticationManager;
 
     private final ClientHandlerMetricSource metrics = new ClientHandlerMetricSource();
 
@@ -71,9 +71,9 @@ public class TestServer {
 
     TestServer(@Nullable TestSslConfig testSslConfig, @Nullable SecurityConfiguration securityConfiguration) {
         this.testSslConfig = testSslConfig;
-        this.securityConfiguration = securityConfiguration == null
-                ? mock(SecurityConfiguration.class)
-                : securityConfiguration;
+        this.authenticationManager = securityConfiguration == null
+                ? new DummyAuthenticationManager()
+                : new AuthenticationManagerImpl(securityConfiguration);
         this.generator = new ConfigurationTreeGenerator(ClientConnectorConfiguration.KEY, NetworkConfiguration.KEY);
         this.configurationManager = new ConfigurationManager(
                 List.of(ClientConnectorConfiguration.KEY, NetworkConfiguration.KEY),
@@ -97,6 +97,7 @@ public class TestServer {
 
     ClientHandlerModule start(TestInfo testInfo) {
         configurationManager.start();
+        authenticationManager.start();
 
         clientConnectorConfig().change(
                 local -> local
@@ -131,7 +132,7 @@ public class TestServer {
                 () -> CompletableFuture.completedFuture(UUID.randomUUID()),
                 mock(MetricManager.class),
                 metrics,
-                authenticationManager(),
+                authenticationManager,
                 new HybridClockImpl(),
                 new AlwaysSyncedSchemaSyncService(),
                 mock(CatalogService.class),
@@ -151,10 +152,4 @@ public class TestServer {
         var registry = configurationManager.configurationRegistry();
         return registry.getConfiguration(ClientConnectorConfiguration.KEY);
     }
-
-    private AuthenticationManager authenticationManager() {
-        AuthenticationManagerImpl authenticationManager = new AuthenticationManagerImpl();
-        securityConfiguration.listen(authenticationManager);
-        return authenticationManager;
-    }
 }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java
index 274afd20d7..b15a7ae99a 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java
@@ -294,15 +294,12 @@ public class ClientHandlerModule implements IgniteComponent {
 
                             ClientInboundMessageHandler messageHandler = createInboundMessageHandler(
                                     configuration, clusterId, connectionId);
-                            authenticationManager.listen(messageHandler);
 
                             ch.pipeline().addLast(
                                     new ClientMessageDecoder(),
                                     messageHandler
                             );
 
-                            ch.closeFuture().addListener(future -> authenticationManager.stopListen(messageHandler));
-
                             metrics.connectionsInitiatedIncrement();
                         } finally {
                             busyLock.leaveBusy();
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java
index c66169a81d..81dfa2c129 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.client.handler;
 
+import static org.apache.ignite.internal.util.CompletableFutures.falseCompletedFuture;
 import static org.apache.ignite.lang.ErrorGroups.Client.HANDSHAKE_HEADER_ERR;
 import static org.apache.ignite.lang.ErrorGroups.Client.PROTOCOL_COMPATIBILITY_ERR;
 import static org.apache.ignite.lang.ErrorGroups.Client.PROTOCOL_ERR;
@@ -31,9 +32,12 @@ import io.netty.handler.codec.DecoderException;
 import java.util.BitSet;
 import java.util.EnumMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Consumer;
 import javax.net.ssl.SSLException;
 import org.apache.ignite.client.handler.configuration.ClientConnectorView;
@@ -90,6 +94,7 @@ import org.apache.ignite.internal.client.proto.ErrorExtensions;
 import org.apache.ignite.internal.client.proto.HandshakeExtension;
 import org.apache.ignite.internal.client.proto.ProtocolVersion;
 import org.apache.ignite.internal.client.proto.ResponseFlags;
+import org.apache.ignite.internal.event.EventListener;
 import org.apache.ignite.internal.hlc.HybridClock;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
 import org.apache.ignite.internal.jdbc.proto.JdbcQueryCursorHandler;
@@ -105,8 +110,9 @@ import org.apache.ignite.internal.security.authentication.AuthenticationRequest;
 import org.apache.ignite.internal.security.authentication.UserDetails;
 import org.apache.ignite.internal.security.authentication.UsernamePasswordRequest;
 import org.apache.ignite.internal.security.authentication.event.AuthenticationEvent;
-import org.apache.ignite.internal.security.authentication.event.AuthenticationListener;
-import org.apache.ignite.internal.security.authentication.event.AuthenticationProviderEvent;
+import org.apache.ignite.internal.security.authentication.event.AuthenticationEventParameters;
+import org.apache.ignite.internal.security.authentication.event.AuthenticationProviderEventParameters;
+import org.apache.ignite.internal.security.authentication.event.UserEventParameters;
 import org.apache.ignite.internal.sql.engine.QueryProcessor;
 import org.apache.ignite.internal.table.IgniteTablesInternal;
 import org.apache.ignite.internal.table.distributed.schema.SchemaSyncService;
@@ -127,7 +133,7 @@ import org.jetbrains.annotations.Nullable;
  * Handles messages from thin clients.
  */
 @SuppressWarnings({"rawtypes", "unchecked"})
-public class ClientInboundMessageHandler extends ChannelInboundHandlerAdapter implements AuthenticationListener {
+public class ClientInboundMessageHandler extends ChannelInboundHandlerAdapter implements EventListener<AuthenticationEventParameters> {
     /** The logger. */
     private static final IgniteLogger LOG = Loggers.forClass(ClientInboundMessageHandler.class);
 
@@ -170,8 +176,11 @@ public class ClientInboundMessageHandler extends ChannelInboundHandlerAdapter im
     /** Context. */
     private ClientContext clientContext;
 
+    /** Read-write lock. Protects {@link #clientContext}. */
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
     /** Chanel handler context. */
-    private ChannelHandlerContext channelHandlerContext;
+    private volatile ChannelHandlerContext channelHandlerContext;
 
     /** Primary replicas update counter. */
     private final AtomicLong primaryReplicaMaxStartTime;
@@ -258,6 +267,20 @@ public class ClientInboundMessageHandler extends ChannelInboundHandlerAdapter im
         this.primaryReplicaMaxStartTime = new AtomicLong(HybridTimestamp.MIN_VALUE.longValue());
     }
 
+    @Override
+    public void handlerAdded(ChannelHandlerContext ctx) {
+        authenticationEventsToSubscribe().forEach(event -> {
+            authenticationManager.listen(event, this);
+        });
+    }
+
+    @Override
+    public void handlerRemoved(ChannelHandlerContext ctx) {
+        authenticationEventsToSubscribe().forEach(event -> {
+            authenticationManager.removeListener(event, this);
+        });
+    }
+
     @Override
     public void channelRegistered(ChannelHandlerContext ctx) throws Exception {
         channelHandlerContext = ctx;
@@ -315,10 +338,19 @@ public class ClientInboundMessageHandler extends ChannelInboundHandlerAdapter im
             var features = BitSet.valueOf(unpacker.readPayload(featuresLen));
 
             Map<HandshakeExtension, Object> extensions = extractExtensions(unpacker);
-            AuthenticationRequest<?, ?> authenticationRequest = createAuthenticationRequest(extensions);
-            UserDetails userDetails = authenticationManager.authenticate(authenticationRequest);
 
-            clientContext = new ClientContext(clientVer, clientCode, features, userDetails);
+            // It's necessary to perform authentication and update the client context while holding a write lock.
+            // This prevents a race condition where authentication succeeds but the context isn't updated in time.
+            // In such a scenario, we might receive an authentication event and attempt to close the connection,
+            // but fail because the context is still null.
+            readWriteLock.writeLock().lock();
+            try {
+                AuthenticationRequest<?, ?> authenticationRequest = createAuthenticationRequest(extensions);
+                UserDetails userDetails = authenticationManager.authenticate(authenticationRequest);
+                clientContext = new ClientContext(clientVer, clientCode, features, userDetails);
+            } finally {
+                readWriteLock.writeLock().unlock();
+            }
 
             if (LOG.isDebugEnabled()) {
                 LOG.debug("Handshake [connectionId=" + connectionId + ", remoteAddress=" + ctx.channel().remoteAddress() + "]: "
@@ -798,30 +830,6 @@ public class ClientInboundMessageHandler extends ChannelInboundHandlerAdapter im
         return clock.now().longValue();
     }
 
-    @Override
-    public void onEvent(AuthenticationEvent event) {
-        switch (event.type()) {
-            case AUTHENTICATION_ENABLED:
-                closeConnection();
-                break;
-            case AUTHENTICATION_PROVIDER_REMOVED:
-            case AUTHENTICATION_PROVIDER_UPDATED:
-                AuthenticationProviderEvent providerEvent = (AuthenticationProviderEvent) event;
-                if (clientContext != null && clientContext.userDetails().providerName().equals(providerEvent.name())) {
-                    closeConnection();
-                }
-                break;
-            default:
-                break;
-        }
-    }
-
-    private void closeConnection() {
-        if (channelHandlerContext != null) {
-            channelHandlerContext.close();
-        }
-    }
-
     private void sendNotification(long requestId, @Nullable Consumer<ClientMessagePacker> writer, @Nullable Throwable err) {
         if (err != null) {
             writeError(requestId, -1, err, channelHandlerContext, true);
@@ -849,4 +857,60 @@ public class ClientInboundMessageHandler extends ChannelInboundHandlerAdapter im
         // This is fine, because the client registers a listener before sending the request.
         return (writer, err) -> sendNotification(requestId, writer, err);
     }
+
+    @Override
+    public CompletableFuture<Boolean> notify(AuthenticationEventParameters parameters, @Nullable Throwable exception) {
+        if (shouldCloseConnection(parameters)) {
+            LOG.warn("Closing connection due to authentication event [connectionId=" + connectionId + ", remoteAddress="
+                    + channelHandlerContext.channel().remoteAddress() + ", event=" + parameters.type() + ']');
+            closeConnection();
+        }
+        return falseCompletedFuture();
+    }
+
+    private boolean shouldCloseConnection(AuthenticationEventParameters parameters) {
+        switch (parameters.type()) {
+            case AUTHENTICATION_ENABLED:
+                return true;
+            case AUTHENTICATION_PROVIDER_REMOVED:
+            case AUTHENTICATION_PROVIDER_UPDATED:
+                return currentUserAffected((AuthenticationProviderEventParameters) parameters);
+            case USER_REMOVED:
+            case USER_UPDATED:
+                return currentUserAffected((UserEventParameters) parameters);
+            default:
+                return false;
+        }
+    }
+
+    private boolean currentUserAffected(AuthenticationProviderEventParameters parameters) {
+        readWriteLock.readLock().lock();
+        try {
+            return clientContext != null && clientContext.userDetails().providerName().equals(parameters.name());
+        } finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    private boolean currentUserAffected(UserEventParameters parameters) {
+        return clientContext != null
+                && clientContext.userDetails().providerName().equals(parameters.providerName())
+                && clientContext.userDetails().username().equals(parameters.username());
+    }
+
+    private void closeConnection() {
+        if (channelHandlerContext != null) {
+            channelHandlerContext.close();
+        }
+    }
+
+    private static Set<AuthenticationEvent> authenticationEventsToSubscribe() {
+        return Set.of(
+                AuthenticationEvent.AUTHENTICATION_ENABLED,
+                AuthenticationEvent.AUTHENTICATION_PROVIDER_UPDATED,
+                AuthenticationEvent.AUTHENTICATION_PROVIDER_REMOVED,
+                AuthenticationEvent.USER_UPDATED,
+                AuthenticationEvent.USER_REMOVED
+        );
+    }
 }
diff --git a/modules/client-handler/src/test/java/org/apache/ignite/client/handler/ClientInboundMessageHandlerTest.java b/modules/client-handler/src/test/java/org/apache/ignite/client/handler/ClientInboundMessageHandlerTest.java
deleted file mode 100644
index e1f0b00261..0000000000
--- a/modules/client-handler/src/test/java/org/apache/ignite/client/handler/ClientInboundMessageHandlerTest.java
+++ /dev/null
@@ -1,275 +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.ignite.client.handler;
-
-import static org.apache.ignite.internal.configuration.validation.TestValidationUtil.mockValidationContext;
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
-import static org.awaitility.Awaitility.await;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.is;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.lenient;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.buffer.UnpooledByteBufAllocator;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelHandlerContext;
-import java.io.IOException;
-import java.time.Duration;
-import java.util.UUID;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Consumer;
-import org.apache.ignite.client.handler.configuration.ClientConnectorConfiguration;
-import org.apache.ignite.compute.IgniteCompute;
-import org.apache.ignite.configuration.NamedListView;
-import org.apache.ignite.configuration.validation.ValidationContext;
-import org.apache.ignite.internal.catalog.CatalogService;
-import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
-import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
-import org.apache.ignite.internal.configuration.validation.TestValidationUtil;
-import org.apache.ignite.internal.hlc.HybridClock;
-import org.apache.ignite.internal.security.authentication.AuthenticationManager;
-import org.apache.ignite.internal.security.authentication.AuthenticationManagerImpl;
-import org.apache.ignite.internal.security.authentication.basic.BasicAuthenticationProviderChange;
-import org.apache.ignite.internal.security.authentication.configuration.AuthenticationProviderView;
-import org.apache.ignite.internal.security.authentication.configuration.validator.AuthenticationProvidersValidator;
-import org.apache.ignite.internal.security.authentication.validator.AuthenticationProvidersValidatorImpl;
-import org.apache.ignite.internal.security.configuration.SecurityChange;
-import org.apache.ignite.internal.security.configuration.SecurityConfiguration;
-import org.apache.ignite.internal.sql.engine.QueryProcessor;
-import org.apache.ignite.internal.table.IgniteTablesInternal;
-import org.apache.ignite.internal.table.distributed.schema.SchemaSyncService;
-import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
-import org.apache.ignite.internal.tx.impl.IgniteTransactionsImpl;
-import org.apache.ignite.network.ClusterNode;
-import org.apache.ignite.network.ClusterNodeImpl;
-import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.NetworkAddress;
-import org.apache.ignite.network.TopologyService;
-import org.apache.ignite.sql.IgniteSql;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.mockito.Mock;
-import org.mockito.junit.jupiter.MockitoExtension;
-import org.msgpack.core.MessagePack;
-
-@ExtendWith(MockitoExtension.class)
-@ExtendWith(ConfigurationExtension.class)
-class ClientInboundMessageHandlerTest extends BaseIgniteAbstractTest {
-    private static final Duration TIMEOUT_OF_DURING = Duration.ofSeconds(2);
-
-    private static final String PROVIDER_NAME = "basic";
-
-    @InjectConfiguration
-    private ClientConnectorConfiguration configuration;
-
-    @InjectConfiguration
-    private SecurityConfiguration securityConfiguration;
-
-    @Mock
-    private IgniteTablesInternal igniteTables;
-
-    @Mock
-    private IgniteTransactionsImpl igniteTransactions;
-
-    @Mock
-    private QueryProcessor processor;
-
-    @Mock
-    private IgniteCompute compute;
-
-    @Mock
-    private TopologyService topologyService;
-
-    @Mock
-    private ClusterService clusterService;
-
-    @Mock
-    private IgniteSql sql;
-
-    @Mock
-    private CompletableFuture<UUID> clusterId;
-
-    @Mock
-    private ClientHandlerMetricSource metrics;
-
-    @Mock
-    private HybridClock clock;
-
-    @Mock
-    private SchemaSyncService schemaSyncService;
-
-    @Mock
-    private CatalogService catalogService;
-
-    @Mock
-    private ChannelHandlerContext ctx;
-
-    @Mock
-    private Channel channel;
-
-    @Mock
-    private ChannelFuture channelFuture;
-
-    private ClientInboundMessageHandler handler;
-
-    private final AtomicBoolean ctxClosed = new AtomicBoolean(false);
-
-    @BeforeEach
-    void setUp() throws Exception {
-        doReturn(topologyService).when(clusterService).topologyService();
-
-        ClusterNode node = new ClusterNodeImpl("node1", "node1", new NetworkAddress("localhost", 10800));
-        doReturn(node).when(topologyService).localMember();
-
-        doReturn(UUID.randomUUID()).when(clusterId).join();
-
-        doReturn(channelFuture).when(channel).closeFuture();
-
-        doReturn(new UnpooledByteBufAllocator(true)).when(ctx).alloc();
-        doReturn(channel).when(ctx).channel();
-        lenient().doAnswer(invocation -> {
-            ctxClosed.set(true);
-            return null;
-        }).when(ctx).close();
-
-        AuthenticationManager authenticationManager = new AuthenticationManagerImpl();
-        AtomicLong clientIdGen = new AtomicLong(0);
-
-        handler = new ClientInboundMessageHandler(
-                igniteTables,
-                igniteTransactions,
-                processor,
-                configuration.value(),
-                compute,
-                clusterService,
-                sql,
-                clusterId,
-                metrics,
-                authenticationManager,
-                clock,
-                schemaSyncService,
-                catalogService,
-                clientIdGen.incrementAndGet(),
-                new ClientPrimaryReplicaTracker(null, catalogService, clock, schemaSyncService)
-        );
-
-        authenticationManager.listen(handler);
-        securityConfiguration.listen(authenticationManager);
-
-        changeConfiguration(change -> {
-            change.changeEnabled(true);
-            change.changeAuthentication().changeProviders()
-                    .create(PROVIDER_NAME, providerChange -> {
-                        providerChange.convert(BasicAuthenticationProviderChange.class)
-                                .changeUsers()
-                                .create("admin", user -> user.changePassword("password"))
-                                .create("admin1", user -> user.changePassword("password"));
-                    });
-        });
-
-        handler.channelRegistered(ctx);
-    }
-
-    @Test
-    void disableAuthentication() throws IOException {
-        handshake();
-
-        changeConfiguration(change -> change.changeEnabled(false));
-
-        await().during(TIMEOUT_OF_DURING).untilAtomic(ctxClosed, is(false));
-    }
-
-    @Test
-    void enableAuthentication() throws IOException {
-        changeConfiguration(change -> change.changeEnabled(false));
-
-        handshake();
-
-        changeConfiguration(change -> change.changeEnabled(true));
-
-        await().untilAtomic(ctxClosed, is(true));
-    }
-
-    @Test
-    void changeProvider() throws IOException {
-        handshake();
-
-        changeConfiguration(change -> {
-            change.changeEnabled(true);
-            change.changeAuthentication().changeProviders().update(PROVIDER_NAME, providerChange -> {
-                providerChange.convert(BasicAuthenticationProviderChange.class)
-                        .changeUsers().update("admin", user -> user.changePassword("new-password"));
-            });
-        });
-
-        await().untilAtomic(ctxClosed, is(true));
-    }
-
-    private void handshake() throws IOException {
-        var packer = MessagePack.newDefaultBufferPacker();
-        packer.packInt(3); // Major.
-        packer.packInt(0); // Minor.
-        packer.packInt(0); // Patch.
-
-        packer.packInt(2); // Client type: general purpose.
-
-        packer.packBinaryHeader(0); // Features.
-        packer.packInt(3); // Extensions.
-        packer.packString("authn-type");
-        packer.packString(PROVIDER_NAME);
-        packer.packString("authn-identity");
-        packer.packString("admin");
-        packer.packString("authn-secret");
-        packer.packString("password");
-
-        ByteBuf byteBuf = Unpooled.wrappedBuffer(packer.toByteArray());
-
-        handler.channelRead(ctx, byteBuf);
-
-        verify(ctx).writeAndFlush(any());
-    }
-
-    private void changeConfiguration(Consumer<SecurityChange> changeConsumer) {
-        assertThat(securityConfiguration.change(changeConsumer), willCompleteSuccessfully());
-        validateConfiguration();
-    }
-
-    private void validateConfiguration() {
-        ValidationContext<NamedListView<? extends AuthenticationProviderView>> ctx = mockValidationContext(
-                null,
-                securityConfiguration.value().authentication().providers()
-        );
-
-        doReturn(securityConfiguration.value()).when(ctx).getNewRoot(SecurityConfiguration.KEY);
-
-        TestValidationUtil.validate(
-                AuthenticationProvidersValidatorImpl.INSTANCE,
-                mock(AuthenticationProvidersValidator.class),
-                ctx
-        );
-    }
-}
diff --git a/modules/client-handler/src/testFixtures/java/org/apache/ignite/client/handler/DummyAuthenticationManager.java b/modules/client-handler/src/testFixtures/java/org/apache/ignite/client/handler/DummyAuthenticationManager.java
new file mode 100644
index 0000000000..4dabcbe216
--- /dev/null
+++ b/modules/client-handler/src/testFixtures/java/org/apache/ignite/client/handler/DummyAuthenticationManager.java
@@ -0,0 +1,60 @@
+/*
+ * 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.ignite.client.handler;
+
+import org.apache.ignite.internal.event.EventListener;
+import org.apache.ignite.internal.security.authentication.AuthenticationManager;
+import org.apache.ignite.internal.security.authentication.AuthenticationRequest;
+import org.apache.ignite.internal.security.authentication.UserDetails;
+import org.apache.ignite.internal.security.authentication.event.AuthenticationEvent;
+import org.apache.ignite.internal.security.authentication.event.AuthenticationEventParameters;
+
+/**
+ * Dummy authentication manager that always returns {@link UserDetails#UNKNOWN}.
+ */
+public class DummyAuthenticationManager implements AuthenticationManager {
+    @Override
+    public void listen(AuthenticationEvent evt, EventListener<? extends AuthenticationEventParameters> listener) {
+
+    }
+
+    @Override
+    public void removeListener(AuthenticationEvent evt, EventListener<? extends AuthenticationEventParameters> listener) {
+
+    }
+
+    @Override
+    public void start() {
+
+    }
+
+    @Override
+    public void stop() throws Exception {
+
+    }
+
+    @Override
+    public boolean authenticationEnabled() {
+        return false;
+    }
+
+    @Override
+    public UserDetails authenticate(AuthenticationRequest<?, ?> authenticationRequest) {
+        return UserDetails.UNKNOWN;
+    }
+}
diff --git a/modules/client/src/test/java/org/apache/ignite/client/ClientAuthenticationTest.java b/modules/client/src/test/java/org/apache/ignite/client/ClientAuthenticationTest.java
index 5741879c3f..44d613b64e 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/ClientAuthenticationTest.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/ClientAuthenticationTest.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.security.exception.InvalidCredentialsException;
 import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
@@ -39,22 +38,13 @@ import org.junit.jupiter.api.extension.ExtendWith;
 @SuppressWarnings({"resource", "ThrowableNotThrown"})
 @ExtendWith(ConfigurationExtension.class)
 public class ClientAuthenticationTest extends BaseIgniteAbstractTest {
-    @SuppressWarnings("unused")
-    @InjectConfiguration
+    @InjectConfiguration(rootName = "security")
     private SecurityConfiguration securityConfiguration;
 
     private TestServer server;
 
     private IgniteClient client;
 
-    @BeforeEach
-    public void beforeEach() {
-        securityConfiguration.change(change -> {
-            change.changeEnabled(false);
-            change.changeAuthentication().changeProviders().delete("basic");
-        }).join();
-    }
-
     @AfterEach
     public void afterEach() throws Exception {
         IgniteUtils.closeAll(client, server);
diff --git a/modules/client/src/test/java/org/apache/ignite/client/TestClientHandlerModule.java b/modules/client/src/test/java/org/apache/ignite/client/TestClientHandlerModule.java
index 6247c51525..b161f4df7b 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/TestClientHandlerModule.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/TestClientHandlerModule.java
@@ -49,8 +49,6 @@ import org.apache.ignite.internal.hlc.HybridClock;
 import org.apache.ignite.internal.manager.IgniteComponent;
 import org.apache.ignite.internal.placementdriver.PlacementDriver;
 import org.apache.ignite.internal.security.authentication.AuthenticationManager;
-import org.apache.ignite.internal.security.authentication.AuthenticationManagerImpl;
-import org.apache.ignite.internal.security.configuration.SecurityConfiguration;
 import org.apache.ignite.internal.sql.engine.QueryProcessor;
 import org.apache.ignite.internal.table.IgniteTablesInternal;
 import org.apache.ignite.internal.table.distributed.schema.AlwaysSyncedSchemaSyncService;
@@ -101,7 +99,7 @@ public class TestClientHandlerModule implements IgniteComponent {
     private final NettyBootstrapFactory bootstrapFactory;
 
     /** Security configuration. */
-    private final SecurityConfiguration securityConfiguration;
+    private final AuthenticationManager authenticationManager;
 
     /**
      * Constructor.
@@ -115,8 +113,9 @@ public class TestClientHandlerModule implements IgniteComponent {
      * @param compute Compute.
      * @param clusterId Cluster id.
      * @param metrics Metrics.
-     * @param securityConfiguration Security configuration.
+     * @param authenticationManager Authentication manager.
      * @param clock Clock.
+     * @param placementDriver Placement driver.
      */
     public TestClientHandlerModule(
             Ignite ignite,
@@ -128,7 +127,7 @@ public class TestClientHandlerModule implements IgniteComponent {
             IgniteCompute compute,
             UUID clusterId,
             ClientHandlerMetricSource metrics,
-            SecurityConfiguration securityConfiguration,
+            AuthenticationManager authenticationManager,
             HybridClock clock,
             PlacementDriver placementDriver) {
         assert ignite != null;
@@ -144,7 +143,7 @@ public class TestClientHandlerModule implements IgniteComponent {
         this.compute = compute;
         this.clusterId = clusterId;
         this.metrics = metrics;
-        this.securityConfiguration = securityConfiguration;
+        this.authenticationManager = authenticationManager;
         this.clock = clock;
         this.placementDriver = placementDriver;
     }
@@ -216,7 +215,7 @@ public class TestClientHandlerModule implements IgniteComponent {
                                         ignite.sql(),
                                         CompletableFuture.completedFuture(clusterId),
                                         metrics,
-                                        authenticationManager(securityConfiguration),
+                                        authenticationManager,
                                         clock,
                                         new AlwaysSyncedSchemaSyncService(),
                                         catalogService,
@@ -305,10 +304,4 @@ public class TestClientHandlerModule implements IgniteComponent {
             super.channelRead(ctx, msg);
         }
     }
-
-    private AuthenticationManager authenticationManager(SecurityConfiguration securityConfiguration) {
-        AuthenticationManagerImpl manager = new AuthenticationManagerImpl();
-        securityConfiguration.listen(manager);
-        return manager;
-    }
 }
diff --git a/modules/client/src/test/java/org/apache/ignite/client/TestServer.java b/modules/client/src/test/java/org/apache/ignite/client/TestServer.java
index 402cae73ca..32c376a0cd 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/TestServer.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/TestServer.java
@@ -39,6 +39,7 @@ import org.apache.ignite.client.fakes.FakeIgnite;
 import org.apache.ignite.client.fakes.FakeInternalTable;
 import org.apache.ignite.client.handler.ClientHandlerMetricSource;
 import org.apache.ignite.client.handler.ClientHandlerModule;
+import org.apache.ignite.client.handler.DummyAuthenticationManager;
 import org.apache.ignite.client.handler.FakeCatalogService;
 import org.apache.ignite.client.handler.FakePlacementDriver;
 import org.apache.ignite.client.handler.configuration.ClientConnectorConfiguration;
@@ -82,6 +83,8 @@ public class TestServer implements AutoCloseable {
 
     private final ClientHandlerMetricSource metrics;
 
+    private final AuthenticationManager authenticationManager;
+
     private final Ignite ignite;
 
     private final FakePlacementDriver placementDriver = new FakePlacementDriver(FakeInternalTable.PARTITIONS);
@@ -191,28 +194,31 @@ public class TestServer implements AutoCloseable {
         metrics = new ClientHandlerMetricSource();
         metrics.enable();
 
-        SecurityConfiguration securityConfigurationOnInit = securityConfiguration == null
-                ? mock(SecurityConfiguration.class)
-                : securityConfiguration;
-
         if (clock == null) {
             clock = new HybridClockImpl();
         }
 
+        if (securityConfiguration == null) {
+            authenticationManager = new DummyAuthenticationManager();
+        } else {
+            authenticationManager = new AuthenticationManagerImpl(securityConfiguration);
+            authenticationManager.start();
+        }
+
         module = shouldDropConnection != null
                 ? new TestClientHandlerModule(
-                        ignite,
-                        cfg,
-                        bootstrapFactory,
-                        shouldDropConnection,
-                        responseDelay,
-                        clusterService,
-                        compute,
-                        clusterId,
-                        metrics,
-                        securityConfigurationOnInit,
-                        clock,
-                        placementDriver)
+                ignite,
+                cfg,
+                bootstrapFactory,
+                shouldDropConnection,
+                responseDelay,
+                clusterService,
+                compute,
+                clusterId,
+                metrics,
+                authenticationManager,
+                clock,
+                placementDriver)
                 : new ClientHandlerModule(
                         ((FakeIgnite) ignite).queryEngine(),
                         (IgniteTablesInternal) ignite.tables(),
@@ -225,7 +231,7 @@ public class TestServer implements AutoCloseable {
                         () -> CompletableFuture.completedFuture(clusterId),
                         mock(MetricManager.class),
                         metrics,
-                        authenticationManager(securityConfigurationOnInit),
+                        authenticationManager,
                         clock,
                         new AlwaysSyncedSchemaSyncService(),
                         new FakeCatalogService(FakeInternalTable.PARTITIONS),
@@ -297,6 +303,7 @@ public class TestServer implements AutoCloseable {
     @Override
     public void close() throws Exception {
         module.stop();
+        authenticationManager.stop();
         bootstrapFactory.stop();
         cfg.stop();
         generator.close();
@@ -317,10 +324,4 @@ public class TestServer implements AutoCloseable {
             throw new IOError(e);
         }
     }
-
-    private AuthenticationManager authenticationManager(SecurityConfiguration securityConfiguration) {
-        AuthenticationManagerImpl authenticationManager = new AuthenticationManagerImpl();
-        securityConfiguration.listen(authenticationManager);
-        return authenticationManager;
-    }
 }
diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/notifications/ConfigurationNotifier.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/notifications/ConfigurationNotifier.java
index 4d58120469..aaea854fe0 100644
--- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/notifications/ConfigurationNotifier.java
+++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/notifications/ConfigurationNotifier.java
@@ -44,6 +44,7 @@ import org.apache.ignite.configuration.notifications.ConfigurationListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
 import org.apache.ignite.internal.configuration.DynamicConfiguration;
+import org.apache.ignite.internal.configuration.DynamicProperty;
 import org.apache.ignite.internal.configuration.NamedListConfiguration;
 import org.apache.ignite.internal.configuration.tree.ConfigurationVisitor;
 import org.apache.ignite.internal.configuration.tree.InnerNode;
@@ -133,8 +134,10 @@ public class ConfigurationNotifier {
                 Serializable newLeaf = newInnerNode.traverseChild(key, leafNodeVisitor(), true);
 
                 if (newLeaf != oldLeaf) {
+                    // TODO: Remove null check after https://issues.apache.org/jira/browse/IGNITE-21101
+                    DynamicProperty<Serializable> node = config != null ? dynamicProperty(config, key) : null;
                     notifyPublicListeners(
-                            listeners(dynamicProperty(config, key), ctx.notificationNum),
+                            listeners(node, ctx.notificationNum),
                             concat(mapIterable(anyConfigs, anyCfg -> listeners(dynamicProperty(anyCfg, key), ctx.notificationNum))),
                             oldLeaf,
                             newLeaf,
diff --git a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java
index 0856087698..b58ffb54c9 100644
--- a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java
+++ b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java
@@ -76,6 +76,7 @@ import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.configuration.validation.TestConfigurationValidator;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 /**
@@ -107,6 +108,18 @@ public class ConfigurationListenerTest {
     public static class ChildConfigurationSchema {
         @Value(hasDefault = true)
         public String str = "default";
+
+        @NamedConfigValue
+        public EntryConfigurationSchema entries;
+    }
+
+    /**
+     * Entry configuration schema.
+     */
+    @Config
+    public static class EntryConfigurationSchema {
+        @Value(hasDefault = true)
+        public String str = "default";
     }
 
     /**
@@ -610,6 +623,98 @@ public class ConfigurationListenerTest {
         assertEquals(List.of("parent", "elements", "rename"), log);
     }
 
+    /**
+     * Tests notifications validity when a named list element is renamed and then updated a sub-element of the renamed element.
+     */
+    @Test
+    @Disabled("https://issues.apache.org/jira/browse/IGNITE-21101")
+    public void namedListNodeOnRenameAndThenUpdateSubElement() throws Exception {
+        config.change(parent ->
+                parent.changeChildren(elements -> elements.create("name", element -> {
+                    element.changeEntries()
+                            .create("entry", entry -> entry.changeStr("default"));
+                }))
+        ).get(1, SECONDS);
+
+        List<String> log = new ArrayList<>();
+
+        config.listen(ctx -> {
+            log.add("parent");
+
+            return nullCompletedFuture();
+        });
+
+        config.child().listen(ctx -> {
+            log.add("child");
+
+            return nullCompletedFuture();
+        });
+
+        config.children().listen(ctx -> {
+            log.add("children");
+
+            return nullCompletedFuture();
+        });
+
+        config.children().listenElements(new ConfigurationNamedListListener<ChildView>() {
+            /** {@inheritDoc} */
+            @Override
+            public CompletableFuture<?> onCreate(ConfigurationNotificationEvent<ChildView> ctx) {
+                log.add("create");
+
+                return nullCompletedFuture();
+            }
+
+            /** {@inheritDoc} */
+            @Override
+            public CompletableFuture<?> onUpdate(ConfigurationNotificationEvent<ChildView> ctx) {
+                log.add("update");
+
+                return nullCompletedFuture();
+            }
+
+            /** {@inheritDoc} */
+            @Override
+            public CompletableFuture<?> onRename(
+                    ConfigurationNotificationEvent<ChildView> ctx
+            ) {
+                log.add("rename");
+
+                return nullCompletedFuture();
+            }
+
+            /** {@inheritDoc} */
+            @Override
+            public CompletableFuture<?> onDelete(ConfigurationNotificationEvent<ChildView> ctx) {
+                log.add("delete");
+
+                return nullCompletedFuture();
+            }
+        });
+
+        config.children().get("name").entries().get("entry").listen(ctx -> {
+            log.add("entry");
+
+            return nullCompletedFuture();
+        });
+
+        config.change(parent ->
+                parent.changeChildren(elements -> elements
+                        .rename("name", "newName")
+                )
+        ).get(1, SECONDS);
+
+        config.children().get("newName")
+                .entries()
+                .get("entry")
+                .str()
+                .update("foo")
+                .get(1, SECONDS);
+
+        assertEquals(List.of("parent", "elements", "rename"), log);
+    }
+
+
     /**
      * Tests notifications validity when a named list element is deleted.
      */
diff --git a/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/testframework/ConfigurationExtension.java b/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/testframework/ConfigurationExtension.java
index ef0be1ebad..90e3a5bcc3 100644
--- a/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/testframework/ConfigurationExtension.java
+++ b/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/testframework/ConfigurationExtension.java
@@ -35,7 +35,9 @@ import com.typesafe.config.ConfigObject;
 import java.lang.reflect.Field;
 import java.lang.reflect.Parameter;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.ServiceLoader;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
@@ -50,6 +52,7 @@ import org.apache.ignite.internal.configuration.DynamicConfiguration;
 import org.apache.ignite.internal.configuration.DynamicConfigurationChanger;
 import org.apache.ignite.internal.configuration.RootInnerNode;
 import org.apache.ignite.internal.configuration.SuperRoot;
+import org.apache.ignite.internal.configuration.SuperRootChangeImpl;
 import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
 import org.apache.ignite.internal.configuration.direct.KeyPathNode;
 import org.apache.ignite.internal.configuration.hocon.HoconConverter;
@@ -73,7 +76,6 @@ import org.junit.platform.commons.support.HierarchyTraversalMode;
  * JUnit extension to inject configuration instances into test classes.
  *
  * @see InjectConfiguration
- * @see InjectRevisionListenerHolder
  */
 public class ConfigurationExtension implements BeforeEachCallback, AfterEachCallback,
         BeforeAllCallback, AfterAllCallback, ParameterResolver {
@@ -92,6 +94,9 @@ public class ConfigurationExtension implements BeforeEachCallback, AfterEachCall
     /** All {@link PolymorphicConfigInstance} classes in classpath. */
     private static final List<Class<?>> POLYMORPHIC_EXTENSIONS;
 
+    /** Map from root key name to configuration modules. */
+    private static final Map<String, ConfigurationModule> ROOT_KEY_TO_MODULES = new HashMap<>();
+
     static {
         // Automatically find all @InternalConfiguration and @PolymorphicConfigInstance classes
         // to avoid configuring extensions manually in every test.
@@ -103,6 +108,10 @@ public class ConfigurationExtension implements BeforeEachCallback, AfterEachCall
         modules.forEach(configurationModule -> {
             extensions.addAll(configurationModule.schemaExtensions());
             polymorphicExtensions.addAll(configurationModule.polymorphicSchemaExtensions());
+
+            configurationModule.rootKeys().forEach(rootKey -> {
+                ROOT_KEY_TO_MODULES.put(rootKey.key(), configurationModule);
+            });
         });
 
         EXTENSIONS = List.copyOf(extensions);
@@ -234,7 +243,7 @@ public class ConfigurationExtension implements BeforeEachCallback, AfterEachCall
         // RootKey must be mocked, there's no way to instantiate it using a public constructor.
         RootKey rootKey = mock(RootKey.class, withSettings().lenient());
 
-        when(rootKey.key()).thenReturn("mock");
+        when(rootKey.key()).thenReturn(annotation.rootName().isBlank() ? "mock" : annotation.rootName());
         when(rootKey.type()).thenReturn(LOCAL);
         when(rootKey.schemaClass()).thenReturn(schemaClass);
         when(rootKey.internal()).thenReturn(false);
@@ -245,6 +254,10 @@ public class ConfigurationExtension implements BeforeEachCallback, AfterEachCall
 
         HoconConverter.hoconSource(hoconCfg).descend(superRoot);
 
+        if (!annotation.rootName().isBlank()) {
+            patchWithDynamicDefaults(annotation.rootName(), superRoot);
+        }
+
         ConfigurationUtil.addDefaults(superRoot);
 
         if (!annotation.name().isEmpty()) {
@@ -332,4 +345,11 @@ public class ConfigurationExtension implements BeforeEachCallback, AfterEachCall
     private static boolean supportsAsConfigurationType(Class<?> type) {
         return type.getCanonicalName().endsWith("Configuration");
     }
+
+    private static void patchWithDynamicDefaults(String rootName, SuperRoot superRoot) {
+        if (ROOT_KEY_TO_MODULES.containsKey(rootName)) {
+            SuperRootChangeImpl rootChange = new SuperRootChangeImpl(superRoot);
+            ROOT_KEY_TO_MODULES.get(rootName).patchConfigurationWithDynamicDefaults(rootChange);
+        }
+    }
 }
diff --git a/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/testframework/InjectConfiguration.java b/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/testframework/InjectConfiguration.java
index a835f5a087..0ff759b2e4 100644
--- a/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/testframework/InjectConfiguration.java
+++ b/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/testframework/InjectConfiguration.java
@@ -21,6 +21,7 @@ import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
+import org.apache.ignite.configuration.SuperRootChange;
 import org.apache.ignite.configuration.annotation.ConfigurationExtension;
 import org.apache.ignite.configuration.annotation.PolymorphicConfig;
 import org.apache.ignite.configuration.annotation.PolymorphicConfigInstance;
@@ -65,6 +66,13 @@ public @interface InjectConfiguration {
      */
     String name() default "";
 
+    /**
+     * Root name of the configuration. Default empty string value is treated like the absence of the root name. The root name is used to
+     * patch the configuration tree with dynamic configuration defaults
+     * {@link org.apache.ignite.configuration.ConfigurationModule#patchConfigurationWithDynamicDefaults(SuperRootChange)}
+     */
+    String rootName() default "";
+
     /**
      * Array of configuration schema extensions. Every class in the array must be annotated with {@link ConfigurationExtension} and extend
      * some public configuration.
diff --git a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/cluster/ItClusterManagementControllerTest.java b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/cluster/ItClusterManagementControllerTest.java
index c3ffc1482e..6fd5378350 100644
--- a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/cluster/ItClusterManagementControllerTest.java
+++ b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/cluster/ItClusterManagementControllerTest.java
@@ -160,8 +160,6 @@ public class ItClusterManagementControllerTest extends RestTestBase {
     }
 
     private AuthenticationManagerImpl authenticationManager() {
-        AuthenticationManagerImpl manager = new AuthenticationManagerImpl();
-        securityConfiguration.listen(manager);
-        return manager;
+        return new AuthenticationManagerImpl(securityConfiguration);
     }
 }
diff --git a/modules/rest/src/test/java/org/apache/ignite/internal/rest/configuration/ClusterConfigurationControllerTest.java b/modules/rest/src/test/java/org/apache/ignite/internal/rest/configuration/ClusterConfigurationControllerTest.java
index 9d4f033e60..ddfce0cd20 100644
--- a/modules/rest/src/test/java/org/apache/ignite/internal/rest/configuration/ClusterConfigurationControllerTest.java
+++ b/modules/rest/src/test/java/org/apache/ignite/internal/rest/configuration/ClusterConfigurationControllerTest.java
@@ -28,14 +28,21 @@ import jakarta.inject.Named;
 import org.apache.ignite.internal.configuration.ConfigurationRegistry;
 import org.apache.ignite.internal.configuration.presentation.ConfigurationPresentation;
 import org.apache.ignite.internal.configuration.presentation.HoconPresentation;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.security.authentication.AuthenticationManager;
 import org.apache.ignite.internal.security.authentication.AuthenticationManagerImpl;
+import org.apache.ignite.internal.security.configuration.SecurityConfiguration;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
  * Functional test for {@link ClusterConfigurationController}.
  */
 @MicronautTest
+@ExtendWith(ConfigurationExtension.class)
 class ClusterConfigurationControllerTest extends ConfigurationControllerBaseTest {
+    @InjectConfiguration
+    SecurityConfiguration securityConfiguration;
 
     @Inject
     @Client("/management/v1/configuration/cluster/")
@@ -59,6 +66,6 @@ class ClusterConfigurationControllerTest extends ConfigurationControllerBaseTest
     @Bean
     @Factory
     AuthenticationManager authenticationManager() {
-        return new AuthenticationManagerImpl();
+        return new AuthenticationManagerImpl(securityConfiguration);
     }
 }
diff --git a/modules/rest/src/test/java/org/apache/ignite/internal/rest/configuration/NodeConfigurationControllerTest.java b/modules/rest/src/test/java/org/apache/ignite/internal/rest/configuration/NodeConfigurationControllerTest.java
index eff3aba485..f65884300c 100644
--- a/modules/rest/src/test/java/org/apache/ignite/internal/rest/configuration/NodeConfigurationControllerTest.java
+++ b/modules/rest/src/test/java/org/apache/ignite/internal/rest/configuration/NodeConfigurationControllerTest.java
@@ -28,14 +28,21 @@ import jakarta.inject.Named;
 import org.apache.ignite.internal.configuration.ConfigurationRegistry;
 import org.apache.ignite.internal.configuration.presentation.ConfigurationPresentation;
 import org.apache.ignite.internal.configuration.presentation.HoconPresentation;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.security.authentication.AuthenticationManager;
 import org.apache.ignite.internal.security.authentication.AuthenticationManagerImpl;
+import org.apache.ignite.internal.security.configuration.SecurityConfiguration;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
  * Functional test for {@link NodeConfigurationController}.
  */
 @MicronautTest
+@ExtendWith(ConfigurationExtension.class)
 class NodeConfigurationControllerTest extends ConfigurationControllerBaseTest {
+    @InjectConfiguration
+    SecurityConfiguration securityConfiguration;
 
     @Inject
     @Client("/management/v1/configuration/node/")
@@ -59,6 +66,6 @@ class NodeConfigurationControllerTest extends ConfigurationControllerBaseTest {
     @Bean
     @Factory
     AuthenticationManager authenticationManager() {
-        return new AuthenticationManagerImpl();
+        return new AuthenticationManagerImpl(securityConfiguration);
     }
 }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientAuthenticationTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientAuthenticationTest.java
new file mode 100644
index 0000000000..d453bcc7fd
--- /dev/null
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientAuthenticationTest.java
@@ -0,0 +1,206 @@
+/*
+ * 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.ignite.internal.runner.app.client;
+
+import static org.apache.ignite.internal.configuration.hocon.HoconConverter.hoconSource;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrowWithCauseOrSuppressed;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.awaitility.Awaitility.await;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import com.typesafe.config.ConfigFactory;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.client.BasicAuthenticator;
+import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.configuration.ConfigurationRegistry;
+import org.apache.ignite.internal.security.authentication.basic.BasicAuthenticationProviderChange;
+import org.apache.ignite.internal.security.configuration.SecurityConfiguration;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.security.exception.InvalidCredentialsException;
+import org.apache.ignite.sql.Session;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Thin client authentication test.
+ */
+public class ItThinClientAuthenticationTest extends ItAbstractThinClientTest {
+    private static final String PROVIDER_NAME = "default";
+
+    private static final String USERNAME_1 = "admin";
+
+    private static final String PASSWORD_1 = "password";
+
+    private static final String USERNAME_2 = "developer";
+
+    private static final String PASSWORD_2 = "password";
+
+    private IgniteClient clientWithAuth;
+
+    private SecurityConfiguration securityConfiguration;
+
+    private final BasicAuthenticator basicAuthenticator = BasicAuthenticator.builder()
+            .username(USERNAME_1)
+            .password(PASSWORD_1)
+            .build();
+
+    @BeforeEach
+    void setUp() {
+        securityConfiguration = clusterConfigurationRegistry().getConfiguration(SecurityConfiguration.KEY);
+
+        CompletableFuture<Void> enableAuthentication = securityConfiguration.change(change -> {
+            change.changeEnabled(true);
+            change.changeAuthentication()
+                    .changeProviders(providers -> {
+                        providers.namedListKeys().forEach(name -> {
+                            if (!name.equals(PROVIDER_NAME)) {
+                                providers.delete(name);
+                            }
+                        });
+
+                        providers.createOrUpdate(PROVIDER_NAME, provider -> {
+                            provider.convert(BasicAuthenticationProviderChange.class)
+                                    .changeUsers()
+                                    .createOrUpdate(USERNAME_1, user -> user.changePassword(PASSWORD_1))
+                                    .createOrUpdate(USERNAME_2, user -> user.changePassword(PASSWORD_2));
+                        });
+                    });
+        });
+
+        assertThat(enableAuthentication, willCompleteSuccessfully());
+
+        clientWithAuth = IgniteClient.builder()
+                .authenticator(basicAuthenticator)
+                .reconnectThrottlingRetries(0)
+                .addresses(getClientAddresses().toArray(new String[0]))
+                .build();
+
+        await().untilAsserted(() -> checkConnection(clientWithAuth));
+    }
+
+    @AfterEach
+    void tearDown() throws Exception {
+        IgniteUtils.closeAll(clientWithAuth);
+    }
+
+    @Test
+    void connectionIsNotClosedIfAnotherUserUpdated() {
+        assertThat(
+                securityConfiguration.authentication().providers()
+                        .get(PROVIDER_NAME)
+                        .change(change -> {
+                            change.convert(BasicAuthenticationProviderChange.class)
+                                    .changeUsers()
+                                    .update(USERNAME_2, user -> user.changePassword(PASSWORD_2 + "-changed"));
+                        }),
+                willCompleteSuccessfully()
+        );
+
+        // Connection should be alive after update.
+        await().during(3, TimeUnit.SECONDS)
+                .until(() -> checkConnection(clientWithAuth), willCompleteSuccessfully());
+    }
+
+    @Test
+    void connectionIsClosedIfAuthenticationEnabled() {
+        await().until(() -> checkConnection(client()), willThrowWithCauseOrSuppressed(InvalidCredentialsException.class));
+    }
+
+    @Test
+    void connectionIsClosedIfPasswordChanged() {
+        assertThat(securityConfiguration.change(change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .update(PROVIDER_NAME, provider -> {
+                        provider.convert(BasicAuthenticationProviderChange.class)
+                                .changeUsers()
+                                .update(USERNAME_1, user -> user.changePassword("newPassword"));
+                    });
+        }), willCompleteSuccessfully());
+
+        await().until(() -> checkConnection(clientWithAuth), willThrowWithCauseOrSuppressed(InvalidCredentialsException.class));
+    }
+
+    @Test
+    void connectionIsClosedIfUserRemoved() {
+        assertThat(securityConfiguration.change(change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .update(PROVIDER_NAME, provider -> {
+                        provider.convert(BasicAuthenticationProviderChange.class)
+                                .changeUsers()
+                                .delete(USERNAME_1);
+                    });
+        }), willCompleteSuccessfully());
+
+        await().until(() -> checkConnection(clientWithAuth), willThrowWithCauseOrSuppressed(InvalidCredentialsException.class));
+    }
+
+    @Test
+    void renameBasicProviderAndThenChangeUserPassword() {
+        updateClusterConfiguration("{\n"
+                + "security.authentication.providers.basic={\n"
+                + "type=basic,\n"
+                + "users=[{username=newuser,password=newpassword}]},"
+                + "security.authentication.providers.default=null\n"
+                + "}");
+
+        try (IgniteClient client = IgniteClient.builder()
+                .authenticator(BasicAuthenticator.builder().username("newuser").password("newpassword").build())
+                .reconnectThrottlingRetries(0)
+                .addresses(getClientAddresses().toArray(new String[0]))
+                .build()) {
+
+            checkConnection(client);
+
+            securityConfiguration.authentication().providers()
+                    .get("basic")
+                    .change(change -> {
+                        change.convert(BasicAuthenticationProviderChange.class)
+                                .changeUsers()
+                                .update("newuser", user -> user.changePassword("newpassword-changed"));
+                    }).join();
+
+            await().until(() -> checkConnection(client), willThrowWithCauseOrSuppressed(InvalidCredentialsException.class));
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static CompletableFuture<Void> checkConnection(IgniteClient client) {
+        try (Session session = client.sql().createSession()) {
+            return session.executeAsync(null, "select 1 as num, 'hello' as str")
+                    .thenApply(ignored -> null);
+        }
+    }
+
+    private void updateClusterConfiguration(String hocon) {
+        assertThat(
+                clusterConfigurationRegistry().change(hoconSource(ConfigFactory.parseString(hocon).root())),
+                willCompleteSuccessfully()
+        );
+    }
+
+    private ConfigurationRegistry clusterConfigurationRegistry() {
+        IgniteImpl server = (IgniteImpl) server();
+        return server.clusterConfiguration();
+    }
+}
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index 84390fddfe..09876a597d 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -700,10 +700,7 @@ public class IgniteImpl implements Ignite {
     private AuthenticationManager createAuthenticationManager() {
         SecurityConfiguration securityConfiguration = clusterCfgMgr.configurationRegistry()
                 .getConfiguration(SecurityConfiguration.KEY);
-
-        AuthenticationManager manager = new AuthenticationManagerImpl();
-        securityConfiguration.listen(manager);
-        return manager;
+        return new AuthenticationManagerImpl(securityConfiguration);
     }
 
     private RestComponent createRestComponent(String name) {
@@ -830,6 +827,7 @@ public class IgniteImpl implements Ignite {
                             lifecycleManager.startComponents(
                                     catalogManager,
                                     clusterCfgMgr,
+                                    authenticationManager,
                                     placementDriverMgr,
                                     metricManager,
                                     distributionZoneManager,
@@ -1209,4 +1207,10 @@ public class IgniteImpl implements Ignite {
     public CatalogManager catalogManager() {
         return catalogManager;
     }
+
+    /** Returns the cluster's configuration manager. */
+    @TestOnly
+    public ConfigurationRegistry clusterConfigurationRegistry() {
+        return clusterCfgMgr.configurationRegistry();
+    }
 }
diff --git a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManager.java b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManager.java
index 6e43af0a80..86d82cba89 100644
--- a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManager.java
+++ b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManager.java
@@ -17,32 +17,20 @@
 
 package org.apache.ignite.internal.security.authentication;
 
-import org.apache.ignite.configuration.notifications.ConfigurationListener;
-import org.apache.ignite.internal.security.authentication.event.AuthenticationListener;
-import org.apache.ignite.internal.security.configuration.SecurityView;
+import org.apache.ignite.internal.event.EventProducer;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.security.authentication.event.AuthenticationEvent;
+import org.apache.ignite.internal.security.authentication.event.AuthenticationEventParameters;
 
 /**
  * Authentication manager.
  */
-public interface AuthenticationManager extends Authenticator, ConfigurationListener<SecurityView> {
+public interface AuthenticationManager extends Authenticator, IgniteComponent,
+        EventProducer<AuthenticationEvent, AuthenticationEventParameters> {
     /**
      * Check if authentication is enabled.
      *
      * @return {@code true} if authentication is enabled.
      */
     boolean authenticationEnabled();
-
-    /**
-     * Listen to authentication events.
-     *
-     * @param listener Listener.
-     */
-    void listen(AuthenticationListener listener);
-
-    /**
-     * Stop listen to authentication events.
-     *
-     * @param listener Listener.
-     */
-    void stopListen(AuthenticationListener listener);
 }
diff --git a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/UserDetails.java b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/UserDetails.java
index e2a938d128..f3e4fc3c8c 100644
--- a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/UserDetails.java
+++ b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/UserDetails.java
@@ -21,6 +21,8 @@ package org.apache.ignite.internal.security.authentication;
  * Represents the user details.
  */
 public class UserDetails {
+    public static final UserDetails UNKNOWN = new UserDetails("unknown", "unknown");
+
     private final String username;
 
     private final String providerName;
diff --git a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationEvent.java b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationEvent.java
index 04514c37e3..3eaf85b241 100644
--- a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationEvent.java
+++ b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationEvent.java
@@ -17,14 +17,16 @@
 
 package org.apache.ignite.internal.security.authentication.event;
 
+import org.apache.ignite.internal.event.Event;
+
 /**
- * Represents the authentication event.
+ * Represents the authentication event type.
  */
-public interface AuthenticationEvent {
-    /**
-     * Returns the event type.
-     *
-     * @return the event type.
-     */
-    EventType type();
+public enum AuthenticationEvent implements Event {
+    AUTHENTICATION_ENABLED,
+    AUTHENTICATION_DISABLED,
+    AUTHENTICATION_PROVIDER_REMOVED,
+    AUTHENTICATION_PROVIDER_UPDATED,
+    USER_UPDATED,
+    USER_REMOVED
 }
diff --git a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/EventType.java b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationEventParameters.java
similarity index 81%
rename from modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/EventType.java
rename to modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationEventParameters.java
index 0418a7a9ee..2829f89efd 100644
--- a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/EventType.java
+++ b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationEventParameters.java
@@ -17,12 +17,11 @@
 
 package org.apache.ignite.internal.security.authentication.event;
 
+import org.apache.ignite.internal.event.EventParameters;
+
 /**
- * Represents the authentication event type.
+ * Authentication event parameters.
  */
-public enum EventType {
-    AUTHENTICATION_ENABLED,
-    AUTHENTICATION_DISABLED,
-    AUTHENTICATION_PROVIDER_REMOVED,
-    AUTHENTICATION_PROVIDER_UPDATED
+public interface AuthenticationEventParameters extends EventParameters {
+    AuthenticationEvent type();
 }
diff --git a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationProviderEvent.java b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationProviderEventParameters.java
similarity index 57%
rename from modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationProviderEvent.java
rename to modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationProviderEventParameters.java
index 2484921233..9d3a92a6d1 100644
--- a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationProviderEvent.java
+++ b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationProviderEventParameters.java
@@ -17,36 +17,36 @@
 
 package org.apache.ignite.internal.security.authentication.event;
 
-import static org.apache.ignite.internal.security.authentication.event.EventType.AUTHENTICATION_PROVIDER_REMOVED;
-import static org.apache.ignite.internal.security.authentication.event.EventType.AUTHENTICATION_PROVIDER_UPDATED;
+import static org.apache.ignite.internal.security.authentication.event.AuthenticationEvent.AUTHENTICATION_PROVIDER_REMOVED;
+import static org.apache.ignite.internal.security.authentication.event.AuthenticationEvent.AUTHENTICATION_PROVIDER_UPDATED;
 
 /**
  * Represents the authentication provider event.
  */
-public class AuthenticationProviderEvent implements AuthenticationEvent {
-    private final EventType type;
+public class AuthenticationProviderEventParameters implements AuthenticationEventParameters {
+    private final AuthenticationEvent type;
 
-    private final String name;
+    private final String providerName;
 
-    private AuthenticationProviderEvent(EventType type, String name) {
+    private AuthenticationProviderEventParameters(AuthenticationEvent type, String providerName) {
         this.type = type;
-        this.name = name;
+        this.providerName = providerName;
     }
 
-    public static AuthenticationProviderEvent updated(String name) {
-        return new AuthenticationProviderEvent(AUTHENTICATION_PROVIDER_UPDATED, name);
+    public static AuthenticationProviderEventParameters updated(String name) {
+        return new AuthenticationProviderEventParameters(AUTHENTICATION_PROVIDER_UPDATED, name);
     }
 
-    public static AuthenticationProviderEvent removed(String name) {
-        return new AuthenticationProviderEvent(AUTHENTICATION_PROVIDER_REMOVED, name);
+    public static AuthenticationProviderEventParameters removed(String name) {
+        return new AuthenticationProviderEventParameters(AUTHENTICATION_PROVIDER_REMOVED, name);
     }
 
     @Override
-    public EventType type() {
+    public AuthenticationEvent type() {
         return type;
     }
 
     public String name() {
-        return name;
+        return providerName;
     }
 }
diff --git a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationSwitchedParameters.java b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationSwitchedParameters.java
new file mode 100644
index 0000000000..c7927f009d
--- /dev/null
+++ b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationSwitchedParameters.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.ignite.internal.security.authentication.event;
+
+/**
+ * Authentication event parameters.
+ */
+public class AuthenticationSwitchedParameters implements AuthenticationEventParameters {
+    private final AuthenticationEvent event;
+
+    private AuthenticationSwitchedParameters(AuthenticationEvent event) {
+        this.event = event;
+    }
+
+    @Override
+    public AuthenticationEvent type() {
+        return event;
+    }
+
+    /**
+     * Creates parameters for authentication switched event.
+     *
+     * @param enabled {@code true} if authentication is enabled, {@code false} otherwise.
+     * @return Parameters for authentication switched event.
+     */
+    public static AuthenticationSwitchedParameters enabled(boolean enabled) {
+        if (enabled) {
+            return new AuthenticationSwitchedParameters(AuthenticationEvent.AUTHENTICATION_ENABLED);
+        } else {
+            return new AuthenticationSwitchedParameters(AuthenticationEvent.AUTHENTICATION_DISABLED);
+        }
+    }
+}
diff --git a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/UserDetails.java b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/UserEventParameters.java
similarity index 53%
copy from modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/UserDetails.java
copy to modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/UserEventParameters.java
index e2a938d128..3f38c5c023 100644
--- a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/UserDetails.java
+++ b/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/UserEventParameters.java
@@ -15,26 +15,42 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.security.authentication;
+package org.apache.ignite.internal.security.authentication.event;
 
 /**
- * Represents the user details.
+ * User event parameters.
  */
-public class UserDetails {
-    private final String username;
+public class UserEventParameters implements AuthenticationEventParameters {
+    private final AuthenticationEvent type;
 
     private final String providerName;
 
-    public UserDetails(String username, String providerName) {
-        this.username = username;
+    private final String userName;
+
+    private UserEventParameters(AuthenticationEvent type, String providerName, String userName) {
+        this.type = type;
         this.providerName = providerName;
+        this.userName = userName;
     }
 
-    public String username() {
-        return username;
+    @Override
+    public AuthenticationEvent type() {
+        return type;
     }
 
     public String providerName() {
         return providerName;
     }
+
+    public String username() {
+        return userName;
+    }
+
+    public static UserEventParameters updated(String providerName, String name) {
+        return new UserEventParameters(AuthenticationEvent.USER_UPDATED, providerName, name);
+    }
+
+    public static UserEventParameters removed(String providerName, String name) {
+        return new UserEventParameters(AuthenticationEvent.USER_REMOVED, providerName, name);
+    }
 }
diff --git a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImpl.java b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImpl.java
index 168ad6004a..90ccdc3c4a 100644
--- a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImpl.java
+++ b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImpl.java
@@ -17,27 +17,31 @@
 
 package org.apache.ignite.internal.security.authentication;
 
-import static org.apache.ignite.internal.security.authentication.event.EventType.AUTHENTICATION_DISABLED;
-import static org.apache.ignite.internal.security.authentication.event.EventType.AUTHENTICATION_ENABLED;
+import static org.apache.ignite.internal.security.authentication.AuthenticationUtils.findBasicProviderName;
 import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture;
 
 import java.util.ArrayList;
+import java.util.Comparator;
 import java.util.List;
 import java.util.Objects;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.stream.Collectors;
 import org.apache.ignite.configuration.NamedListView;
-import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import org.apache.ignite.internal.event.AbstractEventProducer;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.security.authentication.basic.BasicAuthenticationProviderConfiguration;
 import org.apache.ignite.internal.security.authentication.configuration.AuthenticationProviderView;
 import org.apache.ignite.internal.security.authentication.configuration.AuthenticationView;
 import org.apache.ignite.internal.security.authentication.event.AuthenticationEvent;
-import org.apache.ignite.internal.security.authentication.event.AuthenticationListener;
-import org.apache.ignite.internal.security.authentication.event.AuthenticationProviderEvent;
+import org.apache.ignite.internal.security.authentication.event.AuthenticationEventParameters;
+import org.apache.ignite.internal.security.authentication.event.AuthenticationProviderEventFactory;
+import org.apache.ignite.internal.security.authentication.event.SecurityEnabledDisabledEventFactory;
+import org.apache.ignite.internal.security.authentication.event.UserEventFactory;
+import org.apache.ignite.internal.security.configuration.SecurityConfiguration;
 import org.apache.ignite.internal.security.configuration.SecurityView;
 import org.apache.ignite.security.exception.InvalidCredentialsException;
 import org.apache.ignite.security.exception.UnsupportedAuthenticationTypeException;
@@ -47,17 +51,99 @@ import org.jetbrains.annotations.TestOnly;
 /**
  * Implementation of {@link Authenticator}.
  */
-public class AuthenticationManagerImpl implements AuthenticationManager {
+public class AuthenticationManagerImpl
+        extends AbstractEventProducer<AuthenticationEvent, AuthenticationEventParameters>
+        implements AuthenticationManager {
     private static final IgniteLogger LOG = Loggers.forClass(AuthenticationManagerImpl.class);
 
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    /**
+     * Security configuration.
+     */
+    private final SecurityConfiguration securityConfiguration;
+
+    /**
+     * Security configuration listener. Refreshes the list of authenticators when the configuration changes.
+     */
+    private final ConfigurationListener<SecurityView> securityConfigurationListener;
 
-    private final List<AuthenticationListener> listeners = new CopyOnWriteArrayList<>();
+    /**
+     * Security enabled/disabled event factory. Fires events when security is enabled/disabled.
+     */
+    private final SecurityEnabledDisabledEventFactory securityEnabledDisabledEventFactory;
 
+    /**
+     * User event factory. Fires events when a basic user is created/updated/deleted.
+     */
+    private final UserEventFactory userEventFactory;
+
+    /**
+     * Authentication provider event factory. Fires events when an authentication provider is created/updated/deleted.
+     */
+    private final AuthenticationProviderEventFactory providerEventFactory;
+
+    /**
+     * Read-write lock for the list of authenticators and the authentication enabled flag.
+     */
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+
+    /**
+     * List of authenticators.
+     */
     private List<Authenticator> authenticators = new ArrayList<>();
 
+    /**
+     * Authentication enabled flag.
+     */
     private boolean authEnabled = false;
 
+    /**
+     * Constructor.
+     *
+     * @param securityConfiguration Security configuration.
+     */
+    public AuthenticationManagerImpl(SecurityConfiguration securityConfiguration) {
+        this.securityConfiguration = securityConfiguration;
+
+        securityConfigurationListener = ctx -> {
+            refreshProviders(ctx.newValue());
+            return nullCompletedFuture();
+        };
+
+        securityEnabledDisabledEventFactory = new SecurityEnabledDisabledEventFactory(this::fireEvent);
+
+        userEventFactory = new UserEventFactory(this::fireEvent);
+
+        providerEventFactory = new AuthenticationProviderEventFactory(
+                securityConfiguration,
+                userEventFactory,
+                this::fireEvent
+        );
+    }
+
+    @Override
+    public void start() {
+        securityConfiguration.listen(securityConfigurationListener);
+        securityConfiguration.enabled().listen(securityEnabledDisabledEventFactory);
+        securityConfiguration.authentication().providers().listenElements(providerEventFactory);
+
+        String basicAuthenticationProviderName = findBasicProviderName(securityConfiguration.authentication().providers().value());
+        BasicAuthenticationProviderConfiguration basicAuthenticationProviderConfiguration = (BasicAuthenticationProviderConfiguration)
+                securityConfiguration.authentication().providers().get(basicAuthenticationProviderName);
+        basicAuthenticationProviderConfiguration.users().listenElements(userEventFactory);
+    }
+
+    @Override
+    public void stop() throws Exception {
+        securityConfiguration.stopListen(securityConfigurationListener);
+        securityConfiguration.enabled().stopListen(securityEnabledDisabledEventFactory);
+        securityConfiguration.authentication().providers().stopListenElements(providerEventFactory);
+
+        String basicAuthenticationProviderName = findBasicProviderName(securityConfiguration.authentication().providers().value());
+        BasicAuthenticationProviderConfiguration basicAuthenticationProviderConfiguration = (BasicAuthenticationProviderConfiguration)
+                securityConfiguration.authentication().providers().get(basicAuthenticationProviderName);
+        basicAuthenticationProviderConfiguration.users().stopListenElements(userEventFactory);
+    }
+
     /**
      * {@inheritDoc}
      */
@@ -72,7 +158,7 @@ public class AuthenticationManagerImpl implements AuthenticationManager {
                         .findFirst()
                         .orElseThrow(() -> new InvalidCredentialsException("Authentication failed"));
             } else {
-                return new UserDetails("Unknown", "Unknown");
+                return UserDetails.UNKNOWN;
             }
         } finally {
             rwLock.readLock().unlock();
@@ -91,35 +177,17 @@ public class AuthenticationManagerImpl implements AuthenticationManager {
         }
     }
 
-    @Override
-    public CompletableFuture<?> onUpdate(ConfigurationNotificationEvent<SecurityView> ctx) {
-        if (refreshProviders(ctx.newValue())) {
-            emitEvents(ctx);
-        }
-
-        return nullCompletedFuture();
-    }
-
-    private boolean refreshProviders(@Nullable SecurityView view) {
+    private void refreshProviders(@Nullable SecurityView view) {
         rwLock.writeLock().lock();
         try {
             if (view == null || !view.enabled()) {
                 authEnabled = false;
-                authenticators = List.of();
-            } else if (view.enabled() && view.authentication().providers().size() != 0) {
+            } else {
                 authenticators = providersFromAuthView(view.authentication());
                 authEnabled = true;
-            } else {
-                LOG.error("Invalid configuration: security is enabled, but no providers. Leaving the old settings");
-
-                return false;
             }
-
-            return true;
         } catch (Exception exception) {
             LOG.error("Couldn't refresh authentication providers. Leaving the old settings", exception);
-
-            return false;
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -129,70 +197,21 @@ public class AuthenticationManagerImpl implements AuthenticationManager {
         NamedListView<? extends AuthenticationProviderView> providers = view.providers();
 
         return providers.stream()
+                .sorted(Comparator.comparing((AuthenticationProviderView o) -> o.name()))
                 .map(AuthenticatorFactory::create)
                 .collect(Collectors.toList());
     }
 
-    private void emitEvents(ConfigurationNotificationEvent<SecurityView> ctx) {
-        SecurityView oldValue = ctx.oldValue();
-        SecurityView newValue = ctx.newValue();
-
-        // Authentication enabled/disabled.
-        if ((oldValue == null || oldValue.enabled()) && !newValue.enabled()) {
-            notifyListeners(() -> AUTHENTICATION_DISABLED);
-        } else if ((oldValue == null || !oldValue.enabled()) && newValue.enabled()) {
-            notifyListeners(() -> AUTHENTICATION_ENABLED);
-        }
-
-        if (oldValue != null) {
-            // Authentication providers removed.
-            oldValue.authentication()
-                    .providers()
-                    .stream()
-                    .map(AuthenticationProviderView::name)
-                    .filter(it -> newValue.authentication().providers().get(it) == null)
-                    .map(AuthenticationProviderEvent::removed)
-                    .forEach(this::notifyListeners);
-
-            // Authentication providers updated.
-            oldValue.authentication()
-                    .providers()
-                    .stream()
-                    .filter(oldProvider -> {
-                        AuthenticationProviderView newProvider = newValue.authentication().providers().get(oldProvider.name());
-                        return newProvider != null && !AuthenticationProviderEqualityVerifier.areEqual(oldProvider, newProvider);
-                    })
-                    .map(AuthenticationProviderView::name)
-                    .map(AuthenticationProviderEvent::updated)
-                    .forEach(this::notifyListeners);
-        }
+    private CompletableFuture<Void> fireEvent(AuthenticationEventParameters parameters) {
+        return fireEvent(parameters.type(), parameters);
     }
 
-    private void notifyListeners(AuthenticationEvent event) {
-        listeners.forEach(listener -> {
-            try {
-                listener.onEvent(event);
-            } catch (Exception exception) {
-                LOG.error("Couldn't notify listener", exception);
-            }
-        });
-    }
 
     @Override
     public boolean authenticationEnabled() {
         return authEnabled;
     }
 
-    @Override
-    public void listen(AuthenticationListener listener) {
-        listeners.add(listener);
-    }
-
-    @Override
-    public void stopListen(AuthenticationListener listener) {
-        listeners.remove(listener);
-    }
-
     @TestOnly
     public void authEnabled(boolean authEnabled) {
         this.authEnabled = authEnabled;
diff --git a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationProviderEqualityVerifier.java b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationProviderEqualityVerifier.java
deleted file mode 100644
index 59f1a316ea..0000000000
--- a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationProviderEqualityVerifier.java
+++ /dev/null
@@ -1,86 +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.ignite.internal.security.authentication;
-
-import java.util.Objects;
-import org.apache.ignite.configuration.NamedListView;
-import org.apache.ignite.internal.security.authentication.basic.BasicAuthenticationProviderView;
-import org.apache.ignite.internal.security.authentication.basic.BasicUserView;
-import org.apache.ignite.internal.security.authentication.configuration.AuthenticationProviderView;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Equality verifier for {@link AuthenticationProviderView}.
- */
-public class AuthenticationProviderEqualityVerifier {
-    /**
-     * Checks if two {@link AuthenticationProviderView} are equal.
-     *
-     * @param o1 First object.
-     * @param o2 Second object.
-     * @return {@code true} if objects are equal, {@code false} otherwise.
-     */
-    public static boolean areEqual(@Nullable AuthenticationProviderView o1, @Nullable AuthenticationProviderView o2) {
-        if (o1 == o2) {
-            return true;
-        }
-
-        if (o1 == null || o2 == null) {
-            return false;
-        }
-
-        if (o1.getClass() != o2.getClass()) {
-            return false;
-        }
-
-        if (!o1.type().equals(o2.type())) {
-            return false;
-        }
-
-        if (!o1.name().equals(o2.name())) {
-            return false;
-        }
-
-        if (o1 instanceof BasicAuthenticationProviderView) {
-            return areEqual((BasicAuthenticationProviderView) o1, (BasicAuthenticationProviderView) o2);
-        }
-
-        return false;
-    }
-
-    private static boolean areEqual(BasicAuthenticationProviderView o1, BasicAuthenticationProviderView o2) {
-        NamedListView<? extends BasicUserView> users1 = o1.users();
-        NamedListView<? extends BasicUserView> users2 = o2.users();
-        if (users1.size() != users2.size()) {
-            return false;
-        }
-
-        for (BasicUserView basicUser1View : users1) {
-            BasicUserView basicUser2View = users2.get(basicUser1View.username());
-            if (basicUser2View == null) {
-                return false;
-            }
-            if (!Objects.equals(basicUser1View.username(), basicUser2View.username())
-                    || !Objects.equals(basicUser1View.password(), basicUser2View.password())) {
-                return false;
-            }
-        }
-
-        return true;
-    }
-}
diff --git a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationUtils.java b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationUtils.java
new file mode 100644
index 0000000000..113b857729
--- /dev/null
+++ b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationUtils.java
@@ -0,0 +1,56 @@
+/*
+ * 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.ignite.internal.security.authentication;
+
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.internal.security.authentication.basic.BasicAuthenticationProviderView;
+import org.apache.ignite.internal.security.authentication.basic.BasicProviderNotFoundException;
+import org.apache.ignite.internal.security.authentication.configuration.AuthenticationProviderView;
+
+/**
+ * Utility class for authentication.
+ */
+public final class AuthenticationUtils {
+    private AuthenticationUtils() {
+        // No-op.
+    }
+
+    /**
+     * Find the name of the basic authentication provider.
+     *
+     * @param providerViews Authentication provider views.
+     * @return Name of the basic authentication provider.
+     */
+    public static String findBasicProviderName(NamedListView<? extends AuthenticationProviderView> providerViews) {
+        return findBasicProvider(providerViews).name();
+    }
+
+    /**
+     * Find the basic authentication provider.
+     *
+     * @param providerViews Authentication provider views.
+     * @return Basic authentication provider.
+     */
+    public static BasicAuthenticationProviderView findBasicProvider(NamedListView<? extends AuthenticationProviderView> providerViews) {
+        return providerViews.stream()
+                .filter(BasicAuthenticationProviderView.class::isInstance)
+                .map(BasicAuthenticationProviderView.class::cast)
+                .findFirst()
+                .orElseThrow(BasicProviderNotFoundException::new);
+    }
+}
diff --git a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticatorFactory.java b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticatorFactory.java
index d9da4fc7b8..8799932c9c 100644
--- a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticatorFactory.java
+++ b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticatorFactory.java
@@ -22,21 +22,21 @@ import org.apache.ignite.internal.security.authentication.basic.BasicAuthenticat
 import org.apache.ignite.internal.security.authentication.basic.BasicAuthenticator;
 import org.apache.ignite.internal.security.authentication.basic.BasicUser;
 import org.apache.ignite.internal.security.authentication.configuration.AuthenticationProviderView;
-import org.apache.ignite.security.AuthenticationType;
 
 /** Factory for {@link Authenticator}. */
 class AuthenticatorFactory {
     static Authenticator create(AuthenticationProviderView view) {
-        AuthenticationType type = AuthenticationType.parse(view.type());
-        if (type == AuthenticationType.BASIC) {
+        if (view instanceof BasicAuthenticationProviderView) {
             BasicAuthenticationProviderView basicAuthProviderView = (BasicAuthenticationProviderView) view;
             return new BasicAuthenticator(
                     view.name(),
-                    basicAuthProviderView.users().stream().map(basicUserView -> new BasicUser(basicUserView.username(),
-                    basicUserView.password())).collect(Collectors.toList())
+                    basicAuthProviderView.users()
+                            .stream()
+                            .map(basicUserView -> new BasicUser(basicUserView.username(), basicUserView.password()))
+                            .collect(Collectors.toList())
             );
         } else {
-            throw new IllegalArgumentException("Unexpected authentication type: " + type);
+            throw new IllegalArgumentException("Unexpected authentication provider view: " + view);
         }
     }
 }
diff --git a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/SecurityConfigurationModule.java b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/SecurityConfigurationModule.java
index 96f7f23d08..e62d163b67 100644
--- a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/SecurityConfigurationModule.java
+++ b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/SecurityConfigurationModule.java
@@ -36,11 +36,11 @@ import org.apache.ignite.internal.security.configuration.SecurityConfiguration;
  */
 @AutoService(ConfigurationModule.class)
 public class SecurityConfigurationModule implements ConfigurationModule {
-    private static final String DEFAULT_PROVIDER_NAME = "default";
+    static final String DEFAULT_PROVIDER_NAME = "default";
 
-    private static final String DEFAULT_USERNAME = "ignite";
+    static final String DEFAULT_USERNAME = "ignite";
 
-    private static final String DEFAULT_PASSWORD = "ignite";
+    static final String DEFAULT_PASSWORD = "ignite";
 
     @Override
     public ConfigurationType type() {
diff --git a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationListener.java b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/basic/BasicProviderNotFoundException.java
similarity index 61%
rename from modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationListener.java
rename to modules/security/src/main/java/org/apache/ignite/internal/security/authentication/basic/BasicProviderNotFoundException.java
index 1f6178ad75..3460ce2f94 100644
--- a/modules/security-api/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationListener.java
+++ b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/basic/BasicProviderNotFoundException.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.security.authentication.event;
+package org.apache.ignite.internal.security.authentication.basic;
 
-/**
- * Authentication events listener.
- */
-public interface AuthenticationListener {
-    /**
-     * Handle authentication event.
-     */
-    void onEvent(AuthenticationEvent event);
+import org.apache.ignite.internal.lang.IgniteInternalException;
+import org.apache.ignite.lang.ErrorGroups.Authentication;
+
+/** Thrown when there are no basic provider defined in the authentication configuration. */
+public class BasicProviderNotFoundException extends IgniteInternalException {
+    public BasicProviderNotFoundException() {
+        super(Authentication.BASIC_PROVIDER_ERR, "Basic authentication provider is not found");
+    }
 }
diff --git a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationProviderEventFactory.java b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationProviderEventFactory.java
new file mode 100644
index 0000000000..fd4e2edbab
--- /dev/null
+++ b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/event/AuthenticationProviderEventFactory.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.security.authentication.event;
+
+import static org.apache.ignite.internal.security.authentication.configuration.AuthenticationProviderConfigurationSchema.TYPE_BASIC;
+import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.security.authentication.basic.BasicAuthenticationProviderConfiguration;
+import org.apache.ignite.internal.security.authentication.configuration.AuthenticationProviderConfiguration;
+import org.apache.ignite.internal.security.authentication.configuration.AuthenticationProviderView;
+import org.apache.ignite.internal.security.configuration.SecurityConfiguration;
+
+/**
+ * Event factory for authentication provider configuration changes. Fires events when authentication providers are added, removed or
+ * updated.
+ */
+public class AuthenticationProviderEventFactory implements ConfigurationNamedListListener<AuthenticationProviderView> {
+    private final SecurityConfiguration securityConfiguration;
+
+    private final UserEventFactory userEventFactory;
+
+    private final Function<AuthenticationEventParameters, CompletableFuture<Void>> notifier;
+
+    /**
+     * Constructor.
+     *
+     * @param securityConfiguration Security configuration.
+     * @param userEventFactory User event factory.
+     * @param notifier Notifier.
+     */
+    public AuthenticationProviderEventFactory(
+            SecurityConfiguration securityConfiguration,
+            UserEventFactory userEventFactory,
+            Function<AuthenticationEventParameters, CompletableFuture<Void>> notifier
+    ) {
+        this.securityConfiguration = securityConfiguration;
+        this.userEventFactory = userEventFactory;
+        this.notifier = notifier;
+    }
+
+    @Override
+    public CompletableFuture<?> onCreate(ConfigurationNotificationEvent<AuthenticationProviderView> ctx) {
+        onCreate(ctx.newValue());
+        return nullCompletedFuture();
+    }
+
+    private void onCreate(AuthenticationProviderView providerView) {
+        if (TYPE_BASIC.equals(providerView.type())) {
+            AuthenticationProviderConfiguration configuration = securityConfiguration.authentication()
+                    .providers()
+                    .get(providerView.name());
+            if (configuration != null) {
+                BasicAuthenticationProviderConfiguration basicCfg = (BasicAuthenticationProviderConfiguration) configuration;
+                basicCfg.users().listenElements(userEventFactory);
+            }
+        }
+    }
+
+    @Override
+    public CompletableFuture<?> onRename(ConfigurationNotificationEvent<AuthenticationProviderView> ctx) {
+        onCreate(ctx.newValue());
+        return notifier.apply(AuthenticationProviderEventParameters.removed(ctx.oldValue().name()));
+    }
+
+    @Override
+    public CompletableFuture<?> onDelete(ConfigurationNotificationEvent<AuthenticationProviderView> ctx) {
+        return notifier.apply(AuthenticationProviderEventParameters.removed(ctx.oldValue().name()));
+    }
+
+    @Override
+    public CompletableFuture<?> onUpdate(ConfigurationNotificationEvent<AuthenticationProviderView> ctx) {
+        if (TYPE_BASIC.equals(ctx.oldValue().type()) && ctx.oldValue().type().equals(ctx.newValue().type())) {
+            return nullCompletedFuture();
+        } else {
+            return notifier.apply(AuthenticationProviderEventParameters.updated(ctx.newValue().name()));
+        }
+    }
+}
diff --git a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/event/SecurityEnabledDisabledEventFactory.java b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/event/SecurityEnabledDisabledEventFactory.java
new file mode 100644
index 0000000000..c343c73649
--- /dev/null
+++ b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/event/SecurityEnabledDisabledEventFactory.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.ignite.internal.security.authentication.event;
+
+import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+
+/**
+ * Event factory for authentication switch changes. Fires events when authentication is enabled or disabled.
+ */
+public class SecurityEnabledDisabledEventFactory implements ConfigurationListener<Boolean> {
+    private final Function<AuthenticationEventParameters, CompletableFuture<Void>> notifier;
+
+    public SecurityEnabledDisabledEventFactory(Function<AuthenticationEventParameters, CompletableFuture<Void>> notifier) {
+        this.notifier = notifier;
+    }
+
+    @Override
+    public CompletableFuture<?> onUpdate(ConfigurationNotificationEvent<Boolean> ctx) {
+        Boolean oldValue = ctx.oldValue();
+        Boolean newValue = ctx.newValue();
+
+        if (oldValue == null || !oldValue.equals(newValue)) {
+            return notifier.apply(AuthenticationSwitchedParameters.enabled(newValue));
+        } else {
+            return nullCompletedFuture();
+        }
+    }
+}
diff --git a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/event/UserEventFactory.java b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/event/UserEventFactory.java
new file mode 100644
index 0000000000..76f488e0cd
--- /dev/null
+++ b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/event/UserEventFactory.java
@@ -0,0 +1,58 @@
+/*
+ * 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.ignite.internal.security.authentication.event;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.security.authentication.AuthenticationUtils;
+import org.apache.ignite.internal.security.authentication.basic.BasicUserView;
+import org.apache.ignite.internal.security.authentication.configuration.AuthenticationView;
+
+/**
+ * Event factory for user configuration changes. Fires events when basic users are added, removed or updated.
+ */
+public class UserEventFactory implements ConfigurationNamedListListener<BasicUserView> {
+    private final Function<AuthenticationEventParameters, CompletableFuture<Void>> notifier;
+
+    public UserEventFactory(Function<AuthenticationEventParameters, CompletableFuture<Void>> notifier) {
+        this.notifier = notifier;
+    }
+
+    @Override
+    public CompletableFuture<?> onRename(ConfigurationNotificationEvent<BasicUserView> ctx) {
+        AuthenticationView authenticationView = ctx.oldValue(AuthenticationView.class);
+        String basicProviderName = AuthenticationUtils.findBasicProviderName(authenticationView.providers());
+        return notifier.apply(UserEventParameters.removed(basicProviderName, ctx.oldValue().username()));
+    }
+
+    @Override
+    public CompletableFuture<?> onDelete(ConfigurationNotificationEvent<BasicUserView> ctx) {
+        AuthenticationView authenticationView = ctx.oldValue(AuthenticationView.class);
+        String basicProviderName = AuthenticationUtils.findBasicProviderName(authenticationView.providers());
+        return notifier.apply(UserEventParameters.removed(basicProviderName, ctx.oldValue().username()));
+    }
+
+    @Override
+    public CompletableFuture<?> onUpdate(ConfigurationNotificationEvent<BasicUserView> ctx) {
+        AuthenticationView authenticationView = ctx.oldValue(AuthenticationView.class);
+        String basicProviderName = AuthenticationUtils.findBasicProviderName(authenticationView.providers());
+        return notifier.apply(UserEventParameters.updated(basicProviderName, ctx.oldValue().username()));
+    }
+}
diff --git a/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImplTest.java b/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImplTest.java
index 5a941be645..8a93661d50 100644
--- a/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImplTest.java
+++ b/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImplTest.java
@@ -17,202 +17,284 @@
 
 package org.apache.ignite.internal.security.authentication;
 
-import static org.apache.ignite.internal.security.authentication.event.EventType.AUTHENTICATION_DISABLED;
-import static org.apache.ignite.internal.security.authentication.event.EventType.AUTHENTICATION_ENABLED;
-import static org.apache.ignite.internal.security.authentication.event.EventType.AUTHENTICATION_PROVIDER_REMOVED;
-import static org.apache.ignite.internal.security.authentication.event.EventType.AUTHENTICATION_PROVIDER_UPDATED;
+import static org.apache.ignite.internal.security.authentication.event.AuthenticationEvent.AUTHENTICATION_DISABLED;
+import static org.apache.ignite.internal.security.authentication.event.AuthenticationEvent.AUTHENTICATION_ENABLED;
+import static org.apache.ignite.internal.security.authentication.event.AuthenticationEvent.AUTHENTICATION_PROVIDER_REMOVED;
+import static org.apache.ignite.internal.security.authentication.event.AuthenticationEvent.AUTHENTICATION_PROVIDER_UPDATED;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.apache.ignite.internal.util.CompletableFutures.falseCompletedFuture;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertInstanceOf;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 
-import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.function.Consumer;
 import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
 import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.event.EventListener;
 import org.apache.ignite.internal.security.authentication.basic.BasicAuthenticationProviderChange;
 import org.apache.ignite.internal.security.authentication.event.AuthenticationEvent;
-import org.apache.ignite.internal.security.authentication.event.AuthenticationListener;
-import org.apache.ignite.internal.security.authentication.event.AuthenticationProviderEvent;
+import org.apache.ignite.internal.security.authentication.event.AuthenticationEventParameters;
+import org.apache.ignite.internal.security.authentication.event.AuthenticationProviderEventParameters;
+import org.apache.ignite.internal.security.authentication.event.UserEventParameters;
 import org.apache.ignite.internal.security.configuration.SecurityChange;
 import org.apache.ignite.internal.security.configuration.SecurityConfiguration;
-import org.apache.ignite.internal.security.configuration.SecurityView;
 import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
 import org.apache.ignite.security.exception.InvalidCredentialsException;
 import org.apache.ignite.security.exception.UnsupportedAuthenticationTypeException;
+import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
 @ExtendWith(ConfigurationExtension.class)
 class AuthenticationManagerImplTest extends BaseIgniteAbstractTest {
-    private static final String PROVIDER = "basic";
+    private static final String PROVIDER = SecurityConfigurationModule.DEFAULT_PROVIDER_NAME;
 
-    private static final String USERNAME = "admin";
+    private static final String USERNAME = SecurityConfigurationModule.DEFAULT_USERNAME;
 
-    private static final String PASSWORD = "password";
+    private static final String PASSWORD = SecurityConfigurationModule.DEFAULT_PASSWORD;
 
     private static final UsernamePasswordRequest USERNAME_PASSWORD_REQUEST = new UsernamePasswordRequest(USERNAME, PASSWORD);
 
-    private final AuthenticationManagerImpl manager = new AuthenticationManagerImpl();
+    @InjectConfiguration(polymorphicExtensions = CustomAuthenticationProviderConfigurationSchema.class, rootName = "security")
+    private SecurityConfiguration securityConfiguration;
 
-    private final List<AuthenticationEvent> events = new ArrayList<>();
+    private AuthenticationManagerImpl manager;
 
-    private final AuthenticationListener listener = events::add;
+    private final List<AuthenticationEventParameters> events = new CopyOnWriteArrayList<>();
 
-    @InjectConfiguration
-    private SecurityConfiguration securityConfiguration;
+    private final EventListener<AuthenticationEventParameters> listener = (parameters, exception) -> {
+        events.add(parameters);
+        return falseCompletedFuture();
+    };
 
     @BeforeEach
     void setUp() {
-        manager.listen(listener);
-    }
+        manager = new AuthenticationManagerImpl(securityConfiguration);
 
-    @Test
-    public void enableAuth() {
-        // when
-        enableAuthentication();
+        Arrays.stream(AuthenticationEvent.values()).forEach(event -> manager.listen(event, listener));
 
-        // then
-        // successful authentication with valid credentials
+        manager.start();
+    }
 
-        assertEquals(USERNAME, manager.authenticate(USERNAME_PASSWORD_REQUEST).username());
+    @AfterEach
+    void tearDown() throws Exception {
+        Arrays.stream(AuthenticationEvent.values()).forEach(event -> manager.removeListener(event, listener));
 
-        // and failed authentication with invalid credentials
-        assertThrows(InvalidCredentialsException.class,
-                () -> manager.authenticate(new UsernamePasswordRequest(USERNAME, "invalid-password")));
+        manager.stop();
+    }
+
+    @Test
+    void shouldFireEventWhenAuthenticationIsEnabled() {
+        // When authentication is enabled.
+        enableAuthentication();
 
+        // Then event is fired.
         assertEquals(1, events.size());
         assertEquals(AUTHENTICATION_ENABLED, events.get(0).type());
     }
 
     @Test
-    public void leaveOldSettingWhenInvalidConfiguration() {
-        // when
-        SecurityView oldValue = securityConfiguration.value();
+    void shouldFireEventsWhenAuthenticationIsEnabledAndThenDisabled() {
+        // When authentication is enabled.
+        enableAuthentication();
 
-        SecurityView invalidAuthView = mutateConfiguration(
-                securityConfiguration, change -> {
-                    change.changeEnabled(true);
-                })
-                .value();
-        manager.onUpdate(new StubSecurityViewEvent(oldValue, invalidAuthView)).join();
+        // And then disabled.
+        disableAuthentication();
 
-        // then
-        // authentication is still disabled
-        UsernamePasswordRequest emptyCredentials = new UsernamePasswordRequest("", "");
+        // Then event is fired.
+        assertEquals(2, events.size());
+        assertEquals(AUTHENTICATION_ENABLED, events.get(0).type());
+        assertEquals(AUTHENTICATION_DISABLED, events.get(1).type());
+    }
 
-        assertEquals("Unknown", manager.authenticate(emptyCredentials).username());
+    @Test
+    void shouldFireUserUpdatedEventWhenUserPasswordIsChanged() {
+        mutateConfiguration(securityConfiguration, change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .update(PROVIDER, provider -> {
+                        provider.convert(BasicAuthenticationProviderChange.class)
+                                .changeUsers(users ->
+                                        users.update(USERNAME, user -> user.changePassword("new-password"))
+                                );
+                    });
+        });
 
-        assertEquals(0, events.size());
+        assertEquals(1, events.size());
+        UserEventParameters userEventParameters = (UserEventParameters) events.get(0);
+        assertEquals(AuthenticationEvent.USER_UPDATED, userEventParameters.type());
+        assertEquals(USERNAME, userEventParameters.username());
+        assertEquals(PROVIDER, userEventParameters.providerName());
     }
 
     @Test
-    public void disableAuthEmptyProviders() {
-        //when
-        enableAuthentication();
+    void shouldFireUserRemovedEventWhenUserIsDeleted() {
+        mutateConfiguration(securityConfiguration, change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .update(PROVIDER, provider -> {
+                        provider.convert(BasicAuthenticationProviderChange.class)
+                                .changeUsers(users ->
+                                        users.delete(USERNAME)
+                                );
+                    });
+        });
 
-        // then
+        assertEquals(1, events.size());
+        UserEventParameters userEventParameters = (UserEventParameters) events.get(0);
+        assertEquals(AuthenticationEvent.USER_REMOVED, userEventParameters.type());
+        assertEquals(USERNAME, userEventParameters.username());
+        assertEquals(PROVIDER, userEventParameters.providerName());
+    }
 
-        // disable authentication
-        SecurityView currentView = securityConfiguration.value();
+    @Test
+    void shouldFireUserRemovedEventWhenUserIsRenamed() {
+        mutateConfiguration(securityConfiguration, change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .update(PROVIDER, provider -> {
+                        provider.convert(BasicAuthenticationProviderChange.class)
+                                .changeUsers(users ->
+                                        users.rename(USERNAME, USERNAME + "-renamed")
+                                );
+                    });
+        });
 
-        SecurityView disabledView = mutateConfiguration(
-                securityConfiguration, change -> {
-                    change.changeAuthentication().changeProviders(providers -> providers.delete(PROVIDER));
-                    change.changeEnabled(false);
-                })
-                .value();
+        assertEquals(1, events.size());
+        UserEventParameters userEventParameters = (UserEventParameters) events.get(0);
+        assertEquals(AuthenticationEvent.USER_REMOVED, userEventParameters.type());
+        assertEquals(USERNAME, userEventParameters.username());
+        assertEquals(PROVIDER, userEventParameters.providerName());
+    }
 
-        manager.onUpdate(new StubSecurityViewEvent(currentView, disabledView)).join();
+    @Test
+    void shouldFireEventsWhenProviderIsRenamedAndUserPasswordChanged() {
+        mutateConfiguration(securityConfiguration, change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .rename(PROVIDER, PROVIDER + "-renamed");
+        });
+
+        mutateConfiguration(securityConfiguration, change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .update(PROVIDER + "-renamed", provider -> {
+                        provider.convert(BasicAuthenticationProviderChange.class)
+                                .changeUsers(users ->
+                                        users.update(USERNAME, user -> user.changePassword("new-password"))
+                                );
+                    });
+        });
 
-        // then
-        // authentication is disabled
-        UsernamePasswordRequest emptyCredentials = new UsernamePasswordRequest("", "");
+        assertEquals(2, events.size());
 
-        assertEquals("Unknown", manager.authenticate(emptyCredentials).username());
+        AuthenticationProviderEventParameters providerEventParameters = (AuthenticationProviderEventParameters) events.get(0);
+        assertEquals(AUTHENTICATION_PROVIDER_REMOVED, providerEventParameters.type());
+        assertEquals(PROVIDER, providerEventParameters.name());
 
-        assertEquals(3, events.size());
-        assertEquals(AUTHENTICATION_ENABLED, events.get(0).type());
-        assertEquals(AUTHENTICATION_DISABLED, events.get(1).type());
-        AuthenticationProviderEvent removed = assertInstanceOf(AuthenticationProviderEvent.class, events.get(2));
-        assertEquals(AUTHENTICATION_PROVIDER_REMOVED, removed.type());
-        assertEquals(PROVIDER, removed.name());
+        UserEventParameters userEventParameters = (UserEventParameters) events.get(1);
+        assertEquals(AuthenticationEvent.USER_UPDATED, userEventParameters.type());
+        assertEquals(USERNAME, userEventParameters.username());
+        assertEquals(PROVIDER + "-renamed", userEventParameters.providerName());
     }
 
     @Test
-    public void disableAuthNotEmptyProviders() {
-        //when
-        enableAuthentication();
+    void shouldFireProviderUpdatedEventWhenCustomProviderPropertyIsChanged() {
+        mutateConfiguration(securityConfiguration, change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .create("custom", provider -> {
+                        provider.convert(CustomAuthenticationProviderChange.class)
+                                .changeCustomProperty("custom-property");
+                    });
+        });
+
+        mutateConfiguration(securityConfiguration, change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .update("custom", provider -> {
+                        provider.convert(CustomAuthenticationProviderChange.class)
+                                .changeCustomProperty("custom-property2");
+                    });
+        });
 
-        // disable authentication
-        SecurityView currentView = securityConfiguration.value();
-
-        SecurityView disabledView = mutateConfiguration(
-                securityConfiguration, change -> {
-                    change.changeEnabled(false);
-                })
-                .value();
+        assertEquals(1, events.size());
+        AuthenticationProviderEventParameters authenticationProviderEventParameters = (AuthenticationProviderEventParameters) events.get(0);
+        assertEquals(AUTHENTICATION_PROVIDER_UPDATED, authenticationProviderEventParameters.type());
+        assertEquals("custom", authenticationProviderEventParameters.name());
+    }
 
-        manager.onUpdate(new StubSecurityViewEvent(currentView, disabledView)).join();
+    @Test
+    void shouldFireProviderRemovedEventWhenProviderIsRenamed() {
+        mutateConfiguration(securityConfiguration, change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .rename(PROVIDER, PROVIDER + "-renamed");
+        });
 
-        // then
-        // authentication is disabled
-        UsernamePasswordRequest emptyCredentials = new UsernamePasswordRequest("", "");
+        assertEquals(1, events.size());
+        AuthenticationProviderEventParameters authenticationProviderEventParameters = (AuthenticationProviderEventParameters) events.get(0);
+        assertEquals(AUTHENTICATION_PROVIDER_REMOVED, authenticationProviderEventParameters.type());
+        assertEquals(PROVIDER, authenticationProviderEventParameters.name());
+    }
 
-        assertEquals("Unknown", manager.authenticate(emptyCredentials).username());
+    @Test
+    void shouldNotFireProviderUpdatedEventForChangesInBasicProviderUsers() {
+        mutateConfiguration(securityConfiguration, change -> {
+            change.changeAuthentication()
+                    .changeProviders()
+                    .update(PROVIDER, provider -> {
+                        provider.convert(BasicAuthenticationProviderChange.class)
+                                .changeUsers(users ->
+                                        users.create("new-user", user -> user.changePassword("new-password"))
+                                );
+                    });
+        });
 
-        assertEquals(2, events.size());
-        assertEquals(AUTHENTICATION_ENABLED, events.get(0).type());
-        assertEquals(AUTHENTICATION_DISABLED, events.get(1).type());
+        assertEquals(0, events.size());
     }
 
     @Test
-    public void changedCredentials() {
+    void shouldAuthenticateWithValidCredentials() {
         // when
         enableAuthentication();
 
         // then
-        // change authentication settings - change password
-        SecurityView currentView = securityConfiguration.value();
-
-        SecurityView adminNewPasswordView = mutateConfiguration(
-                securityConfiguration, change -> {
-                    change.changeAuthentication().changeProviders(providers -> providers.update(PROVIDER, provider -> {
-                        provider.convert(BasicAuthenticationProviderChange.class)
-                                .changeUsers(users ->
-                                        users.update(USERNAME, user -> user.changePassword("new-password"))
-                                );
-                    }));
-                })
-                .value();
-
-        manager.onUpdate(new StubSecurityViewEvent(currentView, adminNewPasswordView)).join();
+        // successful authentication with valid credentials
+        assertEquals(USERNAME, manager.authenticate(USERNAME_PASSWORD_REQUEST).username());
+    }
 
-        assertThrows(InvalidCredentialsException.class, () -> manager.authenticate(USERNAME_PASSWORD_REQUEST));
+    @Test
+    void shouldThrowInvalidCredentialsExceptionForInvalidCredentials() {
+        // when
+        enableAuthentication();
 
         // then
-        // successful authentication with the new password
-        UsernamePasswordRequest adminNewPasswordCredentials = new UsernamePasswordRequest(USERNAME, "new-password");
+        // failed authentication with invalid credentials
+        assertThrows(InvalidCredentialsException.class,
+                () -> manager.authenticate(new UsernamePasswordRequest(USERNAME, "invalid-password")));
+    }
 
-        assertEquals(USERNAME, manager.authenticate(adminNewPasswordCredentials).username());
+    @Test
+    void shouldReturnUnknownUserDetailsWhenAuthenticationIsDisabled() {
+        // when
+        disableAuthentication();
 
-        assertEquals(2, events.size());
-        assertEquals(AUTHENTICATION_ENABLED, events.get(0).type());
-        AuthenticationProviderEvent removed = assertInstanceOf(AuthenticationProviderEvent.class, events.get(1));
-        assertEquals(AUTHENTICATION_PROVIDER_UPDATED, removed.type());
-        assertEquals(PROVIDER, removed.name());
+        // then
+        assertEquals(UserDetails.UNKNOWN, manager.authenticate(USERNAME_PASSWORD_REQUEST));
     }
 
     @Test
-    public void exceptionsDuringAuthentication() {
+    public void shouldAuthenticateWithFallbackOnSequentialAuthenticatorExceptions() {
         UsernamePasswordRequest credentials = new UsernamePasswordRequest("admin", "password");
 
         Authenticator authenticator1 = mock(Authenticator.class);
@@ -239,28 +321,16 @@ class AuthenticationManagerImplTest extends BaseIgniteAbstractTest {
     }
 
     private void enableAuthentication() {
-        SecurityView oldValue = securityConfiguration.value();
-
-        SecurityView adminPasswordView = mutateConfiguration(
-                securityConfiguration, change -> {
-                    change.changeAuthentication().changeProviders(providers -> providers.create(PROVIDER, provider -> {
-                        provider.convert(BasicAuthenticationProviderChange.class)
-                                .changeUsers(users ->
-                                        users.create(USERNAME, user -> user.changePassword(PASSWORD))
-                                );
-                    }));
-                    change.changeEnabled(true);
-                })
-                .value();
+        mutateConfiguration(securityConfiguration, change -> change.changeEnabled(true));
+    }
 
-        manager.onUpdate(new StubSecurityViewEvent(oldValue, adminPasswordView)).join();
+    private void disableAuthentication() {
+        mutateConfiguration(securityConfiguration, change -> change.changeEnabled(false));
     }
 
-    private static SecurityConfiguration mutateConfiguration(SecurityConfiguration configuration,
-            Consumer<SecurityChange> consumer) {
+    private static void mutateConfiguration(SecurityConfiguration configuration, Consumer<SecurityChange> consumer) {
         CompletableFuture<SecurityConfiguration> future = configuration.change(consumer)
                 .thenApply(unused -> configuration);
         assertThat(future, willCompleteSuccessfully());
-        return future.join();
     }
 }
diff --git a/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/CustomAuthenticationProviderConfigurationSchema.java b/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/CustomAuthenticationProviderConfigurationSchema.java
index 0d9e70e61f..71c5b175bb 100644
--- a/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/CustomAuthenticationProviderConfigurationSchema.java
+++ b/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/CustomAuthenticationProviderConfigurationSchema.java
@@ -18,10 +18,14 @@
 package org.apache.ignite.internal.security.authentication;
 
 import org.apache.ignite.configuration.annotation.PolymorphicConfigInstance;
+import org.apache.ignite.configuration.annotation.Value;
 import org.apache.ignite.internal.security.authentication.configuration.AuthenticationProviderConfigurationSchema;
 
 /** Custom authentication configuration. */
 @PolymorphicConfigInstance(CustomAuthenticationProviderConfigurationSchema.TYPE)
 public class CustomAuthenticationProviderConfigurationSchema extends AuthenticationProviderConfigurationSchema {
     static final String TYPE = "custom";
+
+    @Value(hasDefault = true)
+    public String customProperty = "customValue";
 }