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/29 14:26:32 UTC

[1/4] incubator-ignite git commit: # init marshaller cache on daemon nodes

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-sprint-7 f45f96d0c -> 15f3612d3


# init marshaller cache on daemon nodes


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

Branch: refs/heads/ignite-sprint-7
Commit: 3231f93373a2f84b7e0ea64d969b53121b6fe323
Parents: 01d842a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 26 14:34:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 26 15:28:52 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/GridKernalContextImpl.java  |   2 +-
 .../processors/cache/GridCacheProcessor.java    |  62 ++++--
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 4 files changed, 245 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3231f933/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 65107a7..8abb135 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -899,7 +899,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** {@inheritDoc} */
     @Override public boolean clientNode() {
-        return cfg.isClientMode();
+        return cfg.isClientMode() || cfg.isDaemon();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3231f933/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 2f7f22c..8124594 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -265,7 +265,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         // Suppress warning if at least one ATOMIC cache found.
         perf.add("Enable ATOMIC mode if not using transactions (set 'atomicityMode' to ATOMIC)",
-                 cfg.getAtomicityMode() == ATOMIC);
+            cfg.getAtomicityMode() == ATOMIC);
 
         // Suppress warning if at least one non-FULL_SYNC mode found.
         perf.add("Disable fully synchronous writes (set 'writeSynchronizationMode' to PRIMARY_SYNC or FULL_ASYNC)",
@@ -341,8 +341,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 "Serializable transactions are disabled while default transaction isolation is SERIALIZABLE " +
                     "(most likely misconfiguration - either update 'isTxSerializableEnabled' or " +
                     "'defaultTxIsolationLevel' properties) for cache: " + U.maskName(cc.getName()),
-                    "Serializable transactions are disabled while default transaction isolation is SERIALIZABLE " +
-                        "for cache: " + U.maskName(cc.getName()));
+                "Serializable transactions are disabled while default transaction isolation is SERIALIZABLE " +
+                    "for cache: " + U.maskName(cc.getName()));
 
         if (cc.isWriteBehindEnabled()) {
             if (cfgStore == null)
@@ -527,8 +527,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings( {"unchecked"})
     @Override public void start() throws IgniteCheckedException {
-        if (ctx.config().isDaemon())
+        if (ctx.config().isDaemon()) {
+            sharedCtx = createSharedContext(ctx, CU.startStoreSessionListeners(ctx, null));
+
+            for (GridCacheSharedManager mgr : sharedCtx.managers())
+                mgr.start(sharedCtx);
+
             return;
+        }
 
         DeploymentMode depMode = ctx.config().getDeploymentMode();
 
@@ -667,8 +673,45 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart() throws IgniteCheckedException {
         try {
-            if (ctx.config().isDaemon())
+            if (ctx.config().isDaemon()) {
+                for (CacheConfiguration ccfg : ctx.config().getCacheConfiguration()) {
+                    if (CU.isMarshallerCache(ccfg.getName())) {
+                        CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
+
+                        initialize(ccfg, cacheObjCtx);
+
+                        GridCacheContext ctx = createCache(ccfg, null, CacheType.MARSHALLER, cacheObjCtx, true);
+
+                        ctx.dynamicDeploymentId(IgniteUuid.randomUuid());
+
+                        sharedCtx.addCacheContext(ctx);
+
+                        GridCacheAdapter cache = ctx.cache();
+
+                        String name = ccfg.getName();
+
+                        caches.put(maskNull(name), cache);
+
+                        startCache(cache);
+
+                        break;
+                    }
+                }
+
+                marshallerCache().context().preloader().syncFuture().listen(new CIX1<IgniteInternalFuture<?>>() {
+                    @Override public void applyx(IgniteInternalFuture<?> f) throws IgniteCheckedException {
+                        ctx.marshallerContext().onMarshallerCachePreloaded(ctx);
+                    }
+                });
+
+                for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
+                    mgr.onKernalStart();
+
+                for (GridCacheAdapter<?, ?> cache : caches.values())
+                    onKernalStart(cache);
+
                 return;
+            }
 
             ClusterNode locNode = ctx.discovery().localNode();
 
@@ -808,9 +851,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
-        if (ctx.config().isDaemon())
-            return;
-
         for (String cacheName : stopSeq) {
             GridCacheAdapter<?, ?> cache = caches.remove(maskNull(cacheName));
 
@@ -850,9 +890,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @Override public void onKernalStop(boolean cancel) {
         cacheStartedLatch.countDown();
 
-        if (ctx.config().isDaemon())
-            return;
-
         for (String cacheName : stopSeq) {
             GridCacheAdapter<?, ?> cache = caches.get(maskNull(cacheName));
 
@@ -1390,6 +1427,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         Collection<DynamicCacheChangeRequest> reqs,
         AffinityTopologyVersion topVer
     ) throws IgniteCheckedException {
+        if (ctx.isDaemon())
+            return;
+
         for (DynamicCacheChangeRequest req : reqs) {
             assert req.start() : req;
             assert req.cacheType() != null : req;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3231f933/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
new file mode 100644
index 0000000..a0f4074
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
@@ -0,0 +1,192 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+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.junits.common.*;
+
+import java.io.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.internal.IgniteNodeAttributes.*;
+
+/**
+ *
+ */
+public class IgniteDaemonNodeMarshallerCacheTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean daemon;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDaemon(daemon);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMarshalOnDaemonNode1() throws Exception {
+        marshalOnDaemonNode(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMarshalOnDaemonNode2() throws Exception {
+        marshalOnDaemonNode(false);
+    }
+
+    /**
+     * @param startFirst If {@code true} daemon node is started first.
+     * @throws Exception If failed.
+     */
+    public void marshalOnDaemonNode(boolean startFirst) throws Exception {
+        int nodeIdx = 0;
+
+        if (!startFirst) {
+            Ignite ignite1 = startGrid(nodeIdx++);
+
+            assertFalse("true".equals(ignite1.cluster().localNode().attribute(ATTR_DAEMON)));
+        }
+
+        daemon = true;
+
+        Ignite daemonNode = startGrid(nodeIdx++);
+
+        assertEquals("true", daemonNode.cluster().localNode().attribute(ATTR_DAEMON));
+
+        daemon = false;
+
+        if (startFirst) {
+            Ignite ignite1 = startGrid(nodeIdx++);
+
+            assertFalse("true".equals(ignite1.cluster().localNode().attribute(ATTR_DAEMON)));
+        }
+
+        awaitPartitionMapExchange();
+
+        TestClass1 res1 = daemonNode.compute(daemonNode.cluster().forRemotes()).call(new TestCallable1());
+
+        assertNotNull(res1);
+        assertEquals(111, res1.val);
+
+        Ignite ignite2 = startGrid(nodeIdx);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setRebalanceMode(SYNC);
+        ccfg.setCacheMode(REPLICATED);
+
+        IgniteCache<Object, Object> cache = ignite2.getOrCreateCache(ccfg);
+
+        awaitPartitionMapExchange();
+
+        cache.put(1, new TestClass1(1));
+        cache.put(2, new TestClass2(2));
+
+        TestClass2 res2 = daemonNode.compute(daemonNode.cluster().forRemotes()).call(new TestCallable2());
+
+        assertNotNull(res2);
+        assertEquals(222, res2.val);
+    }
+
+    /**
+     *
+     */
+    private static class TestCallable1 implements IgniteCallable<TestClass1> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public TestClass1 call() throws Exception {
+            assertFalse("true".equals(ignite.cluster().localNode().attribute(ATTR_DAEMON)));
+
+            return new TestClass1(111);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestCallable2 implements IgniteCallable<TestClass2> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public TestClass2 call() throws Exception {
+            assertFalse("true".equals(ignite.cluster().localNode().attribute(ATTR_DAEMON)));
+
+            return new TestClass2(222);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestClass1 implements Serializable {
+        /** */
+        public int val;
+
+        /**
+         * @param val Value.
+         */
+        public TestClass1(int val) {
+            this.val = val;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestClass2 implements Serializable {
+        /** */
+        public int val;
+
+        /**
+         * @param val Value.
+         */
+        public TestClass2(int val) {
+            this.val = val;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3231f933/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 6ff83e2..df22290 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -94,6 +94,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(GridFailFastNodeFailureDetectionSelfTest.class);
         suite.addTestSuite(OffHeapTieredTransactionSelfTest.class);
         suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
+        suite.addTestSuite(IgniteDaemonNodeMarshallerCacheTest.class);
 
         return suite;
     }


[2/4] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-7' into ignite-gg-10453

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


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

Branch: refs/heads/ignite-sprint-7
Commit: d86882c3597080c62877ba36b916dda29089b900
Parents: 3231f93 6e23608
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 26 16:55:32 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 26 16:55:32 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      |   6 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 151 ++++++-----
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 103 +++++--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   3 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   4 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 265 ++++++++++++++++++-
 6 files changed, 451 insertions(+), 81 deletions(-)
----------------------------------------------------------------------



[3/4] incubator-ignite git commit: # ignite-gg-10453 removed test (test is invalid after fix)

Posted by sb...@apache.org.
# ignite-gg-10453 removed test (test is invalid after fix)


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

Branch: refs/heads/ignite-sprint-7
Commit: d51db291037866b03ac828430cfdcc914c2f79ed
Parents: d86882c
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 29 09:21:05 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 29 09:21:05 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 -------------------
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 2 files changed, 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d51db291/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
deleted file mode 100644
index c56ad1c..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
+++ /dev/null
@@ -1,119 +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;
-
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
-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 static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Test cache operations with daemon node.
- */
-public class GridCacheDaemonNodeStopSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Daemon flag. */
-    protected boolean daemon;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        daemon = false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        c.setDaemon(daemon);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
-
-        c.setConnectorConfiguration(null);
-
-        CacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(LOCAL);
-        cc.setAtomicityMode(ATOMIC);
-
-        TransactionConfiguration cfg = new TransactionConfiguration();
-
-        c.setTransactionConfiguration(cfg);
-
-        c.setCacheConfiguration(cc);
-
-        return c;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStartStop() throws Exception {
-        try {
-            daemon = true;
-
-            final IgniteEx daemonNode = startGrid(0);
-
-            final IgniteInternalFuture<Object> f = GridTestUtils.runAsync(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    daemonNode.context().marshallerContext().getClass(1,
-                        GridCacheDaemonNodeStopSelfTest.class.getClassLoader());
-
-                    return null;
-                }
-            });
-
-            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    f.get(300);
-
-                    return null;
-                }
-            }, IgniteFutureTimeoutCheckedException.class);
-
-            // Stop grid.
-            stopGrid(0);
-
-            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    f.get(5, TimeUnit.SECONDS);
-
-                    return null;
-                }
-            }, IllegalStateException.class);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d51db291/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
index b83f50f..5947d33 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
@@ -100,7 +100,6 @@ public class IgniteCacheTestSuite3 extends TestSuite {
         suite.addTestSuite(GridCacheDaemonNodeLocalSelfTest.class);
         suite.addTestSuite(GridCacheDaemonNodePartitionedSelfTest.class);
         suite.addTestSuite(GridCacheDaemonNodeReplicatedSelfTest.class);
-        suite.addTestSuite(GridCacheDaemonNodeStopSelfTest.class);
 
         // Write-behind.
         suite.addTest(IgniteCacheWriteBehindTestSuite.suite());


[4/4] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-gg-10453' into ignite-sprint-7

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-gg-10453' 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/15f3612d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/15f3612d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/15f3612d

Branch: refs/heads/ignite-sprint-7
Commit: 15f3612d31a1f39046d4bf5cc7be6e0aed7d02c8
Parents: f45f96d d51db29
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 29 15:25:53 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 29 15:25:53 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/GridKernalContextImpl.java  |   2 +-
 .../processors/cache/GridCacheProcessor.java    |  62 ++++--
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 ------------
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 6 files changed, 245 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/15f3612d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------