You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sk...@apache.org on 2021/05/06 19:09:13 UTC

[ignite] branch master updated: IGNITE-14582 Fixed CQ remote filter deployment to the client nodes. Fixes #9015

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 6041dce  IGNITE-14582 Fixed CQ remote filter deployment to the client nodes. Fixes #9015
6041dce is described below

commit 6041dce13e5462abcb8fb99c14636d5239dd5308
Author: Igor Belyakov <ig...@gmail.com>
AuthorDate: Thu May 6 22:08:22 2021 +0300

    IGNITE-14582 Fixed CQ remote filter deployment to the client nodes. Fixes #9015
    
    Signed-off-by: Slava Koptilin <sl...@gmail.com>
---
 .../continuous/CacheContinuousQueryManager.java    |   8 +-
 .../continuous/GridContinuousProcessor.java        |  84 +++++++-------
 ...CacheContinuousQueryDeploymentToClientTest.java | 128 +++++++++++++++++++++
 .../testsuites/IgniteCacheQuerySelfTestSuite6.java |   4 +-
 4 files changed, 180 insertions(+), 44 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index 219b30d..db10431 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -70,6 +70,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx
 import org.apache.ignite.internal.processors.continuous.GridContinuousHandler;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridLongList;
+import org.apache.ignite.internal.util.lang.gridfunc.IsAllPredicate;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
@@ -80,6 +81,7 @@ import org.apache.ignite.lang.IgniteOutClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.plugin.security.SecurityPermission;
 import org.apache.ignite.resources.LoggerResource;
+import org.apache.ignite.util.AttributeNodeFilter;
 import org.jetbrains.annotations.Nullable;
 
 import static javax.cache.event.EventType.CREATED;
