You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2017/01/18 17:21:43 UTC

[2/3] ignite git commit: IGNITE-4487 Fixed NPE on query execution during concurrent cache destroy. This closes #1388

IGNITE-4487 Fixed NPE on query execution during concurrent cache destroy. This closes #1388


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/177d7f49
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/177d7f49
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/177d7f49

Branch: refs/heads/ignite-comm-balance-master
Commit: 177d7f49934e24e85472a8e44a641c4b57a25914
Parents: 7b711a3
Author: Alexander Menshikov <sh...@gmail.com>
Authored: Wed Jan 18 18:33:45 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Jan 18 18:33:45 2017 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryAdapter.java      |   8 ++
 .../IgniteCacheQueryCacheDestroySelfTest.java   | 142 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 3 files changed, 152 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/177d7f49/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 1fe263d..3ecacf8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -34,6 +34,7 @@ import org.apache.ignite.cache.query.Query;
 import org.apache.ignite.cache.query.QueryMetrics;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException;
@@ -666,6 +667,13 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
 
             nodes = fallbacks(cctx.discovery().topologyVersionEx());
 
+            if (F.isEmpty(nodes))
+                throw new ClusterTopologyException("Failed to execute the query " +
+                    "(all affinity nodes left the grid) [cache=" + cctx.name() +
+                    ", qry=" + qry +
+                    ", startTopVer=" + cctx.versions().last().topologyVersion() +
+                    ", curTopVer=" + qryMgr.queryTopologyVersion().topologyVersion() + ']');
+
             init();
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/177d7f49/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java
new file mode 100644
index 0000000..dc104ff
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java
@@ -0,0 +1,142 @@
+/*
+ * 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;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * The test for the destruction of the cache during the execution of the query
+ */
+public class IgniteCacheQueryCacheDestroySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE_NAME = "cache";
+
+    /** */
+    public static final int GRID_CNT = 3;
+
+    /**
+     * The main test code.
+     */
+    public void testQueue() throws Throwable {
+        startGridsMultiThreaded(GRID_CNT);
+
+        Ignite ig = ignite(0);
+
+        ig.getOrCreateCache(cacheConfiguration());
+
+        final AtomicBoolean stop = new AtomicBoolean();
+        final AtomicReference<Exception> npe = new AtomicReference<>();
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    while (!stop.get())
+                        runQuery();
+                }
+                catch (Exception e) {
+                    NullPointerException npe0 = X.cause(e, NullPointerException.class);
+
+                    if (npe0 != null)
+                        npe.compareAndSet(null, npe0);
+                    else
+                        info("Expected exception: " + e);
+                }
+
+                return null;
+            }
+        }, 6, "query-runner");
+
+        U.sleep(500);
+
+        ig.destroyCache(CACHE_NAME);
+
+        stop.set(true);
+
+        fut.get();
+
+        assertNull(npe.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void runQuery() throws Exception {
+        ScanQuery<String, String> scanQuery = new ScanQuery<String, String>()
+            .setLocal(true)
+            .setFilter(new IgniteBiPredicate<String, String>() {
+                @Override public boolean apply(String key, String p) {
+                    return key != null && key.isEmpty();
+                }
+            });
+
+        Ignite ignite = ignite(ThreadLocalRandom.current().nextInt(GRID_CNT));
+
+        IgniteCache<String, String> example = ignite.cache(CACHE_NAME);
+
+        for (int partition : ignite.affinity(CACHE_NAME).primaryPartitions(ignite.cluster().localNode())) {
+            scanQuery.setPartition(partition);
+
+            try (QueryCursor cursor = example.query(scanQuery)) {
+                for (Object p : cursor) {
+                    String value = (String) ((Cache.Entry)p).getValue();
+
+                    assertNotNull(value);
+                }
+            }
+        }
+    }
+
+    /**
+     * @return Cache configuration for this test.
+     */
+    private CacheConfiguration cacheConfiguration() {
+        CacheConfiguration cfg = new CacheConfiguration(CACHE_NAME);
+
+        cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC)
+            .setCacheMode(CacheMode.PARTITIONED)
+            .setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED)
+            .setRebalanceMode(CacheRebalanceMode.SYNC)
+            .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC)
+            .setRebalanceThrottle(100)
+            .setRebalanceBatchSize(2 * 1024 * 1024)
+            .setBackups(1)
+            .setEagerTtl(false);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/177d7f49/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 1e98eb5..b5e4078 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -92,6 +92,7 @@ import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCa
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryTransformerSelfTest;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.query.IgniteCacheQueryCacheDestroySelfTest;
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryTxSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest;
@@ -238,6 +239,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(CacheQueryNewClientSelfTest.class);
         suite.addTestSuite(CacheOffheapBatchIndexingSingleTypeTest.class);
         suite.addTestSuite(CacheSqlQueryValueCopySelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryCacheDestroySelfTest.class);
 
         return suite;
     }