You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2020/07/14 13:47:24 UTC

[GitHub] [ignite] dgarus opened a new pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

dgarus opened a new pull request #8038:
URL: https://github.com/apache/ignite/pull/8038


   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r481098398



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/impl/TestCertificateSecurityProcessor.java
##########
@@ -70,14 +73,18 @@ public TestCertificateSecurityProcessor(GridKernalContext ctx, Collection<TestSe
 
     /** {@inheritDoc} */
     @Override public SecurityContext authenticateNode(ClusterNode node, SecurityCredentials cred) {
-        return new TestSecurityContext(
+        SecurityContext res = new TestSecurityContext(
             new TestSecuritySubject()
                 .setType(REMOTE_NODE)
                 .setId(node.id())
                 .setAddr(new InetSocketAddress(F.first(node.addresses()), 0))
                 .setLogin("")
                 .setPerms(ALLOW_ALL)
         );
+
+        SECURITY_CONTEXTS.put(res.subject().id(), res);

Review comment:
       Why `SECURITY_CONTEXTS` made static?
   AFAICS it used in a non-static context only.




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

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



[GitHub] [ignite] nizhikov commented on pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#issuecomment-685740705


   We need to check and deprecate existing fields that conatins `securitySubjId`, because now it should be transferred in `SecurityAwareCustomMessageWrapper#secSubjId` field. 


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

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



[GitHub] [ignite] dgarus commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
dgarus commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r483463430



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -836,7 +856,8 @@ else if (type == EVT_CLIENT_NODE_RECONNECTED) {
                                         discoCache0,
                                         notification.getTopSnapshot(),
                                         null,
-                                        notification.getSpanContainer()
+                                        notification.getSpanContainer(),
+                                        null

Review comment:
       changed to getting of current subject id




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

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



[GitHub] [ignite] ingvard commented on a change in pull request #8038: IGNITE-13112 The current security context should be obtained using the IgniteSecurity interface only

Posted by GitBox <gi...@apache.org>.
ingvard commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r620193291



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.managers.discovery;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+public class SecurityAwareCustomMessageWrapper extends CustomMessageWrapper {
+    /**
+     *
+     */
+    private static final long serialVersionUID = 0L;
+
+    /** Security subject id that will be used during message processing on an remote node. */
+    private final UUID secSubjId;
+
+    /**
+     * @param delegate Delegate.
+     */
+    public SecurityAwareCustomMessageWrapper(DiscoveryCustomMessage delegate, UUID secSubjId) {

Review comment:
       Wrong number of params in java docs.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r483041012



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -3082,7 +3115,7 @@ else if (log.isDebugEnabled())
                             evt.discoCache = discoCache;
                         }
 
-                        ctx.event().record(customEvt, evt.discoCache);
+                        withContextIfNeed(evt.secSubjId, ctx.security(), () -> ctx.event().record(customEvt, evt.discoCache));

Review comment:
       We should change the security context for the whole `DiscoveryWorker`




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r488588686



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateNodeLeaveExchangeTask.java
##########
@@ -50,6 +53,11 @@ public ClusterNode node() {
         return true;
     }
 
+    /** {@inheritDoc} */
+    @Override public UUID securitySubjectId() {
+        return null;

Review comment:
       This should be removed.




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

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



[GitHub] [ignite] dgarus commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
dgarus commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r483464190



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -3082,7 +3115,7 @@ else if (log.isDebugEnabled())
                             evt.discoCache = discoCache;
                         }
 
-                        ctx.event().record(customEvt, evt.discoCache);
+                        withContextIfNeed(evt.secSubjId, ctx.security(), () -> ctx.event().record(customEvt, evt.discoCache));

Review comment:
       ok, done




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

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



[GitHub] [ignite] dgarus commented on a change in pull request #8038: IGNITE-13112 The current security context should be obtained using the IgniteSecurity interface only

Posted by GitBox <gi...@apache.org>.
dgarus commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r633285273



##########
File path: modules/clients/src/test/java/org/apache/ignite/internal/TaskEventSubjectIdSelfTest.java
##########
@@ -357,7 +357,7 @@ public void testClient() throws Exception {
         assert evt != null;
 
         assertEquals(EVT_TASK_FINISHED, evt.type());
-        assertEquals(client.id(), evt.subjectId());
+        assertNull(evt.subjectId());

Review comment:
       This check shows that subjectId has to be null when security is disabled.




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

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



[GitHub] [ignite] bmv126 commented on pull request #8038: IGNITE-13112 The current security context should be obtained using the IgniteSecurity interface only

Posted by GitBox <gi...@apache.org>.
bmv126 commented on pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#issuecomment-742230009


   @dgarus 
   What is the plan for this PR. Will this be merged into 2.10 master branch ?


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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r488589247



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaExchangeWorkerTask.java
##########
@@ -44,6 +47,11 @@ public SchemaExchangeWorkerTask(SchemaAbstractDiscoveryMessage msg) {
         return false;
     }
 
+    /** {@inheritDoc} */
+    @Override public UUID securitySubjectId() {
+        return null;

Review comment:
       This should be removed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaNodeLeaveExchangeWorkerTask.java
##########
@@ -44,6 +47,11 @@ public SchemaNodeLeaveExchangeWorkerTask(ClusterNode node) {
         return true;
     }
 
+    /** {@inheritDoc} */
+    @Override public UUID securitySubjectId() {
+        return null;

Review comment:
       This should be removed.




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

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



[GitHub] [ignite] dgarus commented on a change in pull request #8038: IGNITE-13112 The current security context should be obtained using the IgniteSecurity interface only

Posted by GitBox <gi...@apache.org>.
dgarus commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r633359239



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
##########
@@ -1343,4 +1349,43 @@ public void recoveryMode(boolean recoveryMode) {
                 ? ForkJoinPool.commonPool()
                 : config().getAsyncContinuationExecutor();
     }
+
+    /**
+     * Wraps executors to security aware implementations.
+     */
+    private void wrapExecutors() {
+        utilityCachePool = wrapToSecurityAware(utilityCachePool);
+        execSvc = wrapToSecurityAware(execSvc);
+        svcExecSvc = wrapToSecurityAware(svcExecSvc);
+        sysExecSvc = wrapToSecurityAware(sysExecSvc);
+        p2pExecSvc = wrapToSecurityAware(p2pExecSvc);
+        mgmtExecSvc = wrapToSecurityAware(mgmtExecSvc);
+        restExecSvc = wrapToSecurityAware(restExecSvc);

Review comment:
       No, we should not.
   >This code has a strong relation with the initialization order of the executors
   I don't sure what you mean. 
   GridKernalContextImpl gets all executors through the constructor so they should be initialized for this moment.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r488588410



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/TxTimeoutOnPartitionMapExchangeChangeTask.java
##########
@@ -50,6 +53,11 @@ public TxTimeoutOnPartitionMapExchangeChangeMessage message() {
         return false;
     }
 
+    /** {@inheritDoc} */
+    @Override public UUID securitySubjectId() {
+        return null;

Review comment:
       This should be removed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r488588813



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/FinishPreloadingTask.java
##########
@@ -49,6 +52,11 @@ public FinishPreloadingTask(AffinityTopologyVersion topVer, int grpId) {
         return true;
     }
 
+    /** {@inheritDoc} */
+    @Override public UUID securitySubjectId() {
+        return null;

Review comment:
       This should be removed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/ForceRebalanceExchangeTask.java
##########
@@ -47,6 +50,11 @@ public ForceRebalanceExchangeTask(GridDhtPartitionExchangeId exchId, GridCompoun
         return true;
     }
 
+    /** {@inheritDoc} */
+    @Override public UUID securitySubjectId() {
+        return null;

Review comment:
       This should be removed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/RebalanceReassignExchangeTask.java
##########
@@ -46,6 +50,11 @@ public RebalanceReassignExchangeTask(GridDhtPartitionExchangeId exchId, GridDhtP
         return true;
     }
 
+    /** {@inheritDoc} */
+    @Override public UUID securitySubjectId() {
+        return null;

Review comment:
       This should be removed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r483040343



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -836,7 +856,8 @@ else if (type == EVT_CLIENT_NODE_RECONNECTED) {
                                         discoCache0,
                                         notification.getTopSnapshot(),
                                         null,
-                                        notification.getSpanContainer()
+                                        notification.getSpanContainer(),
+                                        null

Review comment:
       Why null here?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -2233,6 +2259,7 @@ public void metricsUpdateEvent(DiscoCache discoCache, ClusterNode node) {
                 discoCache,
                 discoCache.nodeMap.values(),
                 null,
+                null,

Review comment:
       Why null here?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -2212,6 +2237,7 @@ public void clientCacheStartEvent(UUID reqId,
                     null,
                     Collections.<ClusterNode>emptyList(),
                     new ClientCacheChangeDummyDiscoveryMessage(reqId, startReqs, cachesToClose),
+                    null,

Review comment:
       Why null here?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -2623,6 +2650,7 @@ public void scheduleSegmentCheck() {
                                     locNodeOnlyTop),
                                 locNodeOnlyTop,
                                 null,
+                                null,

Review comment:
       Why null here?




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

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



[GitHub] [ignite] ingvard commented on a change in pull request #8038: IGNITE-13112 The current security context should be obtained using the IgniteSecurity interface only

Posted by GitBox <gi...@apache.org>.
ingvard commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r620198071



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeTask.java
##########
@@ -17,19 +17,22 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.UUID;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * Cache statistics mode change task for exchange worker.
  */
-public class CacheStatisticsModeChangeTask implements CachePartitionExchangeWorkerTask {
+public class CacheStatisticsModeChangeTask extends AbstractCachePartitionExchangeWorkerTask {
     /** Discovery message. */
     private final CacheStatisticsModeChangeMessage msg;
 
     /**
      * @param msg Message.
      */
-    public CacheStatisticsModeChangeTask(CacheStatisticsModeChangeMessage msg) {
+    public CacheStatisticsModeChangeTask(UUID secSubjId, CacheStatisticsModeChangeMessage msg) {

Review comment:
       Missing java doc param.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
##########
@@ -53,10 +53,13 @@
      * @param cachesToClose Cache to close.
      */
     public ClientCacheChangeDummyDiscoveryMessage(
+        UUID secSubjId,

Review comment:
       Missing java doc param.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/TxTimeoutOnPartitionMapExchangeChangeTask.java
##########
@@ -31,7 +32,9 @@
      *
      * @param msg Discovery message.
      */
-    public TxTimeoutOnPartitionMapExchangeChangeTask(TxTimeoutOnPartitionMapExchangeChangeMessage msg) {
+    public TxTimeoutOnPartitionMapExchangeChangeTask(UUID secSubjId, TxTimeoutOnPartitionMapExchangeChangeMessage msg) {

Review comment:
       Missing java doc param.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateNodeLeaveExchangeTask.java
##########
@@ -32,7 +33,9 @@
      *
      * @param node Node that has left the grid.
      */
-    public WalStateNodeLeaveExchangeTask(ClusterNode node) {
+    public WalStateNodeLeaveExchangeTask(UUID secSubjId, ClusterNode node) {

Review comment:
       Missing java doc param.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/FinishPreloadingTask.java
##########
@@ -36,7 +37,9 @@
     /**
      * @param topVer Topology version.
      */
-    public FinishPreloadingTask(AffinityTopologyVersion topVer, int grpId, long rebalanceId) {
+    public FinishPreloadingTask(UUID secSubjId, AffinityTopologyVersion topVer, int grpId, long rebalanceId) {

Review comment:
       Missing java doc param.




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

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



[GitHub] [ignite] dgarus commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
dgarus commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r483463800



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -2212,6 +2237,7 @@ public void clientCacheStartEvent(UUID reqId,
                     null,
                     Collections.<ClusterNode>emptyList(),
                     new ClientCacheChangeDummyDiscoveryMessage(reqId, startReqs, cachesToClose),
+                    null,

Review comment:
       changed to getton of current subject id

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -2212,6 +2237,7 @@ public void clientCacheStartEvent(UUID reqId,
                     null,
                     Collections.<ClusterNode>emptyList(),
                     new ClientCacheChangeDummyDiscoveryMessage(reqId, startReqs, cachesToClose),
+                    null,

Review comment:
       changed to getting of current subject id




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r488587759



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
##########
@@ -70,6 +73,11 @@ public ClientCacheChangeDummyDiscoveryMessage(
         return true;
     }
 
+    /** {@inheritDoc} */
+    @Override public UUID securitySubjectId() {
+        return null;

Review comment:
       This should be removed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r481098398



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/impl/TestCertificateSecurityProcessor.java
##########
@@ -70,14 +73,18 @@ public TestCertificateSecurityProcessor(GridKernalContext ctx, Collection<TestSe
 
     /** {@inheritDoc} */
     @Override public SecurityContext authenticateNode(ClusterNode node, SecurityCredentials cred) {
-        return new TestSecurityContext(
+        SecurityContext res = new TestSecurityContext(
             new TestSecuritySubject()
                 .setType(REMOTE_NODE)
                 .setId(node.id())
                 .setAddr(new InetSocketAddress(F.first(node.addresses()), 0))
                 .setLogin("")
                 .setPerms(ALLOW_ALL)
         );
+
+        SECURITY_CONTEXTS.put(res.subject().id(), res);

Review comment:
       Why do you mad `SECURITY_CONTEXTS` static?
   AFAICS it used in a non-static context only.




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

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



[GitHub] [ignite] dgarus commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
dgarus commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r483464070



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -2623,6 +2650,7 @@ public void scheduleSegmentCheck() {
                                     locNodeOnlyTop),
                                 locNodeOnlyTop,
                                 null,
+                                null,

Review comment:
       changed to getting of current subject id




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r488587604



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeTask.java
##########
@@ -40,6 +43,11 @@ public CacheStatisticsModeChangeTask(CacheStatisticsModeChangeMessage msg) {
         return false;
     }
 
+    /** {@inheritDoc} */
+    @Override public UUID securitySubjectId() {
+        return null;

Review comment:
       This should be removed.




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

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



[GitHub] [ignite] dgarus commented on a change in pull request #8038: IGNITE-13112 The current security context should be obtained using the IgniteSecurity interface only

Posted by GitBox <gi...@apache.org>.
dgarus commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r633359239



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
##########
@@ -1343,4 +1349,43 @@ public void recoveryMode(boolean recoveryMode) {
                 ? ForkJoinPool.commonPool()
                 : config().getAsyncContinuationExecutor();
     }
+
+    /**
+     * Wraps executors to security aware implementations.
+     */
+    private void wrapExecutors() {
+        utilityCachePool = wrapToSecurityAware(utilityCachePool);
+        execSvc = wrapToSecurityAware(execSvc);
+        svcExecSvc = wrapToSecurityAware(svcExecSvc);
+        sysExecSvc = wrapToSecurityAware(sysExecSvc);
+        p2pExecSvc = wrapToSecurityAware(p2pExecSvc);
+        mgmtExecSvc = wrapToSecurityAware(mgmtExecSvc);
+        restExecSvc = wrapToSecurityAware(restExecSvc);

Review comment:
       No, we should not.
   >This code has a strong relation with the initialization order of the executors
   
   I don't sure what you mean. 
   GridKernalContextImpl gets all executors through the constructor so they should be initialized for this moment.




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

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



[GitHub] [ignite] dgarus commented on a change in pull request #8038: IGNITE-13112 The current security context should be obtained using the IgniteSecurity interface only

Posted by GitBox <gi...@apache.org>.
dgarus commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r633285273



##########
File path: modules/clients/src/test/java/org/apache/ignite/internal/TaskEventSubjectIdSelfTest.java
##########
@@ -357,7 +357,7 @@ public void testClient() throws Exception {
         assert evt != null;
 
         assertEquals(EVT_TASK_FINISHED, evt.type());
-        assertEquals(client.id(), evt.subjectId());
+        assertNull(evt.subjectId());

Review comment:
       This check shows that subjectId has to be null when security is disabled.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
##########
@@ -1343,4 +1349,43 @@ public void recoveryMode(boolean recoveryMode) {
                 ? ForkJoinPool.commonPool()
                 : config().getAsyncContinuationExecutor();
     }
+
+    /**
+     * Wraps executors to security aware implementations.
+     */
+    private void wrapExecutors() {
+        utilityCachePool = wrapToSecurityAware(utilityCachePool);
+        execSvc = wrapToSecurityAware(execSvc);
+        svcExecSvc = wrapToSecurityAware(svcExecSvc);
+        sysExecSvc = wrapToSecurityAware(sysExecSvc);
+        p2pExecSvc = wrapToSecurityAware(p2pExecSvc);
+        mgmtExecSvc = wrapToSecurityAware(mgmtExecSvc);
+        restExecSvc = wrapToSecurityAware(restExecSvc);

Review comment:
       No, we should not.
   >This code has a strong relation with the initialization order of the executors
   I don't sure what you mean. 
   GridKernalContextImpl gets all executors through the constructor so they should be initialized for this moment.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
##########
@@ -1343,4 +1349,43 @@ public void recoveryMode(boolean recoveryMode) {
                 ? ForkJoinPool.commonPool()
                 : config().getAsyncContinuationExecutor();
     }
+
+    /**
+     * Wraps executors to security aware implementations.
+     */
+    private void wrapExecutors() {
+        utilityCachePool = wrapToSecurityAware(utilityCachePool);
+        execSvc = wrapToSecurityAware(execSvc);
+        svcExecSvc = wrapToSecurityAware(svcExecSvc);
+        sysExecSvc = wrapToSecurityAware(sysExecSvc);
+        p2pExecSvc = wrapToSecurityAware(p2pExecSvc);
+        mgmtExecSvc = wrapToSecurityAware(mgmtExecSvc);
+        restExecSvc = wrapToSecurityAware(restExecSvc);

Review comment:
       No, we should not.
   >This code has a strong relation with the initialization order of the executors
   
   I don't sure what you mean. 
   GridKernalContextImpl gets all executors through the constructor so they should be initialized for this moment.




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

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



[GitHub] [ignite] dgarus commented on a change in pull request #8038: IGNITE-13112 CacheEvent#subjectId is always null for cache events with types EVT_CACHE_STARTED and EVT_CACHE_STOPPED

Posted by GitBox <gi...@apache.org>.
dgarus commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r483463647



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -2233,6 +2259,7 @@ public void metricsUpdateEvent(DiscoCache discoCache, ClusterNode node) {
                 discoCache,
                 discoCache.nodeMap.values(),
                 null,
+                null,

Review comment:
       changed to getton of current subject id




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

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



[GitHub] [ignite] ingvard commented on a change in pull request #8038: IGNITE-13112 The current security context should be obtained using the IgniteSecurity interface only

Posted by GitBox <gi...@apache.org>.
ingvard commented on a change in pull request #8038:
URL: https://github.com/apache/ignite/pull/8038#discussion_r620165392



##########
File path: modules/clients/src/test/java/org/apache/ignite/internal/TaskEventSubjectIdSelfTest.java
##########
@@ -357,7 +357,7 @@ public void testClient() throws Exception {
         assert evt != null;
 
         assertEquals(EVT_TASK_FINISHED, evt.type());
-        assertEquals(client.id(), evt.subjectId());
+        assertNull(evt.subjectId());

Review comment:
       Why did you do the less strict check? Not null does not mean that the read value is correct.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
##########
@@ -1343,4 +1349,43 @@ public void recoveryMode(boolean recoveryMode) {
                 ? ForkJoinPool.commonPool()
                 : config().getAsyncContinuationExecutor();
     }
+
+    /**
+     * Wraps executors to security aware implementations.
+     */
+    private void wrapExecutors() {
+        utilityCachePool = wrapToSecurityAware(utilityCachePool);
+        execSvc = wrapToSecurityAware(execSvc);
+        svcExecSvc = wrapToSecurityAware(svcExecSvc);
+        sysExecSvc = wrapToSecurityAware(sysExecSvc);
+        p2pExecSvc = wrapToSecurityAware(p2pExecSvc);
+        mgmtExecSvc = wrapToSecurityAware(mgmtExecSvc);
+        restExecSvc = wrapToSecurityAware(restExecSvc);
+        affExecSvc = wrapToSecurityAware(affExecSvc);
+        idxExecSvc = wrapToSecurityAware(idxExecSvc);
+        qryExecSvc = wrapToSecurityAware(qryExecSvc);
+        schemaExecSvc = wrapToSecurityAware(schemaExecSvc);
+
+        if (customExecSvcs != null) {
+            Map<String, ExecutorService> res = new HashMap<>();
+
+            for (Map.Entry<String, ? extends ExecutorService> e : customExecSvcs.entrySet())
+                res.put(e.getKey(), wrapToSecurityAware(e.getValue()));
+
+            customExecSvcs = res;
+        }
+
+        stripedExecSvc = wrapToSecurityAware(stripedExecSvc);
+        dataStreamExecSvc = wrapToSecurityAware(dataStreamExecSvc);
+    }
+
+    /** */
+    private StripedExecutor wrapToSecurityAware(StripedExecutor original) {
+        return original != null ? new SecurityAwareStripedExecutor(this, original) : null;
+    }
+
+    /** */

Review comment:
       Empty java docs.https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-JavadocComments

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeTask.java
##########
@@ -17,19 +17,22 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.UUID;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * Cache statistics mode change task for exchange worker.
  */
-public class CacheStatisticsModeChangeTask implements CachePartitionExchangeWorkerTask {
+public class CacheStatisticsModeChangeTask extends AbstractCachePartitionExchangeWorkerTask {
     /** Discovery message. */
     private final CacheStatisticsModeChangeMessage msg;
 
     /**
      * @param msg Message.
      */
-    public CacheStatisticsModeChangeTask(CacheStatisticsModeChangeMessage msg) {
+    public CacheStatisticsModeChangeTask(UUID secSubjId, CacheStatisticsModeChangeMessage msg) {

Review comment:
       Missing java doc pram.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
##########
@@ -1343,4 +1349,43 @@ public void recoveryMode(boolean recoveryMode) {
                 ? ForkJoinPool.commonPool()
                 : config().getAsyncContinuationExecutor();
     }
+
+    /**
+     * Wraps executors to security aware implementations.
+     */
+    private void wrapExecutors() {
+        utilityCachePool = wrapToSecurityAware(utilityCachePool);
+        execSvc = wrapToSecurityAware(execSvc);
+        svcExecSvc = wrapToSecurityAware(svcExecSvc);
+        sysExecSvc = wrapToSecurityAware(sysExecSvc);
+        p2pExecSvc = wrapToSecurityAware(p2pExecSvc);
+        mgmtExecSvc = wrapToSecurityAware(mgmtExecSvc);
+        restExecSvc = wrapToSecurityAware(restExecSvc);
+        affExecSvc = wrapToSecurityAware(affExecSvc);
+        idxExecSvc = wrapToSecurityAware(idxExecSvc);
+        qryExecSvc = wrapToSecurityAware(qryExecSvc);
+        schemaExecSvc = wrapToSecurityAware(schemaExecSvc);
+
+        if (customExecSvcs != null) {
+            Map<String, ExecutorService> res = new HashMap<>();
+
+            for (Map.Entry<String, ? extends ExecutorService> e : customExecSvcs.entrySet())
+                res.put(e.getKey(), wrapToSecurityAware(e.getValue()));
+
+            customExecSvcs = res;
+        }
+
+        stripedExecSvc = wrapToSecurityAware(stripedExecSvc);
+        dataStreamExecSvc = wrapToSecurityAware(dataStreamExecSvc);
+    }
+
+    /** */
+    private StripedExecutor wrapToSecurityAware(StripedExecutor original) {

Review comment:
       Empty java docs. https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-JavadocComments

##########
File path: modules/clients/src/test/java/org/apache/ignite/internal/TaskEventSubjectIdSelfTest.java
##########
@@ -309,14 +309,14 @@ public void testClosure() throws Exception {
         assert evt != null;
 
         assertEquals(EVT_TASK_FINISHED, evt.type());
-        assertEquals(nodeId, evt.subjectId());
+        assertNull(evt.subjectId());
 
         assert !it.hasNext();
     }
 
     /**
      * Events for class tasks that was started from external clients should contain
-     * client subject id instead of the node where it was started. This test checks it.
+     * subject id if security enabled, otherwise null.

Review comment:
       ID - In large letters.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateNodeLeaveExchangeTask.java
##########
@@ -32,7 +33,9 @@
      *
      * @param node Node that has left the grid.
      */
-    public WalStateNodeLeaveExchangeTask(ClusterNode node) {
+    public WalStateNodeLeaveExchangeTask(UUID secSubjId, ClusterNode node) {

Review comment:
       Missing java doc pram.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/ForceRebalanceExchangeTask.java
##########
@@ -34,7 +35,9 @@
      * @param exchId Exchange ID.
      * @param forcedRebFut Rebalance future.
      */
-    public ForceRebalanceExchangeTask(GridDhtPartitionExchangeId exchId, GridCompoundFuture<Boolean, Boolean> forcedRebFut) {
+    public ForceRebalanceExchangeTask(UUID secSubjId, GridDhtPartitionExchangeId exchId, GridCompoundFuture<Boolean, Boolean> forcedRebFut) {

Review comment:
       Missing java doc param.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/TxTimeoutOnPartitionMapExchangeChangeTask.java
##########
@@ -31,7 +32,9 @@
      *
      * @param msg Discovery message.
      */
-    public TxTimeoutOnPartitionMapExchangeChangeTask(TxTimeoutOnPartitionMapExchangeChangeMessage msg) {
+    public TxTimeoutOnPartitionMapExchangeChangeTask(UUID secSubjId, TxTimeoutOnPartitionMapExchangeChangeMessage msg) {

Review comment:
       Missing java doc pram.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/StopCachesOnClientReconnectExchangeTask.java
##########
@@ -33,10 +34,14 @@
     @GridToStringInclude
     private final Collection<GridCacheAdapter> stoppedCaches;
 
+    /** Security subject id. */
+    private final UUID secSubjId;
+
     /**
      * @param stoppedCaches Collection of stopped caches.
      */
-    public StopCachesOnClientReconnectExchangeTask(Collection<GridCacheAdapter> stoppedCaches) {
+    public StopCachesOnClientReconnectExchangeTask(UUID secSubjId, Collection<GridCacheAdapter> stoppedCaches) {

Review comment:
       java doc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
##########
@@ -1153,6 +1153,9 @@ public void start(
 
             startProcessor(new IgnitePluginProcessor(ctx, cfg, plugins));
 
+            // Start security processors.
+            startProcessor(securityProcessor());

Review comment:
       Why do we start it here before  ```startProcessor(new PoolProcessor(ctx)); ```? Is it a problem from polls wrapping?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
##########
@@ -1343,4 +1349,43 @@ public void recoveryMode(boolean recoveryMode) {
                 ? ForkJoinPool.commonPool()
                 : config().getAsyncContinuationExecutor();
     }
+
+    /**
+     * Wraps executors to security aware implementations.
+     */
+    private void wrapExecutors() {
+        utilityCachePool = wrapToSecurityAware(utilityCachePool);
+        execSvc = wrapToSecurityAware(execSvc);
+        svcExecSvc = wrapToSecurityAware(svcExecSvc);
+        sysExecSvc = wrapToSecurityAware(sysExecSvc);
+        p2pExecSvc = wrapToSecurityAware(p2pExecSvc);
+        mgmtExecSvc = wrapToSecurityAware(mgmtExecSvc);
+        restExecSvc = wrapToSecurityAware(restExecSvc);

Review comment:
       Should we throw an exception if the wrap method returns null? This code has a strong relation with the initialization order of the executors. This should be done after all.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityAwareAdapter.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.security;
+
+import java.util.UUID;
+import org.apache.ignite.internal.GridKernalContext;
+
+import static org.apache.ignite.internal.processors.security.SecurityUtils.isAuthentificated;
+
+/** */
+public abstract class SecurityAwareAdapter {

Review comment:
       Empty comment

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityAwareAdapter.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.security;
+
+import java.util.UUID;
+import org.apache.ignite.internal.GridKernalContext;
+
+import static org.apache.ignite.internal.processors.security.SecurityUtils.isAuthentificated;
+
+/** */
+public abstract class SecurityAwareAdapter {
+    /** No op security context. */
+    private static final OperationSecurityContext NO_OP_SEC_CXT = new OperationSecurityContext() {
+        @Override public void close() {
+            // No-op.
+        }
+    };
+
+    /** */
+    private final GridKernalContext ctx;

Review comment:
       Empty doc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.managers.discovery;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+public class SecurityAwareCustomMessageWrapper extends CustomMessageWrapper {

Review comment:
       https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-JavadocComments
   Every type should start with at least minimal Javadoc comments including description in the following form
   Every method, field or initializer public, private or protected in top-level, inner or anonymous type should have at least minimal Javadoc comments including description and description of parameters using @param, @return and @throws Javadoc tags, where applicable

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityAwareAdapter.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.security;
+
+import java.util.UUID;
+import org.apache.ignite.internal.GridKernalContext;
+
+import static org.apache.ignite.internal.processors.security.SecurityUtils.isAuthentificated;
+
+/** */
+public abstract class SecurityAwareAdapter {
+    /** No op security context. */
+    private static final OperationSecurityContext NO_OP_SEC_CXT = new OperationSecurityContext() {
+        @Override public void close() {
+            // No-op.
+        }
+    };
+
+    /** */
+    private final GridKernalContext ctx;
+
+    /** */
+    private final UUID secSubjId;
+
+    /** */
+    protected SecurityAwareAdapter(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        secSubjId = isAuthentificated(ctx) ? ctx.security().securityContext().subject().id() : null;
+    }
+
+    /** */

Review comment:
       Empty

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.managers.discovery;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+public class SecurityAwareCustomMessageWrapper extends CustomMessageWrapper {
+    /**
+     *
+     */
+    private static final long serialVersionUID = 0L;
+
+    /** Security subject id that will be used during message processing on an remote node. */
+    private final UUID secSubjId;

Review comment:
       ID - you use small letters.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.managers.discovery;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+public class SecurityAwareCustomMessageWrapper extends CustomMessageWrapper {
+    /**
+     *
+     */
+    private static final long serialVersionUID = 0L;
+
+    /** Security subject id that will be used during message processing on an remote node. */
+    private final UUID secSubjId;
+
+    /**
+     * @param delegate Delegate.
+     */
+    public SecurityAwareCustomMessageWrapper(DiscoveryCustomMessage delegate, UUID secSubjId) {

Review comment:
       Wrong number of prams in java docs.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
##########
@@ -2847,7 +2881,8 @@ public NotificationEvent(
             DiscoCache discoCache,
             Collection<ClusterNode> topSnapshot,
             @Nullable DiscoveryCustomMessage data,
-            SpanContainer spanContainer
+            SpanContainer spanContainer,
+            @Nullable UUID secSubjId

Review comment:
       Skipped in java docs.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
##########
@@ -285,4 +348,27 @@ public SandboxInvocationHandler(IgniteSandbox sandbox, Object original) {
             });
         }
     }
+
+    /**
+     * Runnable that can throw exceptions.
+     */
+    @FunctionalInterface
+    public static interface RunnableX extends Runnable {
+        /**
+         * Runnable body.
+         *
+         * @throws Exception If failed.
+         */
+        void runx() throws Exception;

Review comment:
       public void (ignite code style for interfaces).

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityAwareExecutorService.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.security;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.processors.security.SecurityAwareCallable.convertToSecurityAware;
+
+/**
+ * Executes tasks with a security context that was in force when executor's method was called.
+ */
+public class SecurityAwareExecutorService implements ExecutorService {
+    /** */
+    private final GridKernalContext ctx;

Review comment:
       Please add docs to class

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.managers.discovery;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+public class SecurityAwareCustomMessageWrapper extends CustomMessageWrapper {
+    /**
+     *
+     */
+    private static final long serialVersionUID = 0L;

Review comment:
       Empty

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityAwareStripedExecutor.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.security;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.util.StripedExecutor;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.processors.security.SecurityAwareCallable.convertToSecurityAware;
+
+/**
+ *
+ */
+public class SecurityAwareStripedExecutor extends StripedExecutor {

Review comment:
       Docs.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/AbstractCachePartitionExchangeWorkerTask.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+public abstract class AbstractCachePartitionExchangeWorkerTask implements CachePartitionExchangeWorkerTask {

Review comment:
       https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-JavadocComments
   Every type should start with at least minimal Javadoc comments including description in the following form
   Every method, field or initializer public, private or protected in top-level, inner or anonymous type should have at least minimal Javadoc comments including description and description of parameters using @param, @return and @throws Javadoc tags, where applicable

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.managers.discovery;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+public class SecurityAwareCustomMessageWrapper extends CustomMessageWrapper {
+    /**
+     *
+     */
+    private static final long serialVersionUID = 0L;
+
+    /** Security subject id that will be used during message processing on an remote node. */
+    private final UUID secSubjId;
+
+    /**
+     * @param delegate Delegate.
+     */
+    public SecurityAwareCustomMessageWrapper(DiscoveryCustomMessage delegate, UUID secSubjId) {
+        super(delegate);
+
+        this.secSubjId = secSubjId;
+    }
+
+    /**
+     * @return Security subject id.
+     */
+    public UUID securitySubjectId() {

Review comment:
       The return message isn't consistent to field declaration, it should be equals.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityAwareIoPool.java
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.security;
+
+import java.util.concurrent.Executor;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.plugin.extensions.communication.IoPool;
+import org.jetbrains.annotations.NotNull;
+
+/** */
+public class SecurityAwareIoPool implements IoPool {

Review comment:
       Docs here

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/AbstractCachePartitionExchangeWorkerTask.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+public abstract class AbstractCachePartitionExchangeWorkerTask implements CachePartitionExchangeWorkerTask {
+    /** Security subject id. */
+    private final UUID secSubjId;

Review comment:
       ID (java docs)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityAwareRunnable.java
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.security;
+
+import org.apache.ignite.internal.GridKernalContext;
+
+/**
+ * The runnable executes the run method with a security context that was actual when the runnable was created.
+ */
+public class SecurityAwareRunnable extends SecurityAwareAdapter implements Runnable {
+    /** */
+    private final Runnable original;
+

Review comment:
       Docs

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartitionExchangeWorkerTask.java
##########
@@ -25,4 +27,9 @@
      * @return {@code False} if exchange merge should stop if this task is found in exchange worker queue.
      */
     boolean skipForExchangeMerge();
+
+    /**
+     * @return Security subject id.

Review comment:
       ID

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/AbstractCachePartitionExchangeWorkerTask.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+public abstract class AbstractCachePartitionExchangeWorkerTask implements CachePartitionExchangeWorkerTask {
+    /** Security subject id. */
+    private final UUID secSubjId;
+
+    /**
+     * @param secSubjId Security subject id.

Review comment:
       ID (java docs).

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
##########
@@ -354,12 +341,18 @@ public void addEvent(
                 hasNewVal,
                 oldVal0,
                 hasOldVal,
-                subjId,
+                cacheEventSubjectId(type, cctx),
                 cloClsName,
                 taskName));
         }
     }
 
+    /** */
+    private UUID cacheEventSubjectId(int type, GridCacheContext<?, ?> cctx) {

Review comment:
       Empty java doc.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/AbstractRemoteSecurityContextCheckTest.java
##########
@@ -298,9 +298,16 @@ public void checkResult() {
 
         /** */
         public Verifier initiator(IgniteEx initiator) {
+            return initiator(secSubjectId(initiator));
+        }
+
+        /** */
+        public Verifier initiator(UUID secSubjId) {

Review comment:
       java doc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
##########
@@ -53,10 +53,13 @@
      * @param cachesToClose Cache to close.
      */
     public ClientCacheChangeDummyDiscoveryMessage(
+        UUID secSubjId,

Review comment:
       Missing java doc pram.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
##########
@@ -1604,7 +1593,7 @@ else if (interceptorVal != val0)
                     val != null,
                     evtOld,
                     evtOld != null || hasValueUnlocked(),
-                    subjId, null, taskName,
+                    null, taskName,

Review comment:
       Code style formatting.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/RebalanceReassignExchangeTask.java
##########
@@ -33,7 +34,9 @@
      * @param exchId Exchange ID.
      * @param exchFut Exchange future.
      */
-    public RebalanceReassignExchangeTask(GridDhtPartitionExchangeId exchId, GridDhtPartitionsExchangeFuture exchFut) {
+    public RebalanceReassignExchangeTask(UUID secSubjId, GridDhtPartitionExchangeId exchId, GridDhtPartitionsExchangeFuture exchFut) {

Review comment:
       Java doc.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
##########
@@ -389,8 +387,7 @@ public boolean loadedValue(@Nullable IgniteInternalTx tx,
         long expireTime,
         boolean evt,
         boolean keepBinary,
-        AffinityTopologyVersion topVer,
-        UUID subjId)
+        AffinityTopologyVersion topVer)

Review comment:
       java doc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/FinishPreloadingTask.java
##########
@@ -36,7 +37,9 @@
     /**
      * @param topVer Topology version.
      */
-    public FinishPreloadingTask(AffinityTopologyVersion topVer, int grpId, long rebalanceId) {
+    public FinishPreloadingTask(UUID secSubjId, AffinityTopologyVersion topVer, int grpId, long rebalanceId) {

Review comment:
       Missing java doc pram.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaExchangeWorkerTask.java
##########
@@ -33,7 +34,9 @@
      *
      * @param msg Message.
      */
-    public SchemaExchangeWorkerTask(SchemaAbstractDiscoveryMessage msg) {
+    public SchemaExchangeWorkerTask(UUID secSubjId, SchemaAbstractDiscoveryMessage msg) {

Review comment:
       Java doc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/StopCachesOnClientReconnectExchangeTask.java
##########
@@ -33,10 +34,14 @@
     @GridToStringInclude
     private final Collection<GridCacheAdapter> stoppedCaches;
 
+    /** Security subject id. */
+    private final UUID secSubjId;

Review comment:
       ID

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/SecurityAwareGridRestCommandHandler.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.rest.handlers;
+
+import java.util.Collection;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.processors.security.OperationSecurityContext;
+
+/**
+ * Handler setups a security context related to a subject id from the request.
+ */
+public class SecurityAwareGridRestCommandHandler implements GridRestCommandHandler {
+    /** */
+    private final GridKernalContext ctx;

Review comment:
       Empty comment

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/SecurityAwareGridRestCommandHandler.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.rest.handlers;
+
+import java.util.Collection;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.processors.security.OperationSecurityContext;
+
+/**
+ * Handler setups a security context related to a subject id from the request.
+ */
+public class SecurityAwareGridRestCommandHandler implements GridRestCommandHandler {
+    /** */
+    private final GridKernalContext ctx;
+
+    /** */
+    private final GridRestCommandHandler original;

Review comment:
       Empty comment

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/OperationSecurityContext.java
##########
@@ -39,6 +39,13 @@
         this.secCtx = secCtx;
     }
 
+    /** */
+    OperationSecurityContext() {

Review comment:
       Empty comment

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaNodeLeaveExchangeWorkerTask.java
##########
@@ -35,7 +36,9 @@
      *
      * @param node Node.
      */
-    public SchemaNodeLeaveExchangeWorkerTask(ClusterNode node) {
+    public SchemaNodeLeaveExchangeWorkerTask(UUID secSubjId, ClusterNode node) {

Review comment:
       Java doc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheUpdateTimeout.java
##########
@@ -26,6 +28,9 @@
     /** */
     private final GridCacheSharedContext cctx;
 
+    /** Security subject id. */
+    private final UUID secSubjId;

Review comment:
       ID https://en.wikipedia.org/wiki/Identifier && https://www.lingvolive.com/en-us/translate/en-ru/ID

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityAwareAdapter.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.security;
+
+import java.util.UUID;
+import org.apache.ignite.internal.GridKernalContext;
+
+import static org.apache.ignite.internal.processors.security.SecurityUtils.isAuthentificated;
+
+/** */
+public abstract class SecurityAwareAdapter {
+    /** No op security context. */
+    private static final OperationSecurityContext NO_OP_SEC_CXT = new OperationSecurityContext() {
+        @Override public void close() {
+            // No-op.
+        }
+    };
+
+    /** */
+    private final GridKernalContext ctx;
+
+    /** */
+    private final UUID secSubjId;

Review comment:
       Empty

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityAwareCallable.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.security;
+
+import java.util.Collection;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.GridKernalContext;
+
+/**
+ * The callable executes the call method with a security context that was actual when the calleble was created.
+ */
+public class SecurityAwareCallable<T> extends SecurityAwareAdapter implements Callable<T> {
+    /** */

Review comment:
       empty

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
##########
@@ -143,6 +143,13 @@ public StripedExecutor(
         }
     }
 
+    /** */
+    protected StripedExecutor() {

Review comment:
       java doc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityAwareCallable.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.security;
+
+import java.util.Collection;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.GridKernalContext;
+
+/**
+ * The callable executes the call method with a security context that was actual when the calleble was created.
+ */
+public class SecurityAwareCallable<T> extends SecurityAwareAdapter implements Callable<T> {
+    /** */
+    public static <A> Collection<? extends Callable<A>> convertToSecurityAware(GridKernalContext ctx,
+        Collection<? extends Callable<A>> tasks) {
+        return tasks.stream().map(t -> new SecurityAwareCallable<>(ctx, t)).collect(Collectors.toList());
+    }
+
+    /** Original callable. */
+    private final Callable<T> original;
+
+    /** */
+    public SecurityAwareCallable(GridKernalContext ctx, Callable<T> original) {

Review comment:
       empty java doc

##########
File path: modules/core/src/main/java/org/apache/ignite/thread/IgniteStripedThreadPoolExecutor.java
##########
@@ -75,6 +75,11 @@ public IgniteStripedThreadPoolExecutor(
         }
     }
 
+    /** */
+    protected IgniteStripedThreadPoolExecutor() {

Review comment:
       java doc

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/AbstractCacheEventsTest.java
##########
@@ -0,0 +1,603 @@
+/*
+ * 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.security.events;
+
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import javax.cache.Cache;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.client.ClientCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.events.CacheEvent;
+import org.apache.ignite.events.CacheQueryExecutedEvent;
+import org.apache.ignite.events.CacheQueryReadEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.client.GridClientData;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.security.AbstractSecurityTest;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.Collections.singleton;
+import static org.apache.ignite.events.EventType.EVT_CACHE_ENTRY_CREATED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_ENTRY_DESTROYED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_LOCKED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_PUT;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_REMOVED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNLOCKED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;
+
+/**
+ * Security subject id of cache events have to refer to subject that initiates cache operation.
+ */
+public abstract class AbstractCacheEventsTest extends AbstractSecurityTest {
+    /** Counter to name caches. */
+    private static final AtomicInteger COUNTER = new AtomicInteger();
+
+    /** Node that registers event listeners. */
+    protected static final String LISTENER_NODE = "listener_node";
+
+    /** Client node. */
+    protected static final String CLNT = "client";
+
+    /** Server node. */
+    protected static final String SRV = "server";
+
+    /** Key. */
+    protected static final String KEY = "key";
+
+    /** Initiate value. */
+    public static final String INIT_VAL = "init_val";
+
+    /** Value. */
+    protected static final String VAL = "val";
+
+    /** Events latch. */
+    private static CountDownLatch evtsLatch;
+
+    /** Local events count. */
+    private static final AtomicInteger evtCnt = new AtomicInteger();
+
+    /** Error message - actual subject is not expected. */
+    private static final AtomicReference<String> error = new AtomicReference<>();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridAllowAll(LISTENER_NODE);
+        startGridAllowAll(SRV);
+        startGridAllowAll("additional_srv");
+
+        startClientAllowAll(CLNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setConnectorConfiguration(new ConnectorConfiguration())
+            .setIncludeEventTypes(EVT_CACHE_ENTRY_CREATED,
+                EVT_CACHE_ENTRY_DESTROYED,
+                EVT_CACHE_OBJECT_PUT,
+                EVT_CACHE_OBJECT_READ,
+                EVT_CACHE_OBJECT_REMOVED,
+                EVT_CACHE_OBJECT_LOCKED,
+                EVT_CACHE_OBJECT_UNLOCKED,
+                EVT_CACHE_QUERY_EXECUTED,
+                EVT_CACHE_QUERY_OBJECT_READ);
+    }
+
+    /**
+     *
+     */
+    protected abstract CacheAtomicityMode cacheAtomicityMode();
+
+    /**
+     *
+     */
+    protected abstract GridTestUtils.ConsumerX<String> operation();
+
+    /**
+     *
+     */
+    protected abstract String expectedLogin();
+
+    /**
+     *
+     */
+    protected abstract int eventType();
+
+    /**
+     *
+     */
+    protected abstract TestOperation testOperation();
+
+    /**
+     *
+     */
+    @Test
+    public void testCacheEvent() throws Exception {

Review comment:
       java docs.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/impl/TestCertificateSecurityProcessor.java
##########
@@ -54,6 +54,9 @@
     /** Permissions. */
     public static final Map<String, SecurityPermissionSet> PERMS = new ConcurrentHashMap<>();
 
+    /** */
+    private final Map<UUID, SecurityContext> secCtxs = new ConcurrentHashMap<>();

Review comment:
       java docs

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/compute/closure/ClientsComputeTaskRemoteSecurityContextTest.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.security.compute.closure;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.client.ClientCompute;
+import org.apache.ignite.client.Config;
+import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeJobResultPolicy;
+import org.apache.ignite.compute.ComputeTask;
+import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.configuration.ClientConnectorConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.ThinClientConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientCompute;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientFactory;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.client.message.GridClientTaskResultBean;
+import org.apache.ignite.internal.processors.rest.request.GridRestTaskRequest;
+import org.apache.ignite.internal.processors.security.AbstractRemoteSecurityContextCheckTest;
+import org.apache.ignite.internal.processors.security.AbstractTestSecurityPluginProvider;
+import org.apache.ignite.internal.processors.security.SecurityUtils;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecurityCredentialsBasicProvider;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static java.util.Collections.singletonList;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Testing operation security context when the compute task is executed on remote nodes.
+ */
+@RunWith(Parameterized.class)
+public class ClientsComputeTaskRemoteSecurityContextTest extends AbstractRemoteSecurityContextCheckTest {
+    /** */
+    @Parameterized.Parameters(name = "async={0}")
+    public static Iterable<Boolean[]> data() {
+        return Arrays.asList(new Boolean[] {false}, new Boolean[] {true});
+    }
+
+    /** */
+    @Parameterized.Parameter()
+    public boolean async;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridAllowAll(SRV_RUN);
+
+        startGridAllowAll(SRV_CHECK);
+
+        startClientAllowAll(CLNT_CHECK).cluster().state(ClusterState.ACTIVE);
+    }
+
+    /** */
+    @Test
+    public void testGridClient() throws Exception {
+        GridClientConfiguration cfg = new GridClientConfiguration()
+            .setServers(singletonList("127.0.0.1:11211"))
+            .setSecurityCredentialsProvider(new SecurityCredentialsBasicProvider(new SecurityCredentials("grid", "")))
+            .setBalancer(nodes ->
+                nodes.stream().findFirst().orElseThrow(NoSuchElementException::new));
+
+        try (GridClient clnt = GridClientFactory.start(cfg)) {
+            VERIFIER.initiator(clnt.id());
+
+            setupVerifier(VERIFIER);
+
+            GridClientCompute comp = clnt.compute().projection(clnt.compute().nodes(nodesToRunIds()));
+
+            if (async)
+                comp.executeAsync(ComputeTaskClosure.class.getName(), null).get();
+            else
+                comp.execute(ComputeTaskClosure.class.getName(), null);
+
+            VERIFIER.checkResult();
+        }
+    }
+
+    /** */
+    @Test

Review comment:
       java docs

##########
File path: modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
##########
@@ -2502,6 +2503,29 @@ public static long sleep_and_can_fail() {
         }
     }
 
+    /**
+     * Consumer that can throw exceptions.
+     */
+    @FunctionalInterface
+    public static interface ConsumerX<T> extends Consumer<T> {
+        /**
+         * Consumer body.
+         *
+         * @throws Exception If failed.
+         */
+        void acceptX(T t) throws Exception;

Review comment:
       public void

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/CacheCreateDestroyEventsTest.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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.security.events;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.client.Config;
+import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.events.CacheEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.security.AbstractSecurityTest;
+import org.apache.ignite.internal.processors.security.impl.TestSecurityPluginProvider;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecuritySubject;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.events.EventType.EVT_CACHE_STARTED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.DESTROY_CACHE;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.GET_OR_CREATE_CACHE;
+import static org.apache.ignite.plugin.security.SecurityPermissionSetBuilder.ALLOW_ALL;
+
+/**
+ * Test that an event's local listener and an event's remote filter get correct subjectId
+ * when a server (client) node create or destroy a cache.
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+@RunWith(Parameterized.class)
+public class CacheCreateDestroyEventsTest extends AbstractSecurityTest {
+    /** Counter. */
+    private static final AtomicInteger COUNTER = new AtomicInteger();
+
+    /** Node that registers event listeners. */
+    private static final String LISTENER_NODE = "listener_node";
+
+    /** Client node. */
+    static final String CLNT = "client";
+
+    /** Server node. */
+    static final String SRV = "server";
+
+    /** Events latch. */
+    private static CountDownLatch evtsLatch;
+
+    /** */
+    private static final AtomicInteger rmtLoginCnt = new AtomicInteger();
+
+    /** */
+    private static final AtomicInteger locLoginCnt = new AtomicInteger();
+
+    /** */
+    @Parameterized.Parameter()
+    public int cacheCnt;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String login;
+
+    /** */
+    @Parameterized.Parameter(2)
+    public int evtType;
+
+    /** */
+    @Parameterized.Parameter(3)
+    public TestOperation op;
+
+    /** Parameters. */
+    @Parameterized.Parameters(name = "cacheCnt={0}, evtNode={1}, evtType={2}, op={3}")
+    public static Iterable<Object[]> data() {
+        return Arrays.asList(
+            new Object[] {1, SRV, EVT_CACHE_STARTED, TestOperation.GET_OR_CREATE_CACHE},
+            new Object[] {1, CLNT, EVT_CACHE_STARTED, TestOperation.GET_OR_CREATE_CACHE},
+            new Object[] {1, SRV, EVT_CACHE_STARTED, TestOperation.CREATE_CACHE},
+            new Object[] {1, CLNT, EVT_CACHE_STARTED, TestOperation.CREATE_CACHE},
+            new Object[] {1, SRV, EVT_CACHE_STOPPED, TestOperation.DESTROY_CACHE},
+            new Object[] {1, CLNT, EVT_CACHE_STOPPED, TestOperation.DESTROY_CACHE},
+            new Object[] {2, SRV, EVT_CACHE_STARTED, TestOperation.CREATE_CACHES},
+            new Object[] {2, CLNT, EVT_CACHE_STARTED, TestOperation.CREATE_CACHES},
+            new Object[] {2, SRV, EVT_CACHE_STOPPED, TestOperation.DESTROY_CACHES},
+            new Object[] {2, CLNT, EVT_CACHE_STOPPED, TestOperation.DESTROY_CACHES},
+            new Object[] {1, "thin", EVT_CACHE_STARTED, TestOperation.THIN_CLIENT_CREATE_CACHE},
+            new Object[] {1, "thin", EVT_CACHE_STARTED, TestOperation.THIN_CLIENT_GET_OR_CREATE},
+            new Object[] {1, "thin", EVT_CACHE_STOPPED, TestOperation.THIN_CLIENT_DESTROY_CACHE},
+            new Object[] {3, "new_client_node", EVT_CACHE_STARTED, TestOperation.START_NODE},
+            new Object[] {3, "new_server_node", EVT_CACHE_STARTED, TestOperation.START_NODE},
+            new Object[] {2, SRV, EVT_CACHE_STARTED, TestOperation.CHANGE_CLUSTER_STATE},
+            new Object[] {2, CLNT, EVT_CACHE_STARTED, TestOperation.CHANGE_CLUSTER_STATE},
+            new Object[] {2, SRV, EVT_CACHE_STOPPED, TestOperation.CHANGE_CLUSTER_STATE},
+            new Object[] {2, CLNT, EVT_CACHE_STOPPED, TestOperation.CHANGE_CLUSTER_STATE},
+            new Object[] {1, "rest", EVT_CACHE_STARTED, TestOperation.REST_GET_OR_CREATE_CACHE},
+            new Object[] {1, "rest", EVT_CACHE_STOPPED, TestOperation.REST_DESTROY_CACHE}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridAllowAll(LISTENER_NODE);
+        startGridAllowAll(SRV);
+        startClientAllowAll(CLNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopGrid("new_client_node");
+        stopGrid("new_server_node");
+    }
+
+    /** */
+    private GridTestUtils.ConsumerX<Collection<CacheConfiguration>> operation() {

Review comment:
       empty

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/CacheCreateDestroyEventsTest.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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.security.events;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.client.Config;
+import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.events.CacheEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.security.AbstractSecurityTest;
+import org.apache.ignite.internal.processors.security.impl.TestSecurityPluginProvider;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecuritySubject;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.events.EventType.EVT_CACHE_STARTED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.DESTROY_CACHE;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.GET_OR_CREATE_CACHE;
+import static org.apache.ignite.plugin.security.SecurityPermissionSetBuilder.ALLOW_ALL;
+
+/**
+ * Test that an event's local listener and an event's remote filter get correct subjectId
+ * when a server (client) node create or destroy a cache.
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+@RunWith(Parameterized.class)
+public class CacheCreateDestroyEventsTest extends AbstractSecurityTest {
+    /** Counter. */
+    private static final AtomicInteger COUNTER = new AtomicInteger();
+
+    /** Node that registers event listeners. */
+    private static final String LISTENER_NODE = "listener_node";
+
+    /** Client node. */
+    static final String CLNT = "client";
+
+    /** Server node. */
+    static final String SRV = "server";
+
+    /** Events latch. */
+    private static CountDownLatch evtsLatch;
+
+    /** */

Review comment:
       Empty

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/CacheCreateDestroyEventsTest.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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.security.events;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.client.Config;
+import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.events.CacheEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.security.AbstractSecurityTest;
+import org.apache.ignite.internal.processors.security.impl.TestSecurityPluginProvider;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecuritySubject;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.events.EventType.EVT_CACHE_STARTED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.DESTROY_CACHE;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.GET_OR_CREATE_CACHE;
+import static org.apache.ignite.plugin.security.SecurityPermissionSetBuilder.ALLOW_ALL;
+
+/**
+ * Test that an event's local listener and an event's remote filter get correct subjectId
+ * when a server (client) node create or destroy a cache.
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+@RunWith(Parameterized.class)
+public class CacheCreateDestroyEventsTest extends AbstractSecurityTest {
+    /** Counter. */
+    private static final AtomicInteger COUNTER = new AtomicInteger();
+
+    /** Node that registers event listeners. */
+    private static final String LISTENER_NODE = "listener_node";
+
+    /** Client node. */
+    static final String CLNT = "client";
+
+    /** Server node. */
+    static final String SRV = "server";
+
+    /** Events latch. */
+    private static CountDownLatch evtsLatch;
+
+    /** */
+    private static final AtomicInteger rmtLoginCnt = new AtomicInteger();
+
+    /** */
+    private static final AtomicInteger locLoginCnt = new AtomicInteger();
+
+    /** */
+    @Parameterized.Parameter()
+    public int cacheCnt;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String login;
+
+    /** */
+    @Parameterized.Parameter(2)
+    public int evtType;
+
+    /** */
+    @Parameterized.Parameter(3)
+    public TestOperation op;
+
+    /** Parameters. */
+    @Parameterized.Parameters(name = "cacheCnt={0}, evtNode={1}, evtType={2}, op={3}")
+    public static Iterable<Object[]> data() {
+        return Arrays.asList(
+            new Object[] {1, SRV, EVT_CACHE_STARTED, TestOperation.GET_OR_CREATE_CACHE},
+            new Object[] {1, CLNT, EVT_CACHE_STARTED, TestOperation.GET_OR_CREATE_CACHE},
+            new Object[] {1, SRV, EVT_CACHE_STARTED, TestOperation.CREATE_CACHE},
+            new Object[] {1, CLNT, EVT_CACHE_STARTED, TestOperation.CREATE_CACHE},
+            new Object[] {1, SRV, EVT_CACHE_STOPPED, TestOperation.DESTROY_CACHE},
+            new Object[] {1, CLNT, EVT_CACHE_STOPPED, TestOperation.DESTROY_CACHE},
+            new Object[] {2, SRV, EVT_CACHE_STARTED, TestOperation.CREATE_CACHES},
+            new Object[] {2, CLNT, EVT_CACHE_STARTED, TestOperation.CREATE_CACHES},
+            new Object[] {2, SRV, EVT_CACHE_STOPPED, TestOperation.DESTROY_CACHES},
+            new Object[] {2, CLNT, EVT_CACHE_STOPPED, TestOperation.DESTROY_CACHES},
+            new Object[] {1, "thin", EVT_CACHE_STARTED, TestOperation.THIN_CLIENT_CREATE_CACHE},
+            new Object[] {1, "thin", EVT_CACHE_STARTED, TestOperation.THIN_CLIENT_GET_OR_CREATE},
+            new Object[] {1, "thin", EVT_CACHE_STOPPED, TestOperation.THIN_CLIENT_DESTROY_CACHE},
+            new Object[] {3, "new_client_node", EVT_CACHE_STARTED, TestOperation.START_NODE},
+            new Object[] {3, "new_server_node", EVT_CACHE_STARTED, TestOperation.START_NODE},
+            new Object[] {2, SRV, EVT_CACHE_STARTED, TestOperation.CHANGE_CLUSTER_STATE},
+            new Object[] {2, CLNT, EVT_CACHE_STARTED, TestOperation.CHANGE_CLUSTER_STATE},
+            new Object[] {2, SRV, EVT_CACHE_STOPPED, TestOperation.CHANGE_CLUSTER_STATE},
+            new Object[] {2, CLNT, EVT_CACHE_STOPPED, TestOperation.CHANGE_CLUSTER_STATE},
+            new Object[] {1, "rest", EVT_CACHE_STARTED, TestOperation.REST_GET_OR_CREATE_CACHE},
+            new Object[] {1, "rest", EVT_CACHE_STOPPED, TestOperation.REST_DESTROY_CACHE}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridAllowAll(LISTENER_NODE);
+        startGridAllowAll(SRV);
+        startClientAllowAll(CLNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopGrid("new_client_node");
+        stopGrid("new_server_node");
+    }
+
+    /** */
+    private GridTestUtils.ConsumerX<Collection<CacheConfiguration>> operation() {
+        switch (op) {
+            case GET_OR_CREATE_CACHE:
+                return ccfgs -> grid(login).getOrCreateCache(ccfgs.iterator().next());
+
+            case CREATE_CACHE:
+                return ccfgs -> grid(login).createCache(ccfgs.iterator().next());
+
+            case DESTROY_CACHE:
+                return ccfgs -> grid(login).destroyCache(ccfgs.iterator().next().getName());
+
+            case CREATE_CACHES:
+                return ccfgs -> grid(login).createCaches(ccfgs);
+
+            case DESTROY_CACHES:
+                return ccfgs -> grid(login).destroyCaches(ccfgs.stream().map(CacheConfiguration::getName).collect(Collectors.toSet()));
+
+            case THIN_CLIENT_GET_OR_CREATE:
+                return ccfgs -> {
+                    try (IgniteClient clnt = startClient()) {
+                        clnt.getOrCreateCache(ccfgs.iterator().next().getName());
+                    }
+                };
+
+            case THIN_CLIENT_CREATE_CACHE:
+                return ccfgs -> {
+                    try (IgniteClient clnt = startClient()) {
+                        clnt.createCache(ccfgs.iterator().next().getName());
+                    }
+                };
+
+            case THIN_CLIENT_DESTROY_CACHE:
+                return ccfgs -> {
+                    try (IgniteClient clnt = startClient()) {
+                        clnt.destroyCache(ccfgs.iterator().next().getName());
+                    }
+                };
+
+            case START_NODE:
+                return ccfgs -> {
+                    try {
+                        startGrid(getConfiguration(login,
+                            new TestSecurityPluginProvider(login, "", ALLOW_ALL, false))
+                            .setClientMode(login.contains("client"))
+                            .setCacheConfiguration(ccfgs.toArray(
+                                new CacheConfiguration[] {new CacheConfiguration("test_cache_" + COUNTER.incrementAndGet())}))
+                        );
+                    }
+                    catch (Exception e) {
+                        throw new RuntimeException(e);
+                    }
+                };
+
+            case CHANGE_CLUSTER_STATE:
+                return ccfgs -> {
+                    grid(login).cluster().state(ClusterState.INACTIVE);
+
+                    grid(login).cluster().state(ClusterState.ACTIVE);
+                };
+
+            case REST_GET_OR_CREATE_CACHE:
+                return ccfgs -> handleRestRequest(GET_OR_CREATE_CACHE, ccfgs.iterator().next().getName());
+
+            case REST_DESTROY_CACHE:
+                return ccfgs -> handleRestRequest(DESTROY_CACHE, ccfgs.iterator().next().getName());
+
+            default:
+                throw new IllegalArgumentException("Unknown operation " + op);
+        }
+    }
+
+    /** */
+    private void handleRestRequest(GridRestCommand cmd, String cacheName) {
+        final GridRestCacheRequest req = new GridRestCacheRequest();
+
+        req.command(cmd);
+        req.credentials(new SecurityCredentials("rest", ""));
+        req.cacheName(cacheName);
+
+        try {
+            restProtocolHandler(grid(SRV)).handle(req);
+        }
+        catch (Exception e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /** */
+    @Test
+    public void testDynamicCreateDestroyCache() throws Exception {

Review comment:
       empty

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/AbstractCacheEventsTest.java
##########
@@ -0,0 +1,603 @@
+/*
+ * 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.security.events;
+
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import javax.cache.Cache;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.client.ClientCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.events.CacheEvent;
+import org.apache.ignite.events.CacheQueryExecutedEvent;
+import org.apache.ignite.events.CacheQueryReadEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.client.GridClientData;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.security.AbstractSecurityTest;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.Collections.singleton;
+import static org.apache.ignite.events.EventType.EVT_CACHE_ENTRY_CREATED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_ENTRY_DESTROYED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_LOCKED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_PUT;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_REMOVED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNLOCKED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;
+
+/**
+ * Security subject id of cache events have to refer to subject that initiates cache operation.
+ */
+public abstract class AbstractCacheEventsTest extends AbstractSecurityTest {
+    /** Counter to name caches. */
+    private static final AtomicInteger COUNTER = new AtomicInteger();
+
+    /** Node that registers event listeners. */
+    protected static final String LISTENER_NODE = "listener_node";
+
+    /** Client node. */
+    protected static final String CLNT = "client";
+
+    /** Server node. */
+    protected static final String SRV = "server";
+
+    /** Key. */
+    protected static final String KEY = "key";
+
+    /** Initiate value. */
+    public static final String INIT_VAL = "init_val";
+
+    /** Value. */
+    protected static final String VAL = "val";
+
+    /** Events latch. */
+    private static CountDownLatch evtsLatch;
+
+    /** Local events count. */
+    private static final AtomicInteger evtCnt = new AtomicInteger();
+
+    /** Error message - actual subject is not expected. */
+    private static final AtomicReference<String> error = new AtomicReference<>();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridAllowAll(LISTENER_NODE);
+        startGridAllowAll(SRV);
+        startGridAllowAll("additional_srv");
+
+        startClientAllowAll(CLNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setConnectorConfiguration(new ConnectorConfiguration())
+            .setIncludeEventTypes(EVT_CACHE_ENTRY_CREATED,
+                EVT_CACHE_ENTRY_DESTROYED,
+                EVT_CACHE_OBJECT_PUT,
+                EVT_CACHE_OBJECT_READ,
+                EVT_CACHE_OBJECT_REMOVED,
+                EVT_CACHE_OBJECT_LOCKED,
+                EVT_CACHE_OBJECT_UNLOCKED,
+                EVT_CACHE_QUERY_EXECUTED,
+                EVT_CACHE_QUERY_OBJECT_READ);
+    }
+
+    /**
+     *
+     */
+    protected abstract CacheAtomicityMode cacheAtomicityMode();

Review comment:
       java docs

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/TaskAndJobEventsTest.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.security.events;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.client.ClientCompute;
+import org.apache.ignite.client.Config;
+import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeJobResultPolicy;
+import org.apache.ignite.compute.ComputeTask;
+import org.apache.ignite.compute.ComputeTaskTimeoutException;
+import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.configuration.ClientConnectorConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.ThinClientConfiguration;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.events.JobEvent;
+import org.apache.ignite.events.TaskEvent;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientCompute;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientException;
+import org.apache.ignite.internal.client.GridClientFactory;
+import org.apache.ignite.internal.client.thin.ClientServerError;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.client.message.GridClientTaskResultBean;
+import org.apache.ignite.internal.processors.rest.request.GridRestTaskRequest;
+import org.apache.ignite.internal.processors.security.AbstractSecurityTest;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecurityCredentialsBasicProvider;
+import org.apache.ignite.plugin.security.SecuritySubject;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static java.util.Collections.singletonList;
+import static org.apache.ignite.events.EventType.EVT_JOB_CANCELLED;
+import static org.apache.ignite.events.EventType.EVT_JOB_FAILED;
+import static org.apache.ignite.events.EventType.EVT_JOB_FINISHED;
+import static org.apache.ignite.events.EventType.EVT_JOB_MAPPED;
+import static org.apache.ignite.events.EventType.EVT_JOB_QUEUED;
+import static org.apache.ignite.events.EventType.EVT_JOB_RESULTED;
+import static org.apache.ignite.events.EventType.EVT_JOB_STARTED;
+import static org.apache.ignite.events.EventType.EVT_TASK_FAILED;
+import static org.apache.ignite.events.EventType.EVT_TASK_FINISHED;
+import static org.apache.ignite.events.EventType.EVT_TASK_REDUCED;
+import static org.apache.ignite.events.EventType.EVT_TASK_STARTED;
+import static org.apache.ignite.events.EventType.EVT_TASK_TIMEDOUT;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Tests that an event's local listener and an event's remote filter get correct subjectId when task's or job's
+ * operations are performed.
+ */
+@RunWith(Parameterized.class)
+public class TaskAndJobEventsTest extends AbstractSecurityTest {
+    /** Types of events. */
+    private static final int[] EVENT_TYPES = new int[] {
+        EVT_TASK_STARTED, EVT_TASK_FINISHED, EVT_TASK_REDUCED,
+        EVT_JOB_MAPPED, EVT_JOB_RESULTED, EVT_JOB_STARTED, EVT_JOB_FINISHED, EVT_JOB_QUEUED};
+
+    /** Types of events. */
+    private static final int[] TIMEDOUT_EVENT_TYPES = new int[] {
+        EVT_TASK_STARTED, EVT_TASK_TIMEDOUT, EVT_TASK_FAILED, EVT_JOB_MAPPED,
+        EVT_JOB_QUEUED, EVT_JOB_STARTED, EVT_JOB_CANCELLED, EVT_JOB_FAILED
+    };
+
+    /** Job's sleep time. */
+    private static final long TIME_TO_SLEEP = 2000L;
+
+    /** Timeout. */
+    private static final long TIMEOUT = 1000L;
+
+    /** Remote events. */
+    private static final Set<Integer> rmtSet = Collections.synchronizedSet(new HashSet<>());
+
+    /** Local events. */
+    private static final Set<Integer> locSet = Collections.synchronizedSet(new HashSet<>());
+
+    /** Test task name. */
+    private static final String TASK_NAME = "org.apache.ignite.internal.processors.security.events.TaskAndJobEventsTest$TestComputeTask";
+
+    /** Node that registers event listeners. */
+    private static final String LISTENER_NODE = "listener_node";
+
+    /** Client node. */
+    private static final String CLNT = "client";
+
+    /** Server node. */
+    private static final String SRV = "server";
+
+    /** Expected login. */
+    @Parameterized.Parameter
+    public String expLogin;
+
+    /** Async mode. */
+    @Parameterized.Parameter(1)
+    public Boolean async;
+
+    /** Task should be timed out. */
+    @Parameterized.Parameter(2)
+    public boolean timedout;
+
+    /** Parameters. */
+    @Parameterized.Parameters(name = "expLogin={0}, async={1}, timedout={2}")
+    public static Iterable<Object[]> data() {
+        List<Object[]> res = new ArrayList<>();
+
+        Stream.of(SRV, CLNT, "thin", "rest", "grid").forEach(login -> {
+            res.add(new Object[] {login, false, false});
+            res.add(new Object[] {login, true, false});
+
+            if (!"grid".equals(login)) {
+                res.add(new Object[] {login, false, true});
+                res.add(new Object[] {login, true, true});
+            }
+        });
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridAllowAll(LISTENER_NODE);
+        startGridAllowAll(SRV);
+        startClientAllowAll(CLNT);
+    }
+
+    /** */
+    @Test
+    public void test() throws Exception {
+        int[] evtTypes = timedout ? TIMEDOUT_EVENT_TYPES : EVENT_TYPES;
+
+        Arrays.stream(evtTypes).forEach(type -> {
+            locSet.add(type);
+            rmtSet.add(type);
+        });
+
+        UUID taskLsnrId = grid(LISTENER_NODE).events().remoteListen(
+            new IgniteBiPredicate<UUID, Event>() {
+                @IgniteInstanceResource IgniteEx ign;
+
+                @Override public boolean apply(UUID uuid, Event evt) {
+                    locSet.remove(evt.type());
+
+                    onEvent(ign, evt, expLogin);
+
+                    return true;
+                }
+            },
+            new IgnitePredicate<Event>() {
+                @IgniteInstanceResource IgniteEx ign;
+
+                @Override public boolean apply(Event evt) {
+                    rmtSet.remove(evt.type());
+
+                    onEvent(ign, evt, expLogin);
+
+                    return true;
+                }
+            }, evtTypes);
+
+        try {
+            if (timedout) {
+                assertThrowsWithCause(operation(),
+                    "thin".equals(expLogin) ? ClientServerError.class : ComputeTaskTimeoutException.class);
+            }
+            else
+                operation().run();
+
+            waitForCondition(locSet::isEmpty, 10_000);
+        }
+        finally {
+            grid(LISTENER_NODE).events().stopRemoteListen(taskLsnrId);
+        }
+
+        assertTrue("Remote filter. Events that are not happen: " + rmtSet.stream().map(U::gridEventName),
+            rmtSet.isEmpty());
+        assertTrue("Local listener. Events that are not happen: " + locSet.stream().map(U::gridEventName),
+            locSet.isEmpty());
+    }
+
+    /** */
+    private GridTestUtils.RunnableX operation() {

Review comment:
       java docs

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/compute/closure/ClientsComputeTaskRemoteSecurityContextTest.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.security.compute.closure;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.client.ClientCompute;
+import org.apache.ignite.client.Config;
+import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeJobResultPolicy;
+import org.apache.ignite.compute.ComputeTask;
+import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.configuration.ClientConnectorConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.ThinClientConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientCompute;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientFactory;
+import org.apache.ignite.internal.processors.rest.GridRestCommand;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.client.message.GridClientTaskResultBean;
+import org.apache.ignite.internal.processors.rest.request.GridRestTaskRequest;
+import org.apache.ignite.internal.processors.security.AbstractRemoteSecurityContextCheckTest;
+import org.apache.ignite.internal.processors.security.AbstractTestSecurityPluginProvider;
+import org.apache.ignite.internal.processors.security.SecurityUtils;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecurityCredentialsBasicProvider;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static java.util.Collections.singletonList;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Testing operation security context when the compute task is executed on remote nodes.
+ */
+@RunWith(Parameterized.class)
+public class ClientsComputeTaskRemoteSecurityContextTest extends AbstractRemoteSecurityContextCheckTest {
+    /** */
+    @Parameterized.Parameters(name = "async={0}")

Review comment:
       java docs

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/CacheEventsTest.java
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.security.events;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.client.Config;
+import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientDataConfiguration;
+import org.apache.ignite.internal.client.GridClientException;
+import org.apache.ignite.internal.client.GridClientFactory;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.RestQueryRequest;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecurityCredentialsBasicProvider;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
+import static org.apache.ignite.events.EventType.EVT_CACHE_ENTRY_CREATED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_ENTRY_DESTROYED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_LOCKED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_PUT;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_REMOVED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNLOCKED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;
+
+/**
+ * Tests that an event's local listener and an event's remote filter get correct subjectId when cache's operations are
+ * performed.
+ */
+@RunWith(Parameterized.class)
+public class CacheEventsTest extends AbstractCacheEventsTest {
+    /** Cache atomicity mode. */
+    @Parameterized.Parameter()
+    public CacheAtomicityMode atomicMode;
+
+    /** Expected login. */
+    @Parameterized.Parameter(1)
+    public String expLogin;
+
+    /** Event type. */
+    @Parameterized.Parameter(2)
+    public int evtType;
+
+    /** Test operation. */
+    @Parameterized.Parameter(3)
+    public TestOperation op;
+
+    /** Parameters. */
+    @Parameterized.Parameters(name = "atomicMode={0}, expLogin={1}, evtType={2}, op={3}")
+    public static Iterable<Object[]> data() {
+        List<Object[]> lst = Arrays.asList(
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.PUT},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.PUT},
+            new Object[] {"thin", EVT_CACHE_OBJECT_PUT, TestOperation.PUT},
+            new Object[] {"rest", EVT_CACHE_OBJECT_PUT, TestOperation.PUT},
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.PUT},
+
+            new Object[] {CLNT, EVT_CACHE_ENTRY_CREATED, TestOperation.PUT},
+            new Object[] {SRV, EVT_CACHE_ENTRY_CREATED, TestOperation.PUT},
+            new Object[] {"thin", EVT_CACHE_ENTRY_CREATED, TestOperation.PUT},
+            new Object[] {"rest", EVT_CACHE_ENTRY_CREATED, TestOperation.PUT},
+            new Object[] {"grid", EVT_CACHE_ENTRY_CREATED, TestOperation.PUT},
+
+            new Object[] {CLNT, EVT_CACHE_ENTRY_DESTROYED, TestOperation.PUT},
+            new Object[] {SRV, EVT_CACHE_ENTRY_DESTROYED, TestOperation.PUT},
+            new Object[] {"thin", EVT_CACHE_ENTRY_DESTROYED, TestOperation.PUT},
+            new Object[] {"rest", EVT_CACHE_ENTRY_DESTROYED, TestOperation.PUT},
+            new Object[] {"grid", EVT_CACHE_ENTRY_DESTROYED, TestOperation.PUT},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ASYNC},
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ALL},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ALL},
+            new Object[] {"thin", EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ALL},
+            new Object[] {"rest", EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ALL},
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ALL},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ALL_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ALL_ASYNC},
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.PUT_ALL_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.PUT_IF_ABSENT},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.PUT_IF_ABSENT},
+            new Object[] {"thin", EVT_CACHE_OBJECT_PUT, TestOperation.PUT_IF_ABSENT},
+            new Object[] {"rest", EVT_CACHE_OBJECT_PUT, TestOperation.PUT_IF_ABSENT},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.PUT_IF_ABSENT_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.PUT_IF_ABSENT_ASYNC},
+            new Object[] {"rest", EVT_CACHE_OBJECT_PUT, TestOperation.PUT_IF_ABSENT_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET},
+            new Object[] {"thin", EVT_CACHE_OBJECT_READ, TestOperation.GET},
+            new Object[] {"rest", EVT_CACHE_OBJECT_READ, TestOperation.GET},
+            new Object[] {"grid", EVT_CACHE_OBJECT_READ, TestOperation.GET},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_ASYNC},
+            new Object[] {"grid", EVT_CACHE_OBJECT_READ, TestOperation.GET_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_ENTRY},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_ENTRY},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_ENTRY_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_ENTRY_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_ENTRIES},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_ENTRIES},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_ENTRIES_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_ENTRIES_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL},
+            new Object[] {"thin", EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL},
+            new Object[] {"rest", EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL},
+            new Object[] {"grid", EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL_ASYNC},
+            new Object[] {"grid", EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL_OUT_TX},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL_OUT_TX},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL_OUT_TX_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_ALL_OUT_TX_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT},
+            new Object[] {"thin", EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT},
+            new Object[] {"rest", EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT},
+            new Object[] {"thin", EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT},
+            new Object[] {"rest", EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT_IF_ABSENT},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT_IF_ABSENT},
+            new Object[] {"rest", EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT_IF_ABSENT},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT_IF_ABSENT_PUT_CASE},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT_IF_ABSENT_PUT_CASE},
+            new Object[] {"rest", EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT_IF_ABSENT_PUT_CASE},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT_IF_ABSENT_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_PUT_IF_ABSENT_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT_IF_ABSENT_PUT_CASE_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_PUT_IF_ABSENT_PUT_CASE_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REMOVE},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REMOVE},
+            new Object[] {"thin", EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REMOVE},
+            new Object[] {"rest", EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REMOVE},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_REMOVED, TestOperation.GET_AND_REMOVE},
+            new Object[] {SRV, EVT_CACHE_OBJECT_REMOVED, TestOperation.GET_AND_REMOVE},
+            new Object[] {"thin", EVT_CACHE_OBJECT_REMOVED, TestOperation.GET_AND_REMOVE},
+            new Object[] {"rest", EVT_CACHE_OBJECT_REMOVED, TestOperation.GET_AND_REMOVE},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REMOVE_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REMOVE_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_REMOVED, TestOperation.GET_AND_REMOVE_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_REMOVED, TestOperation.GET_AND_REMOVE_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE},
+            new Object[] {SRV, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE},
+            new Object[] {"thin", EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE},
+            new Object[] {"rest", EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE},
+            new Object[] {"grid", EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ASYNC},
+            new Object[] {"grid", EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_VAL},
+            new Object[] {SRV, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_VAL},
+            new Object[] {"thin", EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_VAL},
+            new Object[] {"rest", EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_VAL},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_VAL_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_VAL_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ALL},
+            new Object[] {SRV, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ALL},
+            new Object[] {"thin", EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ALL},
+            new Object[] {"rest", EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ALL},
+            new Object[] {"grid", EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ALL},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ALL_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ALL_ASYNC},
+            new Object[] {"grid", EVT_CACHE_OBJECT_REMOVED, TestOperation.REMOVE_ALL_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE},
+            new Object[] {"thin", EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE},
+            new Object[] {"rest", EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE},
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_ASYNC},
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_VAL},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_VAL},
+            new Object[] {"thin", EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_VAL},
+            new Object[] {"rest", EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_VAL},
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_VAL},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_VAL_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_VAL_ASYNC},
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.REPLACE_VAL_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REPLACE},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REPLACE},
+            new Object[] {"thin", EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REPLACE},
+            new Object[] {"rest", EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REPLACE},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REPLACE_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_READ, TestOperation.GET_AND_REPLACE_ASYNC},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_REPLACE},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_REPLACE},
+            new Object[] {"thin", EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_REPLACE},
+            new Object[] {"rest", EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_REPLACE},
+
+            new Object[] {CLNT, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_REPLACE_ASYNC},
+            new Object[] {SRV, EVT_CACHE_OBJECT_PUT, TestOperation.GET_AND_REPLACE_ASYNC},
+
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.APPEND},
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.APPEND_ASYNC},
+
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.PREPEND},
+            new Object[] {"grid", EVT_CACHE_OBJECT_PUT, TestOperation.PREPEND_ASYNC},
+
+            new Object[] {SRV, EVT_CACHE_OBJECT_LOCKED, TestOperation.LOCK},
+            new Object[] {CLNT, EVT_CACHE_OBJECT_LOCKED, TestOperation.LOCK},
+            new Object[] {SRV, EVT_CACHE_OBJECT_UNLOCKED, TestOperation.LOCK},
+            new Object[] {CLNT, EVT_CACHE_OBJECT_UNLOCKED, TestOperation.LOCK},
+
+            new Object[] {SRV, EVT_CACHE_OBJECT_LOCKED, TestOperation.LOCK_ALL},
+            new Object[] {CLNT, EVT_CACHE_OBJECT_LOCKED, TestOperation.LOCK_ALL},
+            new Object[] {SRV, EVT_CACHE_OBJECT_UNLOCKED, TestOperation.LOCK_ALL},
+            new Object[] {CLNT, EVT_CACHE_OBJECT_UNLOCKED, TestOperation.LOCK_ALL},
+
+            new Object[] {CLNT, EVT_CACHE_QUERY_EXECUTED, TestOperation.SCAN_QUERY},
+            new Object[] {SRV, EVT_CACHE_QUERY_EXECUTED, TestOperation.SCAN_QUERY},
+            new Object[] {"thin", EVT_CACHE_QUERY_EXECUTED, TestOperation.SCAN_QUERY},
+            new Object[] {"rest", EVT_CACHE_QUERY_EXECUTED, TestOperation.SCAN_QUERY},
+
+            new Object[] {CLNT, EVT_CACHE_QUERY_OBJECT_READ, TestOperation.SCAN_QUERY},
+            new Object[] {SRV, EVT_CACHE_QUERY_OBJECT_READ, TestOperation.SCAN_QUERY},
+            new Object[] {"thin", EVT_CACHE_QUERY_OBJECT_READ, TestOperation.SCAN_QUERY},
+            new Object[] {"rest", EVT_CACHE_QUERY_OBJECT_READ, TestOperation.SCAN_QUERY}
+        );
+
+        List<Object[]> res = new ArrayList<>();
+
+        for (Object[] arr : lst) {
+            res.add(new Object[] {CacheAtomicityMode.TRANSACTIONAL, arr[0], arr[1], arr[2]});
+
+            int evt = (int)arr[1];
+
+            TestOperation op = (TestOperation)arr[2];
+            // todo IGNITE-13490 Cache's operation getAndXXX doesn't trigger a CacheEvent with type EVT_CACHE_OBJECT_READ.
+            // This condition should be removed when the issue will be done.
+            if (evt == EVT_CACHE_OBJECT_READ && op.name().contains("GET_AND_"))
+                continue;
+
+            if (evt == EVT_CACHE_OBJECT_LOCKED || evt == EVT_CACHE_OBJECT_UNLOCKED)
+                continue;
+
+            res.add(new Object[] {CacheAtomicityMode.ATOMIC, arr[0], arr[1], arr[2]});
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode cacheAtomicityMode() {
+        return atomicMode;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String expectedLogin() {
+        return expLogin;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int eventType() {
+        return evtType;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected TestOperation testOperation() {
+        return op;
+    }
+
+    /** */
+    @Override protected GridTestUtils.ConsumerX<String> operation() {

Review comment:
       java docs.




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

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



[GitHub] [ignite] dgarus closed pull request #8038: IGNITE-13112 The current security context should be obtained using the IgniteSecurity interface only

Posted by GitBox <gi...@apache.org>.
dgarus closed pull request #8038:
URL: https://github.com/apache/ignite/pull/8038


   


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

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

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