You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/24 08:32:09 UTC

[1/5] incubator-ignite git commit: ignite-1022 do not need to start new thread inside tx on utility cache

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-sprint-7 359b43144 -> b3e2bd488


ignite-1022 do not need to start new thread inside tx on utility cache


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

Branch: refs/heads/ignite-sprint-7
Commit: 74dc6d5a23c7a40841cbb41357e1e3bb2404a93c
Parents: 1c66078
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 10:54:37 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 10:54:37 2015 +0300

----------------------------------------------------------------------
 .../datastructures/DataStructuresProcessor.java | 64 +++++++-------------
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |  4 +-
 .../GridCacheSetAbstractSelfTest.java           |  8 +--
 3 files changed, 27 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/74dc6d5a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 473a2ac..e6335b6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -297,8 +297,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
-        removeDataStructure(new IgniteCallable<Void>() {
-            @Override public Void call() throws Exception {
+        removeDataStructure(new IgniteOutClosureX<Void>() {
+            @Override public Void applyx() throws IgniteCheckedException {
                 dsCacheCtx.gate().enter();
 
                 try {
@@ -306,9 +306,6 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
                     removeInternal(key, GridCacheAtomicSequenceValue.class);
                 }
-                catch (Exception e) {
-                    throw new IgniteCheckedException("Failed to remove sequence by name: " + name, e);
-                }
                 finally {
                     dsCacheCtx.gate().leave();
                 }
@@ -429,11 +426,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
             if (err != null)
                 throw err;
 
-            dataStructure = ctx.closure().callLocalSafe(new Callable<T>() {
-                @Override public T call() throws Exception {
-                    return c.applyx();
-                }
-            }, false).get();
+            dataStructure = c.applyx();
 
             tx.commit();
         }
@@ -453,16 +446,13 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         awaitInitialization();
 
-        removeDataStructure(new IgniteCallable<Void>() {
-            @Override public Void call() throws Exception {
+        removeDataStructure(new IgniteOutClosureX<Void>() {
+            @Override public Void applyx() throws IgniteCheckedException {
                 dsCacheCtx.gate().enter();
 
                 try {
                     removeInternal(new GridCacheInternalKeyImpl(name), GridCacheAtomicLongValue.class);
                 }
-                catch (Exception e) {
-                    throw new IgniteCheckedException("Failed to remove atomic long by name: " + name, e);
-                }
                 finally {
                     dsCacheCtx.gate().leave();
                 }
@@ -479,7 +469,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
      * @param afterRmv Optional closure to run after data structure removed.
      * @throws IgniteCheckedException If failed.
      */
-    private <T> void removeDataStructure(IgniteCallable<T> c,
+    private <T> void removeDataStructure(IgniteOutClosureX<T> c,
         String name,
         DataStructureType type,
         @Nullable IgniteInClosureX<T> afterRmv)
@@ -515,7 +505,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
             if (!exists)
                 return;
 
-            rmvInfo = ctx.closure().callLocalSafe(c, false).get();
+            rmvInfo = c.applyx();
 
             tx.commit();
         }
@@ -611,8 +601,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         awaitInitialization();
 
-        removeDataStructure(new IgniteCallable<Void>() {
-            @Override public Void call() throws Exception {
+        removeDataStructure(new IgniteOutClosureX<Void>() {
+            @Override public Void applyx() throws IgniteCheckedException {
                 dsCacheCtx.gate().enter();
 
                 try {
@@ -620,9 +610,6 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
                     removeInternal(key, GridCacheAtomicReferenceValue.class);
                 }
-                catch (Exception e) {
-                    throw new IgniteCheckedException("Failed to remove atomic reference by name: " + name, e);
-                }
                 finally {
                     dsCacheCtx.gate().leave();
                 }
@@ -718,8 +705,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         awaitInitialization();
 
-        removeDataStructure(new IgniteCallable<Void>() {
-            @Override public Void call() throws Exception {
+        removeDataStructure(new IgniteOutClosureX<Void>() {
+            @Override public Void applyx() throws IgniteCheckedException {
                 dsCacheCtx.gate().enter();
 
                 try {
@@ -727,9 +714,6 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
                     removeInternal(key, GridCacheAtomicStampedValue.class);
                 }
-                catch (Exception e) {
-                    throw new IgniteCheckedException("Failed to remove atomic stamped by name: " + name, e);
-                }
                 finally {
                     dsCacheCtx.gate().leave();
                 }
@@ -806,6 +790,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
     /**
      * @param cfg Collection configuration.
      * @return Cache name.
+     * @throws IgniteCheckedException If failed.
      */
     private String compatibleConfiguration(CollectionConfiguration cfg) throws IgniteCheckedException {
         List<CacheCollectionInfo> caches = utilityDataCache.localPeek(DATA_STRUCTURES_CACHE_KEY, null, null);
@@ -835,8 +820,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         awaitInitialization();
 
-        IgniteCallable<GridCacheQueueHeader> rmv = new IgniteCallable<GridCacheQueueHeader>() {
-            @Override public GridCacheQueueHeader call() throws Exception {
+        IgniteOutClosureX<GridCacheQueueHeader> rmv = new IgniteOutClosureX<GridCacheQueueHeader>() {
+            @Override public GridCacheQueueHeader applyx() throws IgniteCheckedException {
                 return (GridCacheQueueHeader)retryRemove(cctx.cache(), new GridCacheQueueHeaderKey(name));
             }
         };
@@ -910,11 +895,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
             final GridCacheContext cacheCtx = ctx.cache().internalCache(cacheName).context();
 
-            col = ctx.closure().callLocalSafe(new Callable<T>() {
-                @Override public T call() throws Exception {
-                    return c.applyx(cacheCtx);
-                }
-            }, false).get();
+            col = c.applyx(cacheCtx);
 
             tx.commit();
         }
@@ -1055,8 +1036,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         awaitInitialization();
 
-        removeDataStructure(new IgniteCallable<Void>() {
-            @Override public Void call() throws Exception {
+        removeDataStructure(new IgniteOutClosureX<Void>() {
+            @Override public Void applyx() throws IgniteCheckedException {
                 GridCacheInternal key = new GridCacheInternalKeyImpl(name);
 
                 dsCacheCtx.gate().enter();
@@ -1079,11 +1060,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                         tx.setRollbackOnly();
 
                     return null;
-                } catch (Error | Exception e) {
-                    U.error(log, "Failed to remove data structure: " + key, e);
-
-                    throw e;
-                } finally {
+                }
+                finally {
                     dsCacheCtx.gate().leave();
                 }
             }
@@ -1257,8 +1235,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         awaitInitialization();
 
-        IgniteCallable<GridCacheSetHeader> rmv = new IgniteCallable<GridCacheSetHeader>() {
-            @Override public GridCacheSetHeader call() throws Exception {
+        IgniteOutClosureX<GridCacheSetHeader> rmv = new IgniteOutClosureX<GridCacheSetHeader>() {
+            @Override public GridCacheSetHeader applyx() throws IgniteCheckedException {
                 return (GridCacheSetHeader)retryRemove(cctx.cache(), new GridCacheSetHeaderKey(name));
             }
         };

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/74dc6d5a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
index 33fe127..49ee641 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
@@ -455,11 +455,11 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends IgniteColl
     }
 
     /**
-     * TODO: IGNITE-583.
-     *
      * @throws Exception If failed.
      */
     public void testIterator() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-583");
+
         final String queueName = UUID.randomUUID().toString();
 
         info("Queue name: " + queueName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/74dc6d5a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
index 6383f91..74114d9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
@@ -553,20 +553,20 @@ public abstract class GridCacheSetAbstractSelfTest extends IgniteCollectionAbstr
     }
 
     /**
-     * TODO: IGNITE-584.
-     *  
      * @throws Exception If failed.
      */
     public void testNodeJoinsAndLeaves() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-584");
+
         testNodeJoinsAndLeaves(false);
     }
 
     /**
-     * TODO: IGNITE-584.
-     *
      * @throws Exception If failed.
      */
     public void testNodeJoinsAndLeavesCollocated() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-584");
+
         testNodeJoinsAndLeaves(true);
     }
 


[3/5] incubator-ignite git commit: # ignite-1022 added test

Posted by sb...@apache.org.
# ignite-1022 added test


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

Branch: refs/heads/ignite-sprint-7
Commit: 78a9e69c5779f687407faddf565fa5090d94fee9
Parents: 627d0b5
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 12:51:18 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 12:51:18 2015 +0300

----------------------------------------------------------------------
 .../IgniteDataStructureWithJobTest.java         | 111 +++++++++++++++++++
 .../IgniteCacheDataStructuresSelfTestSuite.java |   1 +
 2 files changed, 112 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/78a9e69c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteDataStructureWithJobTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteDataStructureWithJobTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteDataStructureWithJobTest.java
new file mode 100644
index 0000000..47d96be
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteDataStructureWithJobTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class IgniteDataStructureWithJobTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 2 * 60_000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJobWithRestart() throws Exception {
+        Ignite ignite = startGrid(0);
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        final long endTime = System.currentTimeMillis() + 60_000;
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                while (!Thread.currentThread().isInterrupted() && !stop.get() && U.currentTimeMillis() < endTime) {
+                    try (Ignite ignored = startGrid(1)) {
+                        Thread.sleep(500);
+                    }
+                }
+
+                return null;
+            }
+        });
+
+        try {
+            int iter = 0;
+
+            while (System.currentTimeMillis() < endTime) {
+                try {
+                    ignite.compute().broadcast(new IgniteClosure<IgniteSet, Integer>() {
+                        @Override public Integer apply(IgniteSet set) {
+                            assertNotNull(set);
+
+                            return 1;
+                        }
+                    }, ignite.set("set", new CollectionConfiguration()));
+                }
+                catch (IgniteException ignore) {
+                    // No-op.
+                }
+
+                if (iter++ % 1000 == 0)
+                    log.info("Iteration: " + iter);
+            }
+        }
+        finally {
+            stop.set(true);
+        }
+
+        fut.get();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/78a9e69c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
index a689abb..5cb9514 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
@@ -74,6 +74,7 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgnitePartitionedSetNoBackupsSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedAtomicSetSelfTest.class));
         suite.addTest(new TestSuite(IgnitePartitionedCountDownLatchSelfTest.class));
+        suite.addTest(new TestSuite(IgniteDataStructureWithJobTest.class));
 
         suite.addTest(new TestSuite(GridCachePartitionedSetFailoverSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedOffheapSetFailoverSelfTest.class));


[5/5] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1022' into ignite-sprint-7

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1022' into ignite-sprint-7


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

Branch: refs/heads/ignite-sprint-7
Commit: b3e2bd488e183adf0146f170df94eb762c3056da
Parents: 359b431 de7dcfe
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 24 09:31:43 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 24 09:31:43 2015 +0300

----------------------------------------------------------------------
 .../datastructures/DataStructuresProcessor.java |  64 ++++-------
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |   4 +-
 .../GridCacheSetAbstractSelfTest.java           |  22 ++--
 .../IgniteDataStructureWithJobTest.java         | 111 +++++++++++++++++++
 .../distributed/IgniteCacheManyClientsTest.java |   1 +
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   5 -
 ...achePartitionedMultiNodeFullApiSelfTest.java |  49 ++++++--
 .../IgniteCacheDataStructuresSelfTestSuite.java |   1 +
 8 files changed, 186 insertions(+), 71 deletions(-)
----------------------------------------------------------------------



[4/5] incubator-ignite git commit: # ignite-1022

Posted by sb...@apache.org.
# ignite-1022


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

Branch: refs/heads/ignite-sprint-7
Commit: de7dcfe4e4a10e5b557bf7f6c1776026ed6a845e
Parents: 78a9e69
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 24 09:29:56 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 24 09:29:56 2015 +0300

----------------------------------------------------------------------
 .../datastructures/GridCacheSetAbstractSelfTest.java  | 14 ++++++++------
 .../cache/distributed/IgniteCacheManyClientsTest.java |  1 +
 2 files changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/de7dcfe4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
index 74114d9..0494bcc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
@@ -26,7 +26,6 @@ import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.testframework.*;
 
 import java.util.*;
@@ -556,6 +555,9 @@ public abstract class GridCacheSetAbstractSelfTest extends IgniteCollectionAbstr
      * @throws Exception If failed.
      */
     public void testNodeJoinsAndLeaves() throws Exception {
+        if (collectionCacheMode() == LOCAL)
+            return;
+
         fail("https://issues.apache.org/jira/browse/IGNITE-584");
 
         testNodeJoinsAndLeaves(false);
@@ -565,6 +567,9 @@ public abstract class GridCacheSetAbstractSelfTest extends IgniteCollectionAbstr
      * @throws Exception If failed.
      */
     public void testNodeJoinsAndLeavesCollocated() throws Exception {
+        if (collectionCacheMode() == LOCAL)
+            return;
+
         fail("https://issues.apache.org/jira/browse/IGNITE-584");
 
         testNodeJoinsAndLeaves(true);
@@ -575,9 +580,6 @@ public abstract class GridCacheSetAbstractSelfTest extends IgniteCollectionAbstr
      * @throws Exception If failed.
      */
     private void testNodeJoinsAndLeaves(boolean collocated) throws Exception {
-        if (collectionCacheMode() == LOCAL)
-            return;
-
         CollectionConfiguration colCfg = config(collocated);
 
         Set<Integer> set0 = grid(0).set(SET_NAME, colCfg);
@@ -751,7 +753,7 @@ public abstract class GridCacheSetAbstractSelfTest extends IgniteCollectionAbstr
         IgniteInternalFuture<?> fut;
 
         try {
-                fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     try {
                         while (!stop.get()) {
@@ -781,7 +783,7 @@ public abstract class GridCacheSetAbstractSelfTest extends IgniteCollectionAbstr
 
         for (int i = 0; i < gridCount(); i++) {
             Iterator<GridCacheEntryEx> entries =
-                    ((IgniteKernal)grid(i)).context().cache().internalCache(cctx.name()).map().allEntries0().iterator();
+                (grid(i)).context().cache().internalCache(cctx.name()).map().allEntries0().iterator();
 
             while (entries.hasNext()) {
                 GridCacheEntryEx entry = entries.next();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/de7dcfe4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
index 48884de..1991f01 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
@@ -66,6 +66,7 @@ public class IgniteCacheManyClientsTest extends GridCommonAbstractTest {
         ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinderCleanFrequency(10 * 60_000);
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setJoinTimeout(2 * 60_000);
 
         if (!clientDiscovery)


[2/5] incubator-ignite git commit: # ignite-1022 fixed test

Posted by sb...@apache.org.
# ignite-1022 fixed test


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

Branch: refs/heads/ignite-sprint-7
Commit: 627d0b5d43e78b58ac979a8cadb48ae00f7f0646
Parents: 74dc6d5
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 11:13:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 11:13:20 2015 +0300

----------------------------------------------------------------------
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |  5 --
 ...achePartitionedMultiNodeFullApiSelfTest.java | 49 +++++++++++++++-----
 2 files changed, 38 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/627d0b5d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
index 8343c64..d40e9e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
@@ -192,11 +192,6 @@ public class GridCacheNearOnlyMultiNodeFullApiSelfTest extends GridCachePartitio
         // Not needed for near-only cache.
     }
 
-    /** {@inheritDoc} */
-    @Override public void testNearDhtKeySize() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1029");
-    }
-
     /**
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/627d0b5d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index 84594a2..9445f9c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -299,26 +299,53 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
 
         info("Generating keys for test...");
 
-        IgniteCache<String, Integer> cache0 = jcache(0);
+        IgniteEx ignite0 = null;
+        IgniteEx ignite1 = null;
+        IgniteEx ignite2 = null;
 
-        for (int i = 0; i < 5; i++) {
-            while (true) {
-                String key = UUID.randomUUID().toString();
-
-                if (ignite(0).affinity(null).isPrimary(grid(0).localNode(), key) &&
-                    ignite(0).affinity(null).isBackup(grid(1).localNode(), key)) {
-                    keys.add(key);
+        for (int i = 0; i < gridCount(); i++) {
+            IgniteEx ignite = grid(i);
 
-                    cache0.put(key, i);
+            if (!ignite.configuration().isClientMode()) {
+                if (ignite0 == null)
+                    ignite0 = ignite;
+                else if (ignite1 == null)
+                    ignite1 = ignite;
+                else {
+                    ignite2 = ignite;
 
                     break;
                 }
+
             }
         }
 
+        assertNotNull(ignite0);
+        assertNotNull(ignite1);
+        assertNotNull(ignite2);
+
+        info("Generating keys for test [nodes=" + ignite0.name() + ", " + ignite1.name() + ", " + ignite2.name() + ']');
+
+        IgniteCache<String, Integer> cache0 = ignite0.cache(null);
+
+        int val = 0;
+
+        for (int i = 0; i < 10_000 && keys.size() < 5; i++) {
+            String key = String.valueOf(i);
+
+            if (ignite(0).affinity(null).isPrimary(ignite0.localNode(), key) &&
+                ignite(0).affinity(null).isBackup(ignite1.localNode(), key)) {
+                keys.add(key);
+
+                cache0.put(key, val++);
+            }
+        }
+
+        assertEquals(5, keys.size());
+
         info("Finished generating keys for test.");
 
-        IgniteCache<String, Integer> cache2 = jcache(2);
+        IgniteCache<String, Integer> cache2 = ignite2.cache(null);
 
         assertEquals(Integer.valueOf(0), cache2.get(keys.get(0)));
         assertEquals(Integer.valueOf(1), cache2.get(keys.get(1)));
@@ -326,7 +353,7 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
         assertEquals(0, cache0.localSize(NEAR));
         assertEquals(5, cache0.localSize(CachePeekMode.ALL) - cache0.localSize(NEAR));
 
-        IgniteCache<String, Integer> cache1 = jcache(1);
+        IgniteCache<String, Integer> cache1 = ignite1.cache(null);
 
         assertEquals(0, cache1.localSize(NEAR));
         assertEquals(5, cache1.localSize(CachePeekMode.ALL) - cache1.localSize(NEAR));