@@ -90,6 +92,7 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;
 import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE;
 import static org.apache.ignite.internal.IgniteFeatures.CONT_QRY_SECURITY_AWARE;
 import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CLIENT_MODE;
 
 /**
  * Continuous queries manager.
@@ -747,8 +750,9 @@ public class CacheContinuousQueryManager<K, V> extends GridCacheManagerAdapter<K
         hnd.keepBinary(keepBinary);
         hnd.localOnly(locOnly);
 
-        IgnitePredicate<ClusterNode> pred = (loc || cctx.config().getCacheMode() == CacheMode.LOCAL) ?
-            F.nodeForNodeId(cctx.localNodeId()) : cctx.group().nodeFilter();
+        IgnitePredicate<ClusterNode> pred = (loc || cctx.config().getCacheMode() == CacheMode.LOCAL)
+            ? F.nodeForNodeId(cctx.localNodeId())
+            : new IsAllPredicate<>(cctx.group().nodeFilter(), new AttributeNodeFilter(ATTR_CLIENT_MODE, false));
 
         assert pred != null : cctx.config();
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 293225a..396d217 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -676,13 +676,32 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                 ctx.resource().injectGeneric(nodeFilter);
 
             if (nodeFilter == null || nodeFilter.apply(ctx.discovery().localNode())) {
-                    registerHandler(srcNodeId,
-                        routineId,
-                        hnd,
-                        bufSize,
-                        interval,
-                        autoUnsubscribe,
-                        false);
+                registerHandler(srcNodeId,
+                    routineId,
+                    hnd,
+                    bufSize,
+                    interval,
+                    autoUnsubscribe,
+                    false);
+
+                if (ctx.config().isPeerClassLoadingEnabled()) {
+                    // Peer class loading cannot be performed before a node joins, so we delay the deployment.
+                    // Run the deployment task in the system pool to avoid blocking of the discovery thread.
+                    ctx.discovery().localJoinFuture().listen(f -> ctx.closure().runLocalSafe((GridPlainRunnable)() -> {
+                        try {
+                            hnd.p2pUnmarshal(srcNodeId, ctx);
+                        }
+                        catch (IgniteCheckedException | IgniteException e) {
+                            U.error(log, "Failed to unmarshal continuous routine handler [" +
+                                "routineId=" + routineId +
+                                ", srcNodeId=" + srcNodeId + ']', e);
+
+                            ctx.failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+
+                            unregisterHandler(routineId, hnd, false);
+                        }
+                    }));
+                }
             }
             else {
                 if (log.isDebugEnabled()) {
@@ -699,25 +718,6 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
             ctx.failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
         }
-
-        if (ctx.config().isPeerClassLoadingEnabled()) {
-            // Peer class loading cannot be performed before a node joins, so we delay the deployment.
-            // Run the deployment task in the system pool to avoid blocking of the discovery thread.
-            ctx.discovery().localJoinFuture().listen(f -> ctx.closure().runLocalSafe((GridPlainRunnable)() -> {
-                try {
-                    hnd.p2pUnmarshal(srcNodeId, ctx);
-                }
-                catch (IgniteCheckedException | IgniteException e) {
-                    U.error(log, "Failed to unmarshal continuous routine handler [" +
-                        "routineId=" + routineId +
-                        ", srcNodeId=" + srcNodeId + ']', e);
-
-                    ctx.failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
-
-                    unregisterHandler(routineId, hnd, false);
-                }
-            }));
-        }
     }
 
     /**
@@ -1419,8 +1419,6 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
                     data.p2pUnmarshal(marsh, U.resolveClassLoader(dep.classLoader(), ctx.config()));
                 }
-
-                hnd.p2pUnmarshal(node.id(), ctx);
             }
         }
         catch (IgniteCheckedException e) {
@@ -1456,10 +1454,26 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                     ctx.resource().injectGeneric(prjPred);
 
                 if ((prjPred == null || prjPred.apply(ctx.discovery().node(ctx.localNodeId()))) &&
-                    !locInfos.containsKey(routineId))
+                    !locInfos.containsKey(routineId)) {
+                    if (ctx.config().isPeerClassLoadingEnabled())
+                        hnd.p2pUnmarshal(node.id(), ctx);
+
                     registerHandler(node.id(), routineId, hnd, data.bufferSize(), data.interval(),
                         data.autoUnsubscribe(), false);
 
+                    // Load partition counters.
+                    if (err == null && hnd.isQuery()) {
+                        GridCacheProcessor proc = ctx.cache();
+
+                        if (proc != null) {
+                            GridCacheAdapter cache = ctx.cache().internalCache(hnd.cacheName());
+
+                            if (cache != null && !cache.isLocal() && cache.context().userCache())
+                                req.addUpdateCounters(ctx.localNodeId(), hnd.updateCounters());
+                        }
+                    }
+                }
+
                 if (!data.autoUnsubscribe())
                     // Register routine locally.
                     locInfos.putIfAbsent(routineId, new LocalRoutineInfo(
@@ -1472,18 +1486,6 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             }
         }
 
-        // Load partition counters.
-        if (err == null && hnd.isQuery()) {
-            GridCacheProcessor proc = ctx.cache();
-
-            if (proc != null) {
-                GridCacheAdapter cache = ctx.cache().internalCache(hnd.cacheName());
-
-                if (cache != null && !cache.isLocal() && cache.context().userCache())
-                    req.addUpdateCounters(ctx.localNodeId(), hnd.updateCounters());
-            }
-        }
-
         if (err != null)
             req.addError(ctx.localNodeId(), err);
     }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryDeploymentToClientTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryDeploymentToClientTest.java
new file mode 100644
index 0000000..1eed05f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryDeploymentToClientTest.java
@@ -0,0 +1,128 @@
+/*
+ * 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.query.continuous;
+
+import java.util.Map;
+import javax.cache.configuration.Factory;
+import javax.cache.event.CacheEntryEventFilter;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.AbstractContinuousQuery;
+import org.apache.ignite.cache.query.ContinuousQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Tests for continuous query deployment to client nodes.
+ */
+public class CacheContinuousQueryDeploymentToClientTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "test_cache";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setPeerClassLoadingEnabled(true);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(CACHE_NAME));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * Test starts 1 server node and 1 client node. The client node deploys
+     * CQ for the cache {@link #CACHE_NAME}. After that another client node is started.
+     * Expected that CQ won't be deployed to the new client, since the client doesn't
+     * store any data.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeploymentToNewClient() throws Exception {
+        startGrid(0);
+
+        IgniteEx client1 = startClientGrid(1);
+
+        IgniteCache<Integer, String> cache = client1.cache(CACHE_NAME);
+
+        AbstractContinuousQuery<Integer, String> qry = new ContinuousQuery<Integer, String>()
+            .setLocalListener(evts -> {
+                // No-op.
+            })
+            .setRemoteFilterFactory((Factory<CacheEntryEventFilter<Integer, String>>)() -> evt -> true);
+
+        cache.query(qry);
+
+        IgniteEx client2 = startClientGrid(2);
+
+        GridContinuousProcessor proc = client2.context().continuous();
+
+        assertEquals(0, ((Map<?, ?>)U.field(proc, "locInfos")).size());
+        assertEquals(0, ((Map<?, ?>)U.field(proc, "rmtInfos")).size());
+        assertEquals(0, ((Map<?, ?>)U.field(proc, "startFuts")).size());
+        assertEquals(0, ((Map<?, ?>)U.field(proc, "stopFuts")).size());
+        assertEquals(0, ((Map<?, ?>)U.field(proc, "bufCheckThreads")).size());
+    }
+
+    /**
+     * Test starts 1 server node and 2 client nodes. The first client node deploys
+     * CQ for the cache {@link #CACHE_NAME}.
+     * Expected that CQ won't be deployed to the second client, since the client doesn't
+     * store any data.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeploymentToExistingClient() throws Exception {
+        startGrid(0);
+
+        IgniteEx client1 = startClientGrid(1);
+
+        IgniteCache<Integer, String> cache = client1.cache(CACHE_NAME);
+
+        IgniteEx client2 = startClientGrid(2);
+
+        AbstractContinuousQuery<Integer, String> qry = new ContinuousQuery<Integer, String>()
+            .setLocalListener(evts -> {
+                // No-op.
+            })
+            .setRemoteFilterFactory((Factory<CacheEntryEventFilter<Integer, String>>)() -> evt -> true);
+
+        cache.query(qry);
+
+        GridContinuousProcessor proc = client2.context().continuous();
+
+        assertEquals(0, ((Map<?, ?>)U.field(proc, "locInfos")).size());
+        assertEquals(0, ((Map<?, ?>)U.field(proc, "rmtInfos")).size());
+        assertEquals(0, ((Map<?, ?>)U.field(proc, "startFuts")).size());
+        assertEquals(0, ((Map<?, ?>)U.field(proc, "stopFuts")).size());
+        assertEquals(0, ((Map<?, ?>)U.field(proc, "bufCheckThreads")).size());
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite6.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite6.java
index 76f61a7..624dcc5 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite6.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite6.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.cache.index.StaticCacheDdlKeepStati
 import org.apache.ignite.internal.processors.cache.index.StaticCacheDdlTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchAckTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryAsyncFilterListenerTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryDeploymentToClientTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFilterDeploymentFailedTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryOperationP2PTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryOrderingEventTest;
@@ -71,7 +72,8 @@ import org.junit.runners.Suite;
     MemLeakOnSqlWithClientReconnectTest.class,
     CacheContinuousQueryFilterDeploymentFailedTest.class,
     PerformanceStatisticsQueryTest.class,
-    CacheContinuousQueryFilterDeploymentFailedTest.class
+    CacheContinuousQueryFilterDeploymentFailedTest.class,
+    CacheContinuousQueryDeploymentToClientTest.class
 })
 public class IgniteCacheQuerySelfTestSuite6 {
 }