You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by nt...@apache.org on 2016/03/11 16:05:51 UTC

ignite git commit: IGNITE-2791 Fixed.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2791 [created] ef0425361


IGNITE-2791 Fixed.


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

Branch: refs/heads/ignite-2791
Commit: ef04253613846102e410880047d66de9c5f0e230
Parents: 1f328e4
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu Mar 10 16:42:01 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Mar 11 18:05:20 2016 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java       |  10 +-
 .../GridCacheContinuousQueryConcurrentTest.java | 290 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 3 files changed, 298 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ef042536/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index b3786cd..0f89997 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -1323,11 +1323,13 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             Map<Integer, Long> res = new HashMap<>(cntrMap);
 
             for (GridDhtLocalPartition part : locParts.values()) {
-                Long cntr0 = res.get(part.id());
-                Long cntr1 = part.updateCounter();
+                if (part.primary(topVer)) {
+                    Long cntr0 = res.get(part.id());
+                    Long cntr1 = part.updateCounter();
 
-                if (cntr0 == null || cntr1 > cntr0)
-                    res.put(part.id(), cntr1);
+                    if (cntr0 == null || cntr1 > cntr0)
+                        res.put(part.id(), cntr1);
+                }
             }
 
             return res;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef042536/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryConcurrentTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryConcurrentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryConcurrentTest.java
