You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ni...@apache.org on 2021/04/30 15:11:14 UTC

[ignite] branch master updated: IGNITE-14335 Merges APIs of IgniteAuthenticationProcessor and IgniteSecurity. (#8892)

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

nizhikov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 661ba3f  IGNITE-14335 Merges APIs of IgniteAuthenticationProcessor and IgniteSecurity. (#8892)
661ba3f is described below

commit 661ba3f58c3151f7ecf14c0509ff2ffc412a5b93
Author: Mikhail Petrov <32...@users.noreply.github.com>
AuthorDate: Fri Apr 30 18:10:50 2021 +0300

    IGNITE-14335 Merges APIs of IgniteAuthenticationProcessor and IgniteSecurity. (#8892)
---
 .../src/test/config/jdbc-security-config.xml       |  68 +++
 .../common/RunningQueryInfoCheckInitiatorTest.java |   2 +-
 .../client/suite/IgniteClientTestSuite.java        |   2 +
 ...ttyRestProcessorAuthenticationAbstractTest.java |  10 +-
 ...thenticatorUserManagementAuthorizationTest.java | 112 +++++
 .../JdbcThinAuthenticateConnectionSelfTest.java    |  12 +-
 .../apache/ignite/internal/GridKernalContext.java  |   8 -
 .../ignite/internal/GridKernalContextImpl.java     |  12 -
 .../org/apache/ignite/internal/IgniteKernal.java   |  15 +-
 .../managers/discovery/GridDiscoveryManager.java   |   4 +-
 .../authentication/AuthorizationContext.java       |  89 ----
 .../IgniteAuthenticationProcessor.java             | 502 +++++++++++++--------
 .../processors/cache/GridCacheProcessor.java       |   4 +-
 .../preloader/GridDhtPartitionsExchangeFuture.java |   4 +-
 .../wal/reader/StandaloneGridKernalContext.java    |   6 -
 .../ClientListenerAbstractConnectionContext.java   |  42 +-
 .../odbc/ClientListenerConnectionContext.java      |   8 -
 .../processors/odbc/ClientListenerNioListener.java |  10 -
 .../processors/odbc/ClientListenerProcessor.java   |  12 +-
 .../odbc/jdbc/JdbcConnectionContext.java           |   6 +-
 .../processors/odbc/jdbc/JdbcRequestHandler.java   |  12 -
 .../odbc/odbc/OdbcConnectionContext.java           |   5 +-
 .../processors/odbc/odbc/OdbcRequestHandler.java   |  12 -
 .../platform/client/ClientConnectionContext.java   |   5 +-
 .../platform/client/ClientRequestHandler.java      |   8 +-
 .../processors/rest/GridRestProcessor.java         |  66 +--
 .../handlers/user/UserActionCommandHandler.java    |  13 +-
 .../processors/rest/request/GridRestRequest.java   |  16 +-
 .../processors/security/GridSecurityProcessor.java |  32 ++
 .../processors/security/IgniteSecurity.java        |  26 ++
 .../security/IgniteSecurityProcessor.java          |  20 +
 .../security/NoOpIgniteSecurityProcessor.java      |  22 +-
 .../processors/security/SecurityUtils.java         |  12 +
 .../apache/ignite/internal/util/lang/GridFunc.java |  10 +
 .../plugin/security/AuthenticationContext.java     |  21 -
 .../ignite/spi/discovery/tcp/ServerImpl.java       |   3 +-
 .../spi/systemview/view/ClientConnectionView.java  |  10 +-
 .../Authentication1kUsersNodeRestartTest.java      |  28 +-
 .../AuthenticationConfigurationClusterTest.java    |  80 ++--
 .../AuthenticationOnNotActiveClusterTest.java      |  22 +-
 .../AuthenticationProcessorNodeRestartTest.java    |  58 +--
 .../AuthenticationProcessorSelfTest.java           | 289 +++++++-----
 .../processors/query/h2/CommandProcessor.java      |   6 +-
 .../authentication/SqlUserCommandSelfTest.java     |  55 +--
 44 files changed, 1004 insertions(+), 755 deletions(-)

diff --git a/modules/clients/src/test/config/jdbc-security-config.xml b/modules/clients/src/test/config/jdbc-security-config.xml
new file mode 100644
index 0000000..791c9d2
--- /dev/null
+++ b/modules/clients/src/test/config/jdbc-security-config.xml
@@ -0,0 +1,68 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!--
+    Ignite Spring configuration file.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <!-- JDBC driver should force true value -->
+        <property name="clientMode" value="false"/>
+
+        <property name="igniteInstanceName" value="sqlGrid"/>
+        <property name="localHost" value="127.0.0.1"/>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="dataStorageConfiguration">
+            <bean class="org.apache.ignite.configuration.DataStorageConfiguration">
+                <property name="defaultDataRegionConfiguration">
+                    <bean class="org.apache.ignite.configuration.DataRegionConfiguration">
+                        <property name="persistenceEnabled" value="true"/>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="sqlSchemas">
+            <array>
+                <value>PREDEFINED_CLIENT_SCHEMA</value>
+            </array>
+        </property>
+
+        <property name="peerClassLoadingEnabled" value="true"/>
+        <property name="authenticationEnabled" value="true" />
+    </bean>
+</beans>
diff --git a/modules/clients/src/test/java/org/apache/ignite/common/RunningQueryInfoCheckInitiatorTest.java b/modules/clients/src/test/java/org/apache/ignite/common/RunningQueryInfoCheckInitiatorTest.java
index 16799ee..d807b88 100644
--- a/modules/clients/src/test/java/org/apache/ignite/common/RunningQueryInfoCheckInitiatorTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/common/RunningQueryInfoCheckInitiatorTest.java
@@ -227,7 +227,7 @@ public class RunningQueryInfoCheckInitiatorTest extends JdbcThinAbstractSelfTest
 
             GridTestUtils.runAsync(() -> {
                     try (Connection conn = DriverManager.getConnection(
-                        CFG_URL_PREFIX + "nodeId=" + grid0NodeId + "@modules/clients/src/test/config/jdbc-config.xml")) {
+                        CFG_URL_PREFIX + "nodeId=" + grid0NodeId + "@modules/clients/src/test/config/jdbc-security-config.xml")) {
                         try (Statement stmt = conn.createStatement()) {
                             stmt.execute(sql);
                         }
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
index 0058bac..222cba3 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.processors.rest.ChangeStateCommandHandlerTest;
 import org.apache.ignite.internal.processors.rest.ClientMemcachedProtocolSelfTest;
 import org.apache.ignite.internal.processors.rest.JettyRestProcessorAuthenticationWithCredsSelfTest;
 import org.apache.ignite.internal.processors.rest.JettyRestProcessorAuthenticationWithTokenSelfTest;
+import org.apache.ignite.internal.processors.rest.JettyRestProcessorAuthenticatorUserManagementAuthorizationTest;
 import org.apache.ignite.internal.processors.rest.JettyRestProcessorBaselineSelfTest;
 import org.apache.ignite.internal.processors.rest.JettyRestProcessorBeforeNodeStartSelfTest;
 import org.apache.ignite.internal.processors.rest.JettyRestProcessorGetAllAsArrayTest;
@@ -99,6 +100,7 @@ import org.junit.runners.Suite;
     JettyRestProcessorUnsignedSelfTest.class,
     JettyRestProcessorAuthenticationWithCredsSelfTest.class,
     JettyRestProcessorAuthenticationWithTokenSelfTest.class,
+    JettyRestProcessorAuthenticatorUserManagementAuthorizationTest.class,
     JettyRestProcessorGetAllAsArrayTest.class,
     JettyRestProcessorBaselineSelfTest.class,
     JettyRestProcessorBeforeNodeStartSelfTest.class,
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationAbstractTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationAbstractTest.java
index a16c52a..6ecc2e1 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationAbstractTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationAbstractTest.java
@@ -23,12 +23,12 @@ import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException;
-import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.junit.Test;
 
 import static org.apache.ignite.configuration.WALMode.NONE;
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.authenticate;
 
 /**
  * Test REST with enabled authentication.
@@ -112,9 +112,7 @@ public abstract class JettyRestProcessorAuthenticationAbstractTest extends Jetty
 
         assertTrue(res.asBoolean());
 
-        IgniteAuthenticationProcessor auth = grid(0).context().authentication();
-
-        assertNotNull(auth.authenticate("user1", "password1"));
+        assertNotNull(authenticate(grid(0), "user1", "password1"));
 
         // Update user password.
         ret = content(null, GridRestCommand.UPDATE_USER,
@@ -125,7 +123,7 @@ public abstract class JettyRestProcessorAuthenticationAbstractTest extends Jetty
 
         assertTrue(res.asBoolean());
 
-        assertNotNull(auth.authenticate("user1", "password2"));
+        assertNotNull(authenticate(grid(0), "user1", "password2"));
 
         // Remove user.
         ret = content(null, GridRestCommand.REMOVE_USER,
@@ -137,7 +135,7 @@ public abstract class JettyRestProcessorAuthenticationAbstractTest extends Jetty
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                grid(0).context().authentication().authenticate("user1", "password1");
+                authenticate(grid(0), "user1", "password1");
 
                 return null;
             }
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticatorUserManagementAuthorizationTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticatorUserManagementAuthorizationTest.java
new file mode 100644
index 0000000..5b2e9fd
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticatorUserManagementAuthorizationTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.processors.rest;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
+import org.junit.Test;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.internal.processors.authentication.User.DFAULT_USER_NAME;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.ADD_USER;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.REMOVE_USER;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.UPDATE_USER;
+
+/**
+ * Tests authorization of user management commands executed through REST in case {@link IgniteAuthenticationProcessor}
+ * is used as the Ignite security implementation.
+ */
+public class JettyRestProcessorAuthenticatorUserManagementAuthorizationTest extends JettyRestProcessorCommonSelfTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setAuthenticationEnabled(true);
+
+        cfg.setDataStorageConfiguration(new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(new DataRegionConfiguration()
+                .setMaxSize(100 * 1024 * 1024)
+                .setPersistenceEnabled(true)));
+
+        return cfg;
+    }
+
+    /** Tests authorization of user management commands executed through REST. */
+    @Test
+    public void testUserManagementAuthorization() throws Exception {
+        startGrid().cluster().state(ACTIVE);
+
+        String dfltUserPwd = "ignite";
+        String userLogin = "user";
+        String userPwd = "pwd";
+
+        checkRestRequest(DFAULT_USER_NAME, dfltUserPwd, ADD_USER, userLogin, userPwd, null);
+
+        checkRestRequest(userLogin, userPwd, ADD_USER, "not-allowed-user", userPwd,
+            "User management operations are not allowed for user. [curUser=user]");
+
+        checkRestRequest(userLogin, userPwd, UPDATE_USER, DFAULT_USER_NAME, "new-pwd",
+            "User management operations are not allowed for user. [curUser=user]");
+
+        checkRestRequest(DFAULT_USER_NAME, dfltUserPwd, UPDATE_USER, DFAULT_USER_NAME, "new-pwd", null);
+        dfltUserPwd = "new-pwd";
+
+        checkRestRequest(userLogin, userPwd, UPDATE_USER, userLogin, "new-pwd", null);
+
+        checkRestRequest(DFAULT_USER_NAME, dfltUserPwd, UPDATE_USER, userLogin, "pwd", null);
+
+        checkRestRequest(userLogin, userPwd, REMOVE_USER, DFAULT_USER_NAME, null,
+            "User management operations are not allowed for user. [curUser=user]");
+
+        checkRestRequest(DFAULT_USER_NAME, dfltUserPwd, REMOVE_USER, DFAULT_USER_NAME, null,
+            "Default user cannot be removed.");
+
+        checkRestRequest(DFAULT_USER_NAME, dfltUserPwd, REMOVE_USER, userLogin, null, null);
+    }
+
+    /** Checks REST request execution. */
+    public void checkRestRequest(
+        String login,
+        String pwd,
+        GridRestCommand cmd,
+        String loginParam,
+        String pwdParam,
+        String expErr
+    ) throws Exception {
+       JsonNode res = JSON_MAPPER.readTree(content(null, cmd,
+            "ignite.login", login,
+            "ignite.password", pwd,
+            "user", loginParam,
+            "password", pwdParam == null ? "" : pwdParam));
+
+        if (expErr == null) {
+            assertEquals(0, res.get("successStatus").intValue());
+            assertNull(res.get("error").textValue());
+        }
+        else
+            assertTrue(res.get("error").textValue().contains(expErr));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String signature() {
+        return null;
+    }
+}
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAuthenticateConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAuthenticateConnectionSelfTest.java
index 1912843..b6bcf41 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAuthenticateConnectionSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAuthenticateConnectionSelfTest.java
@@ -25,11 +25,13 @@ import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.junit.Test;
 
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.authenticate;
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.withSecurityContextOnAllNodes;
+
 /**
  * Tests for authenticated an non authenticated JDBC thin connection.
  */
@@ -67,11 +69,11 @@ public class JdbcThinAuthenticateConnectionSelfTest extends JdbcThinAbstractSelf
 
         grid(0).cluster().active(true);
 
-        AuthorizationContext.context(grid(0).context().authentication().authenticate("ignite", "ignite"));
+        AutoCloseable secCtxsHnd = withSecurityContextOnAllNodes(authenticate(grid(0), "ignite", "ignite"));
 
-        grid(0).context().authentication().addUser("another_user", "passwd");
+        grid(0).context().security().createUser("another_user", "passwd".toCharArray());
 
-        AuthorizationContext.clear();
+        secCtxsHnd.close();
     }
 
     /** {@inheritDoc} */
@@ -94,7 +96,7 @@ public class JdbcThinAuthenticateConnectionSelfTest extends JdbcThinAbstractSelf
      */
     @Test
     public void testInvalidUserPassword() {
-        String err = "Unauthenticated sessions are prohibited";
+        String err = "The user name or password is incorrect [userName=null]";
         checkInvalidUserPassword(URL, null, null, err);
 
         err = "The user name or password is incorrect";
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index e0e51fc..f73725f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -37,7 +37,6 @@ import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.managers.systemview.GridSystemViewManager;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
-import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor;
 import org.apache.ignite.internal.processors.cache.persistence.defragmentation.IgniteDefragmentation;
@@ -311,13 +310,6 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public <K, V> DataStreamProcessor<K, V> dataStream();
 
     /**
-     * Gets authentication processor.
-     *
-     * @return Authentication processor.
-     */
-    public IgniteAuthenticationProcessor authentication();
-
-    /**
      * Gets event continuous processor.
      *
      * @return Event continuous processor.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 25f4ed5..3fdc5f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -56,7 +56,6 @@ import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.managers.systemview.GridSystemViewManager;
 import org.apache.ignite.internal.managers.tracing.GridTracingManager;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
-import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
@@ -329,10 +328,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     @GridToStringExclude
     private MvccProcessor coordProc;
 
-    /** */
-    @GridToStringExclude
-    private IgniteAuthenticationProcessor authProc;
-
     /** Diagnostic processor. */
     @GridToStringInclude
     private DiagnosticProcessor diagnosticProcessor;
@@ -712,8 +707,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
             pdsFolderRslvr = (PdsFoldersResolver)comp;
         else if (comp instanceof GridInternalSubscriptionProcessor)
             internalSubscriptionProc = (GridInternalSubscriptionProcessor)comp;
-        else if (comp instanceof IgniteAuthenticationProcessor)
-            authProc = (IgniteAuthenticationProcessor)comp;
         else if (comp instanceof IgniteSecurity)
             security = (IgniteSecurity)comp;
         else if (comp instanceof CompressionProcessor)
@@ -1016,11 +1009,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteAuthenticationProcessor authentication() {
-        return authProc;
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteLogger log(String ctgr) {
         return config().getGridLogger().getLogger(ctgr);
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 5ad333a..f7897a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -1250,7 +1250,13 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 startTimer.finishGlobalStage("Configure binary metadata");
 
                 startProcessor(createComponent(IGridClusterStateProcessor.class, ctx));
-                startProcessor(new IgniteAuthenticationProcessor(ctx));
+
+                if (cfg.isAuthenticationEnabled()) {
+                    IgniteSecurityProcessor sec = (IgniteSecurityProcessor)ctx.security();
+
+                    ((IgniteAuthenticationProcessor)sec.securityProcessor()).startProcessor();
+                }
+
                 startProcessor(new PerformanceStatisticsProcessor(ctx));
                 startProcessor(new GridCacheProcessor(ctx));
                 startProcessor(new IndexProcessor(ctx));
@@ -1568,6 +1574,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     private GridProcessor securityProcessor() throws IgniteCheckedException {
         GridSecurityProcessor prc = createComponent(GridSecurityProcessor.class, ctx);
 
+        if (cfg.isAuthenticationEnabled() && !(prc instanceof IgniteAuthenticationProcessor)) {
+            throw new IgniteCheckedException("Invalid security configuration: both authentication is enabled" +
+                " and external security plugin is provided.");
+        }
+
         return prc != null && prc.enabled()
             ? new IgniteSecurityProcessor(ctx, prc)
             : new NoOpIgniteSecurityProcessor(ctx);
@@ -4357,7 +4368,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             return (T)new GridClusterStateProcessor(ctx);
 
         if (cls.equals(GridSecurityProcessor.class))
-            return null;
+            return ctx.config().isAuthenticationEnabled() ? (T)new IgniteAuthenticationProcessor(ctx) : null;
 
         if (cls.equals(IgniteRestProcessor.class))
             return (T)new GridRestProcessor(ctx);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 39b4c5a..d9ebb48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -75,6 +75,7 @@ import org.apache.ignite.internal.managers.GridManagerAdapter;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.managers.systemview.walker.ClusterNodeViewWalker;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.ClientCacheChangeDummyDiscoveryMessage;
 import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
@@ -171,6 +172,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_USER_NAME;
 import static org.apache.ignite.internal.IgniteVersionUtils.VER;
 import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
 import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
+import static org.apache.ignite.internal.processors.security.SecurityUtils.ifAuthenticationEnabled;
 import static org.apache.ignite.internal.processors.security.SecurityUtils.isSecurityCompatibilityMode;
 import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.NOOP;
 
@@ -768,7 +770,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                         ctx.service().onLocalJoin(discoEvt, discoCache);
 
-                        ctx.authentication().onLocalJoin();
+                        ifAuthenticationEnabled(ctx, IgniteAuthenticationProcessor::onLocalJoin);
 
                         ctx.encryption().onLocalJoin();
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/AuthorizationContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/AuthorizationContext.java
deleted file mode 100644
index 551f7f7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/AuthorizationContext.java
+++ /dev/null
@@ -1,89 +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.processors.authentication;
-
-/**
- * Ignite authentication context.
- */
-public class AuthorizationContext {
-    /** User. */
-    private final User user;
-
-    /** Current authorization context. */
-    private static ThreadLocal<AuthorizationContext> actx = new ThreadLocal<>();
-
-    /**
-     * Creates authentication context.
-     *
-     * @param user Authorized user.
-     */
-    public AuthorizationContext(User user) {
-        assert user != null;
-
-        this.user = user;
-    }
-
-    /**
-     * @return Authorized user.
-     */
-    public String userName() {
-        return user.name();
-    }
-
-    /**
-     * @param op User operation to check.
-     * @throws IgniteAccessControlException If operation check fails: user hasn't permissions for user management
-     *      or try to remove default user.
-     */
-    public void checkUserOperation(UserManagementOperation op) throws IgniteAccessControlException {
-        assert op != null;
-
-        if (user == null)
-            throw new IgniteAccessControlException("Operation not allowed: authorized context is empty.");
-
-        if (!User.DFAULT_USER_NAME.equals(user.name())
-            && !(UserManagementOperation.OperationType.UPDATE == op.type() && user.name().equals(op.user().name())))
-            throw new IgniteAccessControlException("User management operations are not allowed for user. " +
-                "[curUser=" + user.name() + ']');
-
-        if (op.type() == UserManagementOperation.OperationType.REMOVE
-            && User.DFAULT_USER_NAME.equals(op.user().name()))
-            throw new IgniteAccessControlException("Default user cannot be removed.");
-    }
-
-    /**
-     * @param actx Authorization context to set.
-     */
-    public static void context(AuthorizationContext actx) {
-        AuthorizationContext.actx.set(actx);
-    }
-
-    /**
-     * Clear authentication context.
-     */
-    public static void clear() {
-        actx.set(null);
-    }
-
-    /**
-     * @return Current authorization context.
-     */
-    public static AuthorizationContext context() {
-        return actx.get();
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/IgniteAuthenticationProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/IgniteAuthenticationProcessor.java
index 4eca2e4..f490681 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/IgniteAuthenticationProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/IgniteAuthenticationProcessor.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.authentication;
 
 import java.io.Serializable;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -37,7 +38,6 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridTopic;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.managers.communication.GridIoManager;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
@@ -52,6 +52,9 @@ import org.apache.ignite.internal.processors.cache.persistence.metastorage.Metas
 import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetastorageTree;
 import org.apache.ignite.internal.processors.cache.persistence.metastorage.ReadOnlyMetastorage;
 import org.apache.ignite.internal.processors.cache.persistence.metastorage.ReadWriteMetastorage;
+import org.apache.ignite.internal.processors.security.GridSecurityProcessor;
+import org.apache.ignite.internal.processors.security.IgniteSecurityProcessor;
+import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -62,7 +65,13 @@ import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteFutureCancelledException;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.spi.IgniteNodeValidationResult;
+import org.apache.ignite.plugin.security.AuthenticationContext;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecurityException;
+import org.apache.ignite.plugin.security.SecurityPermission;
+import org.apache.ignite.plugin.security.SecurityPermissionSet;
+import org.apache.ignite.plugin.security.SecuritySubject;
+import org.apache.ignite.plugin.security.SecuritySubjectType;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -73,11 +82,20 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.AUTH_PROC;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_AUTHENTICATION_ENABLED;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
+import static org.apache.ignite.internal.processors.authentication.UserManagementOperation.OperationType.ADD;
+import static org.apache.ignite.internal.processors.authentication.UserManagementOperation.OperationType.REMOVE;
+import static org.apache.ignite.internal.processors.authentication.UserManagementOperation.OperationType.UPDATE;
+import static org.apache.ignite.plugin.security.SecurityPermissionSetBuilder.ALLOW_ALL;
+import static org.apache.ignite.plugin.security.SecuritySubjectType.REMOTE_CLIENT;
+import static org.apache.ignite.plugin.security.SecuritySubjectType.REMOTE_NODE;
 
 /**
  *
  */
-public class IgniteAuthenticationProcessor extends GridProcessorAdapter implements MetastorageLifecycleListener {
+public class IgniteAuthenticationProcessor extends GridProcessorAdapter implements GridSecurityProcessor,
+    MetastorageLifecycleListener {
     /** Store user prefix. */
     private static final String STORE_USER_PREFIX = "user.";
 
@@ -100,7 +118,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
     private final Map<IgniteUuid, UserManagementOperation> activeOps = Collections.synchronizedMap(new LinkedHashMap<>());
 
     /** User map. */
-    private ConcurrentMap<String, User> users;
+    private ConcurrentMap<UUID, User> users;
 
     /** Shared context. */
     @GridToStringExclude
@@ -115,9 +133,6 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
     /** Coordinator node. */
     private ClusterNode crdNode;
 
-    /** Is authentication enabled. */
-    private boolean isEnabled;
-
     /** Disconnected flag. */
     private volatile boolean disconnected;
 
@@ -136,9 +151,6 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
     /** Node activate future. */
     private final GridFutureAdapter<Void> activateFut = new GridFutureAdapter<>();
 
-    /** Validate error. */
-    private String validateErr;
-
     /**
      * @param ctx Kernal context.
      */
@@ -146,22 +158,16 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
         super(ctx);
     }
 
-    /** {@inheritDoc} */
-    @Override public void start() throws IgniteCheckedException {
-        super.start();
-
-        isEnabled = ctx.config().isAuthenticationEnabled();
-
-        if (isEnabled && !GridCacheUtils.isPersistenceEnabled(ctx.config())) {
-            isEnabled = false;
-
+    /** Starts processor. */
+    public void startProcessor() throws IgniteCheckedException {
+        if (!GridCacheUtils.isPersistenceEnabled(ctx.config())) {
             throw new IgniteCheckedException("Authentication can be enabled only for cluster with enabled persistence."
                 + " Check the DataRegionConfiguration");
         }
 
         ctx.internalSubscriptionProcessor().registerMetastorageListener(this);
 
-        ctx.addNodeAttribute(IgniteNodeAttributes.ATTR_AUTHENTICATION_ENABLED, isEnabled);
+        ctx.addNodeAttribute(ATTR_AUTHENTICATION_ENABLED, true);
 
         GridDiscoveryManager discoMgr = ctx.discovery();
 
@@ -172,7 +178,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
         discoMgr.setCustomEventListener(UserAcceptedMessage.class, new UserAcceptedListener());
 
         discoLsnr = (evt, discoCache) -> {
-            if (!isEnabled || ctx.isStopping())
+            if (ctx.isStopping())
                 return;
 
             switch (evt.type()) {
@@ -190,7 +196,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
         ctx.event().addDiscoveryEventListener(discoLsnr, DISCO_EVT_TYPES);
 
         ioLsnr = (nodeId, msg, plc) -> {
-            if (!isEnabled || ctx.isStopping())
+            if (ctx.isStopping())
                 return;
 
             if (msg instanceof UserManagementOperationFinishedMessage)
@@ -221,44 +227,31 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
 
     /** {@inheritDoc} */
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
-        if (!isEnabled)
-            return;
-
-        ctx.io().removeMessageListener(GridTopic.TOPIC_AUTH, ioLsnr);
+        if (ioLsnr != null)
+            ctx.io().removeMessageListener(GridTopic.TOPIC_AUTH, ioLsnr);
 
-        ctx.event().removeDiscoveryEventListener(discoLsnr, DISCO_EVT_TYPES);
+        if (discoLsnr != null)
+            ctx.event().removeDiscoveryEventListener(discoLsnr, DISCO_EVT_TYPES);
 
         cancelFutures("Node stopped");
 
-        if (!cancel)
-            exec.shutdown();
-        else
-            exec.shutdownNow();
+        if (exec != null) {
+            if (!cancel)
+                exec.shutdown();
+            else
+                exec.shutdownNow();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void onKernalStop(boolean cancel) {
-        if (!isEnabled)
-            return;
-
         synchronized (mux) {
             cancelFutures("Kernal stopped.");
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
-        super.onKernalStart(active);
-
-        if (validateErr != null)
-            throw new IgniteCheckedException(validateErr);
-    }
-
-    /** {@inheritDoc} */
     @Override public void onDisconnected(IgniteFuture reconnectFut) {
-        if (!isEnabled)
-            return;
-
         synchronized (mux) {
             assert !disconnected;
 
@@ -270,9 +263,6 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> onReconnected(boolean active) {
-        if (!isEnabled)
-            return null;
-
         synchronized (mux) {
             assert disconnected;
 
@@ -282,25 +272,29 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
         }
     }
 
-    /**
-     * Authenticate user.
-     *
-     * @param login User's login.
-     * @param passwd Plain text password.
-     * @return User object on successful authenticate. Otherwise returns {@code null}.
-     * @throws IgniteCheckedException On error.
-     * @throws IgniteAccessControlException On authentication error.
-     */
-    public AuthorizationContext authenticate(String login, String passwd) throws IgniteCheckedException {
-        checkEnabled();
+    /** {@inheritDoc} */
+    @Override public SecurityContext authenticate(AuthenticationContext authCtx) throws IgniteCheckedException {
+        SecurityCredentials creds = authCtx.credentials();
+
+        String login = (String)creds.getLogin();
 
         if (F.isEmpty(login))
             throw new IgniteAccessControlException("The user name or password is incorrect [userName=" + login + ']');
 
+        String passwd = (String)creds.getPassword();
+
+        UUID subjId;
+
         if (ctx.clientNode()) {
-            while (true) {
-                AuthenticateFuture fut;
+            if (ctx.discovery().aliveServerNodes().isEmpty()) {
+                throw new IgniteAccessControlException("No alive server node was found to which the authentication" +
+                    " operation could be delegated. It is possible that the client node has been started with the" +
+                    " \"forceServerMode\" flag enabled and no server node had been started yet.");
+            }
 
+            AuthenticateFuture fut;
+
+            do {
                 synchronized (mux) {
                     ClusterNode rndNode = U.randomServerNode(ctx);
 
@@ -314,15 +308,14 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
                 }
 
                 fut.get();
+            } while (fut.retry());
 
-                if (fut.retry())
-                    continue;
-
-                return new AuthorizationContext(User.create(login));
-            }
+            subjId = toSubjectId(login);
         }
         else
-            return new AuthorizationContext(authenticateOnServer(login, passwd));
+            subjId = authenticateOnServer(login, passwd);
+
+        return new SecurityContextImpl(subjId, login, authCtx.subjectType(), authCtx.address());
     }
 
     /**
@@ -330,7 +323,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
      * @param passwd Password.
      * @throws UserManagementException On error.
      */
-    public static void validate(String login, String passwd) throws UserManagementException {
+    public static void validate(String login, char[] passwd) throws UserManagementException {
         if (F.isEmpty(login))
             throw new UserManagementException("User name is empty");
 
@@ -342,41 +335,25 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
                 "The user name length must be less then 60 bytes in UTF8");
     }
 
-    /**
-     * Adds new user.
-     *
-     * @param login User's login.
-     * @param passwd Plain text password.
-     * @throws IgniteCheckedException On error.
-     */
-    public void addUser(String login, String passwd) throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public void createUser(String login, char[] passwd) throws IgniteCheckedException {
         validate(login, passwd);
 
-        UserManagementOperation op = new UserManagementOperation(User.create(login, passwd),
-            UserManagementOperation.OperationType.ADD);
+        UserManagementOperation op = new UserManagementOperation(User.create(login, new String(passwd)), ADD);
 
         execUserOperation(op).get();
     }
 
-    /**
-     * @param login User name.
-     * @throws IgniteCheckedException On error.
-     */
-    public void removeUser(String login) throws IgniteCheckedException {
-        UserManagementOperation op = new UserManagementOperation(User.create(login),
-            UserManagementOperation.OperationType.REMOVE);
+    /** {@inheritDoc} */
+    @Override public void dropUser(String login) throws IgniteCheckedException {
+        UserManagementOperation op = new UserManagementOperation(User.create(login), REMOVE);
 
         execUserOperation(op).get();
     }
 
-    /**
-     * @param login User name.
-     * @param passwd User password.
-     * @throws IgniteCheckedException On error.
-     */
-    public void updateUser(String login, String passwd) throws IgniteCheckedException {
-        UserManagementOperation op = new UserManagementOperation(User.create(login, passwd),
-            UserManagementOperation.OperationType.UPDATE);
+    /** {@inheritDoc} */
+    @Override public void alterUser(String login, char[] passwd) throws IgniteCheckedException {
+        UserManagementOperation op = new UserManagementOperation(User.create(login, new String(passwd)), UPDATE);
 
         execUserOperation(op).get();
     }
@@ -389,7 +366,13 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
             metastorage.iterate(STORE_USER_PREFIX, (key, val) -> {
                 User u = (User)val;
 
-                users.put(u.name(), u);
+                User cur = users.putIfAbsent(toSubjectId(u.name()), u);
+
+                if (cur != null) {
+                    throw new IllegalStateException("Security users with conflicting IDs were found while reading from" +
+                        " metastorage [logins=" + u.name() + ", " + cur.name() + "]. It is possible that the Ignite" +
+                        " metastorage is corrupted or the specified users were created bypassing the Ignite Security API.");
+                }
             }, true);
         }
         else
@@ -410,24 +393,10 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteNodeValidationResult validateNode(ClusterNode node) {
-        Boolean rmtEnabled = node.attribute(IgniteNodeAttributes.ATTR_AUTHENTICATION_ENABLED);
-
-        if (isEnabled && rmtEnabled == null) {
-            String errMsg = "Failed to add node to topology because user authentication is enabled on cluster and " +
-                "the node doesn't support user authentication [nodeId=" + node.id() + ']';
-
-            return new IgniteNodeValidationResult(node.id(), errMsg);
-        }
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
     @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
         // 1. Collect users info only on coordinator
         // 2. Doesn't collect users info to send on client node due to security reason.
-        if (!isEnabled || !isLocalNodeCoordinator() || dataBag.isJoiningNodeClient())
+        if (!isLocalNodeCoordinator() || dataBag.isJoiningNodeClient())
             return;
 
         synchronized (mux) {
@@ -462,11 +431,9 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
         initUsrs = (InitialUsersData)data.commonData();
     }
 
-    /**
-     * @return {@code true} if authentication is enabled, {@code false} if not.
-     */
-    public boolean enabled() {
-        return isEnabled;
+    /** {@inheritDoc} */
+    @Override public boolean enabled() {
+        return true;
     }
 
     /**
@@ -481,16 +448,6 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
     }
 
     /**
-     *
-     */
-    private void checkEnabled() {
-        if (!isEnabled) {
-            throw new IgniteException("Can not perform the operation because the authentication" +
-                " is not enabled for the cluster.");
-        }
-    }
-
-    /**
      */
     private void addDefaultUser() {
         assert users != null && users.isEmpty();
@@ -498,7 +455,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
         User dfltUser = User.defaultUser();
 
         // Put to local map to be ready for authentication.
-        users.put(dfltUser.name(), dfltUser);
+        users.put(toSubjectId(dfltUser.name()), dfltUser);
 
         // Put to MetaStore when it will be ready.
         exec.execute(new RefreshUsersStorageWorker(new ArrayList<>(Collections.singleton(dfltUser))));
@@ -509,25 +466,22 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
      *
      * @param login User's login.
      * @param passwd Plain text password.
-     * @return User object on successful authenticate. Otherwise returns {@code null}.
+     * @return Authenticated user security ID.
      * @throws IgniteCheckedException On authentication error.
      */
-    private User authenticateOnServer(String login, String passwd) throws IgniteCheckedException {
+    private UUID authenticateOnServer(String login, String passwd) throws IgniteCheckedException {
         assert !ctx.clientNode() : "Must be used on server node";
 
         readyForAuthFut.get();
 
-        User usr;
+        UUID subjId = toSubjectId(login);
 
-        usr = users.get(login);
+        User usr = findUser(subjId, login);
 
-        if (usr == null)
+        if (usr == null || !usr.authorize(passwd))
             throw new IgniteAccessControlException("The user name or password is incorrect [userName=" + login + ']');
 
-        if (usr.authorize(passwd))
-            return usr;
-        else
-            throw new IgniteAccessControlException("The user name or password is incorrect [userName=" + login + ']');
+        return subjId;
     }
 
     /**
@@ -537,7 +491,6 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
      */
     private UserOperationFinishFuture execUserOperation(UserManagementOperation op) throws IgniteCheckedException {
         checkActivate();
-        checkEnabled();
 
         synchronized (mux) {
             if (disconnected) {
@@ -545,12 +498,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
                     + "client node is disconnected.");
             }
 
-            AuthorizationContext actx = AuthorizationContext.context();
-
-            if (actx == null)
-                throw new IgniteAccessControlException("Operation not allowed: authorized context is empty.");
-
-            actx.checkUserOperation(op);
+            checkUserOperation(op);
 
             UserOperationFinishFuture fut = new UserOperationFinishFuture(op.id());
 
@@ -600,7 +548,9 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
 
         String userName = usr.name();
 
-        if (users.containsKey(userName))
+        UUID subjId = toSubjectId(userName);
+
+        if (users.get(subjId) != null)
             throw new UserManagementException("User already exists [login=" + userName + ']');
 
         metastorage.write(STORE_USER_PREFIX + userName, usr);
@@ -608,7 +558,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
         synchronized (mux) {
             activeOps.remove(op.id());
 
-            users.put(userName, usr);
+            users.put(subjId, usr);
         }
     }
 
@@ -621,15 +571,19 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
     private void removeUserLocal(UserManagementOperation op) throws IgniteCheckedException {
         User usr = op.user();
 
-        if (!users.containsKey(usr.name()))
-            throw new UserManagementException("User doesn't exist [userName=" + usr.name() + ']');
+        String login = usr.name();
+
+        UUID subjId = toSubjectId(login);
 
-        metastorage.remove(STORE_USER_PREFIX + usr.name());
+        if (findUser(subjId, login) == null)
+            throw new UserManagementException("User doesn't exist [userName=" + login + ']');
+
+        metastorage.remove(STORE_USER_PREFIX + login);
 
         synchronized (mux) {
             activeOps.remove(op.id());
 
-            users.remove(usr.name());
+            users.remove(subjId);
         }
     }
 
@@ -642,15 +596,19 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
     private void updateUserLocal(UserManagementOperation op) throws IgniteCheckedException {
         User usr = op.user();
 
-        if (!users.containsKey(usr.name()))
-            throw new UserManagementException("User doesn't exist [userName=" + usr.name() + ']');
+        String login = usr.name();
+
+        UUID subjId = toSubjectId(login);
+
+        if (findUser(subjId, login) == null)
+            throw new UserManagementException("User doesn't exist [userName=" + login + ']');
 
-        metastorage.write(STORE_USER_PREFIX + usr.name(), usr);
+        metastorage.write(STORE_USER_PREFIX + login, usr);
 
         synchronized (mux) {
             activeOps.remove(op.id());
 
-            users.put(usr.name(), usr);
+            users.put(subjId, usr);
         }
     }
 
@@ -675,10 +633,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
                     && !ctx.discovery().allNodes().isEmpty()
                     && ctx.discovery().aliveServerNodes().isEmpty()) {
                     U.warn(log, "Cannot find the server coordinator node. "
-                        + "Possible a client is started with forceServerMode=true. " +
-                        "Security warning: user authentication will be disabled on the client.");
-
-                    isEnabled = false;
+                        + "Possible a client is started with forceServerMode=true.");
                 }
                 else
                     assert res != null;
@@ -803,7 +758,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
     private void onAuthenticateRequestMessage(UUID nodeId, UserAuthenticateRequestMessage msg) {
         UserAuthenticateResponseMessage respMsg;
         try {
-            User u = authenticateOnServer(msg.name(), msg.password());
+            authenticateOnServer(msg.name(), msg.password());
 
             respMsg = new UserAuthenticateResponseMessage(msg.id(), null);
         }
@@ -841,9 +796,6 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
             return;
 
         if (F.eq(coordinator().id(), ctx.localNodeId())) {
-            if (!isEnabled)
-                return;
-
             assert initUsrs == null;
 
             // Creates default user on coordinator if it is the first start of PDS cluster
@@ -852,35 +804,6 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
                 addDefaultUser();
         }
         else {
-            Boolean rmtEnabled = coordinator().attribute(IgniteNodeAttributes.ATTR_AUTHENTICATION_ENABLED);
-
-            // The cluster doesn't support authentication (ver < 2.5)
-            if (rmtEnabled == null)
-                rmtEnabled = false;
-
-            if (isEnabled != rmtEnabled) {
-                if (rmtEnabled)
-                    U.warn(log, "User authentication is enabled on cluster. Enables on local node");
-                else {
-                    validateErr = "User authentication is disabled on cluster";
-
-                    return;
-                }
-            }
-
-            isEnabled = rmtEnabled;
-
-            if (!isEnabled) {
-                try {
-                    stop(false);
-                }
-                catch (IgniteCheckedException e) {
-                    U.warn(log, "Unexpected exception on stopped authentication processor", e);
-                }
-
-                return;
-            }
-
             if (ctx.clientNode())
                 return;
 
@@ -894,7 +817,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
                     users.clear();
 
                 for (User u : initUsrs.usrs)
-                    users.put(u.name(), u);
+                    users.put(toSubjectId(u.name()), u);
 
                 exec.execute(new RefreshUsersStorageWorker(initUsrs.usrs));
             }
@@ -972,6 +895,108 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
     }
 
     /**
+     * {@inheritDoc}
+     *
+     * The current implementation of {@link GridSecurityProcessor} allows any Ignite node to join the Ignite cluster
+     * without authentication check.
+     */
+    @Override public SecurityContext authenticateNode(ClusterNode node, SecurityCredentials cred) throws IgniteCheckedException {
+        return new SecurityContextImpl(
+            node.id(),
+            node.attribute(ATTR_IGNITE_INSTANCE_NAME),
+            REMOTE_NODE,
+            new InetSocketAddress(F.first(node.addresses()), 0));
+    }
+
+    /** {@inheritDoc} */
+    @Override public SecuritySubject authenticatedSubject(UUID subjId) throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<SecuritySubject> authenticatedSubjects() throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isGlobalNodeAuthentication() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void authorize(String name, SecurityPermission perm, SecurityContext securityCtx) throws SecurityException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionExpired(UUID subjId) {
+        // No-op.
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * This method works with the assumption that {@link SecurityContext} associated with the Ignite node is stored in
+     * node attributes and is obtained automatically by the Ignite using the node ID
+     * (see {@link IgniteSecurityProcessor#withContext(java.util.UUID)}). Since we use the node ID as the subject ID
+     * during node authentication, this method is used for obtaining security context for thin clients only.
+     * Note, that the returned security context does not contain the address of the security subject.
+     * Since the client node does not store user data, the {@link SecurityContext} returned by the client node does
+     * not contain any user information, address, or username.
+     */
+    @Override public SecurityContext securityContext(UUID subjId) {
+        if (ctx.clientNode())
+            return new SecurityContextImpl(subjId, null, REMOTE_CLIENT, null);
+
+        User user = users.get(subjId);
+
+        return user == null ? null : new SecurityContextImpl(subjId, user.name(), REMOTE_CLIENT, null);
+    }
+
+    /**
+     * Gets the user with the specified ID and login. It is necessary to check the login to make sure that there was
+     * no collision when calculating the user ID.
+     */
+    private User findUser(UUID subjId, String login) {
+        User user = users.get(subjId);
+
+        if (user == null || !user.name().equals(login))
+            return null;
+
+        return user;
+    }
+
+    /** Calculates user id based on specified login. */
+    private UUID toSubjectId(String login) {
+        return UUID.nameUUIDFromBytes(login.getBytes());
+    }
+
+    /**
+     * @param op User operation to check.
+     * @throws IgniteAccessControlException If operation check fails: user hasn't permissions for user management
+     *      or try to remove default user.
+     */
+    public void checkUserOperation(UserManagementOperation op) throws IgniteAccessControlException {
+        assert op != null;
+
+        SecuritySubject subj = ctx.security().securityContext().subject();
+
+        if (subj.type() == REMOTE_NODE) {
+            throw new IgniteAccessControlException("User management operations initiated on behalf of" +
+                " the Ignite node are not expected.");
+        }
+
+        if (!User.DFAULT_USER_NAME.equals(subj.login())
+            && !(UserManagementOperation.OperationType.UPDATE == op.type() && subj.login().equals(op.user().name())))
+            throw new IgniteAccessControlException("User management operations are not allowed for user. " +
+                "[curUser=" + subj.login() + ']');
+
+        if (op.type() == UserManagementOperation.OperationType.REMOVE
+            && User.DFAULT_USER_NAME.equals(op.user().name()))
+            throw new IgniteAccessControlException("Default user cannot be removed.");
+    }
+
+    /**
      * Initial data is collected on coordinator to send to join node.
      */
     private static final class InitialUsersData implements Serializable {
@@ -1008,7 +1033,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
         /** {@inheritDoc} */
         @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd,
             final UserProposedMessage msg) {
-            if (!isEnabled || ctx.isStopping() || ctx.clientNode())
+            if (ctx.isStopping() || ctx.clientNode())
                 return;
 
             if (log.isDebugEnabled())
@@ -1024,7 +1049,7 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
     private final class UserAcceptedListener implements CustomEventListener<UserAcceptedMessage> {
         /** {@inheritDoc} */
         @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd, UserAcceptedMessage msg) {
-            if (!isEnabled || ctx.isStopping())
+            if (ctx.isStopping())
                 return;
 
             if (log.isDebugEnabled())
@@ -1335,4 +1360,99 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
             }
         }
     }
+
+    /** Represents {@link SecuritySubject} implementation. */
+    private static class SecuritySubjectImpl implements SecuritySubject {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Security subject identifier. */
+        private final UUID id;
+
+        /** Security subject login.  */
+        private final String login;
+
+        /** Security subject type. */
+        private final SecuritySubjectType type;
+
+        /** Security subject address. */
+        private final InetSocketAddress addr;
+
+        /** */
+        public SecuritySubjectImpl(UUID id, String login, SecuritySubjectType type, InetSocketAddress addr) {
+            this.id = id;
+            this.login = login;
+            this.type = type;
+            this.addr = addr;
+        }
+
+        /** {@inheritDoc} */
+        @Override public UUID id() {
+            return id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String login() {
+            return login;
+        }
+
+        /** {@inheritDoc} */
+        @Override public SecuritySubjectType type() {
+            return type;
+        }
+
+        /** {@inheritDoc} */
+        @Override public InetSocketAddress address() {
+            return addr;
+        }
+
+        /** {@inheritDoc} */
+        @Override public SecurityPermissionSet permissions() {
+            return ALLOW_ALL;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(SecuritySubjectImpl.class, this);
+        }
+    }
+
+    /** Represents {@link SecurityContext} implementation that ignores any security permission checks. */
+    private static class SecurityContextImpl implements SecurityContext, Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final SecuritySubject subj;
+
+        /** */
+        public SecurityContextImpl(UUID id, String login, SecuritySubjectType type, InetSocketAddress addr) {
+            subj = new SecuritySubjectImpl(id, login, type, addr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public SecuritySubject subject() {
+            return subj;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean taskOperationAllowed(String taskClsName, SecurityPermission perm) {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean cacheOperationAllowed(String cacheName, SecurityPermission perm) {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean serviceOperationAllowed(String srvcName, SecurityPermission perm) {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean systemOperationAllowed(SecurityPermission perm) {
+            return true;
+        }
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 507a35a..5acc048 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -90,6 +90,7 @@ import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
+import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCache;
@@ -219,6 +220,7 @@ import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearE
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isPersistentCache;
 import static org.apache.ignite.internal.processors.cache.ValidationOnNodeJoinUtils.validateHashIdResolvers;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition.DFLT_CACHE_REMOVE_ENTRIES_TTL;
+import static org.apache.ignite.internal.processors.security.SecurityUtils.ifAuthenticationEnabled;
 import static org.apache.ignite.internal.util.IgniteUtils.doInParallel;
 
 /**
@@ -628,7 +630,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             log.debug("Started cache processor.");
 
         ctx.state().cacheProcessorStarted();
-        ctx.authentication().cacheProcessorStarted();
+        ifAuthenticationEnabled(ctx, IgniteAuthenticationProcessor::cacheProcessorStarted);
 
         ctx.systemView().registerFiltrableView(
             CACHE_GRP_PAGE_LIST_VIEW,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index e63d78a..a4ba32e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -72,6 +72,7 @@ import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.pagemem.wal.record.ExchangeRecord;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
+import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
@@ -148,6 +149,7 @@ import static org.apache.ignite.internal.processors.cache.ExchangeDiscoveryEvent
 import static org.apache.ignite.internal.processors.cache.ExchangeDiscoveryEvents.serverLeftEvent;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap.PARTIAL_COUNTERS_MAP_SINCE;
 import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.isSnapshotOperation;
+import static org.apache.ignite.internal.processors.security.SecurityUtils.ifAuthenticationEnabled;
 import static org.apache.ignite.internal.util.IgniteUtils.doInParallel;
 import static org.apache.ignite.internal.util.IgniteUtils.doInParallelUninterruptibly;
 
@@ -2514,7 +2516,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             // Create and destroy caches and cache proxies.
             cctx.cache().onExchangeDone(initialVersion(), exchActions, err);
 
-            cctx.kernalContext().authentication().onActivate();
+            ifAuthenticationEnabled(cctx.kernalContext(), IgniteAuthenticationProcessor::onActivate);
 
             Map<T2<Integer, Integer>, Long> localReserved = partHistSuppliers.getReservations(cctx.localNodeId());
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index 912876b..0178db4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -52,7 +52,6 @@ import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.managers.systemview.GridSystemViewManager;
 import org.apache.ignite.internal.managers.systemview.JmxSystemViewExporterSpi;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
-import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor;
@@ -390,11 +389,6 @@ public class StandaloneGridKernalContext implements GridKernalContext {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteAuthenticationProcessor authentication() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
     @Override public GridContinuousProcessor continuous() {
         return null;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java
index 81c61dc..710befb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java
@@ -22,7 +22,6 @@ import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException;
 import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.util.nio.GridNioSession;
@@ -49,9 +48,6 @@ public abstract class ClientListenerAbstractConnectionContext implements ClientL
     /** Connection ID. */
     private long connId;
 
-    /** Authorization context. */
-    private AuthorizationContext authCtx;
-
     /** User attributes. */
     protected Map<String, String> userAttrs;
 
@@ -92,11 +88,6 @@ public abstract class ClientListenerAbstractConnectionContext implements ClientL
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public AuthorizationContext authorizationContext() {
-        return authCtx;
-    }
-
-    /** {@inheritDoc} */
     @Override public long connectionId() {
         return connId;
     }
@@ -104,33 +95,12 @@ public abstract class ClientListenerAbstractConnectionContext implements ClientL
     /**
      * Perform authentication.
      *
-     * @return Auth context.
      * @throws IgniteCheckedException If failed.
      */
-    protected AuthorizationContext authenticate(GridNioSession ses, String user, String pwd)
-        throws IgniteCheckedException {
-        if (ctx.security().enabled())
-            authCtx = authenticateExternal(ses, user, pwd).authorizationContext();
-        else if (ctx.authentication().enabled()) {
-            if (F.isEmpty(user))
-                throw new IgniteAccessControlException("Unauthenticated sessions are prohibited.");
-
-            authCtx = ctx.authentication().authenticate(user, pwd);
+    protected void authenticate(GridNioSession ses, String user, String pwd) throws IgniteCheckedException {
+        if (!ctx.security().enabled())
+            return;
 
-            if (authCtx == null)
-                throw new IgniteAccessControlException("Unknown authentication error.");
-        }
-        else
-            authCtx = null;
-
-        return authCtx;
-    }
-
-    /**
-     * Do 3-rd party authentication.
-     */
-    private AuthenticationContext authenticateExternal(GridNioSession ses, String user, String pwd)
-        throws IgniteCheckedException {
         SecurityCredentials cred = new SecurityCredentials(user, pwd);
 
         AuthenticationContext authCtx = new AuthenticationContext();
@@ -149,8 +119,6 @@ public abstract class ClientListenerAbstractConnectionContext implements ClientL
                 String.format("The user name or password is incorrect [userName=%s]", user)
             );
         }
-
-        return authCtx;
     }
 
     /** {@inheritDoc} */
@@ -163,8 +131,8 @@ public abstract class ClientListenerAbstractConnectionContext implements ClientL
     protected void initClientDescriptor(String prefix) {
         clientDesc = prefix + ":" + ses.remoteAddress().getHostString() + ":" + ses.remoteAddress().getPort();
 
-        if (authCtx != null)
-            clientDesc += "@" + authCtx.userName();
+        if (secCtx != null)
+            clientDesc += "@" + secCtx.subject().login();
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java
index d0cd6ea..a541a05 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.jetbrains.annotations.Nullable;
@@ -75,13 +74,6 @@ public interface ClientListenerConnectionContext {
     void onDisconnected();
 
     /**
-     * Return connection authorization context.
-     *
-     * @return authorization context.
-     */
-    @Nullable AuthorizationContext authorizationContext();
-
-    /**
      * @return Security context.
      */
     @Nullable SecurityContext securityContext();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
index c88844b..92aec47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
@@ -35,7 +35,6 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext;
 import org.apache.ignite.internal.processors.odbc.odbc.OdbcConnectionContext;
@@ -193,18 +192,9 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<Clie
 
             ClientListenerResponse resp;
 
-            AuthorizationContext authCtx = connCtx.authorizationContext();
-
-            if (authCtx != null)
-                AuthorizationContext.context(authCtx);
-
             try (OperationSecurityContext s = ctx.security().withContext(connCtx.securityContext())) {
                 resp = handler.handle(req);
             }
-            finally {
-                if (authCtx != null)
-                    AuthorizationContext.clear();
-            }
 
             if (resp != null) {
                 if (log.isDebugEnabled()) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java
index 4a51fb0..e25512a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java
@@ -39,7 +39,6 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.systemview.walker.ClientConnectionViewWalker;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext;
 import org.apache.ignite.internal.processors.odbc.odbc.OdbcConnectionContext;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
@@ -412,8 +411,6 @@ public class ClientListenerProcessor extends GridProcessorAdapter {
         GridNioSession ses,
         ClientListenerConnectionContext ctx
     ) {
-        AuthorizationContext authCtx = ctx.authorizationContext();
-
         StringBuilder sb = new StringBuilder();
 
         if (ctx instanceof JdbcConnectionContext)
@@ -432,14 +429,7 @@ public class ClientListenerProcessor extends GridProcessorAdapter {
         String rmtAddrStr = rmtAddr.getHostString() + ":" + rmtAddr.getPort();
         String locAddrStr = locAddr.getHostString() + ":" + locAddr.getPort();
 
-        String login;
-
-        if (authCtx != null)
-            login = authCtx.userName();
-        else if (ctx.securityContext() != null)
-            login = "@" + ctx.securityContext().subject().login();
-        else
-            login = "<anonymous>";
+        String login = ctx.securityContext() == null ? "<anonymous>" : "@" + ctx.securityContext().subject().login();
 
         sb.append("id=" + ctx.connectionId());
         sb.append(", user=").append(login);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
index 695166b..a798ed6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
@@ -26,7 +26,6 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.odbc.ClientListenerAbstractConnectionContext;
 import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
@@ -151,7 +150,6 @@ public class JdbcConnectionContext extends ClientListenerAbstractConnectionConte
         boolean skipReducerOnUpdate = false;
 
         NestedTxMode nestedTxMode = NestedTxMode.DEFAULT;
-        AuthorizationContext actx = null;
 
         if (ver.compareTo(VER_2_1_5) >= 0)
             lazyExec = reader.readBoolean();
@@ -204,7 +202,7 @@ public class JdbcConnectionContext extends ClientListenerAbstractConnectionConte
                 throw new IgniteCheckedException("Handshake error: " + e.getMessage(), e);
             }
 
-            actx = authenticate(ses, user, passwd);
+            authenticate(ses, user, passwd);
         }
 
         protoCtx = new JdbcProtocolContext(ver, features, true);
@@ -226,7 +224,7 @@ public class JdbcConnectionContext extends ClientListenerAbstractConnectionConte
 
         handler = new JdbcRequestHandler(busyLock, sender, maxCursors, distributedJoins, enforceJoinOrder,
             collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, nestedTxMode,
-            dataPageScanEnabled, updateBatchSize, actx, ver, this);
+            dataPageScanEnabled, updateBatchSize, ver, this);
 
         handler.start();
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index 224a9ae..e048208 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -47,7 +47,6 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.jdbc.thin.JdbcThinPartitionAwarenessMappingGroup;
 import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters;
 import org.apache.ignite.internal.processors.bulkload.BulkLoadProcessor;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
@@ -162,9 +161,6 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
     /** Protocol version. */
     private final ClientListenerProtocolVersion protocolVer;
 
-    /** Authentication context */
-    private AuthorizationContext actx;
-
     /** Facade that hides transformations internal cache api entities -> jdbc metadata. */
     private final JdbcMetadataInfo meta;
 
@@ -185,7 +181,6 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
      * @param skipReducerOnUpdate Skip reducer on update flag.
      * @param dataPageScanEnabled Enable scan data page mode.
      * @param updateBatchSize Size of internal batch for DML queries.
-     * @param actx Authentication context.
      * @param protocolVer Protocol version.
      * @param connCtx Jdbc connection context.
      */
@@ -203,7 +198,6 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
         NestedTxMode nestedTxMode,
         @Nullable Boolean dataPageScanEnabled,
         @Nullable Integer updateBatchSize,
-        AuthorizationContext actx,
         ClientListenerProtocolVersion protocolVer,
         JdbcConnectionContext connCtx
     ) {
@@ -236,7 +230,6 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
         this.autoCloseCursors = autoCloseCursors;
         this.nestedTxMode = nestedTxMode;
         this.protocolVer = protocolVer;
-        this.actx = actx;
 
         log = connCtx.kernalContext().log(getClass());
 
@@ -310,9 +303,6 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
             return new JdbcResponse(IgniteQueryErrorCode.UNKNOWN,
                 "Failed to handle JDBC request because node is stopping.");
 
-        if (actx != null)
-            AuthorizationContext.context(actx);
-
         JdbcResponse resp;
         try {
             switch (req.type()) {
@@ -403,8 +393,6 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
             return resp;
         }
         finally {
-            AuthorizationContext.clear();
-
             busyLock.leaveBusy();
         }
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
index 0a94d24..8c8d6b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
@@ -23,7 +23,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.odbc.ClientListenerAbstractConnectionContext;
 import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
@@ -154,7 +153,7 @@ public class OdbcConnectionContext extends ClientListenerAbstractConnectionConte
             nestedTxMode = NestedTxMode.fromByte(nestedTxModeVal);
         }
 
-        AuthorizationContext actx = authenticate(ses, user, passwd);
+        authenticate(ses, user, passwd);
 
         ClientListenerResponseSender sender = new ClientListenerResponseSender() {
             @Override public void send(ClientListenerResponse resp) {
@@ -170,7 +169,7 @@ public class OdbcConnectionContext extends ClientListenerAbstractConnectionConte
         initClientDescriptor("odbc");
 
         handler = new OdbcRequestHandler(ctx, busyLock, sender, maxCursors, distributedJoins, enforceJoinOrder,
-            replicatedOnly, collocated, lazy, skipReducerOnUpdate, actx, nestedTxMode, ver, this);
+            replicatedOnly, collocated, lazy, skipReducerOnUpdate, nestedTxMode, ver, this);
 
         parser = new OdbcMessageParser(ctx, ver);
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
index f80aac3..d808fbd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -38,7 +38,6 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx;
@@ -113,9 +112,6 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
     /** Nested transaction behaviour. */
     private final NestedTxMode nestedTxMode;
 
-    /** Authentication context */
-    private final AuthorizationContext actx;
-
     /** Client version. */
     private ClientListenerProtocolVersion ver;
 
@@ -144,7 +140,6 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
      * @param lazy Lazy flag.
      * @param skipReducerOnUpdate Skip reducer on update flag.
      * @param nestedTxMode Nested transaction mode.
-     * @param actx Authentication context.
      * @param ver Client protocol version.
      */
     public OdbcRequestHandler(
@@ -158,7 +153,6 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
         boolean collocated,
         boolean lazy,
         boolean skipReducerOnUpdate,
-        AuthorizationContext actx,
         NestedTxMode nestedTxMode,
         ClientListenerProtocolVersion ver,
         OdbcConnectionContext connCtx) {
@@ -187,7 +181,6 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
         this.busyLock = busyLock;
         this.sender = sender;
         this.maxCursors = maxCursors;
-        this.actx = actx;
         this.nestedTxMode = nestedTxMode;
         this.ver = ver;
 
@@ -237,9 +230,6 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
             return new OdbcResponse(IgniteQueryErrorCode.UNKNOWN,
                 "Failed to handle ODBC request because node is stopping: " + req);
 
-        if (actx != null)
-            AuthorizationContext.context(actx);
-
         try {
             switch (req.command()) {
                 case QRY_EXEC:
@@ -276,8 +266,6 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
             return new OdbcResponse(IgniteQueryErrorCode.UNKNOWN, "Unsupported ODBC request: " + req);
         }
         finally {
-            AuthorizationContext.clear();
-
             busyLock.leaveBusy();
         }
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
index 2a3b5dd..fa4b663 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
@@ -31,7 +31,6 @@ import org.apache.ignite.configuration.ThinClientConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.odbc.ClientListenerAbstractConnectionContext;
 import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
@@ -214,11 +213,11 @@ public class ClientConnectionContext extends ClientListenerAbstractConnectionCon
             }
         }
 
-        AuthorizationContext authCtx = authenticate(ses, user, pwd);
+        authenticate(ses, user, pwd);
 
         initClientDescriptor("cli");
 
-        handler = new ClientRequestHandler(this, authCtx, currentProtocolContext);
+        handler = new ClientRequestHandler(this, currentProtocolContext);
         parser = new ClientMessageParser(this, currentProtocolContext);
 
         this.ses = ses;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
index 1c820a7..e8a4068 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.platform.client;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
@@ -39,9 +38,6 @@ public class ClientRequestHandler implements ClientListenerRequestHandler {
     /** Client context. */
     private final ClientConnectionContext ctx;
 
-    /** Auth context. */
-    private final AuthorizationContext authCtx;
-
     /** Protocol context. */
     private ClientProtocolContext protocolCtx;
 
@@ -52,14 +48,12 @@ public class ClientRequestHandler implements ClientListenerRequestHandler {
      * Constructor.
      *
      * @param ctx Kernal context.
-     * @param authCtx Authentication context.
      * @param protocolCtx Protocol context.
      */
-    ClientRequestHandler(ClientConnectionContext ctx, AuthorizationContext authCtx, ClientProtocolContext protocolCtx) {
+    ClientRequestHandler(ClientConnectionContext ctx, ClientProtocolContext protocolCtx) {
         assert ctx != null;
 
         this.ctx = ctx;
-        this.authCtx = authCtx;
         this.protocolCtx = protocolCtx;
         log = ctx.kernalContext().log(getClass());
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index 21c5eb3..4df9e31 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -43,7 +43,6 @@ import org.apache.ignite.configuration.ConnectorMessageInterceptor;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientTaskResultBean;
 import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandler;
 import org.apache.ignite.internal.processors.rest.handlers.auth.AuthenticationCommandHandler;
@@ -250,10 +249,9 @@ public class GridRestProcessor extends GridProcessorAdapter implements IgniteRes
         if (log.isDebugEnabled())
             log.debug("Received request from client: " + req);
 
-        boolean authenticationEnabled = ctx.authentication().enabled();
         boolean securityEnabled = ctx.security().enabled();
 
-        if (authenticationEnabled || securityEnabled) {
+        if (securityEnabled) {
             Session ses;
 
             try {
@@ -275,54 +273,25 @@ public class GridRestProcessor extends GridProcessorAdapter implements IgniteRes
                 log.debug("Next clientId and sessionToken were extracted according to request: " +
                     "[clientId=" + req.clientId() + ", sesTok=" + Arrays.toString(req.sessionToken()) + "]");
 
-            if (securityEnabled) {
-                SecurityContext secCtx0 = ses.secCtx;
+            SecurityContext secCtx0 = ses.secCtx;
 
-                try {
-                    if (secCtx0 == null || ses.isTokenExpired(sesTokTtl))
-                        ses.secCtx = secCtx0 = authenticate(req, ses);
+            try {
+                if (secCtx0 == null || ses.isTokenExpired(sesTokTtl))
+                    ses.secCtx = secCtx0 = authenticate(req, ses);
 
-                    try (OperationSecurityContext s = ctx.security().withContext(secCtx0)) {
-                        authorize(req);
-                    }
+                try (OperationSecurityContext s = ctx.security().withContext(secCtx0)) {
+                    authorize(req);
                 }
-                catch (SecurityException e) {
-                    assert secCtx0 != null;
 
-                    return new GridFinishedFuture<>(new GridRestResponse(STATUS_SECURITY_CHECK_FAILED, e.getMessage()));
-                }
-                catch (IgniteCheckedException e) {
-                    return new GridFinishedFuture<>(new GridRestResponse(STATUS_AUTH_FAILED, e.getMessage()));
-                }
+                req.securityContext(secCtx0);
             }
-            else {
-                AuthorizationContext authCtx0 = ses.authCtx;
-
-                try {
-                    if (authCtx0 == null) {
-                        SecurityCredentials creds = credentials(req);
-
-                        String login = null;
-
-                        if (creds.getLogin() instanceof String)
-                            login = (String)creds.getLogin();
-
-                        String pwd = null;
+            catch (SecurityException e) {
+                assert secCtx0 != null;
 
-                        if (creds.getPassword() instanceof String)
-                            pwd = (String)creds.getPassword();
-
-                        if (F.isEmpty(login) || F.isEmpty(pwd))
-                            throw new IgniteAuthenticationException("The user name or password is incorrect");
-
-                        ses.authCtx = ctx.authentication().authenticate(login, pwd);
-                    }
-
-                    req.authorizationContext(ses.authCtx);
-                }
-                catch (IgniteCheckedException e) {
-                    return new GridFinishedFuture<>(new GridRestResponse(STATUS_AUTH_FAILED, e.getMessage()));
-                }
+                return new GridFinishedFuture<>(new GridRestResponse(STATUS_SECURITY_CHECK_FAILED, e.getMessage()));
+            }
+            catch (IgniteCheckedException e) {
+                return new GridFinishedFuture<>(new GridRestResponse(STATUS_AUTH_FAILED, e.getMessage()));
             }
         }
 
@@ -384,7 +353,7 @@ public class GridRestProcessor extends GridProcessorAdapter implements IgniteRes
 
                 assert res != null;
 
-                if ((authenticationEnabled || securityEnabled) && !failed)
+                if (securityEnabled && !failed)
                     res.sessionTokenBytes(req.sessionToken());
 
                 interceptResponse(res, req);
@@ -421,7 +390,7 @@ public class GridRestProcessor extends GridProcessorAdapter implements IgniteRes
         while (true) {
             if (F.isEmpty(sesTok) && clientId == null) {
                 // TODO: In IGNITE 3.0 we should check credentials only for AUTHENTICATE command.
-                if (ctx.authentication().enabled() && req.command() != AUTHENTICATE && req.credentials() == null)
+                if (ctx.security().enabled() && req.command() != AUTHENTICATE && req.credentials() == null)
                     throw new IgniteAuthenticationException("Failed to handle request - session token not found or invalid");
 
                 Session ses = Session.random();
@@ -1079,9 +1048,6 @@ public class GridRestProcessor extends GridProcessorAdapter implements IgniteRes
         /** Security context. */
         private volatile SecurityContext secCtx;
 
-        /** Authorization context. */
-        private volatile AuthorizationContext authCtx;
-
         /** Credentials that can be used for security token invalidation.*/
         private volatile SecurityCredentials creds;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/user/UserActionCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/user/UserActionCommandHandler.java
index 40870a3..04ed963 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/user/UserActionCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/user/UserActionCommandHandler.java
@@ -20,13 +20,12 @@ package org.apache.ignite.internal.processors.rest.handlers.user;
 import java.util.Collection;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
-import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
 import org.apache.ignite.internal.processors.rest.GridRestResponse;
 import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandlerAdapter;
 import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
 import org.apache.ignite.internal.processors.rest.request.RestUserActionRequest;
+import org.apache.ignite.internal.processors.security.IgniteSecurity;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
@@ -65,21 +64,21 @@ public class UserActionCommandHandler extends GridRestCommandHandlerAdapter {
         try {
             GridRestCommand cmd = req.command();
 
-            IgniteAuthenticationProcessor authentication = ctx.authentication();
+            IgniteSecurity security = ctx.security();
 
-            AuthorizationContext.context(req.authorizationContext());
+            security.withContext(req.securityContext());
 
             switch (cmd) {
                 case ADD_USER:
-                    authentication.addUser(req0.user(), req0.password());
+                    security.createUser(req0.user(), req0.password().toCharArray());
                     break;
 
                 case REMOVE_USER:
-                    authentication.removeUser(req0.user());
+                    security.dropUser(req0.user());
                     break;
 
                 case UPDATE_USER:
-                    authentication.updateUser(req0.user(), req0.password());
+                    security.alterUser(req0.user(), req0.password().toCharArray());
                     break;
             }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestRequest.java
index e5c9dc8..91278a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestRequest.java
@@ -21,8 +21,8 @@ import java.net.InetSocketAddress;
 import java.security.cert.Certificate;
 import java.util.Map;
 import java.util.UUID;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
@@ -51,7 +51,7 @@ public class GridRestRequest {
     private GridRestCommand cmd;
 
     /** */
-    private AuthorizationContext authCtx;
+    private SecurityContext secCtx;
 
     /** User attributes. */
     Map<String, String> userAttrs;
@@ -156,17 +156,17 @@ public class GridRestRequest {
     }
 
     /**
-     * @return Authorization context.
+     * @return Security context.
      */
-    @Nullable public AuthorizationContext authorizationContext() {
-        return authCtx;
+    @Nullable public SecurityContext securityContext() {
+        return secCtx;
     }
 
     /**
-     * @param authCtx Authorization context.
+     * @param secCtx Security context.
      */
-    public void authorizationContext(AuthorizationContext authCtx) {
-        this.authCtx = authCtx;
+    public void securityContext(SecurityContext secCtx) {
+        this.secCtx = secCtx;
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/GridSecurityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/GridSecurityProcessor.java
index e1dff22..2a46b30 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/GridSecurityProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/GridSecurityProcessor.java
@@ -133,4 +133,36 @@ public interface GridSecurityProcessor extends GridProcessor {
     public default boolean sandboxEnabled() {
         return false;
     }
+
+    /**
+     * Creates user with the specified login and password.
+     *
+     * @param login Login of the user to be created.
+     * @param pwd User password.
+     * @throws IgniteCheckedException If error occurred.
+     */
+    public default void createUser(String login, char[] pwd) throws IgniteCheckedException {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Alters password of user with the specified login.
+     *
+     * @param login Login of the user which password should be altered.
+     * @param pwd User password to alter.
+     * @throws IgniteCheckedException If error occurred.
+     */
+    public default void alterUser(String login, char[] pwd) throws IgniteCheckedException {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Drops user with the specified login.
+     *
+     * @param login Login of the user to be dropped.
+     * @throws IgniteCheckedException If error occurred.
+     */
+    public default void dropUser(String login) throws IgniteCheckedException {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/IgniteSecurity.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/IgniteSecurity.java
index ee73441..ae97df4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/IgniteSecurity.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/IgniteSecurity.java
@@ -126,4 +126,30 @@ public interface IgniteSecurity {
      * false if it's used a default NoOp implementation.
      */
     public boolean enabled();
+
+    /**
+     * Creates user with the specified login and password.
+     *
+     * @param login Login of the user to be created.
+     * @param pwd User password.
+     * @throws IgniteCheckedException If error occurred.
+     */
+    public void createUser(String login, char[] pwd) throws IgniteCheckedException;
+
+    /**
+     * Alters password of user with the specified login.
+     *
+     * @param login Login of the user which password should be altered.
+     * @param pwd User password to alter.
+     * @throws IgniteCheckedException If error occurred.
+     */
+    public void alterUser(String login, char[] pwd) throws IgniteCheckedException;
+
+    /**
+     * Drops user with the specified login.
+     *
+     * @param login Login of the user to be dropped.
+     * @throws IgniteCheckedException If error occurred.
+     */
+    public void dropUser(String login) throws IgniteCheckedException;
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/IgniteSecurityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/IgniteSecurityProcessor.java
index be7daec..e3dcc70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/IgniteSecurityProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/IgniteSecurityProcessor.java
@@ -352,6 +352,21 @@ public class IgniteSecurityProcessor implements IgniteSecurity, GridProcessor {
         return secPrc.onReconnected(clusterRestarted);
     }
 
+    /** {@inheritDoc} */
+    @Override public void createUser(String login, char[] pwd) throws IgniteCheckedException {
+        secPrc.createUser(login, pwd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void alterUser(String login, char[] pwd) throws IgniteCheckedException {
+        secPrc.alterUser(login, pwd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void dropUser(String login) throws IgniteCheckedException {
+        secPrc.dropUser(login);
+    }
+
     /**
      * Getting local node's security context.
      *
@@ -379,4 +394,9 @@ public class IgniteSecurityProcessor implements IgniteSecurity, GridProcessor {
 
         return null;
     }
+
+    /** @return Security processor implementation to which current security facade delegates operations. */
+    public GridSecurityProcessor securityProcessor() {
+        return secPrc;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/NoOpIgniteSecurityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/NoOpIgniteSecurityProcessor.java
index 22b234b..db08f93 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/NoOpIgniteSecurityProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/NoOpIgniteSecurityProcessor.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal.processors.security;
 
 import java.util.Collection;
 import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
@@ -40,6 +42,9 @@ import static org.apache.ignite.internal.processors.security.SecurityUtils.MSG_S
  * No operation IgniteSecurity.
  */
 public class NoOpIgniteSecurityProcessor extends GridProcessorAdapter implements IgniteSecurity {
+    /** Error message that occurs when trying to perform security operations if security disabled. */
+    public static final String SECURITY_DISABLED_ERROR_MSG = "Operation cannot be performed: Ignite security disabled.";
+
     /** No operation security context. */
     private final OperationSecurityContext opSecCtx = new OperationSecurityContext(this, null);
 
@@ -80,7 +85,7 @@ public class NoOpIgniteSecurityProcessor extends GridProcessorAdapter implements
 
     /** {@inheritDoc} */
     @Override public SecurityContext authenticate(AuthenticationContext ctx) {
-        return null;
+        throw new IgniteException(SECURITY_DISABLED_ERROR_MSG);
     }
 
     /** {@inheritDoc} */
@@ -145,4 +150,19 @@ public class NoOpIgniteSecurityProcessor extends GridProcessorAdapter implements
 
         return null;
     }
+
+    /** {@inheritDoc} */
+    @Override public void createUser(String login, char[] pwd) throws IgniteCheckedException {
+        throw new IgniteException(SECURITY_DISABLED_ERROR_MSG);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void alterUser(String login, char[] pwd) throws IgniteCheckedException {
+        throw new IgniteException(SECURITY_DISABLED_ERROR_MSG);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void dropUser(String login) throws IgniteCheckedException {
+        throw new IgniteException(SECURITY_DISABLED_ERROR_MSG);
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
index 10970a6..05cc40c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
@@ -38,6 +38,7 @@ import java.util.Objects;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.function.Consumer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
@@ -45,6 +46,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridInternalWrapper;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.security.sandbox.IgniteDomainCombiner;
 import org.apache.ignite.internal.processors.security.sandbox.IgniteSandbox;
 import org.apache.ignite.internal.util.typedef.F;
@@ -285,4 +287,14 @@ public class SecurityUtils {
             });
         }
     }
+
+    /** Executes specified operation if authentication is enabled. */
+    public static void ifAuthenticationEnabled(GridKernalContext ctx, Consumer<IgniteAuthenticationProcessor> op) {
+        if (!ctx.config().isAuthenticationEnabled())
+            return;
+
+        IgniteSecurityProcessor sec = (IgniteSecurityProcessor)ctx.security();
+
+        op.accept((IgniteAuthenticationProcessor)sec.securityProcessor());
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index cb53ac4..77de5cd 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -1216,6 +1216,16 @@ public class GridFunc {
     }
 
     /**
+     * Tests if the given array is either {@code null} or empty.
+     *
+     * @param c Array to test.
+     * @return Whether or not the given array is {@code null} or empty.
+     */
+    public static boolean isEmpty(@Nullable char[] c) {
+        return c == null || c.length == 0;
+    }
+
+    /**
      * Tests if the given collection is either {@code null} or empty.
      *
      * @param c Collection to test.
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java
index ada5a3b..df20f3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java
@@ -23,7 +23,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.util.typedef.F;
 
 /**
@@ -45,9 +44,6 @@ public class AuthenticationContext {
     /** */
     private Map<String, Object> nodeAttrs;
 
-    /** Authorization context. */
-    private AuthorizationContext athrCtx;
-
     /** True if this is a client node context. */
     private boolean client;
 
@@ -145,23 +141,6 @@ public class AuthenticationContext {
     }
 
     /**
-     * @return Native Apache Ignite authorization context acquired after authentication or {@code null} if native
-     * Ignite authentication is not used.
-     */
-    public AuthorizationContext authorizationContext() {
-        return athrCtx;
-    }
-
-    /**
-     * Set authorization context acquired after native Apache Ignite authentication.
-     */
-    public AuthenticationContext authorizationContext(AuthorizationContext newVal) {
-        athrCtx = newVal;
-
-        return this;
-    }
-
-    /**
      * @return Client SSL certificates.
      */
     public Certificate[] certificates() {
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 0c1b185..530253f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -177,6 +177,7 @@ import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
 import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION;
 import static org.apache.ignite.internal.IgniteFeatures.TCP_DISCOVERY_MESSAGE_NODE_COMPACT_REPRESENTATION;
 import static org.apache.ignite.internal.IgniteFeatures.nodeSupports;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_AUTHENTICATION_ENABLED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_EVENT_DRIVEN_SERVICE_PROCESSOR_ENABLED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
@@ -1255,7 +1256,7 @@ class ServerImpl extends TcpDiscoveryImpl {
      */
     private void localAuthentication(SecurityCredentials locCred) {
         assert spi.nodeAuth != null;
-        assert locCred != null;
+        assert locCred != null || locNode.attribute(ATTR_AUTHENTICATION_ENABLED) != null;
 
         try {
             SecurityContext subj = spi.nodeAuth.authenticateNode(locNode, locCred);
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/ClientConnectionView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/ClientConnectionView.java
index ef30f3c..3caf464 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/ClientConnectionView.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/ClientConnectionView.java
@@ -18,13 +18,13 @@
 package org.apache.ignite.spi.systemview.view;
 
 import java.net.InetSocketAddress;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext;
 import org.apache.ignite.internal.processors.odbc.odbc.OdbcConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.jetbrains.annotations.Nullable;
 
@@ -81,12 +81,12 @@ public class ClientConnectionView {
 
     /** @return User name. */
     public String user() {
-        if (ctx == null)
-            return null;
+        SecurityContext secCtx = ctx == null ? null : ctx.securityContext();
 
-        AuthorizationContext authCtx = ctx.authorizationContext();
+        if (secCtx == null)
+            return null;
 
-        return authCtx == null ? null : authCtx.userName();
+        return (String)secCtx.subject().login();
     }
 
     /** @return Protocol version. */
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/Authentication1kUsersNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/Authentication1kUsersNodeRestartTest.java
index b0ebbc4..927f752 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/Authentication1kUsersNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/Authentication1kUsersNodeRestartTest.java
@@ -23,11 +23,17 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.security.IgniteSecurity;
+import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.authenticate;
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.withSecurityContextOnAllNodes;
+
 /**
  * Test for {@link IgniteAuthenticationProcessor} on unstable topology.
  */
@@ -72,8 +78,6 @@ public class Authentication1kUsersNodeRestartTest extends GridCommonAbstractTest
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        AuthorizationContext.clear();
-
         stopAllGrids();
 
         super.afterTest();
@@ -88,40 +92,40 @@ public class Authentication1kUsersNodeRestartTest extends GridCommonAbstractTest
 
         grid(0).cluster().active(true);
 
-        IgniteAuthenticationProcessor authenticationProcessor = grid(0).context().authentication();
+        IgniteSecurity sec = grid(0).context().security();
 
-        AuthorizationContext actxDflt = authenticationProcessor.authenticate(User.DFAULT_USER_NAME, "ignite");
+        SecurityContext secCtxDflt = authenticate(grid(0), User.DFAULT_USER_NAME, "ignite");
 
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         IntStream.range(0, USERS_COUNT).parallel().forEach(
             i -> {
-                AuthorizationContext.context(actxDflt);
+                AutoCloseable innerNodeSecHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
                 try {
-                    authenticationProcessor.addUser("test" + i, "init");
+                    sec.createUser("test" + i, "init".toCharArray());
                 }
                 catch (IgniteCheckedException e) {
                     throw new IgniteException(e);
                 }
                 finally {
-                    AuthorizationContext.clear();
+                    IgniteUtils.closeQuiet(innerNodeSecHnd);
                 }
             }
         );
 
         IntStream.range(0, USERS_COUNT).parallel().forEach(
             i -> {
-                AuthorizationContext.context(actxDflt);
+                AutoCloseable innerNodeSecHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
                 try {
-                    authenticationProcessor.updateUser("test" + i, "passwd_" + i);
+                    sec.alterUser("test" + i, ("passwd_" + i).toCharArray());
                 }
                 catch (IgniteCheckedException e) {
                     throw new IgniteException(e);
                 }
                 finally {
-                    AuthorizationContext.clear();
+                    IgniteUtils.closeQuiet(innerNodeSecHnd);
                 }
             }
         );
@@ -130,6 +134,6 @@ public class Authentication1kUsersNodeRestartTest extends GridCommonAbstractTest
 
         startGrid(0);
 
-        authenticationProcessor.authenticate("ignite", "ignite");
+        authenticate(grid(0), "ignite", "ignite");
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationConfigurationClusterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationConfigurationClusterTest.java
index e4e2292c..71f34f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationConfigurationClusterTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationConfigurationClusterTest.java
@@ -23,11 +23,17 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.security.impl.TestSecurityPluginProvider;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.authenticate;
+import static org.apache.ignite.internal.processors.security.NoOpIgniteSecurityProcessor.SECURITY_DISABLED_ERROR_MSG;
+import static org.apache.ignite.plugin.security.SecurityPermissionSetBuilder.ALLOW_ALL;
+
 /**
  * Test for disabled {@link IgniteAuthenticationProcessor}.
  */
@@ -87,7 +93,7 @@ public class AuthenticationConfigurationClusterTest extends GridCommonAbstractTe
      */
     @Test
     public void testServerNodeJoinDisabled() throws Exception {
-        checkNodeJoinDisabled(false);
+        checkNodeJoinFailed(false, false);
     }
 
     /**
@@ -95,7 +101,7 @@ public class AuthenticationConfigurationClusterTest extends GridCommonAbstractTe
      */
     @Test
     public void testClientNodeJoinDisabled() throws Exception {
-        checkNodeJoinDisabled(true);
+        checkNodeJoinFailed(true, false);
     }
 
     /**
@@ -103,7 +109,7 @@ public class AuthenticationConfigurationClusterTest extends GridCommonAbstractTe
      */
     @Test
     public void testServerNodeJoinEnabled() throws Exception {
-        checkNodeJoinEnabled(false);
+        checkNodeJoinFailed(false, true);
     }
 
     /**
@@ -111,43 +117,28 @@ public class AuthenticationConfigurationClusterTest extends GridCommonAbstractTe
      */
     @Test
     public void testClientNodeJoinEnabled() throws Exception {
-        checkNodeJoinEnabled(true);
+        checkNodeJoinFailed(true, true);
     }
 
     /**
+     * Checks that a new node cannot join a cluster with a different authentication enable state.
+     *
      * @param client Is joining node client.
+     * @param authEnabled Whether authentication is enabled on joining node.
      * @throws Exception If failed.
      */
-    private void checkNodeJoinDisabled(boolean client) throws Exception {
-        startGrid(configuration(0, true, false));
-
-        startGrid(configuration(1, false, client));
-
-        grid(0).cluster().active(true);
-
-        AuthorizationContext actx = grid(1).context().authentication().authenticate("ignite", "ignite");
-
-        assertNotNull(actx);
+    private void checkNodeJoinFailed(boolean client, boolean authEnabled) throws Exception {
+        startGrid(configuration(0, authEnabled, false));
 
-        assertEquals("ignite", actx.userName());
-    }
-
-    /**
-     * @param client Is joining node client.
-     * @throws Exception If failed.
-     */
-    private void checkNodeJoinEnabled(boolean client) throws Exception {
-        startGrid(configuration(0, false, false));
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    startGrid(configuration(1, true, client));
+                    startGrid(configuration(1, !authEnabled, client));
 
                     return null;
                 }
             },
-            IgniteCheckedException.class,
-            "User authentication is disabled on cluster");
+            IgniteSpiException.class,
+            "Local node's grid security processor class is not equal to remote node's grid security processor class");
     }
 
     /**
@@ -161,39 +152,35 @@ public class AuthenticationConfigurationClusterTest extends GridCommonAbstractTe
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    grid(0).context().authentication().addUser("test", "test");
+                    grid(0).context().security().createUser("test", "test".toCharArray());
 
                     return null;
                 }
-            }, IgniteException.class,
-            "Can not perform the operation because the authentication is not enabled for the cluster");
+            }, IgniteException.class, SECURITY_DISABLED_ERROR_MSG);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    grid(0).context().authentication().removeUser("test");
+                    grid(0).context().security().dropUser("test");
 
                     return null;
                 }
-            }, IgniteException.class,
-            "Can not perform the operation because the authentication is not enabled for the cluster");
+            }, IgniteException.class, SECURITY_DISABLED_ERROR_MSG);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    grid(0).context().authentication().updateUser("test", "test");
+                    grid(0).context().security().alterUser("test", "test".toCharArray());
 
                     return null;
                 }
-            }, IgniteException.class,
-            "Can not perform the operation because the authentication is not enabled for the cluster");
+            }, IgniteException.class, SECURITY_DISABLED_ERROR_MSG);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    grid(0).context().authentication().authenticate("test", "test");
+                    authenticate(grid(0), "test", "test");
 
                     return null;
                 }
-            }, IgniteException.class,
-            "Can not perform the operation because the authentication is not enabled for the cluster");
+            }, IgniteException.class, SECURITY_DISABLED_ERROR_MSG);
     }
 
     /**
@@ -211,4 +198,17 @@ public class AuthenticationConfigurationClusterTest extends GridCommonAbstractTe
             IgniteCheckedException.class,
             "Authentication can be enabled only for cluster with enabled persistence");
     }
+
+    /** Tests that authentication and security plugin can't be configured at the same time. */
+    @Test
+    public void testBothAuthenticationAndSecurityPluginConfiguration() {
+        GridTestUtils.assertThrowsAnyCause(log, () -> {
+                startGrid(configuration(0, true, false)
+                    .setPluginProviders(new TestSecurityPluginProvider("login", "", ALLOW_ALL, false)));
+
+                return null;
+            },
+            IgniteCheckedException.class,
+            "Invalid security configuration: both authentication is enabled and external security plugin is provided.");
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationOnNotActiveClusterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationOnNotActiveClusterTest.java
index 859cb8f..89f63c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationOnNotActiveClusterTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationOnNotActiveClusterTest.java
@@ -20,11 +20,15 @@ package org.apache.ignite.internal.processors.authentication;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.authenticate;
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.withSecurityContextOnAllNodes;
+
 /**
  * Test for {@link IgniteAuthenticationProcessor}.
  */
@@ -86,10 +90,10 @@ public class AuthenticationOnNotActiveClusterTest extends GridCommonAbstractTest
         startClientGrid(CLI_NODE);
 
         for (int i = 0; i < NODES_COUNT; ++i) {
-            AuthorizationContext actx = grid(i).context().authentication().authenticate("ignite", "ignite");
+            SecurityContext secCtx = authenticate(grid(i), "ignite", "ignite");
 
-            assertNotNull(actx);
-            assertEquals("ignite", actx.userName());
+            assertNotNull(secCtx);
+            assertEquals("ignite", secCtx.subject().login());
         }
     }
 
@@ -104,12 +108,12 @@ public class AuthenticationOnNotActiveClusterTest extends GridCommonAbstractTest
 
         grid(0).cluster().active(true);
 
-        AuthorizationContext actxDflt = grid(0).context().authentication().authenticate(User.DFAULT_USER_NAME, "ignite");
+        SecurityContext secCtxDflt = authenticate(grid(0), User.DFAULT_USER_NAME, "ignite");
 
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         for (int i = 0; i < 10; ++i)
-            grid(0).context().authentication().addUser("test" + i, "passwd");
+            grid(0).context().security().createUser("test" + i, "passwd".toCharArray());
 
         stopAllGrids();
 
@@ -120,10 +124,10 @@ public class AuthenticationOnNotActiveClusterTest extends GridCommonAbstractTest
 
         for (int i = 0; i < NODES_COUNT; ++i) {
             for (int usrCnt = 0; usrCnt < 10; ++usrCnt) {
-                AuthorizationContext actx = grid(i).context().authentication().authenticate("test" + usrCnt, "passwd");
+                SecurityContext secCtx = authenticate(grid(i), "test" + usrCnt, "passwd");
 
-                assertNotNull(actx);
-                assertEquals("test" + usrCnt, actx.userName());
+                assertNotNull(secCtx);
+                assertEquals("test" + usrCnt, secCtx.subject().login());
             }
         }
     }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationProcessorNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationProcessorNodeRestartTest.java
index 29bc9c6..f0c3fe6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationProcessorNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationProcessorNodeRestartTest.java
@@ -25,12 +25,17 @@ import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.authenticate;
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.withSecurityContextOnAllNodes;
+import static org.apache.ignite.internal.processors.authentication.User.DFAULT_USER_NAME;
+
 /**
  * Test for {@link IgniteAuthenticationProcessor} on unstable topology.
  */
@@ -45,7 +50,7 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
     private static final int CLI_NODE = NODES_COUNT - 1;
 
     /** Authorization context for default user. */
-    private AuthorizationContext actxDflt;
+    private SecurityContext secCtxDflt;
 
     /** Random. */
     private static final Random RND = new Random(System.currentTimeMillis());
@@ -89,9 +94,9 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
 
         grid(0).cluster().active(true);
 
-        actxDflt = grid(0).context().authentication().authenticate(User.DFAULT_USER_NAME, "ignite");
+        secCtxDflt = authenticate(grid(0), DFAULT_USER_NAME, "ignite");
 
-        assertNotNull(actxDflt);
+        assertNotNull(secCtxDflt);
     }
 
     /** {@inheritDoc} */
@@ -109,12 +114,12 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
     public void testConcurrentAddUpdateRemoveNodeRestartCoordinator() throws Exception {
         final IgniteInternalFuture restartFut = restartCoordinator();
 
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         final AtomicInteger usrCnt = new AtomicInteger();
 
         GridTestUtils.runMultiThreaded(() -> {
-            AuthorizationContext.context(actxDflt);
+            withSecurityContextOnAllNodes(secCtxDflt);
 
             String user = "test" + usrCnt.getAndIncrement();
 
@@ -124,17 +129,17 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
                     try {
                         switch (state) {
                             case 0:
-                                grid(CLI_NODE).context().authentication().addUser(user, "passwd_" + user);
+                                grid(CLI_NODE).context().security().createUser(user, ("passwd_" + user).toCharArray());
 
                                 break;
 
                             case 1:
-                                grid(CLI_NODE).context().authentication().updateUser(user, "new_passwd_" + user);
+                                grid(CLI_NODE).context().security().alterUser(user, ("new_passwd_" + user).toCharArray());
 
                                 break;
 
                             case 2:
-                                grid(CLI_NODE).context().authentication().removeUser(user);
+                                grid(CLI_NODE).context().security().dropUser(user);
 
                                 break;
 
@@ -170,10 +175,10 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
     public void testConcurrentAuthorize() throws Exception {
         final int testUsersCnt = 10;
 
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         for (int i = 0; i < testUsersCnt; ++i)
-            grid(CLI_NODE).context().authentication().addUser("test" + i, "passwd_test" + i);
+            grid(CLI_NODE).context().security().createUser("test" + i, ("passwd_test" + i).toCharArray());
 
         final IgniteInternalFuture restartFut = GridTestUtils.runAsync(() -> {
             try {
@@ -203,10 +208,9 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
 
             try {
                 while (!restartFut.isDone()) {
-                    AuthorizationContext actx = grid(CLI_NODE).context().authentication()
-                        .authenticate(user, "passwd_" + user);
+                   SecurityContext secCtx = authenticate(grid(CLI_NODE), user, "passwd_" + user);
 
-                    assertNotNull(actx);
+                   assertNotNull(secCtx);
                 }
             }
             catch (ClusterTopologyCheckedException ignored) {
@@ -260,14 +264,14 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
         final AtomicInteger usrCnt = new AtomicInteger();
 
         GridTestUtils.runMultiThreaded(() -> {
-            AuthorizationContext.context(actxDflt);
+            withSecurityContextOnAllNodes(secCtxDflt);
 
             try {
                 while (usrCnt.get() < 200) {
                     String user = "test" + usrCnt.getAndIncrement();
 
                     System.out.println("+++ CREATE  " + user);
-                    grid(0).context().authentication().addUser(user, "init");
+                    grid(0).context().security().createUser(user, "init".toCharArray());
                 }
             }
             catch (Exception e) {
@@ -280,7 +284,7 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
         usrCnt.set(0);
 
         GridTestUtils.runMultiThreaded(() -> {
-            AuthorizationContext.context(actxDflt);
+            withSecurityContextOnAllNodes(secCtxDflt);
 
             try {
                 while (usrCnt.get() < 200) {
@@ -288,7 +292,7 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
 
                     System.out.println("+++ ALTER " + user);
 
-                    grid(0).context().authentication().updateUser(user, "passwd_" + user);
+                    grid(0).context().security().alterUser(user, ("passwd_" + user).toCharArray());
                 }
             }
             catch (Exception e) {
@@ -306,7 +310,7 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
         System.out.println("+++ START");
         startGrid(0);
 
-        AuthorizationContext actx = grid(0).context().authentication().authenticate("ignite", "ignite");
+        authenticate(grid(0), "ignite", "ignite");
     }
 
     /**
@@ -316,22 +320,22 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
     public void testConcurrentAddUpdateRemoveNodeRestartServer() throws Exception {
         IgniteInternalFuture restartFut = loopServerRestarts();
 
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         final AtomicInteger usrCnt = new AtomicInteger();
 
         GridTestUtils.runMultiThreaded(() -> {
-            AuthorizationContext.context(actxDflt);
+            withSecurityContextOnAllNodes(secCtxDflt);
 
             String user = "test" + usrCnt.getAndIncrement();
 
             try {
                 while (!restartFut.isDone()) {
-                    grid(CLI_NODE).context().authentication().addUser(user, "init");
+                    grid(CLI_NODE).context().security().createUser(user, "init".toCharArray());
 
-                    grid(CLI_NODE).context().authentication().updateUser(user, "passwd_" + user);
+                    grid(CLI_NODE).context().security().alterUser(user, ("passwd_" + user).toCharArray());
 
-                    grid(CLI_NODE).context().authentication().removeUser(user);
+                    grid(CLI_NODE).context().security().dropUser(user);
                 }
             }
             catch (Exception e) {
@@ -351,17 +355,17 @@ public class AuthenticationProcessorNodeRestartTest extends GridCommonAbstractTe
     public void testConcurrentFailedOperationNodeRestartServer() throws Exception {
         IgniteInternalFuture restartFut = loopServerRestarts();
 
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
-        grid(CLI_NODE).context().authentication().addUser("test", "test");
+        grid(CLI_NODE).context().security().createUser("test", "test".toCharArray());
 
         GridTestUtils.runMultiThreaded(() -> {
-            AuthorizationContext.context(actxDflt);
+            withSecurityContextOnAllNodes(secCtxDflt);
 
             try {
                 while (!restartFut.isDone()) {
                     GridTestUtils.assertThrows(log, () -> {
-                        grid(CLI_NODE).context().authentication().addUser("test", "test");
+                        grid(CLI_NODE).context().security().createUser("test", "test".toCharArray());
 
                         return null;
                     }, UserManagementException.class, "User already exists");
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationProcessorSelfTest.java
index f8089fa..b11ee47 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/authentication/AuthenticationProcessorSelfTest.java
@@ -17,21 +17,34 @@
 
 package org.apache.ignite.internal.processors.authentication;
 
+import java.util.ArrayList;
 import java.util.Base64;
+import java.util.List;
 import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.security.IgniteSecurity;
+import org.apache.ignite.internal.processors.security.OperationSecurityContext;
+import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.security.AuthenticationContext;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecuritySubject;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
+import static org.apache.ignite.internal.processors.authentication.User.DFAULT_USER_NAME;
+import static org.apache.ignite.plugin.security.SecuritySubjectType.REMOTE_CLIENT;
+
 /**
  * Test for {@link IgniteAuthenticationProcessor}.
  */
@@ -48,8 +61,8 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
     /** Random. */
     private static final Random RND = new Random(System.currentTimeMillis());
 
-    /** Authorization context for default user. */
-    protected AuthorizationContext actxDflt;
+    /** Security context for default user. */
+    protected SecurityContext secCtxDflt;
 
     /**
      * @param len String length.
@@ -102,9 +115,9 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
         grid(0).cluster().active(true);
 
-        actxDflt = grid(0).context().authentication().authenticate(User.DFAULT_USER_NAME, "ignite");
+        secCtxDflt = authenticate(grid(0), DFAULT_USER_NAME, "ignite");
 
-        assertNotNull(actxDflt);
+        assertNotNull(secCtxDflt);
     }
 
     /** {@inheritDoc} */
@@ -120,10 +133,10 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
     @Test
     public void testDefaultUser() throws Exception {
         for (int i = 0; i < NODES_COUNT; ++i) {
-            AuthorizationContext actx = grid(i).context().authentication().authenticate("ignite", "ignite");
+            SecurityContext secCtx = authenticate(grid(i), "ignite", "ignite");
 
-            assertNotNull(actx);
-            assertEquals("ignite", actx.userName());
+            assertNotNull(secCtx);
+            assertEquals("ignite", secCtx.subject().login());
         }
     }
 
@@ -132,24 +145,24 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testDefaultUserUpdate() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
             // Change from all nodes
             for (int nodeIdx = 0; nodeIdx < NODES_COUNT; ++nodeIdx) {
-                grid(nodeIdx).context().authentication().updateUser("ignite", "ignite" + nodeIdx);
+                grid(nodeIdx).context().security().alterUser("ignite", ("ignite" + nodeIdx).toCharArray());
 
                 // Check each change from all nodes
                 for (int i = 0; i < NODES_COUNT; ++i) {
-                    AuthorizationContext actx = grid(i).context().authentication().authenticate("ignite", "ignite" + nodeIdx);
+                    SecurityContext secCtx = authenticate(grid(i), "ignite", "ignite" + nodeIdx);
 
-                    assertNotNull(actx);
-                    assertEquals("ignite", actx.userName());
+                    assertNotNull(secCtx);
+                    assertEquals("ignite", secCtx.subject().login());
                 }
             }
         }
         finally {
-            AuthorizationContext.clear();
+            nodeCtxsHnd.close();
         }
     }
 
@@ -158,7 +171,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testRemoveDefault() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
             for (int i = 0; i < NODES_COUNT; ++i) {
@@ -166,17 +179,17 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
                 GridTestUtils.assertThrows(log, new Callable<Object>() {
                     @Override public Object call() throws Exception {
-                        grid(nodeIdx).context().authentication().removeUser("ignite");
+                        grid(nodeIdx).context().security().dropUser("ignite");
 
                         return null;
                     }
                 }, IgniteAccessControlException.class, "Default user cannot be removed");
 
-                assertNotNull(grid(nodeIdx).context().authentication().authenticate("ignite", "ignite"));
+                assertNotNull(authenticate(grid(0), "ignite", "ignite"));
             }
         }
         finally {
-            AuthorizationContext.context(null);
+            nodeCtxsHnd.close();
         }
     }
 
@@ -185,21 +198,21 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testUserManagementPermission() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
-            grid(0).context().authentication().addUser("test", "test");
+            grid(0).context().security().createUser("test", "test".toCharArray());
 
-            final AuthorizationContext actx = grid(0).context().authentication().authenticate("test", "test");
+            final SecurityContext secCtx = authenticate(grid(0), "test", "test");
 
             for (int i = 0; i < NODES_COUNT; ++i) {
                 final int nodeIdx = i;
 
-                AuthorizationContext.context(actx);
+                AutoCloseable innerNodeCtxsHnd = withSecurityContextOnAllNodes(secCtx);
 
                 GridTestUtils.assertThrows(log, new Callable<Object>() {
                     @Override public Object call() throws Exception {
-                        grid(nodeIdx).context().authentication().addUser("test1", "test1");
+                        grid(nodeIdx).context().security().createUser("test1", "test1".toCharArray());
 
                         return null;
                     }
@@ -207,30 +220,32 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
                 GridTestUtils.assertThrows(log, new Callable<Object>() {
                     @Override public Object call() throws Exception {
-                        grid(nodeIdx).context().authentication().removeUser("test");
+                        grid(nodeIdx).context().security().dropUser("test");
 
                         return null;
                     }
                 }, IgniteAccessControlException.class, "User management operations are not allowed for user");
 
-                grid(nodeIdx).context().authentication().updateUser("test", "new_password");
+                grid(nodeIdx).context().security().alterUser("test", "new_password".toCharArray());
 
-                grid(nodeIdx).context().authentication().updateUser("test", "test");
+                grid(nodeIdx).context().security().alterUser("test", "test".toCharArray());
 
                 // Check error on empty auth context:
-                AuthorizationContext.context(null);
+                innerNodeCtxsHnd.close();
+                nodeCtxsHnd.close();
 
                 GridTestUtils.assertThrows(log, new Callable<Object>() {
                     @Override public Object call() throws Exception {
-                        grid(nodeIdx).context().authentication().removeUser("test");
+                        grid(nodeIdx).context().security().dropUser("test");
 
                         return null;
                     }
-                }, IgniteAccessControlException.class, "Operation not allowed: authorized context is empty");
+                }, IgniteAccessControlException.class,
+                "User management operations initiated on behalf of the Ignite node are not expected.");
             }
         }
         finally {
-            AuthorizationContext.context(null);
+            nodeCtxsHnd.close();
         }
     }
 
@@ -239,26 +254,26 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testProceedUsersOnJoinNode() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
-            grid(0).context().authentication().addUser("test0", "test");
-            grid(0).context().authentication().addUser("test1", "test");
+            grid(0).context().security().createUser("test0", "test".toCharArray());
+            grid(0).context().security().createUser("test1", "test".toCharArray());
 
             int nodeIdx = NODES_COUNT;
 
             startGrid(nodeIdx);
 
-            AuthorizationContext actx0 = grid(nodeIdx).context().authentication().authenticate("test0", "test");
-            AuthorizationContext actx1 = grid(nodeIdx).context().authentication().authenticate("test1", "test");
+            SecurityContext secCtx0 = authenticate(grid(nodeIdx), "test0", "test");
+            SecurityContext secCtx1 = authenticate(grid(nodeIdx), "test1", "test");
 
-            assertNotNull(actx0);
-            assertEquals("test0", actx0.userName());
-            assertNotNull(actx1);
-            assertEquals("test1", actx1.userName());
+            assertNotNull(secCtx0);
+            assertEquals("test0", secCtx0.subject().login());
+            assertNotNull(secCtx1);
+            assertEquals("test1", secCtx1.subject().login());
         }
         finally {
-            AuthorizationContext.context(null);
+            nodeCtxsHnd.close();
         }
     }
 
@@ -267,7 +282,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testAuthenticationInvalidUser() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
             for (int i = 0; i < NODES_COUNT; ++i) {
@@ -275,7 +290,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
                 GridTestUtils.assertThrows(log, new Callable<Object>() {
                     @Override public Object call() throws Exception {
-                        grid(nodeIdx).context().authentication().authenticate("invalid_name", "test");
+                        authenticate(grid(nodeIdx), "invalid_name", "test");
 
                         return null;
                     }
@@ -283,7 +298,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
                 GridTestUtils.assertThrows(log, new Callable<Object>() {
                     @Override public Object call() throws Exception {
-                        grid(nodeIdx).context().authentication().authenticate("test", "invalid_password");
+                        authenticate(grid(nodeIdx), "test", "invalid_password");
 
                         return null;
                     }
@@ -291,7 +306,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
             }
         }
         finally {
-            AuthorizationContext.context(null);
+            nodeCtxsHnd.close();
         }
     }
 
@@ -300,7 +315,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testAddUpdateRemoveUser() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
             for (int i = 0; i < NODES_COUNT; ++i) {
@@ -309,7 +324,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
             }
         }
         finally {
-            AuthorizationContext.context(null);
+            nodeCtxsHnd.close();
         }
     }
 
@@ -318,20 +333,20 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testUpdateUser() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
-            grid(0).context().authentication().addUser("test", "test");
+            grid(0).context().security().createUser("test", "test".toCharArray());
 
-            AuthorizationContext actx = grid(0).context().authentication().authenticate("test", "test");
+            SecurityContext secCtx = authenticate(grid(0), "test", "test");
 
             for (int i = 0; i < NODES_COUNT; ++i) {
                 for (int j = 0; j < NODES_COUNT; ++j)
-                    checkUpdateUser(actx, grid(i), grid(j));
+                    checkUpdateUser(secCtx, grid(i), grid(j));
             }
         }
         finally {
-            AuthorizationContext.context(null);
+            nodeCtxsHnd.close();
         }
     }
 
@@ -340,7 +355,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testUpdateRemoveDoesNotExistsUser() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
             for (int i = 0; i < NODES_COUNT; ++i) {
@@ -348,7 +363,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
                 GridTestUtils.assertThrows(log, new Callable<Object>() {
                     @Override public Object call() throws Exception {
-                        grid(nodeIdx).context().authentication().updateUser("invalid_name", "test");
+                        grid(nodeIdx).context().security().alterUser("invalid_name", "test".toCharArray());
 
                         return null;
                     }
@@ -356,7 +371,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
                 GridTestUtils.assertThrows(log, new Callable<Object>() {
                     @Override public Object call() throws Exception {
-                        grid(nodeIdx).context().authentication().removeUser("invalid_name");
+                        grid(nodeIdx).context().security().dropUser("invalid_name");
 
                         return null;
                     }
@@ -364,7 +379,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
             }
         }
         finally {
-            AuthorizationContext.context(null);
+            nodeCtxsHnd.close();
         }
     }
 
@@ -373,17 +388,17 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testAddAlreadyExistsUser() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
-            grid(0).context().authentication().addUser("test", "test");
+            grid(0).context().security().createUser("test", "test".toCharArray());
 
             for (int i = 0; i < NODES_COUNT; ++i) {
                 final int nodeIdx = i;
 
                 GridTestUtils.assertThrows(log, new Callable<Object>() {
                     @Override public Object call() throws Exception {
-                        grid(nodeIdx).context().authentication().addUser("test", "new_passwd");
+                        grid(nodeIdx).context().security().createUser("test", "new_passwd".toCharArray());
 
                         return null;
                     }
@@ -391,7 +406,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
             }
         }
         finally {
-            AuthorizationContext.context(null);
+            nodeCtxsHnd.close();
         }
     }
 
@@ -400,11 +415,11 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testAuthorizeOnClientDisconnect() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
-        grid(CLI_NODE).context().authentication().addUser("test", "test");
+        grid(CLI_NODE).context().security().createUser("test", "test".toCharArray());
 
-        AuthorizationContext.context(null);
+        nodeCtxsHnd.close();
 
         final IgniteInternalFuture stopServersFut = GridTestUtils.runAsync(new Runnable() {
             @Override public void run() {
@@ -425,10 +440,9 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
         GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     while (!stopServersFut.isDone()) {
-                        AuthorizationContext actx = grid(CLI_NODE).context().authentication()
-                            .authenticate("test", "test");
+                        SecurityContext secCtx = authenticate(grid(CLI_NODE), "test", "test");
 
-                        assertNotNull(actx);
+                        assertNotNull(secCtx);
                     }
 
                     return null;
@@ -449,14 +463,14 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.runMultiThreaded(new Runnable() {
             @Override public void run() {
-                AuthorizationContext.context(actxDflt);
+                withSecurityContextOnAllNodes(secCtxDflt);
                 String user = "test" + usrCnt.getAndIncrement();
 
                 try {
                     for (int i = 0; i < ITERATIONS; ++i) {
-                        grid(CLI_NODE).context().authentication().addUser(user, "passwd_" + user);
+                        grid(CLI_NODE).context().security().createUser(user, ("passwd_" + user).toCharArray());
 
-                        grid(CLI_NODE).context().authentication().removeUser(user);
+                        grid(CLI_NODE).context().security().dropUser(user);
                     }
                 }
                 catch (Exception e) {
@@ -472,13 +486,13 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testUserPersistence() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
             for (int i = 0; i < NODES_COUNT; ++i)
-                grid(i).context().authentication().addUser("test" + i, "passwd" + i);
+                grid(i).context().security().createUser("test" + i, ("passwd" + i).toCharArray());
 
-            grid(CLI_NODE).context().authentication().updateUser("ignite", "new_passwd");
+            grid(CLI_NODE).context().security().alterUser("ignite", "new_passwd".toCharArray());
 
             stopAllGrids();
 
@@ -487,22 +501,20 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
             for (int i = 0; i < NODES_COUNT; ++i) {
                 for (int usrIdx = 0; usrIdx < NODES_COUNT; ++usrIdx) {
-                    AuthorizationContext actx0 = grid(i).context().authentication()
-                        .authenticate("test" + usrIdx, "passwd" + usrIdx);
+                    SecurityContext secCtx0 = authenticate(grid(i), "test" + usrIdx, "passwd" + usrIdx);
 
-                    assertNotNull(actx0);
-                    assertEquals("test" + usrIdx, actx0.userName());
+                    assertNotNull(secCtx0);
+                    assertEquals("test" + usrIdx, secCtx0.subject().login());
                 }
 
-                AuthorizationContext actx = grid(i).context().authentication()
-                    .authenticate("ignite", "new_passwd");
+                SecurityContext secCtx = authenticate(grid(i), "ignite", "new_passwd");
 
-                assertNotNull(actx);
-                assertEquals("ignite", actx.userName());
+                assertNotNull(secCtx);
+                assertEquals("ignite", secCtx.subject().login());
             }
         }
         finally {
-            AuthorizationContext.clear();
+            nodeCtxsHnd.close();
         }
     }
 
@@ -511,10 +523,10 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testDefaultUserPersistence() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        AutoCloseable nodeCtxsHnd = withSecurityContextOnAllNodes(secCtxDflt);
 
         try {
-            grid(CLI_NODE).context().authentication().addUser("test", "passwd");
+            grid(CLI_NODE).context().security().createUser("test", "passwd".toCharArray());
 
             stopAllGrids();
 
@@ -524,22 +536,19 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
             startClientGrid(CLI_NODE);
 
             for (int i = 0; i < NODES_COUNT; ++i) {
-                AuthorizationContext actx = grid(i).context().authentication()
-                    .authenticate("ignite", "ignite");
-
-                assertNotNull(actx);
-                assertEquals("ignite", actx.userName());
+                SecurityContext secCtx = authenticate(grid(i), "ignite", "ignite");
 
-                actx = grid(i).context().authentication()
-                    .authenticate("test", "passwd");
+                assertNotNull(secCtx);
+                assertEquals("ignite", secCtx.subject().login());
 
-                assertNotNull(actx);
-                assertEquals("test", actx.userName());
+                secCtx = authenticate(grid(i), "test", "passwd");
 
+                assertNotNull(secCtx);
+                assertEquals("test", secCtx.subject().login());
             }
         }
         finally {
-            AuthorizationContext.clear();
+            nodeCtxsHnd.close();
         }
     }
 
@@ -548,11 +557,11 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testInvalidUserNamePassword() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                grid(CLI_NODE).context().authentication().addUser(null, "test");
+                grid(CLI_NODE).context().security().createUser(null, "test".toCharArray());
 
                 return null;
             }
@@ -560,7 +569,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                grid(CLI_NODE).context().authentication().addUser("", "test");
+                grid(CLI_NODE).context().security().createUser("", "test".toCharArray());
 
                 return null;
             }
@@ -568,7 +577,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                grid(CLI_NODE).context().authentication().addUser("test", null);
+                grid(CLI_NODE).context().security().createUser("test", null);
 
                 return null;
             }
@@ -576,7 +585,7 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                grid(CLI_NODE).context().authentication().addUser("test", "");
+                grid(CLI_NODE).context().security().createUser("test", "".toCharArray());
 
                 return null;
             }
@@ -584,9 +593,9 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                grid(CLI_NODE).context().authentication().addUser(
+                grid(CLI_NODE).context().security().createUser(
                     "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa",
-                    "a");
+                    "a".toCharArray());
 
                 return null;
             }
@@ -600,15 +609,37 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
 
     }
 
+    /** Test the ability to obtain the security context ot an authenticated user on the remote server node. */
+    @Test
+    public void testRemoteNodeSecurityContext() throws Exception {
+        try (OperationSecurityContext ignored = grid(CLI_NODE).context().security().withContext(secCtxDflt)) {
+            grid(CLI_NODE).context().security().createUser("test", "pwd".toCharArray());
+        }
+
+        SecuritySubject subj = authenticate(grid(0), "test", "pwd").subject();
+
+        for (int i = 1; i < NODES_COUNT; i++) {
+            IgniteSecurity security = ignite(i).context().security();
+
+            try (OperationSecurityContext ignored = security.withContext(subj.id())) {
+                SecuritySubject rmtSubj = security.securityContext().subject();
+
+                assertEquals(subj.id(), rmtSubj.id());
+                assertEquals(i != CLI_NODE ? subj.login() : null, rmtSubj.login());
+                assertEquals(subj.type(), rmtSubj.type());
+            }
+        }
+    }
+
     /**
      * @param passwd User's password to check.
      */
     private void checkInvalidPassword(final String passwd) {
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                grid(CLI_NODE).context().authentication().addUser("test", passwd);
+                grid(CLI_NODE).context().security().createUser("test", passwd.toCharArray());
 
                 return null;
             }
@@ -621,37 +652,65 @@ public class AuthenticationProcessorSelfTest extends GridCommonAbstractTest {
      * @throws Exception On error.
      */
     private void checkAddUpdateRemoveUser(IgniteEx createNode, IgniteEx authNode) throws Exception {
-        createNode.context().authentication().addUser("test", "test");
+        createNode.context().security().createUser("test", "test".toCharArray());
 
-        AuthorizationContext newActx = authNode.context().authentication().authenticate("test", "test");
+        SecurityContext newSecCtx = authenticate(authNode, "test", "test");
 
-        assertNotNull(newActx);
-        assertEquals("test", newActx.userName());
+        assertNotNull(newSecCtx);
+        assertEquals("test", newSecCtx.subject().login());
 
-        createNode.context().authentication().updateUser("test", "newpasswd");
+        createNode.context().security().alterUser("test", "newpasswd".toCharArray());
 
-        newActx = authNode.context().authentication().authenticate("test", "newpasswd");
+        newSecCtx = authenticate(authNode, "test", "newpasswd");
 
-        assertNotNull(newActx);
-        assertEquals("test", newActx.userName());
+        assertNotNull(newSecCtx);
+        assertEquals("test", newSecCtx.subject().login());
 
-        createNode.context().authentication().removeUser("test");
+        createNode.context().security().dropUser("test");
     }
 
     /**
-     * @param actx Authorization context.
+     * @param secCtx Security context.
      * @param updNode Node to execute update operation.
      * @param authNode Node to execute authentication.
      * @throws Exception On error.
      */
-    private void checkUpdateUser(AuthorizationContext actx, IgniteEx updNode, IgniteEx authNode) throws Exception {
+    private void checkUpdateUser(SecurityContext secCtx, IgniteEx updNode, IgniteEx authNode) throws Exception {
         String newPasswd = randomString(16);
 
-        updNode.context().authentication().updateUser("test", newPasswd);
+        updNode.context().security().alterUser("test", newPasswd.toCharArray());
 
-        AuthorizationContext actxNew = authNode.context().authentication().authenticate("test", newPasswd);
+        SecurityContext secCtxNew = authenticate(authNode, "test", newPasswd);
 
-        assertNotNull(actxNew);
-        assertEquals("test", actxNew.userName());
+        assertNotNull(secCtxNew);
+        assertEquals("test", secCtxNew.subject().login());
+    }
+
+    /** Authenticates user on the specified node. */
+    public static SecurityContext authenticate(IgniteEx ignite, String login, String pwd) throws IgniteCheckedException {
+        AuthenticationContext authCtx = new AuthenticationContext();
+
+        authCtx.credentials(new SecurityCredentials(login, pwd));
+        authCtx.subjectType(REMOTE_CLIENT);
+
+        return ignite.context().security().authenticate(authCtx);
+    }
+
+    /**
+     * Sets specified security context on all nodes.
+     *
+     * @return Holder of current security contexts. If closed, all security context will bew restored.
+     */
+    public static AutoCloseable withSecurityContextOnAllNodes(SecurityContext ctx) {
+        List<OperationSecurityContext> oldSecCtxs = new ArrayList<>();
+
+        for (Ignite node : G.allGrids())
+            oldSecCtxs.add(((IgniteEx)node).context().security().withContext(ctx));
+
+        return new AutoCloseable() {
+            @Override public void close() throws Exception {
+               oldSecCtxs.forEach(OperationSecurityContext::close);
+            }
+        };
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
index f81e7f6..eb7d161 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
@@ -658,17 +658,17 @@ public class CommandProcessor {
             else if (cmd instanceof SqlCreateUserCommand) {
                 SqlCreateUserCommand addCmd = (SqlCreateUserCommand)cmd;
 
-                ctx.authentication().addUser(addCmd.userName(), addCmd.password());
+                ctx.security().createUser(addCmd.userName(), addCmd.password().toCharArray());
             }
             else if (cmd instanceof SqlAlterUserCommand) {
                 SqlAlterUserCommand altCmd = (SqlAlterUserCommand)cmd;
 
-                ctx.authentication().updateUser(altCmd.userName(), altCmd.password());
+                ctx.security().alterUser(altCmd.userName(), altCmd.password().toCharArray());
             }
             else if (cmd instanceof SqlDropUserCommand) {
                 SqlDropUserCommand dropCmd = (SqlDropUserCommand)cmd;
 
-                ctx.authentication().removeUser(dropCmd.userName());
+                ctx.security().dropUser(dropCmd.userName());
             }
             else
                 throw new IgniteSQLException("Unsupported DDL operation: " + sql,
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/authentication/SqlUserCommandSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/authentication/SqlUserCommandSelfTest.java
index 24bcb55..aa1574c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/authentication/SqlUserCommandSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/authentication/SqlUserCommandSelfTest.java
@@ -23,15 +23,18 @@ import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException;
-import org.apache.ignite.internal.processors.authentication.User;
 import org.apache.ignite.internal.processors.authentication.UserManagementException;
+import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.authenticate;
+import static org.apache.ignite.internal.processors.authentication.AuthenticationProcessorSelfTest.withSecurityContextOnAllNodes;
+import static org.apache.ignite.internal.processors.authentication.User.DFAULT_USER_NAME;
+
 /**
  * Test for leaks JdbcConnection on SqlFieldsQuery execute.
  */
@@ -39,8 +42,8 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
     /** Nodes count. */
     private static final int NODES_COUNT = 3;
 
-    /** Authorization context for default user. */
-    private AuthorizationContext actxDflt;
+    /** Security context for default user. */
+    private SecurityContext secCtxDflt;
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
@@ -71,9 +74,9 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
 
         grid(0).cluster().active(true);
 
-        actxDflt = grid(0).context().authentication().authenticate(User.DFAULT_USER_NAME, "ignite");
+        secCtxDflt = authenticate(grid(0), DFAULT_USER_NAME, "ignite");
 
-        assertNotNull(actxDflt);
+        assertNotNull(secCtxDflt);
     }
 
     /** {@inheritDoc} */
@@ -88,24 +91,22 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testCreateUpdateDropUser() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         for (int i = 0; i < NODES_COUNT; ++i) {
             userSql(i, "CREATE USER test WITH PASSWORD 'test'");
 
-            AuthorizationContext actx = grid(i).context().authentication()
-                .authenticate("TEST", "test");
+            SecurityContext secCtx = authenticate(grid(i), "TEST", "test");
 
-            assertNotNull(actx);
-            assertEquals("TEST", actx.userName());
+            assertNotNull(secCtx);
+            assertEquals("TEST", secCtx.subject().login());
 
             userSql(i, "ALTER USER test WITH PASSWORD 'newpasswd'");
 
-            actx = grid(i).context().authentication()
-                .authenticate("TEST", "newpasswd");
+            secCtx = authenticate(grid(i), "TEST", "newpasswd");
 
-            assertNotNull(actx);
-            assertEquals("TEST", actx.userName());
+            assertNotNull(secCtx);
+            assertEquals("TEST", secCtx.subject().login());
 
             userSql(i, "DROP USER test");
         }
@@ -116,7 +117,7 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testCreateWithAlreadyExistUser() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
         userSql(0, "CREATE USER test WITH PASSWORD 'test'");
 
         for (int i = 0; i < NODES_COUNT; ++i) {
@@ -137,7 +138,7 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testAlterDropNotExistUser() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         for (int i = 0; i < NODES_COUNT; ++i) {
             final int idx = i;
@@ -174,7 +175,8 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
 
                     return null;
                 }
-            }, IgniteAccessControlException.class, "Operation not allowed: authorized context is empty");
+            }, IgniteAccessControlException.class,
+                "User management operations initiated on behalf of the Ignite node are not expected");
 
             GridTestUtils.assertThrowsAnyCause(log, new Callable<Void>() {
                 @Override public Void call() throws Exception {
@@ -182,7 +184,8 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
 
                     return null;
                 }
-            }, IgniteAccessControlException.class, "Operation not allowed: authorized context is empty");
+            }, IgniteAccessControlException.class,
+                "User management operations initiated on behalf of the Ignite node are not expected");
 
             GridTestUtils.assertThrowsAnyCause(log, new Callable<Void>() {
                 @Override public Void call() throws Exception {
@@ -190,7 +193,8 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
 
                     return null;
                 }
-            }, IgniteAccessControlException.class, "Operation not allowed: authorized context is empty");
+            }, IgniteAccessControlException.class,
+                "User management operations initiated on behalf of the Ignite node are not expected");
         }
     }
 
@@ -199,14 +203,13 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testNotAuthorizedOperation() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         userSql(0, "CREATE USER user0 WITH PASSWORD 'user0'");
 
-        AuthorizationContext actx = grid(0).context().authentication()
-            .authenticate("USER0", "user0");
+        SecurityContext secCtx = authenticate(grid(0), "USER0", "user0");
 
-        AuthorizationContext.context(actx);
+        withSecurityContextOnAllNodes(secCtx);
 
         for (int i = 0; i < NODES_COUNT; ++i) {
             final int idx = i;
@@ -242,7 +245,7 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testDropDefaultUser() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         for (int i = 0; i < NODES_COUNT; ++i) {
             final int idx = i;
@@ -262,7 +265,7 @@ public class SqlUserCommandSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testQuotedUsername() throws Exception {
-        AuthorizationContext.context(actxDflt);
+        withSecurityContextOnAllNodes(secCtxDflt);
 
         userSql(0, "CREATE USER \"test\" with password 'test'");