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/06/03 15:08:33 UTC

[GitHub] [ignite] dgarus opened a new pull request #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897


   


----------------------------------------------------------------
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 #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897#discussion_r435761294



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/AbstractSecurityCacheEventTest.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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 com.google.common.collect.Iterators;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.CacheConfiguration;
+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.security.AbstractSecurityTest;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.security.SecuritySubject;
+
+import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
+
+/**
+ * Abstract class to test correctness of {@link CacheEvent#subjectId}.
+ */
+@SuppressWarnings("rawtypes")
+public abstract class AbstractSecurityCacheEventTest extends AbstractSecurityTest {
+    /** Counter. */
+    protected static final AtomicInteger COUNTER = new AtomicInteger();
+
+    /** Node that registers event listeners. */
+    private static final String LISTENER_NODE = "listener_node";
+
+    /** Events latch. */
+    private static volatile CountDownLatch evtsLatch;
+
+    /** Logins in remote filters. */
+    private static final Collection<String> rmtLogins = new ConcurrentLinkedQueue<>();

Review comment:
       We can just throw exception if we found second login.
   Collection fields not necessary.
   Please, remove it.




----------------------------------------------------------------
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 #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897#discussion_r435758346



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
##########
@@ -153,6 +154,74 @@ public static SecurityContext nodeSecurityContext(Marshaller marsh, ClassLoader
         }
     }
 