new file mode 100644
index 0000000..bab2bad
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryConcurrentTest.java
@@ -0,0 +1,290 @@
+/*
+ * 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.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import javax.cache.configuration.CacheEntryListenerConfiguration;
+import javax.cache.configuration.FactoryBuilder.SingletonFactory;
+import javax.cache.configuration.MutableCacheEntryListenerConfiguration;
+import javax.cache.event.CacheEntryCreatedListener;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryEventFilter;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static java.lang.Thread.currentThread;
+import static java.util.concurrent.Executors.newSingleThreadExecutor;
+import static javax.cache.configuration.FactoryBuilder.factoryOf;
+
+/**
+ *
+ */
+public class GridCacheContinuousQueryConcurrentTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES = 2;
+
+    /** */
+    public static final ExecutorService EXECUTOR_SERVICE = newSingleThreadExecutor();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(NODES);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRegistration() throws Exception {
+        try {
+            final IgniteCache<Integer, String> cache = grid(0).getOrCreateCache(cacheConfiguration());
+
+            for (int i = 0; i < 100; i++) {
+                log.info("Start iteration: " + i);
+
+                final int i0 = i;
+                final AtomicBoolean stop = new AtomicBoolean(false);
+                final CountDownLatch latch = new CountDownLatch(1);
+                final int conQryCnt = 50;
+
+                Future<List<IgniteFuture<String>>> fut = EXECUTOR_SERVICE.submit(
+                    new Callable<List<IgniteFuture<String>>>() {
+                        @Override public List<IgniteFuture<String>> call() throws Exception {
+                            int count = 0;
+                            List<IgniteFuture<String>> futures = new ArrayList<>();
+
+                            while (!stop.get()) {
+                                futures.add(waitForKey(i0, cache, count));
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Started cont query count: " + count);
+
+                                if (++count == conQryCnt)
+                                    latch.countDown();
+                            }
+
+                            return futures;
+                        }
+                    });
+
+                latch.await();
+
+                cache.put(i, "v");
+
+                stop.set(true);
+
+                List<IgniteFuture<String>> contQries = fut.get();
+
+                for (int j = 0; j < contQries.size(); j++)
+                    contQries.get(j).get(2, TimeUnit.SECONDS);
+            }
+        }
+        finally {
+            EXECUTOR_SERVICE.shutdownNow();
+        }
+    }
+
+    /**
+     * @param key Key
+     * @param cache Cache.
+     * @param id ID.
+     * @return Future.
+     */
+    public IgniteFuture<String> waitForKey(Integer key, final IgniteCache<Integer, String> cache, final int id) {
+        String v = cache.get(key);
+
+        // From now on, all futures will be completed immediately (since the key has been
+        // inserted).
+        if (v != null)
+            return new IgniteFinishedFutureImpl<>("immediately");
+
+        final IgniteFuture<String> promise = new IgniteFutureImpl<>(new GridFutureAdapter<String>());
+
+        final CacheEntryListenerConfiguration<Integer, String> cfg =
+            createCacheListener(key, promise, id);
+
+        promise.listen(new IgniteInClosure<IgniteFuture<String>>() {
+            @Override public void apply(IgniteFuture<String> future) {
+                cache.deregisterCacheEntryListener(cfg);
+            }
+        });
+
+        // Start listening.
+        // Assumption: When the call returns, the listener is guaranteed to have been registered.
+        cache.registerCacheEntryListener(cfg);
+
+        // Now must check the cache again, to make sure that we didn't miss the key insert while we
+        // were busy setting up the cache listener.
+        // Check asynchronously.
+        IgniteCache<Integer, String> asyncCache = cache.withAsync();
+        asyncCache.get(key);
+
+        // Complete the promise if the key was inserted concurrently.
+        asyncCache.<String>future().listen(new IgniteInClosure<IgniteFuture<String>>() {
+            @Override public void apply(IgniteFuture<String> f) {
+                String value = f.get();
+
+                if (value != null) {
+                    log.info("Completed by get: " + id);
+
+                    (((GridFutureAdapter)((IgniteFutureImpl)promise).internalFuture())).onDone("by get");
+                }
+            }
+        });
+
+        return promise;
+    }
+
+    /**
+     * @param key Key.
+     * @param result Result.
+     * @param id Listener ID.
+     * @return Listener
+     */
+    private CacheEntryListenerConfiguration<Integer, String> createCacheListener(
+        Integer key,
+        IgniteFuture<String> result,
+        int id) {
+        return new MutableCacheEntryListenerConfiguration<>(
+            factoryOf(new CacheListener(result, id)),
+            new SingletonFactory<>(new KeyEventFilter(key, id)), false, true);
+    }
+
+
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, String> cacheConfiguration() {
+        CacheConfiguration<Integer, String> cfg = new CacheConfiguration<>("test");
+        cfg.setCacheMode(CacheMode.REPLICATED);
+        cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cfg.setAtomicWriteOrderMode(CacheAtomicWriteOrderMode.PRIMARY);
+        cfg.setReadFromBackup(false);
+        cfg.setStartSize(1024);
+
+        return cfg;
+    }
+
+    /**
+     *
+     */
+    private static class CacheListener implements CacheEntryCreatedListener<Integer, String>, Serializable {
+        /** */
+        final IgniteFuture<String> result;
+
+        /** */
+        private final int id;
+
+        /**
+         * @param result Result.
+         * @param id ID.
+         */
+        CacheListener(IgniteFuture<String> result, int id) {
+            this.result = result;
+            this.id = id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onCreated(Iterable<CacheEntryEvent<? extends Integer, ? extends String>> evts) {
+            (((GridFutureAdapter)((IgniteFutureImpl)result).internalFuture())).onDone("by listener");
+        }
+    }
+
+    /**
+     *
+     */
+    private static class KeyEventFilter implements CacheEntryEventFilter<Integer, String>, Serializable {
+        /** */
+        private static final long serialVersionUID = 42L;
+
+        /** */
+        private final Object key;
+
+        /** */
+        private final int id;
+
+        /**
+         * @param key Key.
+         * @param id ID.
+         */
+        KeyEventFilter(Object key, int id) {
+            this.key = key;
+            this.id = id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean evaluate(CacheEntryEvent<? extends Integer, ? extends String> e) {
+            return e.getKey().equals(key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            return this == o || !(o == null || getClass() != o.getClass())
+                && key.equals(((KeyEventFilter) o).key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return key.hashCode();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef042536/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 083af1e..0aa3560 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
@@ -93,6 +93,7 @@ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheCon
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicOffheapValuesTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryConcurrentTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalAtomicSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionAtomicOneNodeTest;
@@ -228,6 +229,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(CacheContinuousQueryFailoverTxOffheapTieredTest.class);
         suite.addTestSuite(CacheContinuousQueryRandomOperationsTest.class);
         suite.addTestSuite(CacheContinuousQueryFactoryFilterTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryConcurrentTest.class);
         suite.addTestSuite(CacheContinuousQueryOperationP2PTest.class);
         suite.addTestSuite(CacheContinuousBatchAckTest.class);
         suite.addTestSuite(CacheContinuousBatchForceServerModeAckTest.class);