You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/02/03 13:03:04 UTC

[1/3] ignite git commit: IGNITE-2454 Fixed "Continuous query notification missed if there is only one node".

Repository: ignite
Updated Branches:
  refs/heads/ignite-2523 07c23931f -> e066650cf


IGNITE-2454 Fixed "Continuous query notification missed if there is only one node".


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

Branch: refs/heads/ignite-2523
Commit: 60c8b072dc08d92e45da2404be09031af165ae48
Parents: e6acce6
Author: Tikhonov Nikolay <ti...@gmail.com>
Authored: Wed Feb 3 13:06:14 2016 +0300
Committer: Tikhonov Nikolay <ti...@gmail.com>
Committed: Wed Feb 3 13:06:14 2016 +0300

----------------------------------------------------------------------
 .../continuous/CacheContinuousQueryHandler.java |   9 +
 ...ntinuousQueryPartitionAtomicOneNodeTest.java |  37 ++++
 ...heContinuousQueryPartitionTxOneNodeTest.java |  37 ++++
 ...tinuousQueryReplicatedAtomicOneNodeTest.java |  31 +++
 ...ontinuousQueryReplicatedOneNodeSelfTest.java | 120 ------------
 ...eContinuousQueryReplicatedTxOneNodeTest.java | 193 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |  10 +-
 7 files changed, 315 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/60c8b072/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index fa54a6b..7e66ad3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -754,6 +754,15 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
             List<CacheContinuousQueryEntry> entries;
 
             synchronized (pendingEvts) {
+                // Received first event.
+                if (curTop == AffinityTopologyVersion.NONE) {
+                    lastFiredEvt = entry.updateCounter();
+
+                    curTop = entry.topologyVersion();
+
+                    return F.asList(entry);
+                }
+
                 if (curTop.compareTo(entry.topologyVersion()) < 0) {
                     if (entry.updateCounter() == 1L && !entry.isBackup()) {
                         entries = new ArrayList<>(pendingEvts.size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/60c8b072/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryPartitionAtomicOneNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryPartitionAtomicOneNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryPartitionAtomicOneNodeTest.java
new file mode 100644
index 0000000..797882b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryPartitionAtomicOneNodeTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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 org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ *
+ */
+public class GridCacheContinuousQueryPartitionAtomicOneNodeTest
+    extends GridCacheContinuousQueryReplicatedTxOneNodeTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/60c8b072/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryPartitionTxOneNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryPartitionTxOneNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryPartitionTxOneNodeTest.java
new file mode 100644
index 0000000..7dd7bd7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryPartitionTxOneNodeTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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 org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ *
+ */
+public class GridCacheContinuousQueryPartitionTxOneNodeTest
+    extends GridCacheContinuousQueryReplicatedTxOneNodeTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/60c8b072/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedAtomicOneNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedAtomicOneNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedAtomicOneNodeTest.java
new file mode 100644
index 0000000..066afa9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedAtomicOneNodeTest.java
@@ -0,0 +1,31 @@
+/*
+ * 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 org.apache.ignite.cache.CacheAtomicityMode;
+
+/**
+ *
+ */
+public class GridCacheContinuousQueryReplicatedAtomicOneNodeTest
+    extends GridCacheContinuousQueryReplicatedTxOneNodeTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/60c8b072/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedOneNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedOneNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedOneNodeSelfTest.java
deleted file mode 100644
index 8152b2a..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedOneNodeSelfTest.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * 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.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import javax.cache.event.CacheEntryEvent;
-import javax.cache.event.CacheEntryListenerException;
-import javax.cache.event.CacheEntryUpdatedListener;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheRebalanceMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.cache.query.ContinuousQuery;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-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;
-
-/**
- * Test for replicated cache with one node.
- */
-public class GridCacheContinuousQueryReplicatedOneNodeSelfTest extends GridCommonAbstractTest {
-    /** IP finder. */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setCacheMode(CacheMode.REPLICATED);
-        cacheCfg.setRebalanceMode(CacheRebalanceMode.SYNC);
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(disco);
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLocal() throws Exception {
-        doTest(true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDistributed() throws Exception {
-        doTest(false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void doTest(boolean loc) throws Exception {
-        try {
-            IgniteCache<String, Integer> cache = startGrid(0).cache(null);
-
-            ContinuousQuery<String, Integer> qry = new ContinuousQuery<>();
-
-            final AtomicInteger cnt = new AtomicInteger();
-            final CountDownLatch latch = new CountDownLatch(10);
-
-            qry.setLocalListener(new CacheEntryUpdatedListener<String, Integer>() {
-                @Override
-                public void onUpdated(Iterable<CacheEntryEvent<? extends String, ? extends Integer>> evts)
-                        throws CacheEntryListenerException {
-                    for (CacheEntryEvent<? extends String, ? extends Integer> evt : evts) {
-                        cnt.incrementAndGet();
-                        latch.countDown();
-                    }
-                }
-            });
-
-            cache.query(qry.setLocal(loc));
-
-            startGrid(1);
-
-            awaitPartitionMapExchange();
-
-            for (int i = 0; i < 10; i++)
-                cache.put("key" + i, i);
-
-            assert latch.await(5000, TimeUnit.MILLISECONDS);
-
-            assertEquals(10, cnt.get());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/60c8b072/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedTxOneNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedTxOneNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedTxOneNodeTest.java
new file mode 100644
index 0000000..271a8d9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedTxOneNodeTest.java
@@ -0,0 +1,193 @@
+/*
+ * 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.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryListenerException;
+import javax.cache.event.CacheEntryUpdatedListener;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.query.ContinuousQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+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;
+
+/**
+ * Test for replicated cache with one node.
+ */
+@SuppressWarnings("Duplicates")
+public class GridCacheContinuousQueryReplicatedTxOneNodeTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setAtomicityMode(atomicMode());
+        cacheCfg.setCacheMode(cacheMode());
+        cacheCfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /**
+     * @return Atomicity mode for a cache.
+     */
+    protected CacheAtomicityMode atomicMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /**
+     * @return Cache mode.
+     */
+    protected CacheMode cacheMode() {
+        return CacheMode.REPLICATED;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocal() throws Exception {
+        if (cacheMode() == CacheMode.REPLICATED)
+            doTest(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDistributed() throws Exception {
+        doTest(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocalOneNode() throws Exception {
+        doTestOneNode(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDistributedOneNode() throws Exception {
+        doTestOneNode(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTest(boolean loc) throws Exception {
+        try {
+            IgniteCache<String, Integer> cache = startGrid(0).cache(null);
+
+            ContinuousQuery<String, Integer> qry = new ContinuousQuery<>();
+
+            final AtomicInteger cnt = new AtomicInteger();
+            final CountDownLatch latch = new CountDownLatch(10);
+
+            qry.setLocalListener(new CacheEntryUpdatedListener<String, Integer>() {
+                @Override
+                public void onUpdated(Iterable<CacheEntryEvent<? extends String, ? extends Integer>> evts)
+                        throws CacheEntryListenerException {
+                    for (CacheEntryEvent<? extends String, ? extends Integer> evt : evts) {
+                        cnt.incrementAndGet();
+                        latch.countDown();
+                    }
+                }
+            });
+
+            cache.query(qry.setLocal(loc));
+
+            startGrid(1);
+
+            awaitPartitionMapExchange();
+
+            for (int i = 0; i < 10; i++)
+                cache.put("key" + i, i);
+
+            assert latch.await(5000, TimeUnit.MILLISECONDS);
+
+            assertEquals(10, cnt.get());
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTestOneNode(boolean loc) throws Exception {
+        try {
+            IgniteCache<String, Integer> cache = startGrid(0).cache(null);
+
+            ContinuousQuery<String, Integer> qry = new ContinuousQuery<>();
+
+            final AtomicInteger cnt = new AtomicInteger();
+            final CountDownLatch latch = new CountDownLatch(10);
+
+            for (int i = 0; i < 10; i++)
+                cache.put("key" + i, i);
+
+            cache.clear();
+
+            qry.setLocalListener(new CacheEntryUpdatedListener<String, Integer>() {
+                @Override public void onUpdated(Iterable<CacheEntryEvent<? extends String, ? extends Integer>> evts)
+                        throws CacheEntryListenerException {
+                    for (CacheEntryEvent<? extends String, ? extends Integer> evt : evts) {
+                        cnt.incrementAndGet();
+                        latch.countDown();
+                    }
+                }
+            });
+
+            cache.query(qry.setLocal(loc));
+
+            for (int i = 0; i < 10; i++)
+                cache.put("key" + i, i);
+
+            assert latch.await(5000, TimeUnit.MILLISECONDS);
+
+            assertEquals(10, cnt.get());
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/60c8b072/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 4b1eafa..359cdf3 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
@@ -79,11 +79,14 @@ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheCon
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicSelfTest;
 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;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionTxOneNodeTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedOnlySelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedAtomicOneNodeTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedOneNodeSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedTxOneNodeTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxSelfTest;
@@ -181,7 +184,10 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedOneNodeSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryReplicatedTxOneNodeTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicOneNodeTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryPartitionTxOneNodeTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryPartitionAtomicOneNodeTest.class);
         suite.addTestSuite(IgniteCacheContinuousQueryClientTest.class);
         suite.addTestSuite(IgniteCacheContinuousQueryClientReconnectTest.class);
         suite.addTestSuite(IgniteCacheContinuousQueryClientTxReconnectTest.class);


[2/3] ignite git commit: 2224 Added Javadoc about new methods to CacheEntry

Posted by vo...@apache.org.
2224 Added Javadoc about new methods to CacheEntry


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

Branch: refs/heads/ignite-2523
Commit: e7de923e6c34bdb4e276bd4314868541087d66f1
Parents: 60c8b07
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Feb 3 13:11:10 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Feb 3 13:11:10 2016 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/cache/CacheEntry.java   | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e7de923e/modules/core/src/main/java/org/apache/ignite/cache/CacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheEntry.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheEntry.java
index d92f9fb..19585a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheEntry.java
@@ -24,9 +24,9 @@ import org.apache.ignite.IgniteCache;
 
 /**
  * Cache entry that extends {@link javax.cache.Cache.Entry} by providing additional entry related information.
- *
- * To get an instance of {@code CacheEntry} use {@link javax.cache.Cache.Entry#unwrap(Class)} method by passing
- * {@code CacheEntry} class to it as the argument.
+ * <p>
+ * To get an instance of {@code CacheEntry} from {@link javax.cache.Cache.Entry} use
+ * {@link javax.cache.Cache.Entry#unwrap(Class)} method by passing {@code CacheEntry} class to it as the argument.
  * <p>
  * {@code CacheEntry} is supported only for {@link javax.cache.Cache.Entry} returned by one of the following methods:
  * <ul>
@@ -36,6 +36,9 @@ import org.apache.ignite.IgniteCache;
  * <li>{@link IgniteCache#randomEntry()}</li>
  * </ul>
  * <p>
+ * To get an instance of {@code CacheEntry} directly use {@link IgniteCache#getEntry(Object)} or
+ * {@link IgniteCache#getEntries(Set)} methods.
+ * <p>
  * {@code CacheEntry} is not supported for {@link javax.cache.Cache#iterator()} because of performance reasons.
  * {@link javax.cache.Cache#iterator()} loads entries from all the cluster nodes and to speed up the load additional
  * information, like entry's version, is ignored.


[3/3] ignite git commit: Merge branch 'master' into ignite-2523

Posted by vo...@apache.org.
Merge branch 'master' into ignite-2523


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

Branch: refs/heads/ignite-2523
Commit: e066650cf4907851dca1d8ef0297f6a1522d6a87
Parents: 07c2393 e7de923
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Feb 3 15:02:52 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 3 15:02:52 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/CacheEntry.java     |   9 +-
 .../continuous/CacheContinuousQueryHandler.java |   9 +
 ...ntinuousQueryPartitionAtomicOneNodeTest.java |  37 ++++
 ...heContinuousQueryPartitionTxOneNodeTest.java |  37 ++++
 ...tinuousQueryReplicatedAtomicOneNodeTest.java |  31 +++
 ...ontinuousQueryReplicatedOneNodeSelfTest.java | 120 ------------
 ...eContinuousQueryReplicatedTxOneNodeTest.java | 193 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |  10 +-
 8 files changed, 321 insertions(+), 125 deletions(-)
----------------------------------------------------------------------