+    /**
+     * Runs passed {@code runnable} with the security context associated
+     * with passed {@code secSubjId} if security is enabled.
+     *
+     * @param secSubjId Security subject id.
+     * @param ctx Grid kernal context.
+     * @param r Runnable.
+     */
+    public static void withContextIfNeed(UUID secSubjId, GridKernalContext ctx, RunnableX r)
+        throws IgniteCheckedException {
+        IgniteSecurity security = ctx.security();
+
+        try {
+            if (security.enabled() && secSubjId != null) {
+                try (OperationSecurityContext s = security.withContext(secSubjId)) {
+                    r.runx();
+                }
+            }
+            else
+                r.runx();
+        }
+        catch (Exception e) {
+            if (e instanceof IgniteCheckedException)
+                throw (IgniteCheckedException)e;
+
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     * Calls passed {@code callable} with the security context associated
+     * with passed {@code secSubjId} if security is enabled.
+     *
+     * @param secSubjId Security subject id.
+     * @param ctx Grid kernal context.
+     * @param c Callable.
+     * @return Result of passed callable.
+     */
+    public static <T> T withContextIfNeed(UUID secSubjId, GridKernalContext ctx, Callable<T> c)
+        throws IgniteCheckedException {
+        IgniteSecurity security = ctx.security();
+
+        try {
+            if (security.enabled() && secSubjId != null) {
+                try (OperationSecurityContext s = security.withContext(secSubjId)) {
+                    return c.call();
+                }
+            }
+
+            return c.call();
+        }
+        catch (Exception e) {
+            if (e instanceof IgniteCheckedException)

Review comment:
       No, we cannot throw Exception, the places when this method is using expect IgniteCheckedException.




----------------------------------------------------------------
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 #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897#issuecomment-658129775


   Do we have a test to cover the scenario when cache created and started from static configuration `IgniteConfiguration#setCacheConfiguration`?


----------------------------------------------------------------
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 #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897#discussion_r435755821



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/AbstractSecurityCacheEventTest.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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 com.google.common.collect.Iterators;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.CacheConfiguration;
+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.security.AbstractSecurityTest;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.security.SecuritySubject;
+
+import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
+
+/**
+ * Abstract class to test correctness of {@link CacheEvent#subjectId}.
+ */
+@SuppressWarnings("rawtypes")
+public abstract class AbstractSecurityCacheEventTest extends AbstractSecurityTest {
+    /** Counter. */
+    protected static final AtomicInteger COUNTER = new AtomicInteger();
+
+    /** Node that registers event listeners. */
+    private static final String LISTENER_NODE = "listener_node";
+
+    /** Events latch. */
+    private static volatile CountDownLatch evtsLatch;
+
+    /** Logins in remote filters. */
+    private static final Collection<String> rmtLogins = new ConcurrentLinkedQueue<>();

Review comment:
       To get as much information as possible about the failure. A failure message will contain info about all logins inside predicates.




----------------------------------------------------------------
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 #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897#discussion_r454290184



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
##########
@@ -127,6 +144,8 @@ public ChangeGlobalStateMessage(
     void exchangeActions(ExchangeActions exchangeActions) {
         assert exchangeActions != null && !exchangeActions.empty() : exchangeActions;
 
+        exchangeActions.securitySubjectId(securitySubjectId());

Review comment:
       Why do we want to do this action inside setter?
   Let's do it outside of it.




----------------------------------------------------------------
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 #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897#discussion_r435736253



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/AbstractSecurityCacheEventTest.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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 com.google.common.collect.Iterators;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.CacheConfiguration;
+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.security.AbstractSecurityTest;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.security.SecuritySubject;
+
+import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
+
+/**
+ * Abstract class to test correctness of {@link CacheEvent#subjectId}.
+ */
+@SuppressWarnings("rawtypes")
+public abstract class AbstractSecurityCacheEventTest extends AbstractSecurityTest {
+    /** Counter. */
+    protected static final AtomicInteger COUNTER = new AtomicInteger();
+
+    /** Node that registers event listeners. */
+    private static final String LISTENER_NODE = "listener_node";
+
+    /** Events latch. */
+    private static volatile CountDownLatch evtsLatch;
+
+    /** Logins in remote filters. */
+    private static final Collection<String> rmtLogins = new ConcurrentLinkedQueue<>();

Review comment:
       Why do we need collection for `rmtLogins` and `locLogins`?
   We expect and assume that it will be single login in this sets - 
   ```
           if (set.size() != 1) {
               fail(msgPrefix + " Expected subject: " + expLogin +
                   ". Actual subjects: " + Iterators.toString(set.iterator()));
           }
   ```




----------------------------------------------------------------
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 #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897#discussion_r435737041



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/AbstractSecurityCacheEventTest.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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 com.google.common.collect.Iterators;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.CacheConfiguration;
+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.security.AbstractSecurityTest;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.security.SecuritySubject;
+
+import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
+
+/**
+ * Abstract class to test correctness of {@link CacheEvent#subjectId}.
+ */
+@SuppressWarnings("rawtypes")
+public abstract class AbstractSecurityCacheEventTest extends AbstractSecurityTest {
+    /** Counter. */
+    protected static final AtomicInteger COUNTER = new AtomicInteger();
+
+    /** Node that registers event listeners. */
+    private static final String LISTENER_NODE = "listener_node";
+
+    /** Events latch. */
+    private static volatile CountDownLatch evtsLatch;
+
+    /** Logins in remote filters. */
+    private static final Collection<String> rmtLogins = new ConcurrentLinkedQueue<>();
+
+    /** Logins in a local listener. */
+    private static final Collection<String> locLogins = new ConcurrentLinkedQueue<>();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridAllowAll(LISTENER_NODE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName).setIncludeEventTypes(EventType.EVTS_CACHE_LIFECYCLE);
+    }
+
+    /** */
+    protected void testCacheEvents(int expTimes, String expLogin, int evtType, Collection<CacheConfiguration> ccfgs,
+        Consumer<Collection<CacheConfiguration>> op) throws Exception {
+        // For the EVT_CACHE_STOPPED event count of local listener should be 1 due to IGNITE-13010.
+        evtsLatch = new CountDownLatch(expTimes + (evtType == EVT_CACHE_STOPPED ? 1 : expTimes));
+
+        rmtLogins.clear();
+        locLogins.clear();
+
+        UUID lsnrId = grid(LISTENER_NODE)
+            .events()
+            .remoteListen(
+                new TestPredicate(ccfgs) {
+                    @Override void register(String login) {
+                        locLogins.add(login);
+                    }
+                },
+                new TestPredicate(ccfgs) {
+                    @Override void register(String login) {
+                        rmtLogins.add(login);
+                    }
+                }, evtType);
+
+        try {
+            // Execute tested operation.
+            op.accept(ccfgs);
+
+            // Waiting for events.
+            evtsLatch.await(10, TimeUnit.SECONDS);
+
+            // Check results.

Review comment:
       This comment can be omitted.




----------------------------------------------------------------
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 #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897#discussion_r435758907



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
##########
@@ -153,6 +154,74 @@ public static SecurityContext nodeSecurityContext(Marshaller marsh, ClassLoader
         }
     }
 
+    /**
+     * Runs passed {@code runnable} with the security context associated
+     * with passed {@code secSubjId} if security is enabled.
+     *
+     * @param secSubjId Security subject id.
+     * @param ctx Grid kernal context.
+     * @param r Runnable.
+     */
+    public static void withContextIfNeed(UUID secSubjId, GridKernalContext ctx, RunnableX r)
+        throws IgniteCheckedException {
+        IgniteSecurity security = ctx.security();
+
+        try {
+            if (security.enabled() && secSubjId != null) {
+                try (OperationSecurityContext s = security.withContext(secSubjId)) {
+                    r.runx();
+                }
+            }
+            else
+                r.runx();
+        }
+        catch (Exception e) {
+            if (e instanceof IgniteCheckedException)
+                throw (IgniteCheckedException)e;
+
+            throw new IgniteCheckedException(e);

Review comment:
       I'll use the run method that doesn't throw an exception.




----------------------------------------------------------------
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 #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897#discussion_r435731554



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
##########
@@ -153,6 +154,74 @@ public static SecurityContext nodeSecurityContext(Marshaller marsh, ClassLoader
         }
     }
 
+    /**
+     * Runs passed {@code runnable} with the security context associated
+     * with passed {@code secSubjId} if security is enabled.
+     *
+     * @param secSubjId Security subject id.
+     * @param ctx Grid kernal context.
+     * @param r Runnable.
+     */
+    public static void withContextIfNeed(UUID secSubjId, GridKernalContext ctx, RunnableX r)
+        throws IgniteCheckedException {
+        IgniteSecurity security = ctx.security();
+
+        try {
+            if (security.enabled() && secSubjId != null) {
+                try (OperationSecurityContext s = security.withContext(secSubjId)) {
+                    r.runx();
+                }
+            }
+            else
+                r.runx();
+        }
+        catch (Exception e) {
+            if (e instanceof IgniteCheckedException)
+                throw (IgniteCheckedException)e;
+
+            throw new IgniteCheckedException(e);

Review comment:
       Why do we need to change the exception here?
   Can't we just throw it?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
##########
@@ -153,6 +154,74 @@ public static SecurityContext nodeSecurityContext(Marshaller marsh, ClassLoader
         }
     }
 
+    /**
+     * Runs passed {@code runnable} with the security context associated
+     * with passed {@code secSubjId} if security is enabled.
+     *
+     * @param secSubjId Security subject id.
+     * @param ctx Grid kernal context.
+     * @param r Runnable.
+     */
+    public static void withContextIfNeed(UUID secSubjId, GridKernalContext ctx, RunnableX r)
+        throws IgniteCheckedException {
+        IgniteSecurity security = ctx.security();
+
+        try {
+            if (security.enabled() && secSubjId != null) {
+                try (OperationSecurityContext s = security.withContext(secSubjId)) {
+                    r.runx();
+                }
+            }
+            else
+                r.runx();
+        }
+        catch (Exception e) {
+            if (e instanceof IgniteCheckedException)
+                throw (IgniteCheckedException)e;
+
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     * Calls passed {@code callable} with the security context associated
+     * with passed {@code secSubjId} if security is enabled.
+     *
+     * @param secSubjId Security subject id.
+     * @param ctx Grid kernal context.
+     * @param c Callable.
+     * @return Result of passed callable.
+     */
+    public static <T> T withContextIfNeed(UUID secSubjId, GridKernalContext ctx, Callable<T> c)
+        throws IgniteCheckedException {
+        IgniteSecurity security = ctx.security();
+
+        try {
+            if (security.enabled() && secSubjId != null) {
+                try (OperationSecurityContext s = security.withContext(secSubjId)) {
+                    return c.call();
+                }
+            }
+
+            return c.call();
+        }
+        catch (Exception e) {
+            if (e instanceof IgniteCheckedException)

Review comment:
       Why do we need to change the exception here?
   Can't we just throw it?




----------------------------------------------------------------
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 #7897: 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 #7897:
URL: https://github.com/apache/ignite/pull/7897#discussion_r435756227



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/security/events/AbstractSecurityCacheEventTest.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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 com.google.common.collect.Iterators;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.CacheConfiguration;
+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.security.AbstractSecurityTest;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.security.SecuritySubject;
+
+import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
+
+/**
+ * Abstract class to test correctness of {@link CacheEvent#subjectId}.
+ */
+@SuppressWarnings("rawtypes")
+public abstract class AbstractSecurityCacheEventTest extends AbstractSecurityTest {
+    /** Counter. */
+    protected static final AtomicInteger COUNTER = new AtomicInteger();
+
+    /** Node that registers event listeners. */
+    private static final String LISTENER_NODE = "listener_node";
+
+    /** Events latch. */
+    private static volatile CountDownLatch evtsLatch;
+
+    /** Logins in remote filters. */
+    private static final Collection<String> rmtLogins = new ConcurrentLinkedQueue<>();
+
+    /** Logins in a local listener. */
+    private static final Collection<String> locLogins = new ConcurrentLinkedQueue<>();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridAllowAll(LISTENER_NODE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName).setIncludeEventTypes(EventType.EVTS_CACHE_LIFECYCLE);
+    }
+
+    /** */
+    protected void testCacheEvents(int expTimes, String expLogin, int evtType, Collection<CacheConfiguration> ccfgs,
+        Consumer<Collection<CacheConfiguration>> op) throws Exception {
+        // For the EVT_CACHE_STOPPED event count of local listener should be 1 due to IGNITE-13010.
+        evtsLatch = new CountDownLatch(expTimes + (evtType == EVT_CACHE_STOPPED ? 1 : expTimes));
+
+        rmtLogins.clear();
+        locLogins.clear();
+
+        UUID lsnrId = grid(LISTENER_NODE)
+            .events()
+            .remoteListen(
+                new TestPredicate(ccfgs) {
+                    @Override void register(String login) {
+                        locLogins.add(login);
+                    }
+                },
+                new TestPredicate(ccfgs) {
+                    @Override void register(String login) {
+                        rmtLogins.add(login);
+                    }
+                }, evtType);
+
+        try {
+            // Execute tested operation.
+            op.accept(ccfgs);
+
+            // Waiting for events.
+            evtsLatch.await(10, TimeUnit.SECONDS);
+
+            // Check results.

Review comment:
       ok




----------------------------------------------------------------
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 #7897: 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 closed pull request #7897:
URL: https://github.com/apache/ignite/pull/7897


   


-- 
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